Extend citus local table utility command support

This commit brings following features:

Foreign key support from citus local tables to reference tables
* Foreign key support from reference tables to citus local tables
  (only with RESTRICT & NO ACTION behavior)
* ALTER TABLE ENABLE/DISABLE trigger command support
* CREATE/DROP/ALTER trigger command support

and disallows:
* ALTER TABLE ATTACH/DETACH PARTITION commands
* CREATE TABLE <postgres table> ATTACH PARTITION <citus local table>
  commands
* Foreign keys from postgres tables to citus local tables
  (the other way was already disallowed)

for citus local tables.
pull/4143/head
Onur Tirtir 2020-09-09 11:37:45 +03:00
parent 17cc810372
commit a58a4395ab
43 changed files with 3347 additions and 496 deletions

View File

@ -818,8 +818,8 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
}
}
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn,
colocationId);
ErrorIfUnsupportedConstraint(relation, distributionMethod, replicationModel,
distributionColumn, colocationId);
ErrorIfUnsupportedPolicy(relation);
@ -987,7 +987,8 @@ EnsureReplicationSettings(Oid relationId, char replicationModel)
extraHint = "";
}
if (replicationModel == REPLICATION_MODEL_STREAMING && ShardReplicationFactor != 1)
if (replicationModel == REPLICATION_MODEL_STREAMING &&
DistributedTableReplicationIsEnabled())
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("replication factors above one are incompatible with %s",

View File

@ -155,6 +155,13 @@ static DistributeObjectOps Any_CreatePolicy = {
.postprocess = NULL,
.address = NULL,
};
static DistributeObjectOps Any_CreateTrigger = {
.deparse = NULL,
.qualify = NULL,
.preprocess = NULL,
.postprocess = PostprocessCreateTriggerStmt,
.address = CreateTriggerStmtObjectAddress,
};
static DistributeObjectOps Any_Grant = {
.deparse = NULL,
.qualify = NULL,
@ -344,6 +351,13 @@ static DistributeObjectOps Routine_AlterObjectDepends = {
.postprocess = NULL,
.address = AlterFunctionDependsStmtObjectAddress,
};
static DistributeObjectOps Trigger_AlterObjectDepends = {
.deparse = NULL,
.qualify = NULL,
.preprocess = NULL,
.postprocess = PostprocessAlterTriggerDependsStmt,
.address = NULL,
};
static DistributeObjectOps Routine_AlterObjectSchema = {
.deparse = DeparseAlterFunctionSchemaStmt,
.qualify = QualifyAlterFunctionSchemaStmt,
@ -435,6 +449,13 @@ static DistributeObjectOps Type_Drop = {
.postprocess = NULL,
.address = NULL,
};
static DistributeObjectOps Trigger_Drop = {
.deparse = NULL,
.qualify = NULL,
.preprocess = PreprocessDropTriggerStmt,
.postprocess = NULL,
.address = NULL,
};
static DistributeObjectOps Type_Rename = {
.deparse = DeparseRenameTypeStmt,
.qualify = QualifyRenameTypeStmt,
@ -442,6 +463,13 @@ static DistributeObjectOps Type_Rename = {
.postprocess = NULL,
.address = RenameTypeStmtObjectAddress,
};
static DistributeObjectOps Trigger_Rename = {
.deparse = NULL,
.qualify = NULL,
.preprocess = NULL,
.postprocess = PostprocessAlterTriggerRenameStmt,
.address = NULL,
};
/*
@ -494,6 +522,11 @@ GetDistributeObjectOps(Node *node)
return &Routine_AlterObjectDepends;
}
case OBJECT_TRIGGER:
{
return &Trigger_AlterObjectDepends;
}
default:
{
return &NoDistributeOps;
@ -677,6 +710,11 @@ GetDistributeObjectOps(Node *node)
return &Any_CreatePolicy;
}
case T_CreateTrigStmt:
{
return &Any_CreateTrigger;
}
case T_DefineStmt:
{
DefineStmt *stmt = castNode(DefineStmt, node);
@ -759,6 +797,11 @@ GetDistributeObjectOps(Node *node)
return &Type_Drop;
}
case OBJECT_TRIGGER:
{
return &Trigger_Drop;
}
default:
{
return &NoDistributeOps;
@ -833,6 +876,11 @@ GetDistributeObjectOps(Node *node)
return &Type_Rename;
}
case OBJECT_TRIGGER:
{
return &Trigger_Rename;
}
default:
{
return &Any_Rename;

View File

@ -36,7 +36,23 @@
#include "utils/ruleutils.h"
#include "utils/syscache.h"
#define BehaviorIsRestrictOrNoAction(x) \
((x) == FKCONSTR_ACTION_NOACTION || (x) == FKCONSTR_ACTION_RESTRICT)
typedef bool (*CheckRelationFunc)(Oid);
/* Local functions forward declarations */
static void EnsureReferencingTableNotReplicated(Oid referencingTableId);
static void EnsureSupportedFKeyOnDistKey(Form_pg_constraint constraintForm);
static void EnsureSupportedFKeyBetweenCitusLocalAndRefTable(Form_pg_constraint
constraintForm,
char
referencingReplicationModel,
char
referencedReplicationModel);
static bool HeapTupleOfForeignConstraintIncludesColumn(HeapTuple heapTuple,
Oid relationId,
int pgConstraintKey,
@ -50,7 +66,10 @@ static void ForeignConstraintFindDistKeys(HeapTuple pgConstraintTuple,
int *referencedAttrIndex);
static List * GetForeignConstraintCommandsInternal(Oid relationId, int flags);
static Oid get_relation_constraint_oid_compat(HeapTuple heapTuple);
static List * GetForeignKeyOidsToCitusLocalTables(Oid relationId);
static List * GetForeignKeyOidsToReferenceTables(Oid relationId);
static List * FilterFKeyOidListByReferencedTableType(List *foreignKeyOidList,
CitusTableType citusTableType);
/*
* ConstraintIsAForeignKeyToReferenceTable checks if the given constraint is a
@ -85,27 +104,21 @@ ConstraintIsAForeignKeyToReferenceTable(char *inputConstaintName, Oid relationId
* ON UPDATE CASCADE options are not used on the distribution key
* of the referencing column.
* - If referencing table is a reference table, error out if the referenced
* table is not a reference table.
* table is a distributed table.
* - If referencing table is a reference table and referenced table is a
* citus local table:
* - ON DELETE/UPDATE SET NULL, ON DELETE/UPDATE SET DEFAULT and
* ON CASCADE options are not used.
* - If referencing or referenced table is distributed table, then the
* other table is not a citus local table.
*/
void
ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDistMethod,
char referencingReplicationModel,
Var *referencingDistKey,
uint32 referencingColocationId)
{
Oid referencingTableId = relation->rd_id;
bool referencingNotReplicated = true;
bool referencingIsCitus = IsCitusTable(referencingTableId);
if (referencingIsCitus)
{
/* ALTER TABLE command is applied over single replicated table */
referencingNotReplicated = SingleReplicatedTable(referencingTableId);
}
else
{
/* Creating single replicated table with foreign constraint */
referencingNotReplicated = (ShardReplicationFactor == 1);
}
int flags = INCLUDE_REFERENCING_CONSTRAINTS;
List *foreignKeyOids = GetForeignKeyOids(referencingTableId, flags);
@ -132,6 +145,12 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
if (!referencedIsCitus && !selfReferencingTable)
{
if (IsCitusLocalTableByDistParams(referencingDistMethod,
referencingReplicationModel))
{
ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(referencedTableId);
}
char *referencedTableName = get_rel_name(referencedTableId);
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
@ -148,6 +167,7 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
/* set referenced table related variables here if table is referencing itself */
char referencedDistMethod = 0;
char referencedReplicationModel = REPLICATION_MODEL_INVALID;
if (!selfReferencingTable)
{
referencedDistMethod = PartitionMethod(referencedTableId);
@ -156,45 +176,51 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
NULL :
DistPartitionKey(referencedTableId);
referencedColocationId = TableColocationId(referencedTableId);
referencedReplicationModel = TableReplicationModel(referencedTableId);
}
else
{
referencedDistMethod = referencingDistMethod;
referencedDistKey = referencingDistKey;
referencedColocationId = referencingColocationId;
referencedReplicationModel = referencingReplicationModel;
}
bool referencingIsReferenceTable = (referencingDistMethod == DISTRIBUTE_BY_NONE);
bool referencedIsReferenceTable = (referencedDistMethod == DISTRIBUTE_BY_NONE);
/*
* We support foreign keys between reference tables. No more checks
* are necessary.
*/
if (referencingIsReferenceTable && referencedIsReferenceTable)
bool referencingIsCitusLocalOrRefTable =
(referencingDistMethod == DISTRIBUTE_BY_NONE);
bool referencedIsCitusLocalOrRefTable =
(referencedDistMethod == DISTRIBUTE_BY_NONE);
if (referencingIsCitusLocalOrRefTable && referencedIsCitusLocalOrRefTable)
{
EnsureSupportedFKeyBetweenCitusLocalAndRefTable(constraintForm,
referencingReplicationModel,
referencedReplicationModel);
ReleaseSysCache(heapTuple);
continue;
}
/*
* Foreign keys from reference tables to distributed tables are not
* supported.
* Foreign keys from citus local tables or reference tables to distributed
* tables are not supported.
*/
if (referencingIsReferenceTable && !referencedIsReferenceTable)
if (referencingIsCitusLocalOrRefTable && !referencedIsCitusLocalOrRefTable)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint "
"since foreign keys from reference tables "
"to distributed tables are not supported"),
errdetail("A reference table can only have reference "
"keys to other reference tables")));
errdetail("A reference table can only have foreign "
"keys to other reference tables or citus "
"local tables")));
}
/*
* To enforce foreign constraints, tables must be co-located unless a
* reference table is referenced.
*/
bool referencedIsReferenceTable =
(referencedReplicationModel == REPLICATION_MODEL_2PC);
if (referencingColocationId == INVALID_COLOCATION_ID ||
(referencingColocationId != referencedColocationId &&
!referencedIsReferenceTable))
@ -226,44 +252,14 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
*/
if (referencingColumnsIncludeDistKey)
{
/*
* ON DELETE SET NULL and ON DELETE SET DEFAULT is not supported. Because we do
* not want to set partition column to NULL or default value.
*/
if (constraintForm->confdeltype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confdeltype == FKCONSTR_ACTION_SETDEFAULT)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL or SET DEFAULT is not supported"
" in ON DELETE operation when distribution "
"key is included in the foreign key constraint")));
}
/*
* ON UPDATE SET NULL, ON UPDATE SET DEFAULT and UPDATE CASCADE is not supported.
* Because we do not want to set partition column to NULL or default value. Also
* cascading update operation would require re-partitioning. Updating partition
* column value is not allowed anyway even outside of foreign key concept.
*/
if (constraintForm->confupdtype == FKCONSTR_ACTION_SETNULL ||
constraintForm->confupdtype == FKCONSTR_ACTION_SETDEFAULT ||
constraintForm->confupdtype == FKCONSTR_ACTION_CASCADE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL, SET DEFAULT or CASCADE is not "
"supported in ON UPDATE operation when "
"distribution key included in the foreign "
"constraint.")));
}
EnsureSupportedFKeyOnDistKey(constraintForm);
}
/*
* if tables are hash-distributed and colocated, we need to make sure that
* the distribution key is included in foreign constraint.
*/
if (!referencedIsReferenceTable && !foreignConstraintOnDistKey)
if (!referencedIsCitusLocalOrRefTable && !foreignConstraintOnDistKey)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
@ -283,24 +279,151 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
* placements always be in the same state (b) executors are aware of reference
* tables and handle concurrency related issues accordingly.
*/
if (!referencingNotReplicated)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Citus Community Edition currently supports "
"foreign key constraints only for "
"\"citus.shard_replication_factor = 1\"."),
errhint("Please change \"citus.shard_replication_factor to "
"1\". To learn more about using foreign keys with "
"other replication factors, please contact us at "
"https://citusdata.com/about/contact_us.")));
}
EnsureReferencingTableNotReplicated(referencingTableId);
ReleaseSysCache(heapTuple);
}
}
/*
* EnsureSupportedFKeyBetweenCitusLocalAndRefTable is a helper function that
* takes a foreign key constraint form for a foreign key between two citus
* tables that are either citus local table or reference table and errors
* out if it it an unsupported foreign key from a reference table to a citus
* local table according to given replication model parameters.
*/
static void
EnsureSupportedFKeyBetweenCitusLocalAndRefTable(Form_pg_constraint fKeyConstraintForm,
char referencingReplicationModel,
char referencedReplicationModel)
{
bool referencingIsReferenceTable =
(referencingReplicationModel == REPLICATION_MODEL_2PC);
bool referencedIsCitusLocalTable =
(referencedReplicationModel != REPLICATION_MODEL_2PC);
if (referencingIsReferenceTable && referencedIsCitusLocalTable)
{
/*
* We only support RESTRICT and NO ACTION behaviors for the
* foreign keys from reference tables to citus local tables.
* This is because, we can't cascade dml operations from citus
* local tables's coordinator placement to the remote placements
* of the reference table.
* Note that for the foreign keys from citus local tables to
* reference tables, we support all foreign key behaviors.
*/
if (!(BehaviorIsRestrictOrNoAction(fKeyConstraintForm->confdeltype) &&
BehaviorIsRestrictOrNoAction(fKeyConstraintForm->confupdtype)))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot define foreign key constraint, "
"foreign keys from reference tables to "
"citus local tables can only be defined "
"with NO ACTION or RESTRICT behaviors")));
}
}
}
/*
* EnsureSupportedFKeyOnDistKey errors out if given foreign key constraint form
* implies an unsupported ON DELETE/UPDATE behavior assuming the referencing column
* is the distribution column of the referencing distributed table.
*/
static void
EnsureSupportedFKeyOnDistKey(Form_pg_constraint fKeyConstraintForm)
{
/*
* ON DELETE SET NULL and ON DELETE SET DEFAULT is not supported. Because we do
* not want to set partition column to NULL or default value.
*/
if (fKeyConstraintForm->confdeltype == FKCONSTR_ACTION_SETNULL ||
fKeyConstraintForm->confdeltype == FKCONSTR_ACTION_SETDEFAULT)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL or SET DEFAULT is not supported "
"in ON DELETE operation when distribution "
"key is included in the foreign key constraint")));
}
/*
* ON UPDATE SET NULL, ON UPDATE SET DEFAULT and UPDATE CASCADE is not supported.
* Because we do not want to set partition column to NULL or default value. Also
* cascading update operation would require re-partitioning. Updating partition
* column value is not allowed anyway even outside of foreign key concept.
*/
if (fKeyConstraintForm->confupdtype == FKCONSTR_ACTION_SETNULL ||
fKeyConstraintForm->confupdtype == FKCONSTR_ACTION_SETDEFAULT ||
fKeyConstraintForm->confupdtype == FKCONSTR_ACTION_CASCADE)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("SET NULL, SET DEFAULT or CASCADE is not "
"supported in ON UPDATE operation when "
"distribution key included in the foreign "
"constraint.")));
}
}
/*
* EnsureReferencingTableNotReplicated takes referencingTableId for the
* referencing table of the foreign key and errors out if it's not a single
* replicated table.
*/
static void
EnsureReferencingTableNotReplicated(Oid referencingTableId)
{
bool referencingNotReplicated = true;
bool referencingIsCitus = IsCitusTable(referencingTableId);
if (referencingIsCitus)
{
/* ALTER TABLE command is applied over single replicated table */
referencingNotReplicated = SingleReplicatedTable(referencingTableId);
}
else
{
/* Creating single replicated table with foreign constraint */
referencingNotReplicated = !DistributedTableReplicationIsEnabled();
}
if (!referencingNotReplicated)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"),
errdetail("Citus Community Edition currently supports "
"foreign key constraints only for "
"\"citus.shard_replication_factor = 1\"."),
errhint("Please change \"citus.shard_replication_factor to "
"1\". To learn more about using foreign keys with "
"other replication factors, please contact us at "
"https://citusdata.com/about/contact_us.")));
}
}
/*
* ErrorOutForFKeyBetweenPostgresAndCitusLocalTable is a helper function to
* error out for foreign keys between postgres local tables and citus local
* tables.
*/
void
ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId)
{
char *localTableName = get_rel_name(localTableId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint as \"%s\" is "
"a postgres local table", localTableName),
errhint("first create a citus local table from the postgres "
"local table using SELECT create_citus_local_table('%s') "
"and execute the ALTER TABLE command to create the "
"foreign key to citus local table", localTableName)));
}
/*
* ForeignConstraintFindDistKeys finds the index of the given distribution columns
* in the given foreign key constraint and returns them in referencingAttrIndex
@ -504,8 +627,35 @@ get_relation_constraint_oid_compat(HeapTuple heapTuple)
/*
* HasForeignKeyToReferenceTable function returns true if any of the foreign
* key constraints on the relation with relationId references to a reference
* HasForeignKeyToCitusLocalTable returns true if any of the foreign key constraints
* on the relation with relationId references to a citus local table.
*/
bool
HasForeignKeyToCitusLocalTable(Oid relationId)
{
List *foreignKeyOidList = GetForeignKeyOidsToCitusLocalTables(relationId);
return list_length(foreignKeyOidList) > 0;
}
/*
* GetForeignKeyOidsToCitusLocalTables returns list of OIDs for the foreign key
* constraints on the given relationId that are referencing to citus local tables.
*/
static List *
GetForeignKeyOidsToCitusLocalTables(Oid relationId)
{
int flags = INCLUDE_REFERENCING_CONSTRAINTS;
List *foreignKeyOidList = GetForeignKeyOids(relationId, flags);
List *fKeyOidsToCitusLocalTables =
FilterFKeyOidListByReferencedTableType(foreignKeyOidList, CITUS_LOCAL_TABLE);
return fKeyOidsToCitusLocalTables;
}
/*
* HasForeignKeyToReferenceTable returns true if any of the foreign key
* constraints on the relation with relationId references to a reference
* table.
*/
bool
@ -518,40 +668,47 @@ HasForeignKeyToReferenceTable(Oid relationId)
/*
* GetForeignKeyOidsToReferenceTables function returns list of OIDs for the
* foreign key constraints on the given relationId that are referencing to
* reference tables.
* GetForeignKeyOidsToReferenceTables returns list of OIDs for the foreign key
* constraints on the given relationId that are referencing to reference tables.
*/
static List *
GetForeignKeyOidsToReferenceTables(Oid relationId)
{
int flags = INCLUDE_REFERENCING_CONSTRAINTS;
List *foreignKeyOids = GetForeignKeyOids(relationId, flags);
List *foreignKeyOidList = GetForeignKeyOids(relationId, flags);
List *fKeyOidsToReferenceTables =
FilterFKeyOidListByReferencedTableType(foreignKeyOidList, REFERENCE_TABLE);
return fKeyOidsToReferenceTables;
}
List *fkeyOidsToReferenceTables = NIL;
/*
* FilterFKeyOidListByReferencedTableType takes a list of foreign key OIDs and
* CitusTableType to filter the foreign key OIDs that CitusTableType matches
* referenced relation's type.
*/
static List *
FilterFKeyOidListByReferencedTableType(List *foreignKeyOidList,
CitusTableType citusTableType)
{
List *filteredFKeyOidList = NIL;
Oid foreignKeyOid = InvalidOid;
foreach_oid(foreignKeyOid, foreignKeyOids)
foreach_oid(foreignKeyOid, foreignKeyOidList)
{
HeapTuple heapTuple =
SearchSysCache1(CONSTROID, ObjectIdGetDatum(foreignKeyOid));
Assert(HeapTupleIsValid(heapTuple));
HeapTuple heapTuple = SearchSysCache1(CONSTROID, ObjectIdGetDatum(foreignKeyOid));
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
Oid referencedTableOid = constraintForm->confrelid;
if (IsCitusTableType(referencedTableOid, REFERENCE_TABLE))
if (IsCitusTableType(referencedTableOid, citusTableType))
{
fkeyOidsToReferenceTables = lappend_oid(fkeyOidsToReferenceTables,
foreignKeyOid);
filteredFKeyOidList = lappend_oid(filteredFKeyOidList, foreignKeyOid);
}
ReleaseSysCache(heapTuple);
}
return fkeyOidsToReferenceTables;
return filteredFKeyOidList;
}
@ -616,18 +773,31 @@ TableReferencing(Oid relationId)
/*
* ConstraintIsAForeignKey returns true if the given constraint name
* is a foreign key defined on the relation.
* ConstraintIsAForeignKey is a wrapper around GetForeignKeyOidByName that
* returns true if the given constraint name identifies a foreign key
* contraint defined on relation with relationId.
*/
bool
ConstraintIsAForeignKey(char *inputConstaintName, Oid relationId)
{
Oid foreignKeyId = GetForeignKeyOidByName(inputConstaintName, relationId);
return OidIsValid(foreignKeyId);
}
/*
* GetForeignKeyOidByName returns OID of the foreign key with name and defined
* on relation with relationId. If there is no such foreign key constraint, then
* this function returns InvalidOid.
*/
Oid
GetForeignKeyOidByName(char *inputConstaintName, Oid relationId)
{
int flags = INCLUDE_REFERENCING_CONSTRAINTS;
List *foreignKeyOids = GetForeignKeyOids(relationId, flags);
Oid foreignKeyOid = FindForeignKeyOidWithName(foreignKeyOids, inputConstaintName);
return OidIsValid(foreignKeyOid);
Oid foreignKeyId = FindForeignKeyOidWithName(foreignKeyOids, inputConstaintName);
return foreignKeyId;
}
@ -788,3 +958,27 @@ GetForeignKeyOids(Oid relationId, int flags)
return foreignKeyOids;
}
/*
* GetReferencedTableId returns OID of the referenced relation for the foreign
* key with foreignKeyId. If there is no such foreign key, then this function
* returns InvalidOid.
*/
Oid
GetReferencedTableId(Oid foreignKeyId)
{
HeapTuple heapTuple = SearchSysCache1(CONSTROID, ObjectIdGetDatum(foreignKeyId));
if (!HeapTupleIsValid(heapTuple))
{
/* no such foreign key */
return InvalidOid;
}
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
Oid referencedTableId = constraintForm->confrelid;
ReleaseSysCache(heapTuple);
return referencedTableId;
}

View File

@ -42,12 +42,32 @@
/* Local functions forward declarations for unsupported command checks */
static void PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement,
const char *queryString);
static void ErrorIfAlterTableDefinesFKeyFromPostgresToCitusLocalTable(
AlterTableStmt *alterTableStatement);
static List * GetAlterTableStmtFKeyConstraintList(AlterTableStmt *alterTableStatement);
static List * GetAlterTableCommandFKeyConstraintList(AlterTableCmd *command);
static bool AlterTableCommandTypeIsTrigger(AlterTableType alterTableType);
static void ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement);
static void ErrorIfCitusLocalTablePartitionCommand(AlterTableCmd *alterTableCmd,
Oid parentRelationId);
static Oid GetPartitionCommandChildRelationId(AlterTableCmd *alterTableCmd,
bool missingOk);
static List * InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
const char *commandString);
static bool AlterInvolvesPartitionColumn(AlterTableStmt *alterTableStatement,
AlterTableCmd *command);
static void ErrorIfUnsupportedAlterAddConstraintStmt(AlterTableStmt *alterTableStatement);
static List * CreateRightShardListForInterShardDDLTask(Oid rightRelationId,
Oid leftRelationId,
List *leftShardList);
static void SetInterShardDDLTaskPlacementList(Task *task,
ShardInterval *leftShardInterval,
ShardInterval *rightShardInterval);
static void SetInterShardDDLTaskRelationShardList(Task *task,
ShardInterval *leftShardInterval,
ShardInterval *rightShardInterval);
/*
* We need to run some of the commands sequentially if there is a foreign constraint
@ -130,53 +150,83 @@ PreprocessDropTableStmt(Node *node, const char *queryString)
/*
* PostprocessCreateTableStmtPartitionOf takes CreateStmt object as a parameter
* but it only processes CREATE TABLE ... PARTITION OF statements and it checks
* if user creates the table as a partition of a distributed table. In that case,
* it distributes partition as well. Since the table itself is a partition,
* CreateDistributedTable will attach it to its parent table automatically after
* distributing it.
* PostprocessCreateTableStmt takes CreateStmt object as a parameter and errors
* out if it creates a table with a foreign key that references to a citus local
* table if pg version is older than 13 (see comment in function).
*
* This function does nothing if the provided CreateStmt is not a CREATE TABLE ...
* PARTITION OF command.
* This function also processes CREATE TABLE ... PARTITION OF statements via
* PostprocessCreateTableStmtPartitionOf function.
*/
List *
void
PostprocessCreateTableStmt(CreateStmt *createStatement, const char *queryString)
{
#if PG_VERSION_NUM < PG_VERSION_13
/*
* Postgres processes foreign key constraints implied by CREATE TABLE
* commands by internally executing ALTER TABLE commands via standard
* process utility starting from PG13. Hence, we will already perform
* unsupported foreign key checks via PreprocessAlterTableStmt function
* in PG13. But for the older version, we need to do unsupported foreign
* key checks here.
*/
/*
* Relation must exist and it is already locked as standard process utility
* is already executed.
*/
bool missingOk = false;
Oid relationId = RangeVarGetRelid(createStatement->relation, NoLock, missingOk);
if (HasForeignKeyToCitusLocalTable(relationId))
{
ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(relationId);
}
#endif
if (createStatement->inhRelations != NIL && createStatement->partbound != NULL)
{
/* process CREATE TABLE ... PARTITION OF command */
PostprocessCreateTableStmtPartitionOf(createStatement, queryString);
}
}
/*
* PostprocessCreateTableStmtPartitionOf processes CREATE TABLE ... PARTITION OF
* statements and it checks if user creates the table as a partition of a distributed
* table. In that case, it distributes partition as well. Since the table itself is a
* partition, CreateDistributedTable will attach it to its parent table automatically
* after distributing it.
*/
static void
PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const
char *queryString)
{
if (createStatement->inhRelations != NIL && createStatement->partbound != NULL)
RangeVar *parentRelation = linitial(createStatement->inhRelations);
bool missingOk = false;
Oid parentRelationId = RangeVarGetRelid(parentRelation, NoLock, missingOk);
/* a partition can only inherit from single parent table */
Assert(list_length(createStatement->inhRelations) == 1);
Assert(parentRelationId != InvalidOid);
/*
* If a partition is being created and if its parent is a distributed
* table, we will distribute this table as well.
*/
if (IsCitusTable(parentRelationId))
{
RangeVar *parentRelation = linitial(createStatement->inhRelations);
bool parentMissingOk = false;
Oid parentRelationId = RangeVarGetRelid(parentRelation, NoLock,
parentMissingOk);
Oid relationId = RangeVarGetRelid(createStatement->relation, NoLock, missingOk);
Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId);
char parentDistributionMethod = DISTRIBUTE_BY_HASH;
char *parentRelationName = generate_qualified_relation_name(parentRelationId);
bool viaDeprecatedAPI = false;
/* a partition can only inherit from single parent table */
Assert(list_length(createStatement->inhRelations) == 1);
Assert(parentRelationId != InvalidOid);
/*
* If a partition is being created and if its parent is a distributed
* table, we will distribute this table as well.
*/
if (IsCitusTable(parentRelationId))
{
bool missingOk = false;
Oid relationId = RangeVarGetRelid(createStatement->relation, NoLock,
missingOk);
Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId);
char parentDistributionMethod = DISTRIBUTE_BY_HASH;
char *parentRelationName = generate_qualified_relation_name(parentRelationId);
bool viaDeprecatedAPI = false;
CreateDistributedTable(relationId, parentDistributionColumn,
parentDistributionMethod, parentRelationName,
viaDeprecatedAPI);
}
CreateDistributedTable(relationId, parentDistributionColumn,
parentDistributionMethod, parentRelationName,
viaDeprecatedAPI);
}
return NIL;
}
@ -325,6 +375,15 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
leftRelationId = IndexGetRelation(leftRelationId, missingOk);
}
/*
* Normally, we would do this check in ErrorIfUnsupportedForeignConstraintExists
* in post process step. However, we skip doing error checks in post process if
* this pre process returns NIL -and this method returns NIL if the left relation
* is a postgres table. So, we need to error out for foreign keys from postgres
* tables to citus local tables here.
*/
ErrorIfAlterTableDefinesFKeyFromPostgresToCitusLocalTable(alterTableStatement);
bool referencingIsLocalTable = !IsCitusTable(leftRelationId);
if (referencingIsLocalTable)
{
@ -375,7 +434,7 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
* only subcommand of ALTER TABLE. It was already checked in
* ErrorIfUnsupportedAlterTableStmt.
*/
Assert(list_length(commandList) <= 1);
Assert(list_length(commandList) == 1);
rightRelationId = RangeVarGetRelid(constraint->pktable, lockmode,
alterTableStatement->missing_ok);
@ -390,6 +449,22 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
constraint->skip_validation = true;
}
}
else if (alterTableType == AT_DropConstraint)
{
char *constraintName = command->name;
if (ConstraintIsAForeignKey(constraintName, leftRelationId))
{
/*
* We only support ALTER TABLE DROP CONSTRAINT ... FOREIGN KEY, if it is
* only subcommand of ALTER TABLE. It was already checked in
* ErrorIfUnsupportedAlterTableStmt.
*/
Assert(list_length(commandList) == 1);
Oid foreignKeyId = GetForeignKeyOidByName(constraintName, leftRelationId);
rightRelationId = GetReferencedTableId(foreignKeyId);
}
}
else if (alterTableType == AT_AddColumn)
{
/*
@ -456,6 +531,18 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
rightRelationId = RangeVarGetRelid(partitionCommand->name, NoLock, false);
}
else if (AlterTableCommandTypeIsTrigger(alterTableType))
{
/*
* We already error'ed out for ENABLE/DISABLE trigger commands for
* other citus table types in ErrorIfUnsupportedAlterTableStmt.
*/
Assert(IsCitusTableType(leftRelationId, CITUS_LOCAL_TABLE));
char *triggerName = command->name;
return CitusLocalTableTriggerCommandDDLJob(leftRelationId, triggerName,
alterTableCommand);
}
/*
* We check and set the execution mode only if we fall into either of first two
@ -502,6 +589,131 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
}
/*
* ErrorIfAlterTableDefinesFKeyFromPostgresToCitusLocalTable errors out if
* given ALTER TABLE statement defines foreign key from a postgres local table
* to a citus local table.
*/
static void
ErrorIfAlterTableDefinesFKeyFromPostgresToCitusLocalTable(
AlterTableStmt *alterTableStatement)
{
List *commandList = alterTableStatement->cmds;
LOCKMODE lockmode = AlterTableGetLockLevel(commandList);
Oid leftRelationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (IsCitusTable(leftRelationId))
{
/* left relation is not a postgres local table, */
return;
}
List *alterTableFKeyConstraints =
GetAlterTableStmtFKeyConstraintList(alterTableStatement);
Constraint *constraint = NULL;
foreach_ptr(constraint, alterTableFKeyConstraints)
{
Oid rightRelationId = RangeVarGetRelid(constraint->pktable, lockmode,
alterTableStatement->missing_ok);
if (IsCitusTableType(rightRelationId, CITUS_LOCAL_TABLE))
{
ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(leftRelationId);
}
}
}
/*
* GetAlterTableStmtFKeyConstraintList returns a list of Constraint objects for
* the foreign keys that given ALTER TABLE statement defines.
*/
static List *
GetAlterTableStmtFKeyConstraintList(AlterTableStmt *alterTableStatement)
{
List *alterTableFKeyConstraintList = NIL;
List *commandList = alterTableStatement->cmds;
AlterTableCmd *command = NULL;
foreach_ptr(command, commandList)
{
List *commandFKeyConstraintList = GetAlterTableCommandFKeyConstraintList(command);
alterTableFKeyConstraintList = list_concat(alterTableFKeyConstraintList,
commandFKeyConstraintList);
}
return alterTableFKeyConstraintList;
}
/*
* GetAlterTableCommandFKeyConstraintList returns a list of Constraint objects
* for the foreign keys that given ALTER TABLE subcommand defines. Note that
* this is only possible if it is an:
* - ADD CONSTRAINT subcommand (explicitly defines) or,
* - ADD COLUMN subcommand (implicitly defines by adding a new column that
* references to another table.
*/
static List *
GetAlterTableCommandFKeyConstraintList(AlterTableCmd *command)
{
List *fkeyConstraintList = NIL;
AlterTableType alterTableType = command->subtype;
if (alterTableType == AT_AddConstraint)
{
Constraint *constraint = (Constraint *) command->def;
if (constraint->contype == CONSTR_FOREIGN)
{
fkeyConstraintList = lappend(fkeyConstraintList, constraint);
}
}
else if (alterTableType == AT_AddColumn)
{
ColumnDef *columnDefinition = (ColumnDef *) command->def;
List *columnConstraints = columnDefinition->constraints;
Constraint *constraint = NULL;
foreach_ptr(constraint, columnConstraints)
{
if (constraint->contype == CONSTR_FOREIGN)
{
fkeyConstraintList = lappend(fkeyConstraintList, constraint);
}
}
}
return fkeyConstraintList;
}
/*
* AlterTableCommandTypeIsTrigger returns true if given alter table command type
* is identifies an ALTER TABLE .. TRIGGER .. command.
*/
static bool
AlterTableCommandTypeIsTrigger(AlterTableType alterTableType)
{
switch (alterTableType)
{
case AT_EnableTrig:
case AT_EnableAlwaysTrig:
case AT_EnableReplicaTrig:
case AT_EnableTrigUser:
case AT_DisableTrig:
case AT_DisableTrigUser:
case AT_EnableTrigAll:
case AT_DisableTrigAll:
{
return true;
}
default:
return false;
}
}
/*
* PreprocessAlterTableStmt issues a warning.
* ALTER TABLE ALL IN TABLESPACE statements have their node type as
@ -868,6 +1080,7 @@ ErrorUnsupportedAlterTableAddColumn(Oid relationId, AlterTableCmd *command,
*/
void
ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
char referencingReplicationModel,
Var *distributionColumn, uint32 colocationId)
{
/*
@ -878,6 +1091,7 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
* and if they are OK, we do not error out for other types of constraints.
*/
ErrorIfUnsupportedForeignConstraintExists(relation, distributionMethod,
referencingReplicationModel,
distributionColumn,
colocationId);
@ -987,12 +1201,17 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
* ALTER TABLE REPLICA IDENTITY
* ALTER TABLE SET ()
* ALTER TABLE RESET ()
* ALTER TABLE ENABLE/DISABLE TRIGGER (only for citus local tables)
*/
static void
ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
{
/* error out if any of the subcommands are unsupported */
List *commandList = alterTableStatement->cmds;
LOCKMODE lockmode = AlterTableGetLockLevel(commandList);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
/* error out if any of the subcommands are unsupported */
AlterTableCmd *command = NULL;
foreach_ptr(command, commandList)
{
@ -1074,12 +1293,10 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
case AT_AttachPartition:
{
Oid relationId = AlterTableLookupRelation(alterTableStatement,
NoLock);
PartitionCmd *partitionCommand = (PartitionCmd *) command->def;
bool missingOK = false;
Oid partitionRelationId = RangeVarGetRelid(partitionCommand->name,
NoLock, missingOK);
lockmode, missingOK);
/* we only allow partitioning commands if they are only subcommand */
if (commandList->length > 1)
@ -1091,6 +1308,8 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
"separately.")));
}
ErrorIfCitusLocalTablePartitionCommand(command, relationId);
if (IsCitusTable(partitionRelationId) &&
!TablesColocated(relationId, partitionRelationId))
{
@ -1115,14 +1334,13 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
"separately.")));
}
ErrorIfCitusLocalTablePartitionCommand(command, relationId);
break;
}
case AT_DropConstraint:
{
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId))
{
return;
@ -1136,17 +1354,41 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
break;
}
case AT_SetNotNull:
case AT_EnableTrig:
case AT_EnableAlwaysTrig:
case AT_EnableReplicaTrig:
case AT_EnableTrigUser:
case AT_DisableTrig:
case AT_DisableTrigUser:
case AT_EnableTrigAll:
case AT_DisableTrigAll:
{
/*
* Postgres already does not allow executing ALTER TABLE TRIGGER
* commands with other subcommands, but let's be on the safe side.
*/
if (commandList->length > 1)
{
ereport(ERROR, (errcode(ERRCODE_INTERNAL_ERROR),
errmsg("cannot execute ENABLE/DISABLE TRIGGER "
"command with other subcommands"),
errhint("You can issue each subcommand separately")));
}
ErrorOutForTriggerIfNotCitusLocalTable(relationId);
break;
}
case AT_SetNotNull:
case AT_ReplicaIdentity:
case AT_ValidateConstraint:
{
/*
* We will not perform any special check for ALTER TABLE DROP CONSTRAINT
* , ALTER TABLE .. ALTER COLUMN .. SET NOT NULL and ALTER TABLE ENABLE/
* DISABLE TRIGGER ALL, ALTER TABLE .. REPLICA IDENTITY .., ALTER TABLE
* .. VALIDATE CONSTRAINT ..
* We will not perform any special check for:
* ALTER TABLE .. ALTER COLUMN .. SET NOT NULL
* ALTER TABLE .. REPLICA IDENTITY ..
* ALTER TABLE .. VALIDATE CONSTRAINT ..
*/
break;
}
@ -1175,6 +1417,52 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
}
/*
* ErrorIfCitusLocalTablePartitionCommand errors out if given alter table subcommand is
* an ALTER TABLE ATTACH / DETACH PARTITION command run for a citus local table.
*/
static void
ErrorIfCitusLocalTablePartitionCommand(AlterTableCmd *alterTableCmd, Oid parentRelationId)
{
AlterTableType alterTableType = alterTableCmd->subtype;
if (alterTableType != AT_AttachPartition && alterTableType != AT_DetachPartition)
{
return;
}
bool missingOK = false;
Oid childRelationId = GetPartitionCommandChildRelationId(alterTableCmd, missingOK);
if (!IsCitusTableType(parentRelationId, CITUS_LOCAL_TABLE) &&
!IsCitusTableType(childRelationId, CITUS_LOCAL_TABLE))
{
return;
}
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot execute ATTACH/DETACH PARTITION command as "
"citus local tables cannot be involved in partition "
"relationships with other tables")));
}
/*
* GetPartitionCommandChildRelationId returns child relationId for given
* ALTER TABLE ATTACH / DETACH PARTITION subcommand.
*/
static Oid
GetPartitionCommandChildRelationId(AlterTableCmd *alterTableCmd, bool missingOk)
{
AlterTableType alterTableType PG_USED_FOR_ASSERTS_ONLY = alterTableCmd->subtype;
Assert(alterTableType == AT_AttachPartition || alterTableType == AT_DetachPartition);
PartitionCmd *partitionCommand = (PartitionCmd *) alterTableCmd->def;
RangeVar *childRelationRangeVar = partitionCommand->name;
Oid childRelationId = RangeVarGetRelid(childRelationRangeVar, AccessExclusiveLock,
missingOk);
return childRelationId;
}
/*
* SetupExecutionModeForAlterTable is the function that is responsible
* for two things for practical purpose for not doing the same checks
@ -1286,7 +1574,8 @@ SetupExecutionModeForAlterTable(Oid relationId, AlterTableCmd *command)
* the distributed tables, thus contradicting our purpose of using
* sequential mode.
*/
if (executeSequentially && !IsCitusTableType(relationId, REFERENCE_TABLE) &&
if (executeSequentially &&
!IsCitusTableType(relationId, CITUS_TABLE_WITH_NO_DIST_KEY) &&
ParallelQueryExecutedInTransaction())
{
char *relationName = get_rel_name(relationId);
@ -1320,65 +1609,40 @@ static List *
InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
const char *commandString)
{
List *taskList = NIL;
List *leftShardList = LoadShardIntervalList(leftRelationId);
ListCell *leftShardCell = NULL;
List *rightShardList = CreateRightShardListForInterShardDDLTask(rightRelationId,
leftRelationId,
leftShardList);
/* lock metadata before getting placement lists */
LockShardListMetadata(leftShardList, ShareLock);
uint64 jobId = INVALID_JOB_ID;
int taskId = 1;
Oid leftSchemaId = get_rel_namespace(leftRelationId);
char *leftSchemaName = get_namespace_name(leftSchemaId);
char *escapedLeftSchemaName = quote_literal_cstr(leftSchemaName);
List *rightShardList = LoadShardIntervalList(rightRelationId);
ListCell *rightShardCell = NULL;
Oid rightSchemaId = get_rel_namespace(rightRelationId);
char *rightSchemaName = get_namespace_name(rightSchemaId);
char *escapedRightSchemaName = quote_literal_cstr(rightSchemaName);
char *escapedCommandString = quote_literal_cstr(commandString);
uint64 jobId = INVALID_JOB_ID;
int taskId = 1;
/*
* If the rightPartitionMethod is a reference table, we need to make sure
* that the tasks are created in a way that the right shard stays the same
* since we only have one placement per worker. This hack is first implemented
* for foreign constraint support from distributed tables to reference tables.
*/
if (IsCitusTableType(rightRelationId, REFERENCE_TABLE))
{
int rightShardCount = list_length(rightShardList);
int leftShardCount = list_length(leftShardList);
Assert(rightShardCount == 1);
ShardInterval *rightShardInterval = (ShardInterval *) linitial(rightShardList);
for (int shardCounter = rightShardCount; shardCounter < leftShardCount;
shardCounter++)
{
rightShardList = lappend(rightShardList, rightShardInterval);
}
}
/* lock metadata before getting placement lists */
LockShardListMetadata(leftShardList, ShareLock);
List *taskList = NIL;
ListCell *leftShardCell = NULL;
ListCell *rightShardCell = NULL;
forboth(leftShardCell, leftShardList, rightShardCell, rightShardList)
{
ShardInterval *leftShardInterval = (ShardInterval *) lfirst(leftShardCell);
uint64 leftShardId = leftShardInterval->shardId;
StringInfo applyCommand = makeStringInfo();
RelationShard *leftRelationShard = CitusMakeNode(RelationShard);
RelationShard *rightRelationShard = CitusMakeNode(RelationShard);
ShardInterval *rightShardInterval = (ShardInterval *) lfirst(rightShardCell);
uint64 leftShardId = leftShardInterval->shardId;
uint64 rightShardId = rightShardInterval->shardId;
leftRelationShard->relationId = leftRelationId;
leftRelationShard->shardId = leftShardId;
rightRelationShard->relationId = rightRelationId;
rightRelationShard->shardId = rightShardId;
StringInfo applyCommand = makeStringInfo();
appendStringInfo(applyCommand, WORKER_APPLY_INTER_SHARD_DDL_COMMAND,
leftShardId, escapedLeftSchemaName, rightShardId,
escapedRightSchemaName, escapedCommandString);
@ -1391,8 +1655,9 @@ InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
task->dependentTaskList = NULL;
task->replicationModel = REPLICATION_MODEL_INVALID;
task->anchorShardId = leftShardId;
task->taskPlacementList = ActiveShardPlacementList(leftShardId);
task->relationShardList = list_make2(leftRelationShard, rightRelationShard);
SetInterShardDDLTaskPlacementList(task, leftShardInterval, rightShardInterval);
SetInterShardDDLTaskRelationShardList(task, leftShardInterval,
rightShardInterval);
taskList = lappend(taskList, task);
}
@ -1401,6 +1666,87 @@ InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
}
/*
* CreateRightShardListForInterShardDDLTask is a helper function that creates
* shard list for the right relation for InterShardDDLTaskList.
*/
static List *
CreateRightShardListForInterShardDDLTask(Oid rightRelationId, Oid leftRelationId,
List *leftShardList)
{
List *rightShardList = LoadShardIntervalList(rightRelationId);
if (!IsCitusTableType(leftRelationId, CITUS_LOCAL_TABLE) &&
IsCitusTableType(rightRelationId, REFERENCE_TABLE))
{
/*
* If the right relation is a reference table and left relation is not
* a citus local table, we need to make sure that the tasks are created
* in a way that the right shard stays the same since we only have one
* placement per worker.
* If left relation is a citus local table, then we don't need to populate
* reference table shards as we will execute ADD/DROP constraint command
* only for coordinator placement of reference table.
*/
ShardInterval *rightShard = (ShardInterval *) linitial(rightShardList);
int leftShardCount = list_length(leftShardList);
rightShardList = GenerateListFromElement(rightShard, leftShardCount);
}
return rightShardList;
}
/*
* SetInterShardDDLTaskPlacementList sets taskPlacementList field of given
* inter-shard DDL task according to passed shard interval arguments.
*/
static void
SetInterShardDDLTaskPlacementList(Task *task, ShardInterval *leftShardInterval,
ShardInterval *rightShardInterval)
{
Oid leftRelationId = leftShardInterval->relationId;
Oid rightRelationId = rightShardInterval->relationId;
if (IsCitusTableType(leftRelationId, REFERENCE_TABLE) &&
IsCitusTableType(rightRelationId, CITUS_LOCAL_TABLE))
{
/*
* If we are defining/dropping a foreign key from a reference table
* to a citus local table, then we will execute ADD/DROP constraint
* command only for coordinator placement of reference table.
*/
task->taskPlacementList = GroupShardPlacementsForTableOnGroup(leftRelationId,
COORDINATOR_GROUP_ID);
}
else
{
uint64 leftShardId = leftShardInterval->shardId;
task->taskPlacementList = ActiveShardPlacementList(leftShardId);
}
}
/*
* SetInterShardDDLTaskRelationShardList sets relationShardList field of given
* inter-shard DDL task according to passed shard interval arguments.
*/
static void
SetInterShardDDLTaskRelationShardList(Task *task, ShardInterval *leftShardInterval,
ShardInterval *rightShardInterval)
{
RelationShard *leftRelationShard = CitusMakeNode(RelationShard);
leftRelationShard->relationId = leftShardInterval->relationId;
leftRelationShard->shardId = leftShardInterval->shardId;
RelationShard *rightRelationShard = CitusMakeNode(RelationShard);
rightRelationShard->relationId = rightShardInterval->relationId;
rightRelationShard->shardId = rightShardInterval->shardId;
task->relationShardList = list_make2(leftRelationShard, rightRelationShard);
}
/*
* AlterInvolvesPartitionColumn checks if the given alter table command
* involves relation's partition column.
@ -1450,12 +1796,14 @@ ErrorIfUnsupportedAlterAddConstraintStmt(AlterTableStmt *alterTableStatement)
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
char distributionMethod = PartitionMethod(relationId);
char referencingReplicationModel = TableReplicationModel(relationId);
Var *distributionColumn = DistPartitionKey(relationId);
uint32 colocationId = TableColocationId(relationId);
Relation relation = relation_open(relationId, ExclusiveLock);
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn,
colocationId);
ErrorIfUnsupportedConstraint(relation, distributionMethod,
referencingReplicationModel,
distributionColumn, colocationId);
relation_close(relation, NoLock);
}

View File

@ -22,16 +22,40 @@
#include "catalog/indexing.h"
#include "catalog/namespace.h"
#include "catalog/pg_trigger.h"
#include "commands/trigger.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/deparser.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/namespace_utils.h"
#include "distributed/shard_utils.h"
#include "distributed/worker_protocol.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
/* appropriate lock modes for the owner relation according to postgres */
#define CREATE_TRIGGER_LOCK_MODE ShareRowExclusiveLock
#define ALTER_TRIGGER_LOCK_MODE AccessExclusiveLock
#define DROP_TRIGGER_LOCK_MODE AccessExclusiveLock
/* local function forward declarations */
static bool IsCreateCitusTruncateTriggerStmt(CreateTrigStmt *createTriggerStmt);
static Value * GetAlterTriggerDependsTriggerNameValue(AlterObjectDependsStmt *
alterTriggerDependsStmt);
static void ErrorIfUnsupportedDropTriggerCommand(DropStmt *dropTriggerStmt);
static RangeVar * GetDropTriggerStmtRelation(DropStmt *dropTriggerStmt);
static void ExtractDropStmtTriggerAndRelationName(DropStmt *dropTriggerStmt,
char **triggerName,
char **relationName);
static void ErrorIfDropStmtDropsMultipleTriggers(DropStmt *dropTriggerStmt);
static int16 GetTriggerTypeById(Oid triggerId);
/*
* GetExplicitTriggerCommandList returns the list of DDL commands to create
* triggers that are explicitly created for the table with relationId. See
@ -63,34 +87,12 @@ GetExplicitTriggerCommandList(Oid relationId)
/*
* GetExplicitTriggerNameList returns a list of trigger names that are explicitly
* created for the table with relationId. See comment of GetExplicitTriggerIdList
* function.
* GetTriggerTupleById returns copy of the heap tuple from pg_trigger for
* the trigger with triggerId. If no such trigger exists, this function returns
* NULL or errors out depending on missingOk.
*/
List *
GetExplicitTriggerNameList(Oid relationId)
{
List *triggerNameList = NIL;
List *triggerIdList = GetExplicitTriggerIdList(relationId);
Oid triggerId = InvalidOid;
foreach_oid(triggerId, triggerIdList)
{
char *triggerHame = GetTriggerNameById(triggerId);
triggerNameList = lappend(triggerNameList, triggerHame);
}
return triggerNameList;
}
/*
* GetTriggerNameById returns name of the trigger identified by triggerId if it
* exists. Otherwise, returns NULL.
*/
char *
GetTriggerNameById(Oid triggerId)
HeapTuple
GetTriggerTupleById(Oid triggerId, bool missingOk)
{
Relation pgTrigger = table_open(TriggerRelationId, AccessShareLock);
@ -111,19 +113,24 @@ GetTriggerNameById(Oid triggerId)
useIndex, NULL, scanKeyCount,
scanKey);
char *triggerName = NULL;
HeapTuple targetHeapTuple = NULL;
HeapTuple heapTuple = systable_getnext(scanDescriptor);
if (HeapTupleIsValid(heapTuple))
{
Form_pg_trigger triggerForm = (Form_pg_trigger) GETSTRUCT(heapTuple);
triggerName = pstrdup(NameStr(triggerForm->tgname));
targetHeapTuple = heap_copytuple(heapTuple);
}
systable_endscan(scanDescriptor);
table_close(pgTrigger, NoLock);
return triggerName;
if (targetHeapTuple == NULL && missingOk == false)
{
ereport(ERROR, (errmsg("could not find heap tuple for trigger with "
"OID %d", triggerId)));
}
return targetHeapTuple;
}
@ -203,45 +210,531 @@ get_relation_trigger_oid_compat(HeapTuple heapTuple)
/*
* ErrorIfUnsupportedCreateTriggerCommand errors out for the CREATE TRIGGER
* command that is run for a citus table if it is not citus_truncate_trigger.
* PostprocessCreateTriggerStmt is called after a CREATE TRIGGER command has
* been executed by standard process utility. This function errors out for
* unsupported commands or creates ddl job for supported CREATE TRIGGER commands.
*/
List *
PostprocessCreateTriggerStmt(Node *node, const char *queryString)
{
CreateTrigStmt *createTriggerStmt = castNode(CreateTrigStmt, node);
if (IsCreateCitusTruncateTriggerStmt(createTriggerStmt))
{
return NIL;
}
RangeVar *relation = createTriggerStmt->relation;
bool missingOk = false;
Oid relationId = RangeVarGetRelid(relation, CREATE_TRIGGER_LOCK_MODE, missingOk);
if (!IsCitusTable(relationId))
{
return NIL;
}
EnsureCoordinator();
ErrorOutForTriggerIfNotCitusLocalTable(relationId);
if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
ObjectAddress objectAddress = GetObjectAddressFromParseTree(node, missingOk);
EnsureDependenciesExistOnAllNodes(&objectAddress);
char *triggerName = createTriggerStmt->trigname;
return CitusLocalTableTriggerCommandDDLJob(relationId, triggerName,
queryString);
}
return NIL;
}
/*
* CreateTriggerStmtObjectAddress finds the ObjectAddress for the trigger that
* is created by given CreateTriggerStmt. If missingOk is false and if trigger
* does not exist, then it errors out.
*
* Note that internal triggers that are created implicitly by postgres for
* foreign key validation already wouldn't be executed via process utility,
* hence there is no need to check that case here.
* Never returns NULL, but the objid in the address can be invalid if missingOk
* was set to true.
*/
ObjectAddress
CreateTriggerStmtObjectAddress(Node *node, bool missingOk)
{
CreateTrigStmt *createTriggerStmt = castNode(CreateTrigStmt, node);
RangeVar *relation = createTriggerStmt->relation;
Oid relationId = RangeVarGetRelid(relation, CREATE_TRIGGER_LOCK_MODE, missingOk);
char *triggerName = createTriggerStmt->trigname;
Oid triggerId = get_trigger_oid(relationId, triggerName, missingOk);
if (triggerId == InvalidOid && missingOk == false)
{
char *relationName = get_rel_name(relationId);
ereport(ERROR, (errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("trigger \"%s\" on relation \"%s\" does not exist",
triggerName, relationName)));
}
ObjectAddress address = { 0 };
ObjectAddressSet(address, TriggerRelationId, triggerId);
return address;
}
/*
* IsCreateCitusTruncateTriggerStmt returns true if given createTriggerStmt
* creates citus_truncate_trigger.
*/
static bool
IsCreateCitusTruncateTriggerStmt(CreateTrigStmt *createTriggerStmt)
{
List *functionNameList = createTriggerStmt->funcname;
RangeVar *functionRangeVar = makeRangeVarFromNameList(functionNameList);
char *functionName = functionRangeVar->relname;
if (strncmp(functionName, CITUS_TRUNCATE_TRIGGER_NAME, NAMEDATALEN) == 0)
{
return true;
}
return false;
}
/*
* CreateTriggerEventExtendNames extends relation name and trigger name with
* shardId, and sets schema name in given CreateTrigStmt.
*/
void
ErrorIfUnsupportedCreateTriggerCommand(CreateTrigStmt *createTriggerStmt)
CreateTriggerEventExtendNames(CreateTrigStmt *createTriggerStmt, char *schemaName,
uint64 shardId)
{
RangeVar *triggerRelation = createTriggerStmt->relation;
RangeVar *relation = createTriggerStmt->relation;
char **relationName = &(relation->relname);
AppendShardIdToName(relationName, shardId);
char **triggerName = &(createTriggerStmt->trigname);
AppendShardIdToName(triggerName, shardId);
char **relationSchemaName = &(relation->schemaname);
SetSchemaNameIfNotExist(relationSchemaName, schemaName);
}
/*
* PostprocessAlterTriggerRenameStmt is called after a ALTER TRIGGER RENAME
* command has been executed by standard process utility. This function errors
* out for unsupported commands or creates ddl job for supported ALTER TRIGGER
* RENAME commands.
*/
List *
PostprocessAlterTriggerRenameStmt(Node *node, const char *queryString)
{
RenameStmt *renameTriggerStmt = castNode(RenameStmt, node);
Assert(renameTriggerStmt->renameType == OBJECT_TRIGGER);
RangeVar *relation = renameTriggerStmt->relation;
bool missingOk = false;
Oid relationId = RangeVarGetRelid(relation, ALTER_TRIGGER_LOCK_MODE, missingOk);
if (!IsCitusTable(relationId))
{
return NIL;
}
EnsureCoordinator();
ErrorOutForTriggerIfNotCitusLocalTable(relationId);
if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
/* use newname as standard process utility already renamed it */
char *triggerName = renameTriggerStmt->newname;
return CitusLocalTableTriggerCommandDDLJob(relationId, triggerName,
queryString);
}
return NIL;
}
/*
* AlterTriggerRenameEventExtendNames extends relation name, old and new trigger
* name with shardId, and sets schema name in given RenameStmt.
*/
void
AlterTriggerRenameEventExtendNames(RenameStmt *renameTriggerStmt, char *schemaName,
uint64 shardId)
{
Assert(renameTriggerStmt->renameType == OBJECT_TRIGGER);
RangeVar *relation = renameTriggerStmt->relation;
char **relationName = &(relation->relname);
AppendShardIdToName(relationName, shardId);
char **triggerOldName = &(renameTriggerStmt->subname);
AppendShardIdToName(triggerOldName, shardId);
char **triggerNewName = &(renameTriggerStmt->newname);
AppendShardIdToName(triggerNewName, shardId);
char **relationSchemaName = &(relation->schemaname);
SetSchemaNameIfNotExist(relationSchemaName, schemaName);
}
/*
* PostprocessAlterTriggerDependsStmt is called after a ALTER TRIGGER DEPENDS ON
* command has been executed by standard process utility. This function errors out
* for unsupported commands or creates ddl job for supported ALTER TRIGGER DEPENDS
* ON commands.
*/
List *
PostprocessAlterTriggerDependsStmt(Node *node, const char *queryString)
{
AlterObjectDependsStmt *alterTriggerDependsStmt =
castNode(AlterObjectDependsStmt, node);
Assert(alterTriggerDependsStmt->objectType == OBJECT_TRIGGER);
RangeVar *relation = alterTriggerDependsStmt->relation;
bool missingOk = false;
Oid relationId = RangeVarGetRelid(relation, ALTER_TRIGGER_LOCK_MODE, missingOk);
if (!IsCitusTable(relationId))
{
return NIL;
}
EnsureCoordinator();
ErrorOutForTriggerIfNotCitusLocalTable(relationId);
if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
Value *triggerNameValue =
GetAlterTriggerDependsTriggerNameValue(alterTriggerDependsStmt);
return CitusLocalTableTriggerCommandDDLJob(relationId, strVal(triggerNameValue),
queryString);
}
return NIL;
}
/*
* AlterTriggerDependsEventExtendNames extends relation name and trigger name
* with shardId, and sets schema name in given AlterObjectDependsStmt.
*/
void
AlterTriggerDependsEventExtendNames(AlterObjectDependsStmt *alterTriggerDependsStmt,
char *schemaName, uint64 shardId)
{
Assert(alterTriggerDependsStmt->objectType == OBJECT_TRIGGER);
RangeVar *relation = alterTriggerDependsStmt->relation;
char **relationName = &(relation->relname);
AppendShardIdToName(relationName, shardId);
Value *triggerNameValue =
GetAlterTriggerDependsTriggerNameValue(alterTriggerDependsStmt);
AppendShardIdToName(&strVal(triggerNameValue), shardId);
char **relationSchemaName = &(relation->schemaname);
SetSchemaNameIfNotExist(relationSchemaName, schemaName);
}
/*
* GetAlterTriggerDependsTriggerName returns Value object for the trigger
* name that given AlterObjectDependsStmt is executed for.
*/
static Value *
GetAlterTriggerDependsTriggerNameValue(AlterObjectDependsStmt *alterTriggerDependsStmt)
{
List *triggerObjectNameList = (List *) alterTriggerDependsStmt->object;
/*
* Before standard process utility, we only have trigger name in "object"
* list. However, standard process utility prepends that list with the
* relationNameList retrieved from AlterObjectDependsStmt->RangeVar and
* we call this method after standard process utility. So, for the further
* usages, it is certain that the last element in "object" list will always
* be the name of the trigger in either before or after standard process
* utility.
*/
Value *triggerNameValue = llast(triggerObjectNameList);
return triggerNameValue;
}
/*
* PreprocessDropTriggerStmt is called before a DROP TRIGGER command has been
* executed by standard process utility. This function errors out for
* unsupported commands or creates ddl job for supported DROP TRIGGER commands.
* The reason we process drop trigger commands before standard process utility
* (unlike the other type of trigger commands) is that we act according to trigger
* type in CitusLocalTableTriggerCommandDDLJob but trigger wouldn't exist after
* standard process utility.
*/
List *
PreprocessDropTriggerStmt(Node *node, const char *queryString)
{
DropStmt *dropTriggerStmt = castNode(DropStmt, node);
Assert(dropTriggerStmt->removeType == OBJECT_TRIGGER);
RangeVar *relation = GetDropTriggerStmtRelation(dropTriggerStmt);
bool missingOk = true;
Oid relationId = RangeVarGetRelid(triggerRelation, AccessShareLock, missingOk);
Oid relationId = RangeVarGetRelid(relation, DROP_TRIGGER_LOCK_MODE, missingOk);
if (!OidIsValid(relationId))
{
/*
* standard_ProcessUtility would already error out if the given table
* does not exist
*/
return;
/* let standard process utility to error out */
return NIL;
}
if (!IsCitusTable(relationId))
{
return NIL;
}
ErrorIfUnsupportedDropTriggerCommand(dropTriggerStmt);
if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
char *triggerName = NULL;
ExtractDropStmtTriggerAndRelationName(dropTriggerStmt, &triggerName, NULL);
return CitusLocalTableTriggerCommandDDLJob(relationId, triggerName,
queryString);
}
return NIL;
}
/*
* ErrorIfUnsupportedDropTriggerCommand errors out for unsupported
* "DROP TRIGGER triggerName ON relationName" commands.
*/
static void
ErrorIfUnsupportedDropTriggerCommand(DropStmt *dropTriggerStmt)
{
RangeVar *relation = GetDropTriggerStmtRelation(dropTriggerStmt);
bool missingOk = false;
Oid relationId = RangeVarGetRelid(relation, DROP_TRIGGER_LOCK_MODE, missingOk);
if (!IsCitusTable(relationId))
{
return;
}
char *functionName = makeRangeVarFromNameList(createTriggerStmt->funcname)->relname;
if (strncmp(functionName, CITUS_TRUNCATE_TRIGGER_NAME, NAMEDATALEN) == 0)
EnsureCoordinator();
ErrorOutForTriggerIfNotCitusLocalTable(relationId);
}
/*
* ErrorOutForTriggerIfNotCitusLocalTable is a helper function to error
* out for unsupported trigger commands depending on the citus table type.
*/
void
ErrorOutForTriggerIfNotCitusLocalTable(Oid relationId)
{
if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
return;
}
char *relationName = triggerRelation->relname;
Assert(relationName != NULL);
ereport(ERROR, (errmsg("cannot create trigger on relation \"%s\" because it "
"is either a distributed table or a reference table",
relationName)));
ereport(ERROR, (errmsg("triggers are only supported for citus local tables")));
}
/*
* GetDropTriggerStmtRelation takes a DropStmt for a trigger object and returns
* RangeVar for the relation that owns the trigger.
*/
static RangeVar *
GetDropTriggerStmtRelation(DropStmt *dropTriggerStmt)
{
Assert(dropTriggerStmt->removeType == OBJECT_TRIGGER);
ErrorIfDropStmtDropsMultipleTriggers(dropTriggerStmt);
List *targetObjectList = dropTriggerStmt->objects;
List *triggerObjectNameList = linitial(targetObjectList);
/*
* The name list that identifies the trigger to be dropped looks like:
* [catalogName, schemaName, relationName, triggerName], where, the first
* two elements are optional. We should take all elements except the
* triggerName to create the range var object that defines the owner
* relation.
*/
int relationNameListLength = list_length(triggerObjectNameList) - 1;
List *relationNameList = list_truncate(list_copy(triggerObjectNameList),
relationNameListLength);
return makeRangeVarFromNameList(relationNameList);
}
/*
* DropTriggerEventExtendNames extends relation name and trigger name with
* shardId, and sets schema name in given DropStmt by recreating "objects"
* list.
*/
void
DropTriggerEventExtendNames(DropStmt *dropTriggerStmt, char *schemaName, uint64 shardId)
{
Assert(dropTriggerStmt->removeType == OBJECT_TRIGGER);
char *triggerName = NULL;
char *relationName = NULL;
ExtractDropStmtTriggerAndRelationName(dropTriggerStmt, &triggerName, &relationName);
AppendShardIdToName(&triggerName, shardId);
Value *triggerNameValue = makeString(triggerName);
AppendShardIdToName(&relationName, shardId);
Value *relationNameValue = makeString(relationName);
Value *schemaNameValue = makeString(pstrdup(schemaName));
List *shardTriggerNameList =
list_make3(schemaNameValue, relationNameValue, triggerNameValue);
dropTriggerStmt->objects = list_make1(shardTriggerNameList);
}
/*
* ExtractDropStmtTriggerAndRelationName extracts triggerName and relationName
* from given dropTriggerStmt if arguments are passed as non-null pointers.
*/
static void
ExtractDropStmtTriggerAndRelationName(DropStmt *dropTriggerStmt, char **triggerName,
char **relationName)
{
ErrorIfDropStmtDropsMultipleTriggers(dropTriggerStmt);
List *targetObjectList = dropTriggerStmt->objects;
List *triggerObjectNameList = linitial(targetObjectList);
int objectNameListLength = list_length(triggerObjectNameList);
if (triggerName != NULL)
{
int triggerNameindex = objectNameListLength - 1;
*triggerName = strVal(safe_list_nth(triggerObjectNameList, triggerNameindex));
}
if (relationName != NULL)
{
int relationNameIndex = objectNameListLength - 2;
*relationName = strVal(safe_list_nth(triggerObjectNameList, relationNameIndex));
}
}
/*
* ErrorIfDropStmtDropsMultipleTriggers errors out if given drop trigger
* command drops more than one trigger. Actually, this can't be the case
* as postgres doesn't support dropping multiple triggers, but we should
* be on the safe side.
*/
static void
ErrorIfDropStmtDropsMultipleTriggers(DropStmt *dropTriggerStmt)
{
List *targetObjectList = dropTriggerStmt->objects;
if (list_length(targetObjectList) > 1)
{
ereport(ERROR, (errcode(ERRCODE_INTERNAL_ERROR),
errmsg("cannot execute DROP TRIGGER command for multiple "
"triggers")));
}
}
/*
* CitusLocalTableTriggerCommandDDLJob creates a ddl job to execute given
* queryString trigger command on shell relation(s) in mx worker(s) and to
* execute necessary ddl task on citus local table shard (if needed).
*/
List *
CitusLocalTableTriggerCommandDDLJob(Oid relationId, char *triggerName,
const char *queryString)
{
DDLJob *ddlJob = palloc0(sizeof(DDLJob));
ddlJob->targetRelationId = relationId;
ddlJob->commandString = queryString;
if (!triggerName)
{
/*
* ENABLE/DISABLE TRIGGER ALL/USER commands do not specify trigger
* name.
*/
ddlJob->taskList = DDLTaskList(relationId, queryString);
return list_make1(ddlJob);
}
bool missingOk = true;
Oid triggerId = get_trigger_oid(relationId, triggerName, missingOk);
if (!OidIsValid(triggerId))
{
/*
* For DROP, ENABLE/DISABLE, ENABLE REPLICA/ALWAYS TRIGGER commands,
* we create ddl job in preprocess. So trigger may not exist.
*/
return NIL;
}
int16 triggerType = GetTriggerTypeById(triggerId);
/* we don't have truncate triggers on shard relations */
if (!TRIGGER_FOR_TRUNCATE(triggerType))
{
ddlJob->taskList = DDLTaskList(relationId, queryString);
}
return list_make1(ddlJob);
}
/*
* GetTriggerTypeById returns trigger type (tgtype) of the trigger identified
* by triggerId if it exists. Otherwise, errors out.
*/
static int16
GetTriggerTypeById(Oid triggerId)
{
bool missingOk = false;
HeapTuple triggerTuple = GetTriggerTupleById(triggerId, missingOk);
Form_pg_trigger triggerForm = (Form_pg_trigger) GETSTRUCT(triggerTuple);
int16 triggerType = triggerForm->tgtype;
heap_freetuple(triggerTuple);
return triggerType;
}
/*
* GetTriggerFunctionId returns OID of the function that the trigger with
* triggerId executes if the trigger exists. Otherwise, errors out.
*/
Oid
GetTriggerFunctionId(Oid triggerId)
{
bool missingOk = false;
HeapTuple triggerTuple = GetTriggerTupleById(triggerId, missingOk);
Form_pg_trigger triggerForm = (Form_pg_trigger) GETSTRUCT(triggerTuple);
Oid functionId = triggerForm->tgfoid;
heap_freetuple(triggerTuple);
return functionId;
}

View File

@ -80,6 +80,9 @@ citus_truncate_trigger(PG_FUNCTION_ARGS)
PG_RETURN_DATUM(PointerGetDatum(NULL));
}
/* we might be truncating multiple relations */
UseCoordinatedTransaction();
if (IsCitusTableType(relationId, APPEND_DISTRIBUTED))
{
Oid schemaId = get_rel_namespace(relationId);
@ -236,12 +239,12 @@ EnsureLocalTableCanBeTruncated(Oid relationId)
/*
* PostprocessTruncateStatement handles few things that should be
* PreprocessTruncateStatement handles few things that should be
* done before standard process utility is called for truncate
* command.
*/
void
PostprocessTruncateStatement(TruncateStmt *truncateStatement)
PreprocessTruncateStatement(TruncateStmt *truncateStatement)
{
ErrorIfUnsupportedTruncateStmt(truncateStatement);
EnsurePartitionTableNotReplicatedForTruncate(truncateStatement);
@ -316,7 +319,7 @@ ExecuteTruncateStmtSequentialIfNecessary(TruncateStmt *command)
{
Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK);
if (IsCitusTableType(relationId, REFERENCE_TABLE) &&
if (IsCitusTableType(relationId, CITUS_TABLE_WITH_NO_DIST_KEY) &&
TableReferenced(relationId))
{
char *relationName = get_rel_name(relationId);

View File

@ -214,13 +214,6 @@ multi_ProcessUtility(PlannedStmt *pstmt,
parsetree = ProcessCreateSubscriptionStmt(createSubStmt);
}
if (IsA(parsetree, CreateTrigStmt))
{
CreateTrigStmt *createTriggerStmt = (CreateTrigStmt *) parsetree;
ErrorIfUnsupportedCreateTriggerCommand(createTriggerStmt);
}
if (IsA(parsetree, CallStmt))
{
CallStmt *callStmt = (CallStmt *) parsetree;
@ -378,7 +371,7 @@ multi_ProcessUtility(PlannedStmt *pstmt,
if (IsA(parsetree, TruncateStmt))
{
PostprocessTruncateStatement((TruncateStmt *) parsetree);
PreprocessTruncateStatement((TruncateStmt *) parsetree);
}
/* only generate worker DDLJobs if propagation is enabled */
@ -555,15 +548,11 @@ multi_ProcessUtility(PlannedStmt *pstmt,
}
}
/*
* We only process CREATE TABLE ... PARTITION OF commands in the function below
* to handle the case when user creates a table as a partition of distributed table.
*/
if (IsA(parsetree, CreateStmt))
{
CreateStmt *createStatement = (CreateStmt *) parsetree;
PostprocessCreateTableStmtPartitionOf(createStatement, queryString);
PostprocessCreateTableStmt(createStatement, queryString);
}
/*

View File

@ -8338,4 +8338,64 @@ get_range_partbound_string(List *bound_datums)
return buf->data;
}
/*
* Collect a list of OIDs of all sequences owned by the specified relation,
* and column if specified. If deptype is not zero, then only find sequences
* with the specified dependency type.
*/
List *
getOwnedSequences_internal(Oid relid, AttrNumber attnum, char deptype)
{
List *result = NIL;
Relation depRel;
ScanKeyData key[3];
SysScanDesc scan;
HeapTuple tup;
depRel = table_open(DependRelationId, AccessShareLock);
ScanKeyInit(&key[0],
Anum_pg_depend_refclassid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(RelationRelationId));
ScanKeyInit(&key[1],
Anum_pg_depend_refobjid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(relid));
if (attnum)
ScanKeyInit(&key[2],
Anum_pg_depend_refobjsubid,
BTEqualStrategyNumber, F_INT4EQ,
Int32GetDatum(attnum));
scan = systable_beginscan(depRel, DependReferenceIndexId, true,
NULL, attnum ? 3 : 2, key);
while (HeapTupleIsValid(tup = systable_getnext(scan)))
{
Form_pg_depend deprec = (Form_pg_depend) GETSTRUCT(tup);
/*
* We assume any auto or internal dependency of a sequence on a column
* must be what we are looking for. (We need the relkind test because
* indexes can also have auto dependencies on columns.)
*/
if (deprec->classid == RelationRelationId &&
deprec->objsubid == 0 &&
deprec->refobjsubid != 0 &&
(deprec->deptype == DEPENDENCY_AUTO || deprec->deptype == DEPENDENCY_INTERNAL) &&
get_rel_relkind(deprec->objid) == RELKIND_SEQUENCE)
{
if (!deptype || deprec->deptype == deptype)
result = lappend_oid(result, deprec->objid);
}
}
systable_endscan(scan);
table_close(depRel, AccessShareLock);
return result;
}
#endif /* (PG_VERSION_NUM >= PG_VERSION_12) && (PG_VERSION_NUM < PG_VERSION_13) */

View File

@ -21,6 +21,7 @@
#include "catalog/indexing.h"
#include "catalog/pg_class.h"
#include "catalog/pg_depend.h"
#include "catalog/pg_proc_d.h"
#include "catalog/pg_rewrite.h"
#include "catalog/pg_rewrite_d.h"
#include "catalog/pg_shdepend.h"
@ -28,6 +29,7 @@
#if PG_VERSION_NUM >= PG_VERSION_13
#include "common/hashfn.h"
#endif
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/listutils.h"
#include "distributed/metadata/dependency.h"
@ -116,6 +118,8 @@ typedef struct ViewDependencyNode
}ViewDependencyNode;
static List * GetRelationTriggerFunctionDepencyList(Oid relationId);
static DependencyDefinition * CreateObjectAddressDependencyDef(Oid classId, Oid objectId);
static ObjectAddress DependencyDefinitionObjectAddress(DependencyDefinition *definition);
/* forward declarations for functions to interact with the ObjectAddressCollector */
@ -857,12 +861,9 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
*/
if (get_typtype(target.objectId) == TYPTYPE_COMPOSITE)
{
DependencyDefinition *dependency = palloc0(sizeof(DependencyDefinition));
dependency->mode = DependencyObjectAddress;
ObjectAddressSet(dependency->data.address,
RelationRelationId,
get_typ_typrelid(target.objectId));
Oid typeRelationId = get_typ_typrelid(target.objectId);
DependencyDefinition *dependency =
CreateObjectAddressDependencyDef(RelationRelationId, typeRelationId);
result = lappend(result, dependency);
}
@ -874,18 +875,32 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
*/
if (type_is_array(target.objectId))
{
DependencyDefinition *dependency = palloc0(sizeof(DependencyDefinition));
dependency->mode = DependencyObjectAddress;
ObjectAddressSet(dependency->data.address,
TypeRelationId,
get_element_type(target.objectId));
Oid typeId = get_element_type(target.objectId);
DependencyDefinition *dependency =
CreateObjectAddressDependencyDef(TypeRelationId, typeId);
result = lappend(result, dependency);
}
break;
}
case RelationRelationId:
{
/*
* Triggers both depend to the relations and to the functions they
* execute. Also, pg_depend records dependencies from triggers to the
* functions but not from relations to their triggers. Given above two,
* we directly expand depencies for the relations to trigger functions.
* That way, we won't attempt to create the trigger as a dependency of
* the relation, which would fail as the relation itself is not created
* yet when ensuring dependencies.
*/
Oid relationId = target.objectId;
List *triggerFunctionDepencyList =
GetRelationTriggerFunctionDepencyList(relationId);
result = list_concat(result, triggerFunctionDepencyList);
}
default:
{
/* no expansion for unsupported types */
@ -896,6 +911,44 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
}
/*
* GetRelationTriggerFunctionDepencyList returns a list of DependencyDefinition
* objects for the functions that triggers of the relation with relationId depends.
*/
static List *
GetRelationTriggerFunctionDepencyList(Oid relationId)
{
List *dependencyList = NIL;
List *triggerIdList = GetExplicitTriggerIdList(relationId);
Oid triggerId = InvalidOid;
foreach_oid(triggerId, triggerIdList)
{
Oid functionId = GetTriggerFunctionId(triggerId);
DependencyDefinition *dependency =
CreateObjectAddressDependencyDef(ProcedureRelationId, functionId);
dependencyList = lappend(dependencyList, dependency);
}
return dependencyList;
}
/*
* CreateObjectAddressDependencyDef returns DependencyDefinition object that
* stores the ObjectAddress for the database object identified by classId and
* objectId.
*/
static DependencyDefinition *
CreateObjectAddressDependencyDef(Oid classId, Oid objectId)
{
DependencyDefinition *dependency = palloc0(sizeof(DependencyDefinition));
dependency->mode = DependencyObjectAddress;
ObjectAddressSet(dependency->data.address, classId, objectId);
return dependency;
}
/*
* DependencyDefinitionObjectAddress returns the object address of the dependency defined
* by the dependency definition, irregardless what the source of the definition is

View File

@ -426,6 +426,30 @@ IsCitusTableViaCatalog(Oid relationId)
}
/*
* IsCitusLocalTableByDistParams returns true if given partitionMethod and
* replicationModel would identify a citus local table.
*/
bool
IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel)
{
return partitionMethod == DISTRIBUTE_BY_NONE &&
replicationModel != REPLICATION_MODEL_2PC;
}
/*
* IsReferenceTableByDistParams returns true if given partitionMethod and
* replicationModel would identify a reference table.
*/
bool
IsReferenceTableByDistParams(char partitionMethod, char replicationModel)
{
return partitionMethod == DISTRIBUTE_BY_NONE &&
replicationModel == REPLICATION_MODEL_2PC;
}
/*
* CitusTableList returns a list that includes all the valid distributed table
* cache entries.
@ -3816,8 +3840,7 @@ ReferenceTableOidList()
Anum_pg_dist_partition_repmodel,
tupleDescriptor, &isNull);
if (partitionMethod == DISTRIBUTE_BY_NONE &&
replicationModel == REPLICATION_MODEL_2PC)
if (IsReferenceTableByDistParams(partitionMethod, replicationModel))
{
Datum relationIdDatum = heap_getattr(heapTuple,
Anum_pg_dist_partition_logicalrelid,

View File

@ -851,3 +851,14 @@ WorkerNodeGetDatum(WorkerNode *workerNode, TupleDesc tupleDescriptor)
return workerNodeDatum;
}
/*
* DistributedTableReplicationIsEnabled returns true if distributed table shards
* are replicated according to ShardReplicationFactor.
*/
bool
DistributedTableReplicationIsEnabled()
{
return (ShardReplicationFactor > 1);
}

View File

@ -817,13 +817,13 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
colocatedShardForeignConstraintCommandList,
List **referenceTableForeignConstraintList)
{
Oid schemaId = get_rel_namespace(shardInterval->relationId);
Oid relationId = shardInterval->relationId;
Oid schemaId = get_rel_namespace(relationId);
char *schemaName = get_namespace_name(schemaId);
char *escapedSchemaName = quote_literal_cstr(schemaName);
int shardIndex = 0;
List *commandList = GetReferencingForeignConstaintCommands(
shardInterval->relationId);
List *commandList = GetReferencingForeignConstaintCommands(relationId);
/* we will only use shardIndex if there is a foreign constraint */
if (commandList != NIL)
@ -844,7 +844,7 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
StringInfo applyForeignConstraintCommand = makeStringInfo();
/* we need to parse the foreign constraint command to get referencing table id */
/* we need to parse the foreign constraint command to get referenced table id */
Oid referencedRelationId = ForeignConstraintGetReferencedTableId(command);
if (referencedRelationId == InvalidOid)
{
@ -857,10 +857,29 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
char *referencedSchemaName = get_namespace_name(referencedSchemaId);
char *escapedReferencedSchemaName = quote_literal_cstr(referencedSchemaName);
if (IsCitusTableType(referencedRelationId, CITUS_TABLE_WITH_NO_DIST_KEY))
if (IsCitusTableType(referencedRelationId, REFERENCE_TABLE))
{
referencedShardId = GetFirstShardId(referencedRelationId);
}
else if (IsCitusTableType(referencedRelationId, CITUS_LOCAL_TABLE))
{
/*
* Only reference tables and citus local tables can have foreign
* keys to citus local tables but we already do not allow copying
* citus local table shards and we don't try to replicate citus
* local table shards. So, the referencing table must be a reference
* table in this context.
*/
Assert(IsCitusTableType(relationId, REFERENCE_TABLE));
/*
* We don't set foreign keys from reference tables to citus local
* tables in worker shard placements of reference tables because
* we don't have the shard placement for citus local table in worker
* nodes.
*/
continue;
}
else
{
referencedShardId = ColocatedShardIdInRelation(referencedRelationId,

View File

@ -407,7 +407,7 @@ NodeIsPrimaryWorker(WorkerNode *node)
/*
* CoordinatorAddedAsWorkerNode returns true if coordinator is added to the
* pg_dist_node. This function also acquires RowExclusiveLock on pg_dist_node
* pg_dist_node. This function also acquires ShareLock on pg_dist_node
* and does not release it to ensure that existency of the coordinator in
* metadata won't be changed until the end of transaction.
*/
@ -416,7 +416,7 @@ CoordinatorAddedAsWorkerNode()
{
bool groupContainsNodes = false;
LockRelationOid(DistNodeRelationId(), RowExclusiveLock);
LockRelationOid(DistNodeRelationId(), ShareLock);
PrimaryNodeForGroup(COORDINATOR_GROUP_ID, &groupContainsNodes);

View File

@ -96,8 +96,8 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
case T_AlterTableStmt:
{
/*
* We append shardId to the very end of table and index names to
* avoid name collisions. We also append shardId to constraint names.
* We append shardId to the very end of table and index, constraint
* and trigger names to avoid name collisions.
*/
AlterTableStmt *alterTableStmt = (AlterTableStmt *) parseTree;
@ -167,6 +167,14 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
AppendShardIdToName(indexName, shardId);
}
}
else if (command->subtype == AT_EnableTrig ||
command->subtype == AT_DisableTrig ||
command->subtype == AT_EnableAlwaysTrig ||
command->subtype == AT_EnableReplicaTrig)
{
char **triggerName = &(command->name);
AppendShardIdToName(triggerName, shardId);
}
}
break;
@ -236,6 +244,33 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
break;
}
case T_CreateTrigStmt:
{
CreateTrigStmt *createTriggerStmt = (CreateTrigStmt *) parseTree;
CreateTriggerEventExtendNames(createTriggerStmt, schemaName, shardId);
break;
}
case T_AlterObjectDependsStmt:
{
AlterObjectDependsStmt *alterTriggerDependsStmt =
(AlterObjectDependsStmt *) parseTree;
ObjectType objectType = alterTriggerDependsStmt->objectType;
if (objectType == OBJECT_TRIGGER)
{
AlterTriggerDependsEventExtendNames(alterTriggerDependsStmt,
schemaName, shardId);
}
else
{
ereport(WARNING, (errmsg("unsafe object type in alter object "
"depends statement"),
errdetail("Object type: %u", (uint32) objectType)));
}
break;
}
case T_DropStmt:
{
DropStmt *dropStmt = (DropStmt *) parseTree;
@ -310,6 +345,10 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
{
DropPolicyEventExtendNames(dropStmt, schemaName, shardId);
}
else if (objectType == OBJECT_TRIGGER)
{
DropTriggerEventExtendNames(dropStmt, schemaName, shardId);
}
else
{
ereport(WARNING, (errmsg("unsafe object type in drop statement"),
@ -456,7 +495,7 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
*newRelationName, NAMEDATALEN - 1)));
}
}
else if (objectType == OBJECT_COLUMN || objectType == OBJECT_TRIGGER)
else if (objectType == OBJECT_COLUMN)
{
char **relationName = &(renameStmt->relation->relname);
char **objectSchemaName = &(renameStmt->relation->schemaname);
@ -466,6 +505,10 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
AppendShardIdToName(relationName, shardId);
}
else if (objectType == OBJECT_TRIGGER)
{
AlterTriggerRenameEventExtendNames(renameStmt, schemaName, shardId);
}
else if (objectType == OBJECT_POLICY)
{
RenamePolicyEventExtendNames(renameStmt, schemaName, shardId);

View File

@ -19,7 +19,6 @@
#include "catalog/indexing.h"
#include "catalog/pg_type.h"
#include "commands/sequence.h"
#include "distributed/create_citus_local_table.h"
#include "distributed/colocation_utils.h"
#include "distributed/listutils.h"
#include "distributed/metadata_utility.h"

View File

@ -204,3 +204,40 @@ ListTake(List *pointerList, int size)
return result;
}
/*
* safe_list_nth first checks if given index is valid and errors out if it is
* not. Otherwise, it directly calls list_nth.
*/
void *
safe_list_nth(const List *list, int index)
{
int listLength = list_length(list);
if (index < 0 || index >= listLength)
{
ereport(ERROR, (errcode(ERRCODE_INTERNAL_ERROR),
errmsg("invalid list access: list length was %d but "
"element at index %d was requested ",
listLength, index)));
}
return list_nth(list, index);
}
/*
* GenerateListFromElement returns a new list with length of listLength
* such that all the elements are identical with input listElement pointer.
*/
List *
GenerateListFromElement(void *listElement, int listLength)
{
List *list = NIL;
for (int i = 0; i < listLength; i++)
{
list = lappend(list, listElement);
}
return list;
}

View File

@ -12,7 +12,9 @@
#include "postgres.h"
#include "catalog/namespace.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/namespace_utils.h"
#include "utils/regproc.h"
/*
* PushOverrideEmptySearchPath pushes search_path to be NIL and sets addCatalog to

View File

@ -10,8 +10,8 @@
*/
#include "postgres.h"
#include "utils/lsyscache.h"
#include "distributed/metadata_cache.h"
#include "distributed/relay_utility.h"
#include "distributed/shard_utils.h"
@ -36,23 +36,3 @@ GetTableLocalShardOid(Oid citusTableOid, uint64 shardId)
return shardRelationOid;
}
/*
* GetReferenceTableLocalShardOid returns OID of the local shard of given
* reference table. Caller of this function must ensure that referenceTableOid
* is owned by a reference table.
*/
Oid
GetReferenceTableLocalShardOid(Oid referenceTableOid)
{
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(referenceTableOid);
/* given OID should belong to a valid reference table */
Assert(cacheEntry != NULL && IsCitusTableTypeCacheEntry(cacheEntry, REFERENCE_TABLE));
const ShardInterval *shardInterval = cacheEntry->sortedShardIntervalArray[0];
uint64 referenceTableShardId = shardInterval->shardId;
return GetTableLocalShardOid(referenceTableOid, referenceTableShardId);
}

View File

@ -52,6 +52,7 @@ extern char * pg_get_triggerdef_command(Oid triggerId);
extern char * generate_relation_name(Oid relid, List *namespaces);
extern char * generate_qualified_relation_name(Oid relid);
extern char * generate_operator_name(Oid operid, Oid arg1, Oid arg2);
extern List * getOwnedSequences_internal(Oid relid, AttrNumber attnum, char deptype);
#endif /* CITUS_RULEUTILS_H */

View File

@ -114,18 +114,23 @@ extern ObjectAddress AlterExtensionUpdateStmtObjectAddress(Node *stmt,
extern bool ConstraintIsAForeignKeyToReferenceTable(char *constraintName,
Oid leftRelationId);
extern void ErrorIfUnsupportedForeignConstraintExists(Relation relation,
char referencingReplicationModel,
char distributionMethod,
Var *distributionColumn,
uint32 colocationId);
extern void ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId);
extern bool ColumnAppearsInForeignKeyToReferenceTable(char *columnName, Oid
relationId);
extern List * GetReferencingForeignConstaintCommands(Oid relationOid);
extern bool HasForeignKeyToCitusLocalTable(Oid relationId);
extern bool HasForeignKeyToReferenceTable(Oid relationOid);
extern bool TableReferenced(Oid relationOid);
extern bool TableReferencing(Oid relationOid);
extern bool ConstraintIsAForeignKey(char *inputConstaintName, Oid relationOid);
extern Oid GetForeignKeyOidByName(char *inputConstaintName, Oid relationId);
extern void ErrorIfTableHasExternalForeignKeys(Oid relationId);
extern List * GetForeignKeyOids(Oid relationId, int flags);
extern Oid GetReferencedTableId(Oid foreignKeyId);
/* function.c - forward declarations */
@ -231,8 +236,8 @@ extern Node * ProcessCreateSubscriptionStmt(CreateSubscriptionStmt *createSubStm
/* table.c - forward declarations */
extern List * PreprocessDropTableStmt(Node *stmt, const char *queryString);
extern List * PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement,
const char *queryString);
extern void PostprocessCreateTableStmt(CreateStmt *createStatement,
const char *queryString);
extern List * PostprocessAlterTableStmtAttachPartition(
AlterTableStmt *alterTableStatement,
const char *queryString);
@ -248,6 +253,7 @@ extern void PostprocessAlterTableStmt(AlterTableStmt *pStmt);
extern void ErrorUnsupportedAlterTableAddColumn(Oid relationId, AlterTableCmd *command,
Constraint *constraint);
extern void ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
char referencingReplicationModel,
Var *distributionColumn, uint32 colocationId);
extern ObjectAddress AlterTableSchemaStmtObjectAddress(Node *stmt,
bool missing_ok);
@ -255,7 +261,7 @@ extern List * MakeNameListFromRangeVar(const RangeVar *rel);
/* truncate.c - forward declarations */
extern void PostprocessTruncateStatement(TruncateStmt *truncateStatement);
extern void PreprocessTruncateStatement(TruncateStmt *truncateStatement);
/* type.c - forward declarations */
extern List * PreprocessCompositeTypeStmt(Node *stmt, const char *queryString);
@ -296,11 +302,27 @@ extern void PostprocessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumComm
/* trigger.c - forward declarations */
extern List * GetExplicitTriggerCommandList(Oid relationId);
extern List * GetExplicitTriggerNameList(Oid relationId);
extern char * GetTriggerNameById(Oid triggerId);
extern HeapTuple GetTriggerTupleById(Oid triggerId, bool missingOk);
extern List * GetExplicitTriggerIdList(Oid relationId);
extern Oid get_relation_trigger_oid_compat(HeapTuple heapTuple);
extern void ErrorIfUnsupportedCreateTriggerCommand(CreateTrigStmt *createTriggerStmt);
extern List * PostprocessCreateTriggerStmt(Node *node, const char *queryString);
extern ObjectAddress CreateTriggerStmtObjectAddress(Node *node, bool missingOk);
extern void CreateTriggerEventExtendNames(CreateTrigStmt *createTriggerStmt,
char *schemaName, uint64 shardId);
extern List * PostprocessAlterTriggerRenameStmt(Node *node, const char *queryString);
extern void AlterTriggerRenameEventExtendNames(RenameStmt *renameTriggerStmt,
char *schemaName, uint64 shardId);
extern List * PostprocessAlterTriggerDependsStmt(Node *node, const char *queryString);
extern void AlterTriggerDependsEventExtendNames(
AlterObjectDependsStmt *alterTriggerDependsStmt,
char *schemaName, uint64 shardId);
extern List * PreprocessDropTriggerStmt(Node *node, const char *queryString);
extern void ErrorOutForTriggerIfNotCitusLocalTable(Oid relationId);
extern void DropTriggerEventExtendNames(DropStmt *dropTriggerStmt, char *schemaName,
uint64 shardId);
extern List * CitusLocalTableTriggerCommandDDLJob(Oid relationId, char *triggerName,
const char *queryString);
extern Oid GetTriggerFunctionId(Oid triggerId);
extern bool ShouldPropagateSetCommand(VariableSetStmt *setStmt);
extern void PostprocessVariableSetStmt(VariableSetStmt *setStmt, const char *setCommand);

View File

@ -107,6 +107,7 @@ extern List * GetTableBuildingCommands(Oid relationId, bool includeSequenceDefau
extern List * GetTableIndexAndConstraintCommands(Oid relationId);
extern bool IndexImpliedByAConstraint(Form_pg_index indexForm);
extern char ShardStorageType(Oid relationId);
extern bool DistributedTableReplicationIsEnabled(void);
extern void CheckDistributedTable(Oid relationId);
extern void CreateAppendDistributedShardPlacements(Oid relationId, int64 shardId,
List *workerNodeList, int

View File

@ -90,5 +90,7 @@ extern ArrayType * DatumArrayToArrayType(Datum *datumArray, int datumCount,
extern HTAB * ListToHashSet(List *pointerList, Size keySize, bool isStringList);
extern char * StringJoin(List *stringList, char delimiter);
extern List * ListTake(List *pointerList, int size);
extern void * safe_list_nth(const List *list, int index);
extern List * GenerateListFromElement(void *listElement, int listLength);
#endif /* CITUS_LISTUTILS_H */

View File

@ -133,10 +133,12 @@ typedef enum
} CitusTableType;
extern bool IsCitusTableType(Oid relationId, CitusTableType tableType);
extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry, CitusTableType
tableType);
extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry,
CitusTableType tableType);
extern bool IsCitusTable(Oid relationId);
extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel);
extern bool IsReferenceTableByDistParams(char partitionMethod, char replicationModel);
extern List * CitusTableList(void);
extern ShardInterval * LoadShardInterval(uint64 shardId);
extern Oid RelationIdForShard(uint64 shardId);

View File

@ -14,6 +14,5 @@
#include "postgres.h"
extern Oid GetTableLocalShardOid(Oid citusTableOid, uint64 shardId);
extern Oid GetReferenceTableLocalShardOid(Oid referenceTableOid);
#endif /* SHARD_UTILS_H */

View File

@ -35,7 +35,16 @@ s/ keyval(1|2|ref)_[0-9]+ / keyval\1_xxxxxxx /g
s/ daily_uniques_[0-9]+ / daily_uniques_xxxxxxx /g
# shard table names for isolation_create_citus_local_table
s/ "citus_local_table_([0-9]+)_[0-9]+" / "citus_local_table_\1_xxxxxxx" /g
s/"citus_local_table_([0-9]+)_[0-9]+"/"citus_local_table_\1_xxxxxxx"/g
# normalize relation oid suffix for the truncate triggers created by citus
s/truncate_trigger_[0-9]+/truncate_trigger_xxxxxxx/g
# (citus_table_triggers.sql)
# postgres generates create trigger commands for triggers with:
# "EXECUTE FUNCTION" in pg12
# "EXECUTE PROCEDURE" in pg11
s/FOR EACH (ROW|STATEMENT)(.*)EXECUTE PROCEDURE/FOR EACH \1\2EXECUTE FUNCTION/g
# In foreign_key_restriction_enforcement, normalize shard names
s/"(on_update_fkey_table_|fkey_)[0-9]+"/"\1xxxxxxx"/g

View File

@ -0,0 +1,442 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1507000;
SET citus.shard_replication_factor TO 1;
SET citus.enable_local_execution TO ON;
SET citus.log_local_commands TO ON;
CREATE SCHEMA citus_local_table_triggers;
SET search_path TO citus_local_table_triggers;
-- ensure that coordinator is added to pg_dist_node
SET client_min_messages to ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
RESET client_min_messages;
CREATE TABLE citus_local_table (value int);
SELECT create_citus_local_table('citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
---------------------------------------------------------------------
-- DELETE trigger --
---------------------------------------------------------------------
BEGIN;
CREATE TABLE distributed_table(value int);
SELECT create_distributed_table('distributed_table', 'value');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE FUNCTION insert_42() RETURNS trigger AS $insert_42$
BEGIN
INSERT INTO distributed_table VALUES (42);
RETURN NEW;
END;
$insert_42$ LANGUAGE plpgsql;
CREATE TRIGGER insert_42_trigger
AFTER DELETE ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION insert_42();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507000, 'citus_local_table_triggers', 'CREATE TRIGGER insert_42_trigger
AFTER DELETE ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION insert_42();')
-- select should print two rows with "42" as delete from citus_local_table will
-- insert 42 per deleted row
DELETE FROM citus_local_table;
NOTICE: executing the command locally: DELETE FROM citus_local_table_triggers.citus_local_table_1507000 citus_local_table
SELECT * FROM distributed_table;
value
---------------------------------------------------------------------
(0 rows)
ROLLBACK;
---------------------------------------------------------------------
-- TRUNCATE trigger --
---------------------------------------------------------------------
BEGIN;
CREATE TABLE reference_table(value int);
SELECT create_reference_table('reference_table');
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507005, 'citus_local_table_triggers', 'CREATE TABLE citus_local_table_triggers.reference_table (value integer)');SELECT worker_apply_shard_ddl_command (1507005, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table_triggers.reference_table OWNER TO postgres')
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE FUNCTION insert_100() RETURNS trigger AS $insert_100$
BEGIN
INSERT INTO reference_table VALUES (100);
RETURN NEW;
END;
$insert_100$ LANGUAGE plpgsql;
CREATE TRIGGER insert_100_trigger
AFTER TRUNCATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
-- As TRUNCATE triggers are executed by utility hook, it's critical to see that they
-- are executed only for once.
-- select should print a row with "100" as truncate from citus_local_table will insert 100
TRUNCATE citus_local_table;
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.reference_table_1507005 (value) VALUES (100)
NOTICE: executing the command locally: TRUNCATE TABLE citus_local_table_triggers.citus_local_table_xxxxx CASCADE
SELECT * FROM reference_table;
NOTICE: executing the command locally: SELECT value FROM citus_local_table_triggers.reference_table_1507005 reference_table
value
---------------------------------------------------------------------
100
(1 row)
ROLLBACK;
---------------------------------------------------------------------
-- INSERT trigger --
---------------------------------------------------------------------
BEGIN;
CREATE TABLE local_table(value int);
CREATE FUNCTION increment_value() RETURNS trigger AS $increment_value$
BEGIN
UPDATE local_table SET value=value+1;
RETURN NEW;
END;
$increment_value$ LANGUAGE plpgsql;
CREATE TRIGGER increment_value_trigger
AFTER INSERT ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION increment_value();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507000, 'citus_local_table_triggers', 'CREATE TRIGGER increment_value_trigger
AFTER INSERT ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION increment_value();')
-- insert initial data to the table that increment_value_trigger will execute for
INSERT INTO local_table VALUES (0);
-- select should print a row with "2" as insert into citus_local_table will
-- increment all rows per inserted row
INSERT INTO citus_local_table VALUES(0), (1);
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.citus_local_table_1507000 AS citus_table_alias (value) VALUES (0), (1)
SELECT * FROM local_table;
value
---------------------------------------------------------------------
2
(1 row)
ROLLBACK;
---------------------------------------------------------------------
-- UPDATE trigger --
---------------------------------------------------------------------
BEGIN;
CREATE FUNCTION error_for_5() RETURNS trigger AS $error_for_5$
BEGIN
IF OLD.value = 5 THEN
RAISE EXCEPTION 'cannot update update for value=5';
END IF;
END;
$error_for_5$ LANGUAGE plpgsql;
CREATE TRIGGER error_for_5_trigger
BEFORE UPDATE OF value ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION error_for_5();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507000, 'citus_local_table_triggers', 'CREATE TRIGGER error_for_5_trigger
BEFORE UPDATE OF value ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION error_for_5();')
-- below update will error out as trigger raises exception
INSERT INTO citus_local_table VALUES (5);
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.citus_local_table_1507000 (value) VALUES (5)
UPDATE citus_local_table SET value=value*2 WHERE value=5;
NOTICE: executing the command locally: UPDATE citus_local_table_triggers.citus_local_table_1507000 citus_local_table SET value = (value OPERATOR(pg_catalog.*) 2) WHERE (value OPERATOR(pg_catalog.=) 5)
ERROR: cannot update update for value=5
ROLLBACK;
---------------------------------------------------------------------
-- Test other trigger commands + weird object names --
---------------------------------------------------------------------
CREATE SCHEMA "interesting!schema";
-- below view is a helper to print triggers on both shell relation and
-- shard relation for "citus_local_table"
CREATE VIEW citus_local_table_triggers AS
SELECT tgname, tgrelid::regclass, tgenabled
FROM pg_trigger
WHERE tgrelid::regclass::text like '"interesting!schema"."citus_local!_table%"'
ORDER BY 1, 2;
CREATE FUNCTION dummy_function() RETURNS trigger AS $dummy_function$
BEGIN
NEW.value := value+1;
RETURN NEW;
END;
$dummy_function$ LANGUAGE plpgsql;
BEGIN;
CREATE TABLE "interesting!schema"."citus_local!_table"(value int);
CREATE TRIGGER initial_truncate_trigger
AFTER TRUNCATE ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
SELECT create_citus_local_table('"interesting!schema"."citus_local!_table"');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- we shouldn't see truncate trigger on shard relation as we drop it
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
initial_truncate_trigger | "interesting!schema"."citus_local!_table" | O
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(2 rows)
ROLLBACK;
CREATE TABLE "interesting!schema"."citus_local!_table"(value int);
SELECT create_citus_local_table('"interesting!schema"."citus_local!_table"');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TRIGGER "trigger\'name"
BEFORE INSERT ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', E'CREATE TRIGGER "trigger\\''name"
BEFORE INSERT ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();')
BEGIN;
CREATE EXTENSION seg;
-- ALTER TRIGGER DEPENDS ON
ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', E'ALTER TRIGGER "trigger\\''name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg;')
-- show that triggers on both shell relation and shard relation are depending on seg
SELECT tgname FROM pg_depend, pg_trigger, pg_extension
WHERE deptype = 'x' and classid='pg_trigger'::regclass and
pg_trigger.oid=pg_depend.objid and extname='seg'
ORDER BY 1;
tgname
---------------------------------------------------------------------
trigger\'name
trigger\'name_1507007
(2 rows)
DROP EXTENSION seg;
-- show that dropping extension drops the triggers automatically
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(1 row)
ROLLBACK;
-- ALTER TRIGGER RENAME
ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" RENAME TO "trigger\'name22";
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', E'ALTER TRIGGER "trigger\\''name" ON "interesting!schema"."citus_local!_table" RENAME TO "trigger\\''name22";')
-- show that triggers on both shell relation and shard relation are renamed
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
trigger\'name22 | "interesting!schema"."citus_local!_table" | O
trigger\'name22_1507007 | "interesting!schema"."citus_local!_table_1507007" | O
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(3 rows)
-- ALTER TABLE DISABLE trigger
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER "trigger\'name22";
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', E'ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER "trigger\\''name22";')
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
trigger\'name22 | "interesting!schema"."citus_local!_table" | D
trigger\'name22_1507007 | "interesting!schema"."citus_local!_table_1507007" | D
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(3 rows)
-- ALTER TABLE ENABLE trigger
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER "trigger\'name22";
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', E'ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER "trigger\\''name22";')
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
trigger\'name22 | "interesting!schema"."citus_local!_table" | O
trigger\'name22_1507007 | "interesting!schema"."citus_local!_table_1507007" | O
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(3 rows)
CREATE TRIGGER another_trigger
AFTER DELETE ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', 'CREATE TRIGGER another_trigger
AFTER DELETE ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();')
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER USER;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', 'ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER USER;')
-- show that all triggers except the internal ones are disabled
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_trigger | "interesting!schema"."citus_local!_table" | D
another_trigger_1507007 | "interesting!schema"."citus_local!_table_1507007" | D
trigger\'name22 | "interesting!schema"."citus_local!_table" | D
trigger\'name22_1507007 | "interesting!schema"."citus_local!_table_1507007" | D
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(5 rows)
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER USER;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', 'ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER USER;')
-- show that all triggers except the internal ones are enabled again
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_trigger | "interesting!schema"."citus_local!_table" | O
another_trigger_1507007 | "interesting!schema"."citus_local!_table_1507007" | O
trigger\'name22 | "interesting!schema"."citus_local!_table" | O
trigger\'name22_1507007 | "interesting!schema"."citus_local!_table_1507007" | O
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(5 rows)
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER ALL;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', 'ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER ALL;')
-- show that all triggers including internal triggers are disabled
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_trigger | "interesting!schema"."citus_local!_table" | D
another_trigger_1507007 | "interesting!schema"."citus_local!_table_1507007" | D
trigger\'name22 | "interesting!schema"."citus_local!_table" | D
trigger\'name22_1507007 | "interesting!schema"."citus_local!_table_1507007" | D
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | D
(5 rows)
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER ALL;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', 'ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER ALL;')
-- show that all triggers including internal triggers are enabled again
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_trigger | "interesting!schema"."citus_local!_table" | O
another_trigger_1507007 | "interesting!schema"."citus_local!_table_1507007" | O
trigger\'name22 | "interesting!schema"."citus_local!_table" | O
trigger\'name22_1507007 | "interesting!schema"."citus_local!_table_1507007" | O
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(5 rows)
DROP TRIGGER another_trigger ON "interesting!schema"."citus_local!_table";
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', 'DROP TRIGGER another_trigger ON "interesting!schema"."citus_local!_table";')
DROP TRIGGER "trigger\'name22" ON "interesting!schema"."citus_local!_table";
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', E'DROP TRIGGER "trigger\\''name22" ON "interesting!schema"."citus_local!_table";')
-- show that drop trigger works as expected
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(1 row)
BEGIN;
CREATE TRIGGER "another_trigger\'name"
AFTER TRUNCATE ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER "another_trigger\'name";
-- show that our truncate trigger is disabled ..
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_trigger\'name | "interesting!schema"."citus_local!_table" | D
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(2 rows)
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER ALL;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507007, 'interesting!schema', 'ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER ALL;')
-- .. and now it is enabled back
SELECT * FROM citus_local_table_triggers;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_trigger\'name | "interesting!schema"."citus_local!_table" | O
truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O
(2 rows)
ROLLBACK;
-- as we create ddl jobs for DROP TRIGGER before standard process utility,
-- it's important to see that we properly handle non-existing triggers
-- and relations
DROP TRIGGER no_such_trigger ON "interesting!schema"."citus_local!_table";
ERROR: trigger "no_such_trigger" for table "citus_local!_table" does not exist
DROP TRIGGER no_such_trigger ON no_such_relation;
ERROR: relation "no_such_relation" does not exist
---------------------------------------------------------------------
-- a complex test case with triggers --
---------------------------------------------------------------------
-- create test tables and some foreign key relationships between them to see
-- that triggers are properly handled when ddl cascades to referencing table
CREATE TABLE another_citus_local_table (value int unique);
SELECT create_citus_local_table('another_citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
ALTER TABLE another_citus_local_table ADD CONSTRAINT fkey_self FOREIGN KEY(value) REFERENCES another_citus_local_table(value);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507008, 'citus_local_table_triggers', 1507008, 'citus_local_table_triggers', 'ALTER TABLE another_citus_local_table ADD CONSTRAINT fkey_self FOREIGN KEY(value) REFERENCES another_citus_local_table(value);')
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(value) REFERENCES another_citus_local_table(value) ON UPDATE CASCADE;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1507000, 'citus_local_table_triggers', 1507008, 'citus_local_table_triggers', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(value) REFERENCES another_citus_local_table(value) ON UPDATE CASCADE;')
CREATE TABLE reference_table(value int);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE FUNCTION insert_100() RETURNS trigger AS $insert_100$
BEGIN
INSERT INTO reference_table VALUES (100);
RETURN NEW;
END;
$insert_100$ LANGUAGE plpgsql;
BEGIN;
CREATE TRIGGER insert_100_trigger
AFTER TRUNCATE ON another_citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
CREATE TRIGGER insert_100_trigger
AFTER TRUNCATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
TRUNCATE another_citus_local_table CASCADE;
NOTICE: truncate cascades to table "citus_local_table"
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.reference_table_1507009 (value) VALUES (100)
NOTICE: executing the command locally: TRUNCATE TABLE citus_local_table_triggers.another_citus_local_table_xxxxx CASCADE
NOTICE: truncate cascades to table "citus_local_table_xxxxx"
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.reference_table_1507009 (value) VALUES (100)
NOTICE: executing the command locally: TRUNCATE TABLE citus_local_table_triggers.citus_local_table_xxxxx CASCADE
-- we should see two rows with "100"
SELECT * FROM reference_table;
NOTICE: executing the command locally: SELECT value FROM citus_local_table_triggers.reference_table_1507009 reference_table
value
---------------------------------------------------------------------
100
100
(2 rows)
ROLLBACK;
BEGIN;
-- update should actually update something to test ON UPDATE CASCADE logic
INSERT INTO another_citus_local_table VALUES (600);
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.another_citus_local_table_1507008 (value) VALUES (600)
INSERT INTO citus_local_table VALUES (600);
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.citus_local_table_1507000 (value) VALUES (600)
CREATE TRIGGER insert_100_trigger
AFTER UPDATE ON another_citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507008, 'citus_local_table_triggers', 'CREATE TRIGGER insert_100_trigger
AFTER UPDATE ON another_citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();')
CREATE TRIGGER insert_100_trigger
AFTER UPDATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507000, 'citus_local_table_triggers', 'CREATE TRIGGER insert_100_trigger
AFTER UPDATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();')
UPDATE another_citus_local_table SET value=value-1;;
NOTICE: executing the command locally: UPDATE citus_local_table_triggers.another_citus_local_table_1507008 another_citus_local_table SET value = (value OPERATOR(pg_catalog.-) 1)
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.reference_table_1507009 (value) VALUES (100)
NOTICE: executing the command locally: INSERT INTO citus_local_table_triggers.reference_table_1507009 (value) VALUES (100)
-- we should see two rows with "100"
SELECT * FROM reference_table;
NOTICE: executing the command locally: SELECT value FROM citus_local_table_triggers.reference_table_1507009 reference_table
value
---------------------------------------------------------------------
100
100
(2 rows)
ROLLBACK;
-- cleanup at exit
DROP SCHEMA citus_local_table_triggers, "interesting!schema" CASCADE;
NOTICE: drop cascades to 11 other objects

View File

@ -147,7 +147,7 @@ BEGIN;
-- cannot attach citus local table to a partitioned distributed table
ALTER TABLE partitioned_table ATTACH PARTITION citus_local_table FOR VALUES FROM (20) TO (30);
ERROR: distributed tables cannot have non-colocated distributed tables as a partition
ERROR: cannot execute ATTACH/DETACH PARTITION command as citus local tables cannot be involved in partition relationships with other tables
ROLLBACK;
-- show that we do not support inheritance relationships --
CREATE TABLE parent_table (a int, b text);
@ -177,7 +177,7 @@ BEGIN;
$update_value$ LANGUAGE plpgsql;
CREATE TRIGGER insert_trigger
AFTER INSERT ON citus_local_table_3
FOR EACH STATEMENT EXECUTE PROCEDURE update_value();
FOR EACH STATEMENT EXECUTE FUNCTION update_value();
SELECT create_citus_local_table('citus_local_table_3');
create_citus_local_table
---------------------------------------------------------------------
@ -416,26 +416,76 @@ NOTICE: executing the command locally: CREATE UNIQUE INDEX uniqueindex2_15040
---- utility command execution ----
---------------------------------------------------------------------
SET search_path TO citus_local_tables_test_schema;
-- any foreign key between citus local tables and other tables cannot be set for now
-- most should error out (for now with meaningless error messages)
-- between citus local tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(a) references citus_local_table_2(a);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1504027, 'citus_local_tables_test_schema', 1504028, 'citus_local_tables_test_schema', 'ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(a) references citus_local_table_2(a);')
-- between citus local tables and reference tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_ref FOREIGN KEY(a) references reference_table(a);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1504027, 'citus_local_tables_test_schema', 1504033, 'citus_local_tables_test_schema', 'ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_ref FOREIGN KEY(a) references reference_table(a);')
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_c FOREIGN KEY(a) references citus_local_table_1(a);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1504033, 'citus_local_tables_test_schema', 1504027, 'citus_local_tables_test_schema', 'ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_c FOREIGN KEY(a) references citus_local_table_1(a);')
ERROR: relation "citus_local_tables_test_schema.citus_local_table_1_1504027" does not exist
-- any foreign key between citus local tables and other tables except reference tables cannot be set
-- more tests at ref_citus_local_fkeys.sql
-- between citus local tables and distributed tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_dist FOREIGN KEY(a) references distributed_table(a);
ERROR: cannot create foreign key constraint since foreign keys from reference tables to distributed tables are not supported
ALTER TABLE distributed_table ADD CONSTRAINT fkey_dist_to_c FOREIGN KEY(a) references citus_local_table_1(a);
ERROR: relation "citus_local_tables_test_schema.citus_local_table_1_1504027" does not exist
ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table
-- between citus local tables and local tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_local FOREIGN KEY(a) references local_table(a);
ERROR: referenced table "local_table" must be a distributed table or a reference table
ALTER TABLE local_table ADD CONSTRAINT fkey_local_to_c FOREIGN KEY(a) references citus_local_table_1(a);
ERROR: cannot create foreign key constraint as "local_table" is a postgres local table
ALTER TABLE local_table
ADD CONSTRAINT fkey_local_to_c FOREIGN KEY(a) references citus_local_table_1(a),
ADD CONSTRAINT fkey_self FOREIGN KEY(a) references local_table(a);
ERROR: cannot create foreign key constraint as "local_table" is a postgres local table
ALTER TABLE local_table
ADD COLUMN b int references citus_local_table_1(a),
ADD COLUMN c int references local_table(a);
ERROR: cannot create foreign key constraint as "local_table" is a postgres local table
CREATE TABLE local_table_4 (
a int unique references citus_local_table_1(a),
b int references local_table_4(a));
ERROR: cannot create foreign key constraint as "local_table_4" is a postgres local table
ALTER TABLE citus_local_table_1 ADD COLUMN b int NOT NULL;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1504027, 'citus_local_tables_test_schema', 'ALTER TABLE citus_local_table_1 ADD COLUMN b int NOT NULL;')
-- show that we added column with NOT NULL
SELECT table_name, column_name, is_nullable
FROM INFORMATION_SCHEMA.COLUMNS
WHERE table_name LIKE 'citus_local_table_1%' AND column_name = 'b'
ORDER BY 1;
table_name | column_name | is_nullable
---------------------------------------------------------------------
citus_local_table_1 | b | NO
citus_local_table_1_1504027 | b | NO
(2 rows)
ALTER TABLE citus_local_table_1 ADD CONSTRAINT unique_a_b UNIQUE (a, b);
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1504027, 'citus_local_tables_test_schema', 'ALTER TABLE citus_local_table_1 ADD CONSTRAINT unique_a_b UNIQUE (a, b);')
-- show that we defined unique constraints
SELECT conrelid::regclass, conname, conkey
FROM pg_constraint
WHERE conrelid::regclass::text LIKE 'citus_local_table_1%' AND contype = 'u'
ORDER BY 1;
conrelid | conname | conkey
---------------------------------------------------------------------
citus_local_table_1_1504027 | unique_a_b_1504027 | {1,2}
citus_local_table_1 | unique_a_b | {1,2}
(2 rows)
CREATE UNIQUE INDEX citus_local_table_1_idx ON citus_local_table_1(b);
NOTICE: executing the command locally: CREATE UNIQUE INDEX citus_local_table_1_idx_1504027 ON citus_local_tables_test_schema.citus_local_table_1_1504027 USING btree (b )
-- show that we successfully defined the unique index
SELECT indexrelid::regclass, indrelid::regclass, indkey
FROM pg_index
WHERE indrelid::regclass::text LIKE 'citus_local_table_1%' AND indexrelid::regclass::text LIKE 'unique_a_b%'
ORDER BY 1;
indexrelid | indrelid | indkey
---------------------------------------------------------------------
unique_a_b | citus_local_table_1 | 1 2
unique_a_b_1504027 | citus_local_table_1_1504027 | 1 2
(2 rows)
-- execute truncate & drop commands for multiple relations to see that we don't break local execution
TRUNCATE citus_local_table_1, citus_local_table_2, distributed_table, local_table, reference_table;
NOTICE: executing the command locally: TRUNCATE TABLE citus_local_tables_test_schema.citus_local_table_1_xxxxx CASCADE
NOTICE: executing the command locally: TRUNCATE TABLE citus_local_tables_test_schema.citus_local_table_2_xxxxx CASCADE
NOTICE: executing the command locally: TRUNCATE TABLE citus_local_tables_test_schema.reference_table_xxxxx CASCADE
DROP TABLE citus_local_table_1, citus_local_table_2, distributed_table, local_table, reference_table;
NOTICE: executing the command locally: DROP TABLE IF EXISTS citus_local_tables_test_schema.reference_table_xxxxx CASCADE
NOTICE: executing the command locally: DROP TABLE IF EXISTS citus_local_tables_test_schema.citus_local_table_2_xxxxx CASCADE
NOTICE: executing the command locally: DROP TABLE IF EXISTS citus_local_tables_test_schema.citus_local_table_1_xxxxx CASCADE
-- cleanup at exit
DROP SCHEMA citus_local_tables_test_schema, "CiTUS!LocalTables" CASCADE;
NOTICE: drop cascades to 23 other objects
NOTICE: drop cascades to 15 other objects

View File

@ -0,0 +1,194 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1508000;
SET citus.shard_replication_factor TO 1;
SET citus.enable_local_execution TO ON;
SET citus.log_local_commands TO ON;
CREATE SCHEMA citus_local_tables_mx;
SET search_path TO citus_local_tables_mx;
-- ensure that coordinator is added to pg_dist_node
SET client_min_messages to ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
RESET client_min_messages;
---------------------------------------------------------------------
-- triggers --
---------------------------------------------------------------------
CREATE TABLE citus_local_table (value int);
SELECT create_citus_local_table('citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- first stop metadata sync to worker_1
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
stop_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
CREATE FUNCTION dummy_function() RETURNS trigger AS $dummy_function$
BEGIN
RAISE EXCEPTION 'a trigger that throws this exception';
END;
$dummy_function$ LANGUAGE plpgsql;
CREATE TRIGGER dummy_function_trigger
BEFORE UPDATE OF value ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION dummy_function();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1508000, 'citus_local_tables_mx', 'CREATE TRIGGER dummy_function_trigger
BEFORE UPDATE OF value ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION dummy_function();')
-- Show that we can sync metadata successfully. That means, we create
-- the function that trigger needs in mx workers too.
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
CREATE EXTENSION seg;
ALTER TRIGGER dummy_function_trigger ON citus_local_table DEPENDS ON EXTENSION seg;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1508000, 'citus_local_tables_mx', 'ALTER TRIGGER dummy_function_trigger ON citus_local_table DEPENDS ON EXTENSION seg;')
ALTER TRIGGER dummy_function_trigger ON citus_local_table RENAME TO renamed_trigger;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1508000, 'citus_local_tables_mx', 'ALTER TRIGGER dummy_function_trigger ON citus_local_table RENAME TO renamed_trigger;')
ALTER TABLE citus_local_table DISABLE TRIGGER ALL;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1508000, 'citus_local_tables_mx', 'ALTER TABLE citus_local_table DISABLE TRIGGER ALL;')
-- show that update trigger mx relation are depending on seg, renamed and disabled.
-- both workers should should print 1.
SELECT run_command_on_workers(
$$
SELECT COUNT(*) FROM pg_depend, pg_trigger, pg_extension
WHERE pg_trigger.tgrelid='citus_local_tables_mx.citus_local_table'::regclass AND
pg_trigger.tgname='renamed_trigger' AND
pg_trigger.tgenabled='D' AND
pg_depend.classid='pg_trigger'::regclass AND
pg_depend.deptype='x' AND
pg_trigger.oid=pg_depend.objid AND
pg_extension.extname='seg'
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
CREATE FUNCTION another_dummy_function() RETURNS trigger AS $another_dummy_function$
BEGIN
RAISE EXCEPTION 'another trigger that throws another exception';
END;
$another_dummy_function$ LANGUAGE plpgsql;
-- Show that we can create the trigger successfully. That means, we create
-- the function that trigger needs in mx worker too when processing CREATE
-- TRIGGER commands.
CREATE TRIGGER another_dummy_function_trigger
AFTER TRUNCATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION another_dummy_function();
-- create some test tables before next three sections
-- and define some foreign keys between them
CREATE TABLE citus_local_table_1(l1 int);
SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table_1(r1 int primary key);
SELECT create_reference_table('reference_table_1');
create_reference_table
---------------------------------------------------------------------
(1 row)
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table_1(r1) ON DELETE CASCADE;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1508001, 'citus_local_tables_mx', 1508002, 'citus_local_tables_mx', 'ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table_1(r1) ON DELETE CASCADE;')
CREATE TABLE citus_local_table_2(l1 int primary key);
SELECT create_citus_local_table('citus_local_table_2');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table_2(r1 int);
SELECT create_reference_table('reference_table_2');
create_reference_table
---------------------------------------------------------------------
(1 row)
ALTER TABLE reference_table_2 ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table_2(l1) ON DELETE RESTRICT;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1508004, 'citus_local_tables_mx', 1508003, 'citus_local_tables_mx', 'ALTER TABLE reference_table_2 ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table_2(l1) ON DELETE RESTRICT;')
CREATE TABLE citus_local_table_3(l1 int);
SELECT create_citus_local_table('citus_local_table_3');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE citus_local_table_4(l1 int primary key);
SELECT create_citus_local_table('citus_local_table_4');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
ALTER TABLE citus_local_table_3 ADD CONSTRAINT fkey_local_to_local FOREIGN KEY(l1) REFERENCES citus_local_table_4(l1) ON UPDATE SET NULL;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1508005, 'citus_local_tables_mx', 1508006, 'citus_local_tables_mx', 'ALTER TABLE citus_local_table_3 ADD CONSTRAINT fkey_local_to_local FOREIGN KEY(l1) REFERENCES citus_local_table_4(l1) ON UPDATE SET NULL;')
-- and switch to worker1
\c - - - :worker_1_port
SET search_path TO citus_local_tables_mx;
---------------------------------------------------------------------
-- foreign key from citus local table to reference table --
---------------------------------------------------------------------
-- show that on delete cascade works
INSERT INTO reference_table_1 VALUES (11);
INSERT INTO citus_local_table_1 VALUES (11);
DELETE FROM reference_table_1 WHERE r1=11;
-- should print 0 rows
SELECT * FROM citus_local_table_1 ORDER BY l1;
l1
---------------------------------------------------------------------
(0 rows)
-- show that we are checking for foreign key constraint, below should fail
INSERT INTO citus_local_table_1 VALUES (2);
ERROR: insert or update on table "citus_local_table_1_xxxxxxx" violates foreign key constraint "fkey_local_to_ref_1508001"
-- below should work
INSERT INTO reference_table_1 VALUES (2);
INSERT INTO citus_local_table_1 VALUES (2);
---------------------------------------------------------------------
-- foreign key from reference table to citus local table --
---------------------------------------------------------------------
-- show that we are checking for foreign key constraint, below should fail
INSERT INTO reference_table_2 VALUES (4);
ERROR: insert or update on table "reference_table_2_1508004" violates foreign key constraint "fkey_ref_to_local_1508004"
-- below should work
INSERT INTO citus_local_table_2 VALUES (4);
INSERT INTO reference_table_2 VALUES (4);
---------------------------------------------------------------------
-- foreign key from citus local table to citus local table --
---------------------------------------------------------------------
-- show that we are checking for foreign key constraint, below should fail
INSERT INTO citus_local_table_3 VALUES (3);
ERROR: insert or update on table "citus_local_table_3_xxxxxxx" violates foreign key constraint "fkey_local_to_local_1508005"
-- below shoud work
INSERT INTO citus_local_table_4 VALUES (3);
INSERT INTO citus_local_table_3 VALUES (3);
UPDATE citus_local_table_4 SET l1=6 WHERE l1=3;
-- show that it prints only one row with l1=null due to ON UPDATE SET NULL
SELECT * FROM citus_local_table_3;
l1
---------------------------------------------------------------------
(1 row)
-- finally show that we do not allow defining foreign key in mx nodes
ALTER TABLE citus_local_table_3 ADD CONSTRAINT fkey_local_to_local_2 FOREIGN KEY(l1) REFERENCES citus_local_table_4(l1);
ERROR: operation is not allowed on this node
-- cleanup at exit
\c - - - :master_port
DROP SCHEMA citus_local_tables_mx CASCADE;
NOTICE: drop cascades to 17 other objects

View File

@ -1,12 +1,13 @@
-- This test file includes tests to show that we do not allow triggers
-- on citus tables. Note that in other regression tests, we already test
-- the successfull citus table creation cases.
-- on distributed tables and reference tables. Note that in other
-- regression tests, we already test the successfull citus table
-- creation cases.
\set VERBOSITY terse
SET citus.next_shard_id TO 1505000;
CREATE SCHEMA table_triggers_schema;
SET search_path TO table_triggers_schema;
---------------------------------------------------------------------
-- show that we do not allow trigger creation on citus tables
-- show that we do not allow trigger creation on distributed & reference tables
---------------------------------------------------------------------
-- create a simple function to be invoked by triggers
CREATE FUNCTION update_value() RETURNS trigger AS $update_value$
@ -32,12 +33,65 @@ SELECT create_reference_table('reference_table');
-- below two should fail
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE PROCEDURE update_value();
ERROR: cannot create trigger on relation "distributed_table" because it is either a distributed table or a reference table
FOR EACH ROW EXECUTE FUNCTION update_value();
ERROR: triggers are only supported for citus local tables
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table
FOR EACH ROW EXECUTE PROCEDURE update_value();
ERROR: cannot create trigger on relation "reference_table" because it is either a distributed table or a reference table
FOR EACH ROW EXECUTE FUNCTION update_value();
ERROR: triggers are only supported for citus local tables
---------------------------------------------------------------------
-- show that we error out for trigger commands on distributed & reference tables
---------------------------------------------------------------------
SET citus.enable_ddl_propagation to OFF;
-- create triggers when ddl propagation is off
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE FUNCTION update_value();
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table
FOR EACH ROW EXECUTE FUNCTION update_value();
-- enable ddl propagation back
SET citus.enable_ddl_propagation to ON;
-- create an extension for "depends on" commands
CREATE EXTENSION seg;
-- below all should error out
ALTER TRIGGER update_value_dist ON distributed_table RENAME TO update_value_dist1;
ERROR: triggers are only supported for citus local tables
ALTER TRIGGER update_value_dist ON distributed_table DEPENDS ON EXTENSION seg;
ERROR: triggers are only supported for citus local tables
DROP TRIGGER update_value_dist ON distributed_table;
ERROR: triggers are only supported for citus local tables
ALTER TABLE distributed_table DISABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ALTER TABLE distributed_table DISABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ALTER TABLE distributed_table DISABLE TRIGGER update_value_dist;
ERROR: triggers are only supported for citus local tables
ALTER TABLE distributed_table ENABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ALTER TABLE distributed_table ENABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ALTER TABLE distributed_table ENABLE TRIGGER update_value_dist;
ERROR: triggers are only supported for citus local tables
-- below all should error out
ALTER TRIGGER update_value_ref ON reference_table RENAME TO update_value_ref1;
ERROR: triggers are only supported for citus local tables
ALTER TRIGGER update_value_ref ON reference_table DEPENDS ON EXTENSION seg;
ERROR: triggers are only supported for citus local tables
DROP TRIGGER update_value_ref ON reference_table;
ERROR: triggers are only supported for citus local tables
ALTER TABLE reference_table DISABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ALTER TABLE reference_table DISABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ALTER TABLE reference_table DISABLE TRIGGER update_value_ref;
ERROR: triggers are only supported for citus local tables
ALTER TABLE reference_table ENABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
ALTER TABLE reference_table ENABLE TRIGGER USER;
ERROR: triggers are only supported for citus local tables
ALTER TABLE reference_table ENABLE TRIGGER update_value_ref;
ERROR: triggers are only supported for citus local tables
---------------------------------------------------------------------
-- show that we do not allow creating citus tables if the
-- table has already triggers
@ -45,11 +99,11 @@ ERROR: cannot create trigger on relation "reference_table" because it is either
CREATE TABLE distributed_table_1 (value int);
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value();
FOR EACH ROW EXECUTE FUNCTION update_value();
CREATE TABLE reference_table_1 (value int);
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value();
FOR EACH ROW EXECUTE FUNCTION update_value();
-- below two should fail
SELECT create_distributed_table('distributed_table_1', 'value');
ERROR: cannot distribute relation "distributed_table_1" because it has triggers
@ -98,4 +152,4 @@ SELECT master_get_table_ddl_events('test_table');
-- cleanup at exit
DROP SCHEMA table_triggers_schema CASCADE;
NOTICE: drop cascades to 7 other objects
NOTICE: drop cascades to 8 other objects

View File

@ -1,101 +0,0 @@
-- This test file includes tests to show that we do not allow triggers
-- on citus tables. Note that in other regression tests, we already test
-- the successfull citus table creation cases.
\set VERBOSITY terse
SET citus.next_shard_id TO 1505000;
CREATE SCHEMA table_triggers_schema;
SET search_path TO table_triggers_schema;
---------------------------------------------------------------------
-- show that we do not allow trigger creation on citus tables
---------------------------------------------------------------------
-- create a simple function to be invoked by triggers
CREATE FUNCTION update_value() RETURNS trigger AS $update_value$
BEGIN
NEW.value := value+1 ;
RETURN NEW;
END;
$update_value$ LANGUAGE plpgsql;
CREATE TABLE distributed_table (value int);
SELECT create_distributed_table('distributed_table', 'value');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table (value int);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
-- below two should fail
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE PROCEDURE update_value();
ERROR: cannot create trigger on relation "distributed_table" because it is either a distributed table or a reference table
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table
FOR EACH ROW EXECUTE PROCEDURE update_value();
ERROR: cannot create trigger on relation "reference_table" because it is either a distributed table or a reference table
---------------------------------------------------------------------
-- show that we do not allow creating citus tables if the
-- table has already triggers
---------------------------------------------------------------------
CREATE TABLE distributed_table_1 (value int);
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value();
CREATE TABLE reference_table_1 (value int);
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value();
-- below two should fail
SELECT create_distributed_table('distributed_table_1', 'value');
ERROR: cannot distribute relation "distributed_table_1" because it has triggers
SELECT create_reference_table('reference_table_1');
ERROR: cannot distribute relation "reference_table_1" because it has triggers
---------------------------------------------------------------------
-- test deparse logic for CREATE TRIGGER commands
-- via master_get_table_ddl_events
---------------------------------------------------------------------
CREATE TABLE test_table (
id int,
text_number text,
text_col text
);
CREATE FUNCTION test_table_trigger_function() RETURNS trigger AS $test_table_trigger_function$
BEGIN
RAISE EXCEPTION 'a meaningless exception';
END;
$test_table_trigger_function$ LANGUAGE plpgsql;
-- in below two, use constraint triggers to test DEFERRABLE | NOT DEFERRABLE syntax
CREATE CONSTRAINT TRIGGER test_table_update
AFTER UPDATE OF id ON test_table
NOT DEFERRABLE
FOR EACH ROW
WHEN (OLD.* IS NOT DISTINCT FROM NEW.* AND OLD.text_number IS NOT NULL)
EXECUTE FUNCTION test_table_trigger_function();
CREATE CONSTRAINT TRIGGER test_table_insert
AFTER INSERT ON test_table
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW
WHEN (NEW.id > 5 OR NEW.text_col IS NOT NULL AND NEW.id < to_number(NEW.text_number, '9999'))
EXECUTE FUNCTION test_table_trigger_function();
CREATE TRIGGER test_table_delete
AFTER DELETE ON test_table
FOR EACH STATEMENT
EXECUTE FUNCTION test_table_trigger_function();
SELECT master_get_table_ddl_events('test_table');
master_get_table_ddl_events
---------------------------------------------------------------------
CREATE TABLE table_triggers_schema.test_table (id integer, text_number text, text_col text)
ALTER TABLE table_triggers_schema.test_table OWNER TO postgres
CREATE TRIGGER test_table_delete AFTER DELETE ON table_triggers_schema.test_table FOR EACH STATEMENT EXECUTE PROCEDURE table_triggers_schema.test_table_trigger_function()
CREATE CONSTRAINT TRIGGER test_table_insert AFTER INSERT ON table_triggers_schema.test_table DEFERRABLE INITIALLY IMMEDIATE FOR EACH ROW WHEN (((new.id OPERATOR(pg_catalog.>) 5) OR ((new.text_col IS NOT NULL) AND ((new.id)::numeric OPERATOR(pg_catalog.<) to_number(new.text_number, '9999'::text))))) EXECUTE PROCEDURE table_triggers_schema.test_table_trigger_function()
CREATE CONSTRAINT TRIGGER test_table_update AFTER UPDATE OF id ON table_triggers_schema.test_table NOT DEFERRABLE INITIALLY IMMEDIATE FOR EACH ROW WHEN (((NOT (old.* IS DISTINCT FROM new.*)) AND (old.text_number IS NOT NULL))) EXECUTE PROCEDURE table_triggers_schema.test_table_trigger_function()
(5 rows)
-- cleanup at exit
DROP SCHEMA table_triggers_schema CASCADE;
NOTICE: drop cascades to 7 other objects

View File

@ -836,7 +836,7 @@ SELECT create_distributed_table('referenced_by_reference_table', 'id');
CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_by_reference_table(id));
SELECT create_reference_table('reference_table');
ERROR: cannot create foreign key constraint since foreign keys from reference tables to distributed tables are not supported
DETAIL: A reference table can only have reference keys to other reference tables
DETAIL: A reference table can only have foreign keys to other reference tables or citus local tables
-- test foreign key creation on CREATE TABLE from + to reference table
DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int);
@ -887,7 +887,7 @@ SELECT create_reference_table('reference_table');
ALTER TABLE reference_table ADD CONSTRAINT fk FOREIGN KEY(referencing_column) REFERENCES referenced_by_reference_table(id);
ERROR: cannot create foreign key constraint since foreign keys from reference tables to distributed tables are not supported
DETAIL: A reference table can only have reference keys to other reference tables
DETAIL: A reference table can only have foreign keys to other reference tables or citus local tables
-- test foreign key creation on ALTER TABLE to reference table
CREATE TABLE references_to_reference_table(id int, referencing_column int);
SELECT create_distributed_table('references_to_reference_table', 'referencing_column');

View File

@ -357,7 +357,7 @@ ORDER BY nodeport;
(2 rows)
-- register after insert trigger
SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()')
SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE FUNCTION reject_large_id()')
ORDER BY nodeport, shardid;
nodename | nodeport | shardid | success | result
---------------------------------------------------------------------
@ -519,7 +519,7 @@ $rb$ LANGUAGE plpgsql;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON objects_1200003
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad();
FOR EACH ROW EXECUTE FUNCTION reject_bad();
\c - - - :master_port
-- test partial failure; worker_1 succeeds, 2 fails
-- in this case, we expect the transaction to abort
@ -570,7 +570,7 @@ $rb$ LANGUAGE plpgsql;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON labs_1200002
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad();
FOR EACH ROW EXECUTE FUNCTION reject_bad();
\c - - - :master_port
BEGIN;
INSERT INTO objects VALUES (1, 'apple');
@ -611,7 +611,7 @@ DROP TRIGGER reject_bad ON objects_1200003;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON objects_1200003
DEFERRABLE INITIALLY DEFERRED
FOR EACH ROW EXECUTE PROCEDURE reject_bad();
FOR EACH ROW EXECUTE FUNCTION reject_bad();
\c - - - :master_port
-- should be the same story as before, just at COMMIT time
BEGIN;
@ -654,7 +654,7 @@ DROP TRIGGER reject_bad ON labs_1200002;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON labs_1200002
DEFERRABLE INITIALLY DEFERRED
FOR EACH ROW EXECUTE PROCEDURE reject_bad();
FOR EACH ROW EXECUTE FUNCTION reject_bad();
\c - - - :master_port
BEGIN;
INSERT INTO objects VALUES (1, 'apple');
@ -856,7 +856,7 @@ $rb$ LANGUAGE plpgsql;
CREATE CONSTRAINT TRIGGER reject_bad_reference
AFTER INSERT ON reference_modifying_xacts_1200006
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad_reference();
FOR EACH ROW EXECUTE FUNCTION reject_bad_reference();
\c - - - :master_port
\set VERBOSITY terse
-- try without wrapping inside a transaction
@ -873,7 +873,7 @@ DROP TRIGGER reject_bad_reference ON reference_modifying_xacts_1200006;
CREATE CONSTRAINT TRIGGER reject_bad_reference
AFTER INSERT ON reference_modifying_xacts_1200006
DEFERRABLE INITIALLY DEFERRED
FOR EACH ROW EXECUTE PROCEDURE reject_bad_reference();
FOR EACH ROW EXECUTE FUNCTION reject_bad_reference();
\c - - - :master_port
\set VERBOSITY terse
-- try without wrapping inside a transaction
@ -944,7 +944,7 @@ $rb$ LANGUAGE plpgsql;
CREATE CONSTRAINT TRIGGER reject_bad_hash
AFTER INSERT ON hash_modifying_xacts_1200007
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad_hash();
FOR EACH ROW EXECUTE FUNCTION reject_bad_hash();
\c - - - :master_port
\set VERBOSITY terse
-- the transaction as a whole should fail
@ -967,7 +967,7 @@ DROP TRIGGER reject_bad_hash ON hash_modifying_xacts_1200007;
CREATE CONSTRAINT TRIGGER reject_bad_hash
AFTER INSERT ON hash_modifying_xacts_1200007
DEFERRABLE INITIALLY DEFERRED
FOR EACH ROW EXECUTE PROCEDURE reject_bad_hash();
FOR EACH ROW EXECUTE FUNCTION reject_bad_hash();
\c - - - :master_port
\set VERBOSITY terse
-- the transaction as a whole should fail
@ -1004,7 +1004,7 @@ ORDER BY s.logicalrelid, sp.shardstate;
CREATE CONSTRAINT TRIGGER reject_bad_reference
AFTER INSERT ON reference_modifying_xacts_1200006
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad_reference();
FOR EACH ROW EXECUTE FUNCTION reject_bad_reference();
\c - - - :master_port
\set VERBOSITY terse
BEGIN;

View File

@ -239,7 +239,7 @@ $rb$ LANGUAGE plpgsql;
CREATE CONSTRAINT TRIGGER reject_bad_mx
AFTER INSERT ON objects_mx_1220103
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad_mx();
FOR EACH ROW EXECUTE FUNCTION reject_bad_mx();
-- test partial failure; statement 1 successed, statement 2 fails
\set VERBOSITY terse
BEGIN;
@ -283,7 +283,7 @@ SELECT * FROM labs_mx WHERE id = 7;
CREATE CONSTRAINT TRIGGER reject_bad_mx
AFTER INSERT ON labs_mx_1220102
DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad_mx();
FOR EACH ROW EXECUTE FUNCTION reject_bad_mx();
BEGIN;
SET LOCAL citus.enable_local_execution TO off;
INSERT INTO objects_mx VALUES (1, 'apple');
@ -333,7 +333,7 @@ DROP TRIGGER reject_bad_mx ON objects_mx_1220103;
CREATE CONSTRAINT TRIGGER reject_bad_mx
AFTER INSERT ON objects_mx_1220103
DEFERRABLE INITIALLY DEFERRED
FOR EACH ROW EXECUTE PROCEDURE reject_bad_mx();
FOR EACH ROW EXECUTE FUNCTION reject_bad_mx();
-- should be the same story as before, just at COMMIT time
BEGIN;
SET LOCAL citus.enable_local_execution TO off;
@ -361,7 +361,7 @@ DROP TRIGGER reject_bad_mx ON labs_mx_1220102;
CREATE CONSTRAINT TRIGGER reject_bad_mx
AFTER INSERT ON labs_mx_1220102
DEFERRABLE INITIALLY DEFERRED
FOR EACH ROW EXECUTE PROCEDURE reject_bad_mx();
FOR EACH ROW EXECUTE FUNCTION reject_bad_mx();
BEGIN;
SET LOCAL citus.enable_local_execution TO off;
INSERT INTO objects_mx VALUES (1, 'apple');

View File

@ -1430,7 +1430,7 @@ COMMIT;
BEGIN;
TRUNCATE partitioning_locks;
SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3;
relation | locktype | mode
relation | locktype | mode
---------------------------------------------------------------------
partitioning_locks | relation | AccessExclusiveLock
partitioning_locks | relation | AccessShareLock

View File

@ -0,0 +1,220 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1506000;
SET citus.shard_replication_factor TO 1;
SET citus.enable_local_execution TO ON;
SET citus.log_local_commands TO ON;
CREATE SCHEMA ref_citus_local_fkeys;
SET search_path TO ref_citus_local_fkeys;
-- ensure that coordinator is added to pg_dist_node
SET client_min_messages to ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
RESET client_min_messages;
-- create test tables
CREATE TABLE citus_local_table(l1 int);
SELECT create_citus_local_table('citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table(r1 int primary key);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
---------------------------------------------------------------------
-- foreign key from citus local table to reference table --
---------------------------------------------------------------------
-- we support ON DELETE CASCADE behaviour in "ALTER TABLE ADD fkey citus_local_table (to reference_table) commands
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON DELETE CASCADE;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON DELETE CASCADE;')
-- show that on delete cascade works
INSERT INTO reference_table VALUES (11);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.reference_table_1506001 (r1) VALUES (11)
INSERT INTO citus_local_table VALUES (11);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_local_table_1506000 (l1) VALUES (11)
DELETE FROM reference_table WHERE r1=11;
NOTICE: executing the command locally: DELETE FROM ref_citus_local_fkeys.reference_table_1506001 reference_table WHERE (r1 OPERATOR(pg_catalog.=) 11)
-- should print 0 rows
SELECT * FROM citus_local_table ORDER BY l1;
NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.citus_local_table_1506000 citus_local_table ORDER BY l1
l1
---------------------------------------------------------------------
(0 rows)
-- show that we support drop constraint
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;')
-- we support ON UPDATE CASCADE behaviour in "ALTER TABLE ADD fkey citus_local_table (to reference table)" commands
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON UPDATE CASCADE;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON UPDATE CASCADE;')
-- show that on update cascade works
INSERT INTO reference_table VALUES (12);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.reference_table_1506001 (r1) VALUES (12)
INSERT INTO citus_local_table VALUES (12);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_local_table_1506000 (l1) VALUES (12)
UPDATE reference_table SET r1=13 WHERE r1=12;
NOTICE: executing the command locally: UPDATE ref_citus_local_fkeys.reference_table_1506001 reference_table SET r1 = 13 WHERE (r1 OPERATOR(pg_catalog.=) 12)
-- should print a row with 13
SELECT * FROM citus_local_table ORDER BY l1;
NOTICE: executing the command locally: SELECT l1 FROM ref_citus_local_fkeys.citus_local_table_1506000 citus_local_table ORDER BY l1
l1
---------------------------------------------------------------------
13
(1 row)
-- drop constraint for next commands
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;')
INSERT INTO citus_local_table VALUES (2);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_local_table_1506000 (l1) VALUES (2)
-- show that we are checking for foreign key constraint while defining, below should fail
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);')
ERROR: insert or update on table "citus_local_table_1506000" violates foreign key constraint "fkey_local_to_ref_1506000"
INSERT INTO reference_table VALUES (2);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.reference_table_1506001 (r1) VALUES (2)
-- this should work
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);')
-- show that we are checking for foreign key constraint after defining, this should fail
INSERT INTO citus_local_table VALUES (1);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_local_table_1506000 (l1) VALUES (1)
ERROR: insert or update on table "citus_local_table_1506000" violates foreign key constraint "fkey_local_to_ref_1506000"
INSERT INTO reference_table VALUES (1);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.reference_table_1506001 (r1) VALUES (1)
-- this should work
INSERT INTO citus_local_table VALUES (1);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_local_table_1506000 (l1) VALUES (1)
-- drop and add constraint for next commands
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;')
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506000, 'ref_citus_local_fkeys', 1506001, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);')
-- show that drop table without CASCADE errors out
DROP TABLE reference_table;
ERROR: cannot drop table reference_table because other objects depend on it
-- this should work
BEGIN;
DROP TABLE reference_table CASCADE;
NOTICE: drop cascades to constraint fkey_local_to_ref on table citus_local_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.reference_table_xxxxx CASCADE
NOTICE: drop cascades to constraint fkey_local_to_ref_1506000 on table ref_citus_local_fkeys.citus_local_table_1506000
ROLLBACK;
-- drop tables finally
DROP TABLE citus_local_table, reference_table;
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.reference_table_xxxxx CASCADE
NOTICE: drop cascades to constraint fkey_local_to_ref_1506000 on table ref_citus_local_fkeys.citus_local_table_1506000
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.citus_local_table_xxxxx CASCADE
---------------------------------------------------------------------
-- foreign key from reference table to citus local table --
---------------------------------------------------------------------
-- first remove worker_2 to test the behavior when replicating a
-- reference table that has a foreign key to a citus local table
-- to a new node
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
-- create test tables
CREATE TABLE citus_local_table(l1 int primary key);
SELECT create_citus_local_table('citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table(r1 int);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
INSERT INTO reference_table VALUES (3);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.reference_table_1506003 (r1) VALUES (3)
-- show that we are checking for foreign key constraint while defining, this should fail
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506003, 'ref_citus_local_fkeys', 1506002, 'ref_citus_local_fkeys', 'ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1);')
ERROR: insert or update on table "reference_table_1506003" violates foreign key constraint "fkey_ref_to_local_1506003"
-- we do not support CASCADE / SET NULL / SET DEFAULT behavior in "ALTER TABLE ADD fkey reference_table (to citus_local_table)" commands
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE CASCADE;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE SET NULL;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE SET DEFAULT;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE CASCADE;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE SET NULL;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE SET DEFAULT;
ERROR: cannot define foreign key constraint, foreign keys from reference tables to citus local tables can only be defined with NO ACTION or RESTRICT behaviors
INSERT INTO citus_local_table VALUES (3);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.citus_local_table_1506002 (l1) VALUES (3)
-- .. but we allow such foreign keys with RESTRICT behavior
BEGIN;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE RESTRICT;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506003, 'ref_citus_local_fkeys', 1506002, 'ref_citus_local_fkeys', 'ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE RESTRICT;')
ROLLBACK;
-- .. and we allow such foreign keys with NO ACTION behavior
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE NO ACTION;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506003, 'ref_citus_local_fkeys', 1506002, 'ref_citus_local_fkeys', 'ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE NO ACTION;')
-- show that we are checking for foreign key constraint after defining, this should fail
INSERT INTO reference_table VALUES (4);
NOTICE: executing the command locally: INSERT INTO ref_citus_local_fkeys.reference_table_1506003 (r1) VALUES (4)
ERROR: insert or update on table "reference_table_1506003" violates foreign key constraint "fkey_ref_to_local_1506003"
-- enable the worker_2 to show that we don't try to set up the foreign keys
-- between reference tables and citus local tables in worker_2 placements of
-- the reference tables
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
NOTICE: Replicating reference table "reference_table" to the node localhost:xxxxx
?column?
---------------------------------------------------------------------
1
(1 row)
-- show that we support drop constraint
BEGIN;
ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506003, 'ref_citus_local_fkeys', 1506002, 'ref_citus_local_fkeys', 'ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;')
ROLLBACK;
-- show that drop table errors as expected
DROP TABLE citus_local_table;
ERROR: cannot drop table citus_local_table because other objects depend on it
-- this should work
DROP TABLE citus_local_table CASCADE;
NOTICE: drop cascades to constraint fkey_ref_to_local on table reference_table
NOTICE: executing the command locally: DROP TABLE IF EXISTS ref_citus_local_fkeys.citus_local_table_xxxxx CASCADE
NOTICE: drop cascades to constraint fkey_ref_to_local_1506003 on table ref_citus_local_fkeys.reference_table_1506003
BEGIN;
CREATE TABLE citus_local_table_1(a int, b int, unique (a,b));
CREATE TABLE citus_local_table_2(a int, b int, unique (a,b));
SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
SELECT create_citus_local_table('citus_local_table_2');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- show that we properly handle multi column foreign keys
ALTER TABLE citus_local_table_1 ADD CONSTRAINT multi_fkey FOREIGN KEY (a, b) REFERENCES citus_local_table_2(a, b);
NOTICE: executing the command locally: SELECT worker_apply_inter_shard_ddl_command (1506004, 'ref_citus_local_fkeys', 1506005, 'ref_citus_local_fkeys', 'ALTER TABLE citus_local_table_1 ADD CONSTRAINT multi_fkey FOREIGN KEY (a, b) REFERENCES citus_local_table_2(a, b);')
COMMIT;
-- cleanup at exit
DROP SCHEMA ref_citus_local_fkeys CASCADE;
NOTICE: drop cascades to 6 other objects

View File

@ -42,6 +42,7 @@ test: multi_mx_function_call_delegation
test: multi_mx_modifications local_shard_execution
test: multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2
test: local_shard_copy
test: citus_local_tables_mx
test: multi_mx_transaction_recovery
test: multi_mx_modifying_xacts
test: multi_mx_explain

View File

@ -14,18 +14,19 @@
# ---
# Tests around schema changes, these are run first, so there's no preexisting objects.
#
# propagate_extension_commands lies just after multi_cluster_management as we do
# remove / add node operations, we do not want any preexisting objects before
# propagate_extension_commands
# ---
test: multi_extension
test: multi_703_upgrade
test: single_node
test: multi_cluster_management
# below tests are placed right after multi_cluster_management as we do
# remove/add node operations and we do not want any preexisting objects
test: alter_role_propagation
test: propagate_extension_commands
test: escape_extension_name
test: ref_citus_local_fkeys
test: multi_test_helpers multi_test_helpers_superuser
test: multi_test_catalog_views
test: multi_table_ddl
@ -46,7 +47,7 @@ test: multi_create_table_constraints multi_master_protocol multi_load_data multi
test: multi_behavioral_analytics_basics multi_behavioral_analytics_single_shard_queries multi_insert_select_non_pushable_queries multi_insert_select multi_behavioral_analytics_create_table_superuser
test: multi_shard_update_delete recursive_dml_with_different_planners_executors
test: insert_select_repartition window_functions dml_recursive multi_insert_select_window
test: multi_insert_select_conflict create_table_triggers
test: multi_insert_select_conflict citus_table_triggers
test: multi_row_insert insert_select_into_local_table multi_create_table_new_features
# following should not run in parallel because it relies on connection counts to workers
@ -300,7 +301,7 @@ test: add_coordinator
test: multi_upgrade_reference_table
test: multi_replicate_reference_table
test: multi_reference_table
test: foreign_key_to_reference_table
test: foreign_key_to_reference_table citus_local_table_triggers
test: replicate_reference_tables_to_coordinator
test: coordinator_shouldhaveshards
test: local_shard_utility_command_execution

View File

@ -999,22 +999,22 @@ SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
(1 row)
ALTER TABLE trigger_table DISABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
value | count
------------------+-------
trigger disabled | 1
trigger enabled | 1
(2 rows)
trigger enabled | 2
(1 row)
ALTER TABLE trigger_table ENABLE TRIGGER ALL;
ERROR: triggers are only supported for citus local tables
INSERT INTO trigger_table VALUES (1, 'trigger disabled');
SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value;
value | count
------------------+-------
trigger disabled | 1
trigger enabled | 2
(2 rows)
trigger enabled | 3
(1 row)
DROP TABLE trigger_table;
-- test ALTER TABLE ALL IN TABLESPACE

View File

@ -0,0 +1,288 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1507000;
SET citus.shard_replication_factor TO 1;
SET citus.enable_local_execution TO ON;
SET citus.log_local_commands TO ON;
CREATE SCHEMA citus_local_table_triggers;
SET search_path TO citus_local_table_triggers;
-- ensure that coordinator is added to pg_dist_node
SET client_min_messages to ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
RESET client_min_messages;
CREATE TABLE citus_local_table (value int);
SELECT create_citus_local_table('citus_local_table');
--------------------
-- DELETE trigger --
--------------------
BEGIN;
CREATE TABLE distributed_table(value int);
SELECT create_distributed_table('distributed_table', 'value');
CREATE FUNCTION insert_42() RETURNS trigger AS $insert_42$
BEGIN
INSERT INTO distributed_table VALUES (42);
RETURN NEW;
END;
$insert_42$ LANGUAGE plpgsql;
CREATE TRIGGER insert_42_trigger
AFTER DELETE ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION insert_42();
-- select should print two rows with "42" as delete from citus_local_table will
-- insert 42 per deleted row
DELETE FROM citus_local_table;
SELECT * FROM distributed_table;
ROLLBACK;
----------------------
-- TRUNCATE trigger --
----------------------
BEGIN;
CREATE TABLE reference_table(value int);
SELECT create_reference_table('reference_table');
CREATE FUNCTION insert_100() RETURNS trigger AS $insert_100$
BEGIN
INSERT INTO reference_table VALUES (100);
RETURN NEW;
END;
$insert_100$ LANGUAGE plpgsql;
CREATE TRIGGER insert_100_trigger
AFTER TRUNCATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
-- As TRUNCATE triggers are executed by utility hook, it's critical to see that they
-- are executed only for once.
-- select should print a row with "100" as truncate from citus_local_table will insert 100
TRUNCATE citus_local_table;
SELECT * FROM reference_table;
ROLLBACK;
--------------------
-- INSERT trigger --
--------------------
BEGIN;
CREATE TABLE local_table(value int);
CREATE FUNCTION increment_value() RETURNS trigger AS $increment_value$
BEGIN
UPDATE local_table SET value=value+1;
RETURN NEW;
END;
$increment_value$ LANGUAGE plpgsql;
CREATE TRIGGER increment_value_trigger
AFTER INSERT ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION increment_value();
-- insert initial data to the table that increment_value_trigger will execute for
INSERT INTO local_table VALUES (0);
-- select should print a row with "2" as insert into citus_local_table will
-- increment all rows per inserted row
INSERT INTO citus_local_table VALUES(0), (1);
SELECT * FROM local_table;
ROLLBACK;
--------------------
-- UPDATE trigger --
--------------------
BEGIN;
CREATE FUNCTION error_for_5() RETURNS trigger AS $error_for_5$
BEGIN
IF OLD.value = 5 THEN
RAISE EXCEPTION 'cannot update update for value=5';
END IF;
END;
$error_for_5$ LANGUAGE plpgsql;
CREATE TRIGGER error_for_5_trigger
BEFORE UPDATE OF value ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION error_for_5();
-- below update will error out as trigger raises exception
INSERT INTO citus_local_table VALUES (5);
UPDATE citus_local_table SET value=value*2 WHERE value=5;
ROLLBACK;
------------------------------------------------------
-- Test other trigger commands + weird object names --
------------------------------------------------------
CREATE SCHEMA "interesting!schema";
-- below view is a helper to print triggers on both shell relation and
-- shard relation for "citus_local_table"
CREATE VIEW citus_local_table_triggers AS
SELECT tgname, tgrelid::regclass, tgenabled
FROM pg_trigger
WHERE tgrelid::regclass::text like '"interesting!schema"."citus_local!_table%"'
ORDER BY 1, 2;
CREATE FUNCTION dummy_function() RETURNS trigger AS $dummy_function$
BEGIN
NEW.value := value+1;
RETURN NEW;
END;
$dummy_function$ LANGUAGE plpgsql;
BEGIN;
CREATE TABLE "interesting!schema"."citus_local!_table"(value int);
CREATE TRIGGER initial_truncate_trigger
AFTER TRUNCATE ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
SELECT create_citus_local_table('"interesting!schema"."citus_local!_table"');
-- we shouldn't see truncate trigger on shard relation as we drop it
SELECT * FROM citus_local_table_triggers;
ROLLBACK;
CREATE TABLE "interesting!schema"."citus_local!_table"(value int);
SELECT create_citus_local_table('"interesting!schema"."citus_local!_table"');
CREATE TRIGGER "trigger\'name"
BEFORE INSERT ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
BEGIN;
CREATE EXTENSION seg;
-- ALTER TRIGGER DEPENDS ON
ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg;
-- show that triggers on both shell relation and shard relation are depending on seg
SELECT tgname FROM pg_depend, pg_trigger, pg_extension
WHERE deptype = 'x' and classid='pg_trigger'::regclass and
pg_trigger.oid=pg_depend.objid and extname='seg'
ORDER BY 1;
DROP EXTENSION seg;
-- show that dropping extension drops the triggers automatically
SELECT * FROM citus_local_table_triggers;
ROLLBACK;
-- ALTER TRIGGER RENAME
ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" RENAME TO "trigger\'name22";
-- show that triggers on both shell relation and shard relation are renamed
SELECT * FROM citus_local_table_triggers;
-- ALTER TABLE DISABLE trigger
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER "trigger\'name22";
SELECT * FROM citus_local_table_triggers;
-- ALTER TABLE ENABLE trigger
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER "trigger\'name22";
SELECT * FROM citus_local_table_triggers;
CREATE TRIGGER another_trigger
AFTER DELETE ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER USER;
-- show that all triggers except the internal ones are disabled
SELECT * FROM citus_local_table_triggers;
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER USER;
-- show that all triggers except the internal ones are enabled again
SELECT * FROM citus_local_table_triggers;
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER ALL;
-- show that all triggers including internal triggers are disabled
SELECT * FROM citus_local_table_triggers;
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER ALL;
-- show that all triggers including internal triggers are enabled again
SELECT * FROM citus_local_table_triggers;
DROP TRIGGER another_trigger ON "interesting!schema"."citus_local!_table";
DROP TRIGGER "trigger\'name22" ON "interesting!schema"."citus_local!_table";
-- show that drop trigger works as expected
SELECT * FROM citus_local_table_triggers;
BEGIN;
CREATE TRIGGER "another_trigger\'name"
AFTER TRUNCATE ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
ALTER TABLE "interesting!schema"."citus_local!_table" DISABLE TRIGGER "another_trigger\'name";
-- show that our truncate trigger is disabled ..
SELECT * FROM citus_local_table_triggers;
ALTER TABLE "interesting!schema"."citus_local!_table" ENABLE TRIGGER ALL;
-- .. and now it is enabled back
SELECT * FROM citus_local_table_triggers;
ROLLBACK;
-- as we create ddl jobs for DROP TRIGGER before standard process utility,
-- it's important to see that we properly handle non-existing triggers
-- and relations
DROP TRIGGER no_such_trigger ON "interesting!schema"."citus_local!_table";
DROP TRIGGER no_such_trigger ON no_such_relation;
---------------------------------------
-- a complex test case with triggers --
---------------------------------------
-- create test tables and some foreign key relationships between them to see
-- that triggers are properly handled when ddl cascades to referencing table
CREATE TABLE another_citus_local_table (value int unique);
SELECT create_citus_local_table('another_citus_local_table');
ALTER TABLE another_citus_local_table ADD CONSTRAINT fkey_self FOREIGN KEY(value) REFERENCES another_citus_local_table(value);
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(value) REFERENCES another_citus_local_table(value) ON UPDATE CASCADE;
CREATE TABLE reference_table(value int);
SELECT create_reference_table('reference_table');
CREATE FUNCTION insert_100() RETURNS trigger AS $insert_100$
BEGIN
INSERT INTO reference_table VALUES (100);
RETURN NEW;
END;
$insert_100$ LANGUAGE plpgsql;
BEGIN;
CREATE TRIGGER insert_100_trigger
AFTER TRUNCATE ON another_citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
CREATE TRIGGER insert_100_trigger
AFTER TRUNCATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
TRUNCATE another_citus_local_table CASCADE;
-- we should see two rows with "100"
SELECT * FROM reference_table;
ROLLBACK;
BEGIN;
-- update should actually update something to test ON UPDATE CASCADE logic
INSERT INTO another_citus_local_table VALUES (600);
INSERT INTO citus_local_table VALUES (600);
CREATE TRIGGER insert_100_trigger
AFTER UPDATE ON another_citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
CREATE TRIGGER insert_100_trigger
AFTER UPDATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION insert_100();
UPDATE another_citus_local_table SET value=value-1;;
-- we should see two rows with "100"
SELECT * FROM reference_table;
ROLLBACK;
-- cleanup at exit
DROP SCHEMA citus_local_table_triggers, "interesting!schema" CASCADE;

View File

@ -140,7 +140,7 @@ BEGIN;
CREATE TRIGGER insert_trigger
AFTER INSERT ON citus_local_table_3
FOR EACH STATEMENT EXECUTE PROCEDURE update_value();
FOR EACH STATEMENT EXECUTE FUNCTION update_value();
SELECT create_citus_local_table('citus_local_table_3');
@ -315,15 +315,8 @@ CREATE UNIQUE INDEX uniqueIndex2 ON "LocalTabLE.1!?!"(id);
SET search_path TO citus_local_tables_test_schema;
-- any foreign key between citus local tables and other tables cannot be set for now
-- most should error out (for now with meaningless error messages)
-- between citus local tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(a) references citus_local_table_2(a);
-- between citus local tables and reference tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_ref FOREIGN KEY(a) references reference_table(a);
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_c FOREIGN KEY(a) references citus_local_table_1(a);
-- any foreign key between citus local tables and other tables except reference tables cannot be set
-- more tests at ref_citus_local_fkeys.sql
-- between citus local tables and distributed tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_dist FOREIGN KEY(a) references distributed_table(a);
@ -331,7 +324,40 @@ ALTER TABLE distributed_table ADD CONSTRAINT fkey_dist_to_c FOREIGN KEY(a) refer
-- between citus local tables and local tables
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_local FOREIGN KEY(a) references local_table(a);
ALTER TABLE local_table ADD CONSTRAINT fkey_local_to_c FOREIGN KEY(a) references citus_local_table_1(a);
ALTER TABLE local_table
ADD CONSTRAINT fkey_local_to_c FOREIGN KEY(a) references citus_local_table_1(a),
ADD CONSTRAINT fkey_self FOREIGN KEY(a) references local_table(a);
ALTER TABLE local_table
ADD COLUMN b int references citus_local_table_1(a),
ADD COLUMN c int references local_table(a);
CREATE TABLE local_table_4 (
a int unique references citus_local_table_1(a),
b int references local_table_4(a));
ALTER TABLE citus_local_table_1 ADD COLUMN b int NOT NULL;
-- show that we added column with NOT NULL
SELECT table_name, column_name, is_nullable
FROM INFORMATION_SCHEMA.COLUMNS
WHERE table_name LIKE 'citus_local_table_1%' AND column_name = 'b'
ORDER BY 1;
ALTER TABLE citus_local_table_1 ADD CONSTRAINT unique_a_b UNIQUE (a, b);
-- show that we defined unique constraints
SELECT conrelid::regclass, conname, conkey
FROM pg_constraint
WHERE conrelid::regclass::text LIKE 'citus_local_table_1%' AND contype = 'u'
ORDER BY 1;
CREATE UNIQUE INDEX citus_local_table_1_idx ON citus_local_table_1(b);
-- show that we successfully defined the unique index
SELECT indexrelid::regclass, indrelid::regclass, indkey
FROM pg_index
WHERE indrelid::regclass::text LIKE 'citus_local_table_1%' AND indexrelid::regclass::text LIKE 'unique_a_b%'
ORDER BY 1;
-- execute truncate & drop commands for multiple relations to see that we don't break local execution
TRUNCATE citus_local_table_1, citus_local_table_2, distributed_table, local_table, reference_table;
DROP TABLE citus_local_table_1, citus_local_table_2, distributed_table, local_table, reference_table;
-- cleanup at exit
DROP SCHEMA citus_local_tables_test_schema, "CiTUS!LocalTables" CASCADE;

View File

@ -0,0 +1,148 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1508000;
SET citus.shard_replication_factor TO 1;
SET citus.enable_local_execution TO ON;
SET citus.log_local_commands TO ON;
CREATE SCHEMA citus_local_tables_mx;
SET search_path TO citus_local_tables_mx;
-- ensure that coordinator is added to pg_dist_node
SET client_min_messages to ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
RESET client_min_messages;
--------------
-- triggers --
--------------
CREATE TABLE citus_local_table (value int);
SELECT create_citus_local_table('citus_local_table');
-- first stop metadata sync to worker_1
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
CREATE FUNCTION dummy_function() RETURNS trigger AS $dummy_function$
BEGIN
RAISE EXCEPTION 'a trigger that throws this exception';
END;
$dummy_function$ LANGUAGE plpgsql;
CREATE TRIGGER dummy_function_trigger
BEFORE UPDATE OF value ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION dummy_function();
-- Show that we can sync metadata successfully. That means, we create
-- the function that trigger needs in mx workers too.
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
CREATE EXTENSION seg;
ALTER TRIGGER dummy_function_trigger ON citus_local_table DEPENDS ON EXTENSION seg;
ALTER TRIGGER dummy_function_trigger ON citus_local_table RENAME TO renamed_trigger;
ALTER TABLE citus_local_table DISABLE TRIGGER ALL;
-- show that update trigger mx relation are depending on seg, renamed and disabled.
-- both workers should should print 1.
SELECT run_command_on_workers(
$$
SELECT COUNT(*) FROM pg_depend, pg_trigger, pg_extension
WHERE pg_trigger.tgrelid='citus_local_tables_mx.citus_local_table'::regclass AND
pg_trigger.tgname='renamed_trigger' AND
pg_trigger.tgenabled='D' AND
pg_depend.classid='pg_trigger'::regclass AND
pg_depend.deptype='x' AND
pg_trigger.oid=pg_depend.objid AND
pg_extension.extname='seg'
$$);
CREATE FUNCTION another_dummy_function() RETURNS trigger AS $another_dummy_function$
BEGIN
RAISE EXCEPTION 'another trigger that throws another exception';
END;
$another_dummy_function$ LANGUAGE plpgsql;
-- Show that we can create the trigger successfully. That means, we create
-- the function that trigger needs in mx worker too when processing CREATE
-- TRIGGER commands.
CREATE TRIGGER another_dummy_function_trigger
AFTER TRUNCATE ON citus_local_table
FOR EACH STATEMENT EXECUTE FUNCTION another_dummy_function();
-- create some test tables before next three sections
-- and define some foreign keys between them
CREATE TABLE citus_local_table_1(l1 int);
SELECT create_citus_local_table('citus_local_table_1');
CREATE TABLE reference_table_1(r1 int primary key);
SELECT create_reference_table('reference_table_1');
ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table_1(r1) ON DELETE CASCADE;
CREATE TABLE citus_local_table_2(l1 int primary key);
SELECT create_citus_local_table('citus_local_table_2');
CREATE TABLE reference_table_2(r1 int);
SELECT create_reference_table('reference_table_2');
ALTER TABLE reference_table_2 ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table_2(l1) ON DELETE RESTRICT;
CREATE TABLE citus_local_table_3(l1 int);
SELECT create_citus_local_table('citus_local_table_3');
CREATE TABLE citus_local_table_4(l1 int primary key);
SELECT create_citus_local_table('citus_local_table_4');
ALTER TABLE citus_local_table_3 ADD CONSTRAINT fkey_local_to_local FOREIGN KEY(l1) REFERENCES citus_local_table_4(l1) ON UPDATE SET NULL;
-- and switch to worker1
\c - - - :worker_1_port
SET search_path TO citus_local_tables_mx;
-----------------------------------------------------------
-- foreign key from citus local table to reference table --
-----------------------------------------------------------
-- show that on delete cascade works
INSERT INTO reference_table_1 VALUES (11);
INSERT INTO citus_local_table_1 VALUES (11);
DELETE FROM reference_table_1 WHERE r1=11;
-- should print 0 rows
SELECT * FROM citus_local_table_1 ORDER BY l1;
-- show that we are checking for foreign key constraint, below should fail
INSERT INTO citus_local_table_1 VALUES (2);
-- below should work
INSERT INTO reference_table_1 VALUES (2);
INSERT INTO citus_local_table_1 VALUES (2);
-----------------------------------------------------------
-- foreign key from reference table to citus local table --
-----------------------------------------------------------
-- show that we are checking for foreign key constraint, below should fail
INSERT INTO reference_table_2 VALUES (4);
-- below should work
INSERT INTO citus_local_table_2 VALUES (4);
INSERT INTO reference_table_2 VALUES (4);
-------------------------------------------------------------
-- foreign key from citus local table to citus local table --
-------------------------------------------------------------
-- show that we are checking for foreign key constraint, below should fail
INSERT INTO citus_local_table_3 VALUES (3);
-- below shoud work
INSERT INTO citus_local_table_4 VALUES (3);
INSERT INTO citus_local_table_3 VALUES (3);
UPDATE citus_local_table_4 SET l1=6 WHERE l1=3;
-- show that it prints only one row with l1=null due to ON UPDATE SET NULL
SELECT * FROM citus_local_table_3;
-- finally show that we do not allow defining foreign key in mx nodes
ALTER TABLE citus_local_table_3 ADD CONSTRAINT fkey_local_to_local_2 FOREIGN KEY(l1) REFERENCES citus_local_table_4(l1);
-- cleanup at exit
\c - - - :master_port
DROP SCHEMA citus_local_tables_mx CASCADE;

View File

@ -1,6 +1,7 @@
-- This test file includes tests to show that we do not allow triggers
-- on citus tables. Note that in other regression tests, we already test
-- the successfull citus table creation cases.
-- on distributed tables and reference tables. Note that in other
-- regression tests, we already test the successfull citus table
-- creation cases.
\set VERBOSITY terse
@ -9,9 +10,9 @@ SET citus.next_shard_id TO 1505000;
CREATE SCHEMA table_triggers_schema;
SET search_path TO table_triggers_schema;
-------------------------------------------------------------
-- show that we do not allow trigger creation on citus tables
-------------------------------------------------------------
-------------------------------------------------------------------------------
-- show that we do not allow trigger creation on distributed & reference tables
-------------------------------------------------------------------------------
-- create a simple function to be invoked by triggers
CREATE FUNCTION update_value() RETURNS trigger AS $update_value$
@ -30,11 +31,54 @@ SELECT create_reference_table('reference_table');
-- below two should fail
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE PROCEDURE update_value();
FOR EACH ROW EXECUTE FUNCTION update_value();
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table
FOR EACH ROW EXECUTE PROCEDURE update_value();
FOR EACH ROW EXECUTE FUNCTION update_value();
--------------------------------------------------------------------------------
-- show that we error out for trigger commands on distributed & reference tables
--------------------------------------------------------------------------------
SET citus.enable_ddl_propagation to OFF;
-- create triggers when ddl propagation is off
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE FUNCTION update_value();
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table
FOR EACH ROW EXECUTE FUNCTION update_value();
-- enable ddl propagation back
SET citus.enable_ddl_propagation to ON;
-- create an extension for "depends on" commands
CREATE EXTENSION seg;
-- below all should error out
ALTER TRIGGER update_value_dist ON distributed_table RENAME TO update_value_dist1;
ALTER TRIGGER update_value_dist ON distributed_table DEPENDS ON EXTENSION seg;
DROP TRIGGER update_value_dist ON distributed_table;
ALTER TABLE distributed_table DISABLE TRIGGER ALL;
ALTER TABLE distributed_table DISABLE TRIGGER USER;
ALTER TABLE distributed_table DISABLE TRIGGER update_value_dist;
ALTER TABLE distributed_table ENABLE TRIGGER ALL;
ALTER TABLE distributed_table ENABLE TRIGGER USER;
ALTER TABLE distributed_table ENABLE TRIGGER update_value_dist;
-- below all should error out
ALTER TRIGGER update_value_ref ON reference_table RENAME TO update_value_ref1;
ALTER TRIGGER update_value_ref ON reference_table DEPENDS ON EXTENSION seg;
DROP TRIGGER update_value_ref ON reference_table;
ALTER TABLE reference_table DISABLE TRIGGER ALL;
ALTER TABLE reference_table DISABLE TRIGGER USER;
ALTER TABLE reference_table DISABLE TRIGGER update_value_ref;
ALTER TABLE reference_table ENABLE TRIGGER ALL;
ALTER TABLE reference_table ENABLE TRIGGER USER;
ALTER TABLE reference_table ENABLE TRIGGER update_value_ref;
---------------------------------------------------------
-- show that we do not allow creating citus tables if the
@ -45,13 +89,13 @@ CREATE TABLE distributed_table_1 (value int);
CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value();
FOR EACH ROW EXECUTE FUNCTION update_value();
CREATE TABLE reference_table_1 (value int);
CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value();
FOR EACH ROW EXECUTE FUNCTION update_value();
-- below two should fail
SELECT create_distributed_table('distributed_table_1', 'value');

View File

@ -0,0 +1,145 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1506000;
SET citus.shard_replication_factor TO 1;
SET citus.enable_local_execution TO ON;
SET citus.log_local_commands TO ON;
CREATE SCHEMA ref_citus_local_fkeys;
SET search_path TO ref_citus_local_fkeys;
-- ensure that coordinator is added to pg_dist_node
SET client_min_messages to ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
RESET client_min_messages;
-- create test tables
CREATE TABLE citus_local_table(l1 int);
SELECT create_citus_local_table('citus_local_table');
CREATE TABLE reference_table(r1 int primary key);
SELECT create_reference_table('reference_table');
-----------------------------------------------------------
-- foreign key from citus local table to reference table --
-----------------------------------------------------------
-- we support ON DELETE CASCADE behaviour in "ALTER TABLE ADD fkey citus_local_table (to reference_table) commands
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON DELETE CASCADE;
-- show that on delete cascade works
INSERT INTO reference_table VALUES (11);
INSERT INTO citus_local_table VALUES (11);
DELETE FROM reference_table WHERE r1=11;
-- should print 0 rows
SELECT * FROM citus_local_table ORDER BY l1;
-- show that we support drop constraint
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
-- we support ON UPDATE CASCADE behaviour in "ALTER TABLE ADD fkey citus_local_table (to reference table)" commands
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1) ON UPDATE CASCADE;
-- show that on update cascade works
INSERT INTO reference_table VALUES (12);
INSERT INTO citus_local_table VALUES (12);
UPDATE reference_table SET r1=13 WHERE r1=12;
-- should print a row with 13
SELECT * FROM citus_local_table ORDER BY l1;
-- drop constraint for next commands
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
INSERT INTO citus_local_table VALUES (2);
-- show that we are checking for foreign key constraint while defining, below should fail
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
INSERT INTO reference_table VALUES (2);
-- this should work
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
-- show that we are checking for foreign key constraint after defining, this should fail
INSERT INTO citus_local_table VALUES (1);
INSERT INTO reference_table VALUES (1);
-- this should work
INSERT INTO citus_local_table VALUES (1);
-- drop and add constraint for next commands
ALTER TABLE citus_local_table DROP CONSTRAINT fkey_local_to_ref;
ALTER TABLE citus_local_table ADD CONSTRAINT fkey_local_to_ref FOREIGN KEY(l1) REFERENCES reference_table(r1);
-- show that drop table without CASCADE errors out
DROP TABLE reference_table;
-- this should work
BEGIN;
DROP TABLE reference_table CASCADE;
ROLLBACK;
-- drop tables finally
DROP TABLE citus_local_table, reference_table;
-----------------------------------------------------------
-- foreign key from reference table to citus local table --
-----------------------------------------------------------
-- first remove worker_2 to test the behavior when replicating a
-- reference table that has a foreign key to a citus local table
-- to a new node
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
-- create test tables
CREATE TABLE citus_local_table(l1 int primary key);
SELECT create_citus_local_table('citus_local_table');
CREATE TABLE reference_table(r1 int);
SELECT create_reference_table('reference_table');
INSERT INTO reference_table VALUES (3);
-- show that we are checking for foreign key constraint while defining, this should fail
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1);
-- we do not support CASCADE / SET NULL / SET DEFAULT behavior in "ALTER TABLE ADD fkey reference_table (to citus_local_table)" commands
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE CASCADE;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE SET NULL;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE SET DEFAULT;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE CASCADE;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE SET NULL;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON UPDATE SET DEFAULT;
INSERT INTO citus_local_table VALUES (3);
-- .. but we allow such foreign keys with RESTRICT behavior
BEGIN;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE RESTRICT;
ROLLBACK;
-- .. and we allow such foreign keys with NO ACTION behavior
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(r1) REFERENCES citus_local_table(l1) ON DELETE NO ACTION;
-- show that we are checking for foreign key constraint after defining, this should fail
INSERT INTO reference_table VALUES (4);
-- enable the worker_2 to show that we don't try to set up the foreign keys
-- between reference tables and citus local tables in worker_2 placements of
-- the reference tables
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
-- show that we support drop constraint
BEGIN;
ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;
ROLLBACK;
-- show that drop table errors as expected
DROP TABLE citus_local_table;
-- this should work
DROP TABLE citus_local_table CASCADE;
BEGIN;
CREATE TABLE citus_local_table_1(a int, b int, unique (a,b));
CREATE TABLE citus_local_table_2(a int, b int, unique (a,b));
SELECT create_citus_local_table('citus_local_table_1');
SELECT create_citus_local_table('citus_local_table_2');
-- show that we properly handle multi column foreign keys
ALTER TABLE citus_local_table_1 ADD CONSTRAINT multi_fkey FOREIGN KEY (a, b) REFERENCES citus_local_table_2(a, b);
COMMIT;
-- cleanup at exit
DROP SCHEMA ref_citus_local_fkeys CASCADE;