Merge pull request #4143 from citusdata/single-placement-table/master-cache-entry-rebased

DESCRIPTION: Introduce citus local tables

The commits in this pr are merged from other sub-pr's:

* community/#3852: Brings lazy&fast table creation logic for create_citus_local_table udf
* community/#3995: Brings extended utility command support for citus local tables
* community/#4133: Brings changes in planner and in several places to integrate citus local tables into our distributed execution logic

We are introducing citus local tables, which a new table type to citus.

To be able to create a citus local table, first we need to add coordinator as a worker
node.
Then, we can create a citus local table via SELECT create_citus_local_table(<tableName>).

Calling this udf from coordinator will actually create a single-shard table whose shard
is on the coordinator.

Also, from the citus metadata perspective, for citus local tables:
* partitionMethod is set to DISTRIBUTE_BY_NONE (like reference tables) and
* replicationModel is set to the current value of citus.replication_model, which
  already can't be equal to REPLICATION_MODEL_2PC, which is only used for reference
  tables internally.

Note that currently we support creating citus local tables only from postgres tables
living in the coordinator.
That means, it is not allowed to execute this udf from worker nodes or it is not allowed
to move shard of a citus local table to any other nodes.

Also, run-time complexity of calling create_citus_local_table udf does not depend
on the size of the relation, that means, creating citus local tables is actually a
non-blocking operation.
This is because, instead of copying the data to a new shard, this udf just does the
following:

* convert input postgres table to the single-shard of the citus local table by suffixing
  the shardId to it's name, constraints, indexes and triggers etc.,
* create a shell table for citus local table in coordinator and in mx-worker nodes when
  metadata sycn is enabled.
* create necessary objects on shell table.

Here, we should also note we can execute queries/dml's from mx worker nodes
as citus local tables are already first class citus tables.

Even more, we brought trigger support for citus local tables.
That means, we can define triggers on citus local tables so that users can define trigger
objects to perform execution of custom functions that might even modify other citus tables
and other postgres tables.

Other than trigger support, citus local tables can also be involved in foreign key relationships
with reference tables.
Here the only restriction is, foreign keys from reference tables to citus local tables cannot
have behaviors other than RESTRICT & NO ACTION behavior.
Other than that, foreign keys between citus local tables and reference tables just work fine.

All in all, citus local tables are actually just local tables living in the coordinator, but natively
accessible from other nodes like other first class citus tables and this enables us to set foreign
keys constraints between very big coordinator tables and reference tables without having to
do any data replication to worker nodes for local tables.
pull/4154/head
Onur Tirtir 2020-09-09 13:02:42 +03:00 committed by GitHub
commit 5e5ba46793
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
94 changed files with 9672 additions and 626 deletions

View File

@ -102,7 +102,6 @@ CallFuncExprRemotely(CallStmt *callStmt, DistObjectCacheEntry *procedure,
ereport(DEBUG1, (errmsg( ereport(DEBUG1, (errmsg(
"will push down CALL for reference tables"))); "will push down CALL for reference tables")));
colocatedWithReferenceTable = true; colocatedWithReferenceTable = true;
Assert(IsCitusTableType(colocatedRelationId, REFERENCE_TABLE));
} }
ShardPlacement *placement = NULL; ShardPlacement *placement = NULL;

View File

@ -0,0 +1,890 @@
/*-------------------------------------------------------------------------
*
* create_citus_local_table.c
*
* This file contains functions to create citus local tables.
*
* A citus local table is composed of a shell relation to wrap the
* the regular postgres relation as its coordinator local shard.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/genam.h"
#include "access/htup_details.h"
#include "catalog/pg_constraint.h"
#include "catalog/pg_trigger.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/colocation_utils.h"
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/listutils.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/namespace_utils.h"
#include "distributed/reference_table_utils.h"
#include "distributed/worker_protocol.h"
#include "distributed/worker_shard_visibility.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/ruleutils.h"
#include "utils/syscache.h"
static void CreateCitusLocalTable(Oid relationId);
static void ErrorIfUnsupportedCreateCitusLocalTable(Relation relation);
static void ErrorIfUnsupportedCitusLocalTableKind(Oid relationId);
static List * GetShellTableDDLEventsForCitusLocalTable(Oid relationId);
static uint64 ConvertLocalTableToShard(Oid relationId);
static void RenameRelationToShardRelation(Oid shellRelationId, uint64 shardId);
static void RenameShardRelationConstraints(Oid shardRelationId, uint64 shardId);
static List * GetConstraintNameList(Oid relationId);
static char * GetRenameShardConstraintCommand(Oid relationId, char *constraintName,
uint64 shardId);
static void RenameShardRelationIndexes(Oid shardRelationId, uint64 shardId);
static char * GetDropTriggerCommand(Oid relationId, char *triggerName);
static char * GetRenameShardIndexCommand(char *indexName, uint64 shardId);
static void RenameShardRelationNonTruncateTriggers(Oid shardRelationId, uint64 shardId);
static char * GetRenameShardTriggerCommand(Oid shardRelationId, char *triggerName,
uint64 shardId);
static void ExecuteAndLogDDLCommandList(List *ddlCommandList);
static void ExecuteAndLogDDLCommand(const char *commandString);
static void DropRelationTruncateTriggers(Oid relationId);
static char * GetDropTriggerCommand(Oid relationId, char *triggerName);
static List * GetExplicitIndexNameList(Oid relationId);
static void DropAndMoveDefaultSequenceOwnerships(Oid sourceRelationId,
Oid targetRelationId);
static void ExtractColumnsOwningSequences(Oid relationId, List **columnNameList,
List **ownedSequenceIdList);
static void DropDefaultColumnDefinition(Oid relationId, char *columnName);
static void TransferSequenceOwnership(Oid ownedSequenceId, Oid targetRelationId,
char *columnName);
static void InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId);
static void FinalizeCitusLocalTableCreation(Oid relationId);
PG_FUNCTION_INFO_V1(create_citus_local_table);
/*
* create_citus_local_table creates a citus local table from the table with
* relationId by executing the internal method CreateCitusLocalTable.
* (See CreateCitusLocalTable function's comment.)
*/
Datum
create_citus_local_table(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
Oid relationId = PG_GETARG_OID(0);
CreateCitusLocalTable(relationId);
PG_RETURN_VOID();
}
/*
* CreateCitusLocalTable is the internal method that creates a citus table
* from the table with relationId. The created table would have the following
* properties:
* - it will have only one shard,
* - its distribution method will be DISTRIBUTE_BY_NONE,
* - its replication model will be ReplicationModel,
* - its replication factor will be set to 1.
* Similar to reference tables, it has only 1 placement. In addition to that, that
* single placement is only allowed to be on the coordinator.
*/
static void
CreateCitusLocalTable(Oid relationId)
{
/*
* These checks should be done before acquiring any locks on relation.
* This is because we don't allow creating citus local tables in worker
* nodes and we don't want to acquire any locks on a table if we are not
* the owner of it.
*/
EnsureCoordinator();
EnsureTableOwner(relationId);
/*
* Lock target relation with an AccessExclusiveLock as we don't want
* multiple backends manipulating this relation. We could actually simply
* lock the relation without opening it. However, we also want to check
* if the relation does not exist or dropped by another backend. Also,
* we open the relation with try_relation_open instead of relation_open
* to give a nice error in case the table is dropped by another backend.
*/
Relation relation = try_relation_open(relationId, AccessExclusiveLock);
ErrorIfUnsupportedCreateCitusLocalTable(relation);
/*
* We immediately close relation with NoLock right after opening it. This is
* because, in this function, we may execute ALTER TABLE commands modifying
* relation's column definitions and postgres does not allow us to do so when
* the table is still open. (See the postgres function CheckTableNotInUse for
* more information)
*/
relation_close(relation, NoLock);
ObjectAddress tableAddress = { 0 };
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
/*
* Ensure dependencies first as we will create shell table on the other nodes
* in the MX case.
*/
EnsureDependenciesExistOnAllNodes(&tableAddress);
/*
* Make sure that existing reference tables have been replicated to all
* the nodes such that we can create foreign keys and joins work
* immediately after creation.
*/
EnsureReferenceTablesExistOnAllNodes();
List *shellTableDDLEvents = GetShellTableDDLEventsForCitusLocalTable(relationId);
char *relationName = get_rel_name(relationId);
Oid relationSchemaId = get_rel_namespace(relationId);
/* below we convert relation with relationId to the shard relation */
uint64 shardId = ConvertLocalTableToShard(relationId);
/*
* As we retrieved the DDL commands necessary to create the shell table
* from scratch, below we simply recreate the shell table executing them
* via process utility.
*/
ExecuteAndLogDDLCommandList(shellTableDDLEvents);
/*
* Set shellRelationId as the relation with relationId now points
* to the shard relation.
*/
Oid shardRelationId = relationId;
Oid shellRelationId = get_relname_relid(relationName, relationSchemaId);
/* assert that we created the shell table properly in the same schema */
Assert(OidIsValid(shellRelationId));
/*
* Move sequence ownerships from shard table to shell table and also drop
* DEFAULT expressions from shard relation as we should evaluate such columns
* in shell table when needed.
*/
DropAndMoveDefaultSequenceOwnerships(shardRelationId, shellRelationId);
InsertMetadataForCitusLocalTable(shellRelationId, shardId);
FinalizeCitusLocalTableCreation(shellRelationId);
}
/*
* ErrorIfUnsupportedCreateCitusLocalTable errors out if we cannot create the
* citus local table from the relation.
*/
static void
ErrorIfUnsupportedCreateCitusLocalTable(Relation relation)
{
if (!RelationIsValid(relation))
{
ereport(ERROR, (errmsg("cannot create citus local table, relation does "
"not exist")));
}
Oid relationId = relation->rd_id;
ErrorIfCoordinatorNotAddedAsWorkerNode();
ErrorIfUnsupportedCitusLocalTableKind(relationId);
EnsureTableNotDistributed(relationId);
/*
* When creating other citus table types, we don't need to check that case as
* EnsureTableNotDistributed already errors out if the given relation implies
* a citus table. However, as we don't mark the relation as citus table, i.e we
* do not use the relation with relationId as the shell relation, parallel
* create_citus_local_table executions would not error out for that relation.
* Hence we need to error out for shard relations too.
*/
ErrorIfRelationIsAKnownShard(relationId);
/*
* We do not allow creating citus local table if the table is involved in a
* foreign key relationship with "any other table". Note that we allow self
* references.
*/
ErrorIfTableHasExternalForeignKeys(relationId);
/* we do not support policies in citus community */
ErrorIfUnsupportedPolicy(relation);
}
/*
* ErrorIfUnsupportedCitusLocalTableKind errors out if the relation kind of
* relation with relationId is not supported for citus local table creation.
*/
static void
ErrorIfUnsupportedCitusLocalTableKind(Oid relationId)
{
const char *relationName = get_rel_name(relationId);
if (IsChildTable(relationId) || IsParentTable(relationId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create citus local table \"%s\", citus local "
"tables cannot be involved in inheritance relationships",
relationName)));
}
if (PartitionTable(relationId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create citus local table \"%s\", citus local "
"tables cannot be partition of other tables",
relationName)));
}
char relationKind = get_rel_relkind(relationId);
if (!(relationKind == RELKIND_RELATION || relationKind == RELKIND_FOREIGN_TABLE))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create citus local table \"%s\", only regular "
"tables and foreign tables are supported for citus local "
"table creation", relationName)));
}
}
/*
* GetShellTableDDLEventsForCitusLocalTable returns a list of DDL commands
* to create the shell table from scratch.
*/
static List *
GetShellTableDDLEventsForCitusLocalTable(Oid relationId)
{
/*
* As we don't allow foreign keys with other tables initially, below we
* only pick self-referencing foreign keys.
*/
List *foreignConstraintCommands =
GetReferencingForeignConstaintCommands(relationId);
/*
* Include DEFAULT clauses for columns getting their default values from
* a sequence.
*/
bool includeSequenceDefaults = true;
List *shellTableDDLEvents = GetTableDDLEvents(relationId,
includeSequenceDefaults);
shellTableDDLEvents = list_concat(shellTableDDLEvents, foreignConstraintCommands);
return shellTableDDLEvents;
}
/*
* ConvertLocalTableToShard first acquires a shardId and then converts the
* given relation with relationId to the shard relation with shardId. That
* means, this function suffixes shardId to:
* - relation name,
* - all the objects "defined on" the relation.
* After converting the given relation, returns the acquired shardId.
*/
static uint64
ConvertLocalTableToShard(Oid relationId)
{
uint64 shardId = GetNextShardId();
RenameRelationToShardRelation(relationId, shardId);
RenameShardRelationConstraints(relationId, shardId);
RenameShardRelationIndexes(relationId, shardId);
/*
* We do not create truncate triggers on shard relation. This is
* because truncate triggers are fired by utility hook and we would
* need to disable them to prevent executing them twice if we don't
* drop the trigger on shard relation.
*/
DropRelationTruncateTriggers(relationId);
/*
* We create INSERT|DELETE|UPDATE triggers on shard relation too.
* This is because citus prevents postgres executor to fire those
* triggers. So, here we suffix such triggers on shard relation
* with shardId.
*/
RenameShardRelationNonTruncateTriggers(relationId, shardId);
return shardId;
}
/*
* RenameRelationToShardRelation appends given shardId to the end of the name
* of relation with shellRelationId.
*/
static void
RenameRelationToShardRelation(Oid shellRelationId, uint64 shardId)
{
char *qualifiedShellRelationName = generate_qualified_relation_name(shellRelationId);
char *shellRelationName = get_rel_name(shellRelationId);
char *shardRelationName = pstrdup(shellRelationName);
AppendShardIdToName(&shardRelationName, shardId);
const char *quotedShardRelationName = quote_identifier(shardRelationName);
StringInfo renameCommand = makeStringInfo();
appendStringInfo(renameCommand, "ALTER TABLE %s RENAME TO %s;",
qualifiedShellRelationName, quotedShardRelationName);
ExecuteAndLogDDLCommand(renameCommand->data);
}
/*
* RenameShardRelationConstraints appends given shardId to the end of the name
* of constraints "defined on" the relation with shardRelationId. This function
* utilizes GetConstraintNameList to pick the constraints to be renamed,
* see more details in function's comment.
*/
static void
RenameShardRelationConstraints(Oid shardRelationId, uint64 shardId)
{
List *constraintNameList = GetConstraintNameList(shardRelationId);
char *constraintName = NULL;
foreach_ptr(constraintName, constraintNameList)
{
const char *commandString =
GetRenameShardConstraintCommand(shardRelationId, constraintName, shardId);
ExecuteAndLogDDLCommand(commandString);
}
}
/*
* GetConstraintNameList returns a list of constraint names "defined on" the
* relation with relationId. Those constraints can be:
* - "check" constraints or,
* - "primary key" constraints or,
* - "unique" constraints or,
* - "trigger" constraints or,
* - "exclusion" constraints or,
* - "foreign key" constraints in which the relation is the "referencing"
* relation (including the self-referencing foreign keys).
*/
static List *
GetConstraintNameList(Oid relationId)
{
List *constraintNameList = NIL;
int scanKeyCount = 1;
ScanKeyData scanKey[1];
Relation pgConstraint = table_open(ConstraintRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid,
BTEqualStrategyNumber, F_OIDEQ, relationId);
bool useIndex = true;
SysScanDesc scanDescriptor = systable_beginscan(pgConstraint,
ConstraintRelidTypidNameIndexId,
useIndex, NULL, scanKeyCount,
scanKey);
HeapTuple heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
char *constraintName = NameStr(constraintForm->conname);
constraintNameList = lappend(constraintNameList, pstrdup(constraintName));
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
table_close(pgConstraint, NoLock);
return constraintNameList;
}
/*
* GetRenameShardConstraintCommand returns DDL command to append given
* shardId to the constrant with constraintName on the relation with
* relationId.
*/
static char *
GetRenameShardConstraintCommand(Oid relationId, char *constraintName, uint64 shardId)
{
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
char *shardConstraintName = pstrdup(constraintName);
AppendShardIdToName(&shardConstraintName, shardId);
const char *quotedShardConstraintName = quote_identifier(shardConstraintName);
const char *quotedConstraintName = quote_identifier(constraintName);
StringInfo renameCommand = makeStringInfo();
appendStringInfo(renameCommand, "ALTER TABLE %s RENAME CONSTRAINT %s TO %s;",
qualifiedRelationName, quotedConstraintName,
quotedShardConstraintName);
return renameCommand->data;
}
/*
* RenameShardRelationIndexes appends given shardId to the end of the names
* of shard relation indexes except the ones that are already renamed via
* RenameShardRelationConstraints. This function utilizes
* GetExplicitIndexNameList to pick the indexes to be renamed, see more
* details in function's comment.
*/
static void
RenameShardRelationIndexes(Oid shardRelationId, uint64 shardId)
{
List *indexNameList = GetExplicitIndexNameList(shardRelationId);
char *indexName = NULL;
foreach_ptr(indexName, indexNameList)
{
const char *commandString = GetRenameShardIndexCommand(indexName, shardId);
ExecuteAndLogDDLCommand(commandString);
}
}
/*
* GetRenameShardIndexCommand returns DDL command to append given shardId to
* the index with indexName.
*/
static char *
GetRenameShardIndexCommand(char *indexName, uint64 shardId)
{
char *shardIndexName = pstrdup(indexName);
AppendShardIdToName(&shardIndexName, shardId);
const char *quotedShardIndexName = quote_identifier(shardIndexName);
const char *quotedIndexName = quote_identifier(indexName);
StringInfo renameCommand = makeStringInfo();
appendStringInfo(renameCommand, "ALTER INDEX %s RENAME TO %s;",
quotedIndexName, quotedShardIndexName);
return renameCommand->data;
}
/*
* RenameShardRelationNonTruncateTriggers appends given shardId to the end of
* the names of shard relation INSERT/DELETE/UPDATE triggers that are explicitly
* created.
*/
static void
RenameShardRelationNonTruncateTriggers(Oid shardRelationId, uint64 shardId)
{
List *triggerIdList = GetExplicitTriggerIdList(shardRelationId);
Oid triggerId = InvalidOid;
foreach_oid(triggerId, triggerIdList)
{
bool missingOk = false;
HeapTuple triggerTuple = GetTriggerTupleById(triggerId, missingOk);
Form_pg_trigger triggerForm = (Form_pg_trigger) GETSTRUCT(triggerTuple);
if (!TRIGGER_FOR_TRUNCATE(triggerForm->tgtype))
{
char *triggerName = NameStr(triggerForm->tgname);
char *commandString =
GetRenameShardTriggerCommand(shardRelationId, triggerName, shardId);
ExecuteAndLogDDLCommand(commandString);
}
heap_freetuple(triggerTuple);
}
}
/*
* GetRenameShardTriggerCommand returns DDL command to append given shardId to
* the trigger with triggerName.
*/
static char *
GetRenameShardTriggerCommand(Oid shardRelationId, char *triggerName, uint64 shardId)
{
char *qualifiedShardRelationName = generate_qualified_relation_name(shardRelationId);
char *shardTriggerName = pstrdup(triggerName);
AppendShardIdToName(&shardTriggerName, shardId);
const char *quotedShardTriggerName = quote_identifier(shardTriggerName);
const char *quotedTriggerName = quote_identifier(triggerName);
StringInfo renameCommand = makeStringInfo();
appendStringInfo(renameCommand, "ALTER TRIGGER %s ON %s RENAME TO %s;",
quotedTriggerName, qualifiedShardRelationName,
quotedShardTriggerName);
return renameCommand->data;
}
/*
* ExecuteAndLogDDLCommandList takes a list of ddl commands and calls
* ExecuteAndLogDDLCommand function for each of them.
*/
static void
ExecuteAndLogDDLCommandList(List *ddlCommandList)
{
char *ddlCommand = NULL;
foreach_ptr(ddlCommand, ddlCommandList)
{
ExecuteAndLogDDLCommand(ddlCommand);
}
}
/*
* ExecuteAndLogDDLCommand takes a ddl command and logs it in DEBUG4 log level.
* Then, parses and executes it via CitusProcessUtility.
*/
static void
ExecuteAndLogDDLCommand(const char *commandString)
{
ereport(DEBUG4, (errmsg("executing \"%s\"", commandString)));
Node *parseTree = ParseTreeNode(commandString);
CitusProcessUtility(parseTree, commandString, PROCESS_UTILITY_TOPLEVEL,
NULL, None_Receiver, NULL);
}
/*
* DropRelationTruncateTriggers drops TRUNCATE triggers that are explicitly
* created on relation with relationId.
*/
static void
DropRelationTruncateTriggers(Oid relationId)
{
List *triggerIdList = GetExplicitTriggerIdList(relationId);
Oid triggerId = InvalidOid;
foreach_oid(triggerId, triggerIdList)
{
bool missingOk = false;
HeapTuple triggerTuple = GetTriggerTupleById(triggerId, missingOk);
Form_pg_trigger triggerForm = (Form_pg_trigger) GETSTRUCT(triggerTuple);
if (TRIGGER_FOR_TRUNCATE(triggerForm->tgtype))
{
char *triggerName = NameStr(triggerForm->tgname);
char *commandString = GetDropTriggerCommand(relationId, triggerName);
ExecuteAndLogDDLCommand(commandString);
}
heap_freetuple(triggerTuple);
}
}
/*
* GetDropTriggerCommand returns DDL command to drop the trigger with triggerName
* on relationId.
*/
static char *
GetDropTriggerCommand(Oid relationId, char *triggerName)
{
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
const char *quotedTriggerName = quote_identifier(triggerName);
/*
* In postgres, the only possible object type that may depend on a trigger
* is the "constraint" object implied by the trigger itself if it is a
* constraint trigger, and it would be an internal dependency so it could
* be dropped without using CASCADE. Other than this, it is also possible
* to define dependencies on trigger via recordDependencyOn api by other
* extensions. We don't handle those kind of dependencies, we just drop
* them with CASCADE.
*/
StringInfo dropCommand = makeStringInfo();
appendStringInfo(dropCommand, "DROP TRIGGER %s ON %s CASCADE;",
quotedTriggerName, qualifiedRelationName);
return dropCommand->data;
}
/*
* GetExplicitIndexNameList returns a list of index names defined "explicitly"
* on the relation with relationId by the "CREATE INDEX" commands. That means,
* all the constraints defined on the relation except:
* - primary indexes,
* - unique indexes and
* - exclusion indexes
* that are actually applied by the related constraints.
*/
static List *
GetExplicitIndexNameList(Oid relationId)
{
int scanKeyCount = 1;
ScanKeyData scanKey[1];
PushOverrideEmptySearchPath(CurrentMemoryContext);
Relation pgIndex = table_open(IndexRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_index_indrelid,
BTEqualStrategyNumber, F_OIDEQ, relationId);
bool useIndex = true;
SysScanDesc scanDescriptor = systable_beginscan(pgIndex, IndexIndrelidIndexId,
useIndex, NULL, scanKeyCount,
scanKey);
List *indexNameList = NIL;
HeapTuple heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_index indexForm = (Form_pg_index) GETSTRUCT(heapTuple);
Oid indexId = indexForm->indexrelid;
bool indexImpliedByConstraint = IndexImpliedByAConstraint(indexForm);
/*
* Skip the indexes that are not implied by explicitly executing
* a CREATE INDEX command.
*/
if (!indexImpliedByConstraint)
{
char *indexName = get_rel_name(indexId);
indexNameList = lappend(indexNameList, pstrdup(indexName));
}
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
table_close(pgIndex, NoLock);
/* revert back to original search_path */
PopOverrideSearchPath();
return indexNameList;
}
/*
* DropAndMoveDefaultSequenceOwnerships drops default column definitions for
* relation with sourceRelationId. Also, for each column that defaults to an
* owned sequence, it grants ownership to the same named column of the relation
* with targetRelationId.
*/
static void
DropAndMoveDefaultSequenceOwnerships(Oid sourceRelationId, Oid targetRelationId)
{
List *columnNameList = NIL;
List *ownedSequenceIdList = NIL;
ExtractColumnsOwningSequences(sourceRelationId, &columnNameList,
&ownedSequenceIdList);
ListCell *columnNameCell = NULL;
ListCell *ownedSequenceIdCell = NULL;
forboth(columnNameCell, columnNameList, ownedSequenceIdCell, ownedSequenceIdList)
{
char *columnName = (char *) lfirst(columnNameCell);
Oid ownedSequenceId = lfirst_oid(ownedSequenceIdCell);
DropDefaultColumnDefinition(sourceRelationId, columnName);
/* column might not own a sequence */
if (OidIsValid(ownedSequenceId))
{
TransferSequenceOwnership(ownedSequenceId, targetRelationId, columnName);
}
}
}
/*
* ExtractColumnsOwningSequences finds each column of relation with relationId
* defaulting to an owned sequence. Then, appends the column name and id of the
* owned sequence -that the column defaults- to the lists passed as NIL initially.
*/
static void
ExtractColumnsOwningSequences(Oid relationId, List **columnNameList,
List **ownedSequenceIdList)
{
Assert(*columnNameList == NIL && *ownedSequenceIdList == NIL);
Relation relation = relation_open(relationId, AccessShareLock);
TupleDesc tupleDescriptor = RelationGetDescr(relation);
for (int attributeIndex = 0; attributeIndex < tupleDescriptor->natts;
attributeIndex++)
{
Form_pg_attribute attributeForm = TupleDescAttr(tupleDescriptor, attributeIndex);
if (attributeForm->attisdropped || !attributeForm->atthasdef)
{
/*
* If this column has already been dropped or it has no DEFAULT
* definition, skip it.
*/
continue;
}
char *columnName = NameStr(attributeForm->attname);
*columnNameList = lappend(*columnNameList, columnName);
List *columnOwnedSequences =
GetSequencesOwnedByColumn(relationId, attributeIndex + 1);
Oid ownedSequenceId = InvalidOid;
if (list_length(columnOwnedSequences) != 0)
{
/*
* A column might only own one sequence. We intentionally use
* GetSequencesOwnedByColumn macro and pick initial oid from the
* list instead of using getOwnedSequence. This is both because
* getOwnedSequence is removed in pg13 and is also because it
* errors out if column does not have any sequences.
*/
Assert(list_length(columnOwnedSequences) == 1);
ownedSequenceId = linitial_oid(columnOwnedSequences);
}
*ownedSequenceIdList = lappend_oid(*ownedSequenceIdList, ownedSequenceId);
}
relation_close(relation, NoLock);
}
/*
* DropDefaultColumnDefinition drops the DEFAULT definiton of the column with
* columnName of the relation with relationId via process utility.
*/
static void
DropDefaultColumnDefinition(Oid relationId, char *columnName)
{
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
const char *quotedColumnName = quote_identifier(columnName);
StringInfo sequenceDropCommand = makeStringInfo();
appendStringInfo(sequenceDropCommand,
"ALTER TABLE %s ALTER COLUMN %s DROP DEFAULT",
qualifiedRelationName, quotedColumnName);
ExecuteAndLogDDLCommand(sequenceDropCommand->data);
}
/*
* TransferSequenceOwnership grants ownership of the sequence with sequenceId
* to the column with targetColumnName of relation with targetRelationId via
* process utility. Note that this function assumes that the target relation
* has a column with targetColumnName which can default to the given sequence.
*/
static void
TransferSequenceOwnership(Oid sequenceId, Oid targetRelationId, char *targetColumnName)
{
char *qualifiedSequenceName = generate_qualified_relation_name(sequenceId);
char *qualifiedTargetRelationName =
generate_qualified_relation_name(targetRelationId);
const char *quotedTargetColumnName = quote_identifier(targetColumnName);
StringInfo sequenceOwnershipCommand = makeStringInfo();
appendStringInfo(sequenceOwnershipCommand, "ALTER SEQUENCE %s OWNED BY %s.%s",
qualifiedSequenceName, qualifiedTargetRelationName,
quotedTargetColumnName);
ExecuteAndLogDDLCommand(sequenceOwnershipCommand->data);
}
/*
* InsertMetadataForCitusLocalTable inserts necessary metadata for the citus
* local table to the following metadata tables:
* pg_dist_partition, pg_dist_shard & pg_dist_placement.
*/
static void
InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId)
{
Assert(OidIsValid(citusLocalTableId));
Assert(shardId != INVALID_SHARD_ID);
char distributionMethod = DISTRIBUTE_BY_NONE;
char replicationModel = ReplicationModel;
Assert(replicationModel != REPLICATION_MODEL_2PC);
uint32 colocationId = INVALID_COLOCATION_ID;
Var *distributionColumn = NULL;
InsertIntoPgDistPartition(citusLocalTableId, distributionMethod,
distributionColumn, colocationId,
replicationModel);
/* set shard storage type according to relation type */
char shardStorageType = ShardStorageType(citusLocalTableId);
text *shardMinValue = NULL;
text *shardMaxValue = NULL;
InsertShardRow(citusLocalTableId, shardId, shardStorageType,
shardMinValue, shardMaxValue);
List *nodeList = list_make1(CoordinatorNodeIfAddedAsWorkerOrError());
int replicationFactor = 1;
int workerStartIndex = 0;
InsertShardPlacementRows(citusLocalTableId, shardId, nodeList,
workerStartIndex, replicationFactor);
}
/*
* FinalizeCitusLocalTableCreation completes creation of the citus local table
* with relationId by performing operations that should be done after creating
* the shard and inserting the metadata.
*/
static void
FinalizeCitusLocalTableCreation(Oid relationId)
{
/*
* If it is a foreign table, then skip creating citus truncate trigger
* as foreign tables do not support truncate triggers.
*/
if (RegularTable(relationId))
{
CreateTruncateTrigger(relationId);
}
if (ShouldSyncTableMetadata(relationId))
{
CreateTableMetadataOnWorkers(relationId);
}
/*
* We've a custom way of foreign key graph invalidation,
* see InvalidateForeignKeyGraph().
*/
if (TableReferenced(relationId) || TableReferencing(relationId))
{
InvalidateForeignKeyGraph();
}
}

View File

@ -103,7 +103,6 @@ static void EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
Oid distributionColumnType, Oid distributionColumnType,
Oid sourceRelationId); Oid sourceRelationId);
static void EnsureLocalTableEmpty(Oid relationId); static void EnsureLocalTableEmpty(Oid relationId);
static void EnsureTableNotDistributed(Oid relationId);
static void EnsureRelationHasNoTriggers(Oid relationId); static void EnsureRelationHasNoTriggers(Oid relationId);
static Oid SupportFunctionForColumn(Var *partitionColumn, Oid accessMethodId, static Oid SupportFunctionForColumn(Var *partitionColumn, Oid accessMethodId,
int16 supportFunctionNumber); int16 supportFunctionNumber);
@ -398,7 +397,7 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
InsertIntoPgDistPartition(relationId, distributionMethod, distributionColumn, InsertIntoPgDistPartition(relationId, distributionMethod, distributionColumn,
colocationId, replicationModel); colocationId, replicationModel);
/* foreign tables does not support TRUNCATE trigger */ /* foreign tables do not support TRUNCATE trigger */
if (RegularTable(relationId)) if (RegularTable(relationId))
{ {
CreateTruncateTrigger(relationId); CreateTruncateTrigger(relationId);
@ -424,7 +423,6 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
CreateReferenceTableShard(relationId); CreateReferenceTableShard(relationId);
} }
if (ShouldSyncTableMetadata(relationId)) if (ShouldSyncTableMetadata(relationId))
{ {
CreateTableMetadataOnWorkers(relationId); CreateTableMetadataOnWorkers(relationId);
@ -820,8 +818,8 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
} }
} }
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn, ErrorIfUnsupportedConstraint(relation, distributionMethod, replicationModel,
colocationId); distributionColumn, colocationId);
ErrorIfUnsupportedPolicy(relation); ErrorIfUnsupportedPolicy(relation);
@ -955,7 +953,7 @@ EnsureLocalTableEmpty(Oid relationId)
/* /*
* EnsureTableNotDistributed errors out if the table is distributed. * EnsureTableNotDistributed errors out if the table is distributed.
*/ */
static void void
EnsureTableNotDistributed(Oid relationId) EnsureTableNotDistributed(Oid relationId)
{ {
char *relationName = get_rel_name(relationId); char *relationName = get_rel_name(relationId);
@ -989,7 +987,8 @@ EnsureReplicationSettings(Oid relationId, char replicationModel)
extraHint = ""; extraHint = "";
} }
if (replicationModel == REPLICATION_MODEL_STREAMING && ShardReplicationFactor != 1) if (replicationModel == REPLICATION_MODEL_STREAMING &&
DistributedTableReplicationIsEnabled())
{ {
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("replication factors above one are incompatible with %s", errmsg("replication factors above one are incompatible with %s",

View File

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

View File

@ -36,23 +36,23 @@
#include "utils/ruleutils.h" #include "utils/ruleutils.h"
#include "utils/syscache.h" #include "utils/syscache.h"
/*
* Flags that can be passed to GetForeignKeyOids to indicate
* which foreign key constraint OIDs are to be extracted
*/
typedef enum ExtractForeignKeyConstrainstMode
{
/* extract the foreign key OIDs where the table is the referencing one */
INCLUDE_REFERENCING_CONSTRAINTS = 1 << 0,
/* extract the foreign key OIDs the table is the referenced one */ #define BehaviorIsRestrictOrNoAction(x) \
INCLUDE_REFERENCED_CONSTRAINTS = 1 << 1, ((x) == FKCONSTR_ACTION_NOACTION || (x) == FKCONSTR_ACTION_RESTRICT)
typedef bool (*CheckRelationFunc)(Oid);
/* exclude the self-referencing foreign keys */
EXCLUDE_SELF_REFERENCES = 1 << 2
} ExtractForeignKeyConstraintMode;
/* Local functions forward declarations */ /* 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, static bool HeapTupleOfForeignConstraintIncludesColumn(HeapTuple heapTuple,
Oid relationId, Oid relationId,
int pgConstraintKey, int pgConstraintKey,
@ -66,8 +66,10 @@ static void ForeignConstraintFindDistKeys(HeapTuple pgConstraintTuple,
int *referencedAttrIndex); int *referencedAttrIndex);
static List * GetForeignConstraintCommandsInternal(Oid relationId, int flags); static List * GetForeignConstraintCommandsInternal(Oid relationId, int flags);
static Oid get_relation_constraint_oid_compat(HeapTuple heapTuple); static Oid get_relation_constraint_oid_compat(HeapTuple heapTuple);
static List * GetForeignKeyOidsToCitusLocalTables(Oid relationId);
static List * GetForeignKeyOidsToReferenceTables(Oid relationId); static List * GetForeignKeyOidsToReferenceTables(Oid relationId);
static List * GetForeignKeyOids(Oid relationId, int flags); static List * FilterFKeyOidListByReferencedTableType(List *foreignKeyOidList,
CitusTableType citusTableType);
/* /*
* ConstraintIsAForeignKeyToReferenceTable checks if the given constraint is a * ConstraintIsAForeignKeyToReferenceTable checks if the given constraint is a
@ -102,27 +104,21 @@ ConstraintIsAForeignKeyToReferenceTable(char *inputConstaintName, Oid relationId
* ON UPDATE CASCADE options are not used on the distribution key * ON UPDATE CASCADE options are not used on the distribution key
* of the referencing column. * of the referencing column.
* - If referencing table is a reference table, error out if the referenced * - 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 void
ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDistMethod, ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDistMethod,
char referencingReplicationModel,
Var *referencingDistKey, Var *referencingDistKey,
uint32 referencingColocationId) uint32 referencingColocationId)
{ {
Oid referencingTableId = relation->rd_id; 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; int flags = INCLUDE_REFERENCING_CONSTRAINTS;
List *foreignKeyOids = GetForeignKeyOids(referencingTableId, flags); List *foreignKeyOids = GetForeignKeyOids(referencingTableId, flags);
@ -149,6 +145,12 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
if (!referencedIsCitus && !selfReferencingTable) if (!referencedIsCitus && !selfReferencingTable)
{ {
if (IsCitusLocalTableByDistParams(referencingDistMethod,
referencingReplicationModel))
{
ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(referencedTableId);
}
char *referencedTableName = get_rel_name(referencedTableId); char *referencedTableName = get_rel_name(referencedTableId);
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION), ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
@ -165,6 +167,7 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
/* set referenced table related variables here if table is referencing itself */ /* set referenced table related variables here if table is referencing itself */
char referencedDistMethod = 0; char referencedDistMethod = 0;
char referencedReplicationModel = REPLICATION_MODEL_INVALID;
if (!selfReferencingTable) if (!selfReferencingTable)
{ {
referencedDistMethod = PartitionMethod(referencedTableId); referencedDistMethod = PartitionMethod(referencedTableId);
@ -173,45 +176,51 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
NULL : NULL :
DistPartitionKey(referencedTableId); DistPartitionKey(referencedTableId);
referencedColocationId = TableColocationId(referencedTableId); referencedColocationId = TableColocationId(referencedTableId);
referencedReplicationModel = TableReplicationModel(referencedTableId);
} }
else else
{ {
referencedDistMethod = referencingDistMethod; referencedDistMethod = referencingDistMethod;
referencedDistKey = referencingDistKey; referencedDistKey = referencingDistKey;
referencedColocationId = referencingColocationId; referencedColocationId = referencingColocationId;
referencedReplicationModel = referencingReplicationModel;
} }
bool referencingIsReferenceTable = (referencingDistMethod == DISTRIBUTE_BY_NONE); bool referencingIsCitusLocalOrRefTable =
bool referencedIsReferenceTable = (referencedDistMethod == DISTRIBUTE_BY_NONE); (referencingDistMethod == DISTRIBUTE_BY_NONE);
bool referencedIsCitusLocalOrRefTable =
/* (referencedDistMethod == DISTRIBUTE_BY_NONE);
* We support foreign keys between reference tables. No more checks if (referencingIsCitusLocalOrRefTable && referencedIsCitusLocalOrRefTable)
* are necessary.
*/
if (referencingIsReferenceTable && referencedIsReferenceTable)
{ {
EnsureSupportedFKeyBetweenCitusLocalAndRefTable(constraintForm,
referencingReplicationModel,
referencedReplicationModel);
ReleaseSysCache(heapTuple); ReleaseSysCache(heapTuple);
continue; continue;
} }
/* /*
* Foreign keys from reference tables to distributed tables are not * Foreign keys from citus local tables or reference tables to distributed
* supported. * tables are not supported.
*/ */
if (referencingIsReferenceTable && !referencedIsReferenceTable) if (referencingIsCitusLocalOrRefTable && !referencedIsCitusLocalOrRefTable)
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint " errmsg("cannot create foreign key constraint "
"since foreign keys from reference tables " "since foreign keys from reference tables "
"to distributed tables are not supported"), "to distributed tables are not supported"),
errdetail("A reference table can only have reference " errdetail("A reference table can only have foreign "
"keys to other reference tables"))); "keys to other reference tables or citus "
"local tables")));
} }
/* /*
* To enforce foreign constraints, tables must be co-located unless a * To enforce foreign constraints, tables must be co-located unless a
* reference table is referenced. * reference table is referenced.
*/ */
bool referencedIsReferenceTable =
(referencedReplicationModel == REPLICATION_MODEL_2PC);
if (referencingColocationId == INVALID_COLOCATION_ID || if (referencingColocationId == INVALID_COLOCATION_ID ||
(referencingColocationId != referencedColocationId && (referencingColocationId != referencedColocationId &&
!referencedIsReferenceTable)) !referencedIsReferenceTable))
@ -243,44 +252,14 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
*/ */
if (referencingColumnsIncludeDistKey) if (referencingColumnsIncludeDistKey)
{ {
/* EnsureSupportedFKeyOnDistKey(constraintForm);
* 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.")));
}
} }
/* /*
* if tables are hash-distributed and colocated, we need to make sure that * if tables are hash-distributed and colocated, we need to make sure that
* the distribution key is included in foreign constraint. * the distribution key is included in foreign constraint.
*/ */
if (!referencedIsReferenceTable && !foreignConstraintOnDistKey) if (!referencedIsCitusLocalOrRefTable && !foreignConstraintOnDistKey)
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"), errmsg("cannot create foreign key constraint"),
@ -300,6 +279,117 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
* placements always be in the same state (b) executors are aware of reference * placements always be in the same state (b) executors are aware of reference
* tables and handle concurrency related issues accordingly. * tables and handle concurrency related issues accordingly.
*/ */
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) if (!referencingNotReplicated)
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
@ -312,9 +402,25 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
"other replication factors, please contact us at " "other replication factors, please contact us at "
"https://citusdata.com/about/contact_us."))); "https://citusdata.com/about/contact_us.")));
} }
}
ReleaseSysCache(heapTuple);
} /*
* 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)));
} }
@ -521,8 +627,35 @@ get_relation_constraint_oid_compat(HeapTuple heapTuple)
/* /*
* HasForeignKeyToReferenceTable function returns true if any of the foreign * HasForeignKeyToCitusLocalTable returns true if any of the foreign key constraints
* key constraints on the relation with relationId references to a reference * 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. * table.
*/ */
bool bool
@ -535,40 +668,47 @@ HasForeignKeyToReferenceTable(Oid relationId)
/* /*
* GetForeignKeyOidsToReferenceTables function returns list of OIDs for the * GetForeignKeyOidsToReferenceTables returns list of OIDs for the foreign key
* foreign key constraints on the given relationId that are referencing to * constraints on the given relationId that are referencing to reference tables.
* reference tables.
*/ */
static List * static List *
GetForeignKeyOidsToReferenceTables(Oid relationId) GetForeignKeyOidsToReferenceTables(Oid relationId)
{ {
int flags = INCLUDE_REFERENCING_CONSTRAINTS; 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; Oid foreignKeyOid = InvalidOid;
foreach_oid(foreignKeyOid, foreignKeyOids) foreach_oid(foreignKeyOid, foreignKeyOidList)
{ {
HeapTuple heapTuple = HeapTuple heapTuple = SearchSysCache1(CONSTROID, ObjectIdGetDatum(foreignKeyOid));
SearchSysCache1(CONSTROID, ObjectIdGetDatum(foreignKeyOid));
Assert(HeapTupleIsValid(heapTuple));
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple); Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
Oid referencedTableOid = constraintForm->confrelid; Oid referencedTableOid = constraintForm->confrelid;
if (IsCitusTableType(referencedTableOid, citusTableType))
if (IsCitusTableType(referencedTableOid, REFERENCE_TABLE))
{ {
fkeyOidsToReferenceTables = lappend_oid(fkeyOidsToReferenceTables, filteredFKeyOidList = lappend_oid(filteredFKeyOidList, foreignKeyOid);
foreignKeyOid);
} }
ReleaseSysCache(heapTuple); ReleaseSysCache(heapTuple);
} }
return fkeyOidsToReferenceTables; return filteredFKeyOidList;
} }
@ -633,18 +773,31 @@ TableReferencing(Oid relationId)
/* /*
* ConstraintIsAForeignKey returns true if the given constraint name * ConstraintIsAForeignKey is a wrapper around GetForeignKeyOidByName that
* is a foreign key defined on the relation. * returns true if the given constraint name identifies a foreign key
* constraint defined on relation with relationId.
*/ */
bool bool
ConstraintIsAForeignKey(char *inputConstaintName, Oid relationId) 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; int flags = INCLUDE_REFERENCING_CONSTRAINTS;
List *foreignKeyOids = GetForeignKeyOids(relationId, flags); List *foreignKeyOids = GetForeignKeyOids(relationId, flags);
Oid foreignKeyOid = FindForeignKeyOidWithName(foreignKeyOids, inputConstaintName); Oid foreignKeyId = FindForeignKeyOidWithName(foreignKeyOids, inputConstaintName);
return foreignKeyId;
return OidIsValid(foreignKeyOid);
} }
@ -674,13 +827,44 @@ FindForeignKeyOidWithName(List *foreignKeyOids, const char *inputConstraintName)
} }
/*
* ErrorIfTableHasExternalForeignKeys errors out if the relation with relationId
* is involved in a foreign key relationship other than the self-referencing ones.
*/
void
ErrorIfTableHasExternalForeignKeys(Oid relationId)
{
int flags = (INCLUDE_REFERENCING_CONSTRAINTS | EXCLUDE_SELF_REFERENCES);
List *foreignKeyIdsTableReferencing = GetForeignKeyOids(relationId, flags);
flags = (INCLUDE_REFERENCED_CONSTRAINTS | EXCLUDE_SELF_REFERENCES);
List *foreignKeyIdsTableReferenced = GetForeignKeyOids(relationId, flags);
List *foreignKeysWithOtherTables = list_concat(foreignKeyIdsTableReferencing,
foreignKeyIdsTableReferenced);
if (list_length(foreignKeysWithOtherTables) == 0)
{
return;
}
const char *relationName = get_rel_name(relationId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("relation \"%s\" is involved in a foreign key relationship "
"with another table", relationName),
errhint("Drop foreign keys with other tables and re-define them "
"with ALTER TABLE commands after the current operation "
"is done.")));
}
/* /*
* GetForeignKeyOids takes in a relationId, and returns a list of OIDs for * GetForeignKeyOids takes in a relationId, and returns a list of OIDs for
* foreign constraints that the relation with relationId is involved according * foreign constraints that the relation with relationId is involved according
* to "flags" argument. See ExtractForeignKeyConstrainstMode enum definition * to "flags" argument. See ExtractForeignKeyConstraintsMode enum definition
* for usage of the flags. * for usage of the flags.
*/ */
static List * List *
GetForeignKeyOids(Oid relationId, int flags) GetForeignKeyOids(Oid relationId, int flags)
{ {
AttrNumber pgConstraintTargetAttrNumber = InvalidAttrNumber; AttrNumber pgConstraintTargetAttrNumber = InvalidAttrNumber;
@ -774,3 +958,27 @@ GetForeignKeyOids(Oid relationId, int flags)
return foreignKeyOids; 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 */ /* 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 ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement);
static void ErrorIfCitusLocalTablePartitionCommand(AlterTableCmd *alterTableCmd,
Oid parentRelationId);
static Oid GetPartitionCommandChildRelationId(AlterTableCmd *alterTableCmd,
bool missingOk);
static List * InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId, static List * InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
const char *commandString); const char *commandString);
static bool AlterInvolvesPartitionColumn(AlterTableStmt *alterTableStatement, static bool AlterInvolvesPartitionColumn(AlterTableStmt *alterTableStatement,
AlterTableCmd *command); AlterTableCmd *command);
static void ErrorIfUnsupportedAlterAddConstraintStmt(AlterTableStmt *alterTableStatement); 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 * We need to run some of the commands sequentially if there is a foreign constraint
@ -130,26 +150,61 @@ PreprocessDropTableStmt(Node *node, const char *queryString)
/* /*
* PostprocessCreateTableStmtPartitionOf takes CreateStmt object as a parameter * PostprocessCreateTableStmt takes CreateStmt object as a parameter and errors
* but it only processes CREATE TABLE ... PARTITION OF statements and it checks * out if it creates a table with a foreign key that references to a citus local
* if user creates the table as a partition of a distributed table. In that case, * table if pg version is older than 13 (see comment in function).
* it distributes partition as well. Since the table itself is a partition,
* CreateDistributedTable will attach it to its parent table automatically after
* distributing it.
* *
* This function does nothing if the provided CreateStmt is not a CREATE TABLE ... * This function also processes CREATE TABLE ... PARTITION OF statements via
* PARTITION OF command. * 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 PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const
char *queryString) char *queryString)
{ {
if (createStatement->inhRelations != NIL && createStatement->partbound != NULL)
{
RangeVar *parentRelation = linitial(createStatement->inhRelations); RangeVar *parentRelation = linitial(createStatement->inhRelations);
bool parentMissingOk = false; bool missingOk = false;
Oid parentRelationId = RangeVarGetRelid(parentRelation, NoLock, Oid parentRelationId = RangeVarGetRelid(parentRelation, NoLock, missingOk);
parentMissingOk);
/* a partition can only inherit from single parent table */ /* a partition can only inherit from single parent table */
Assert(list_length(createStatement->inhRelations) == 1); Assert(list_length(createStatement->inhRelations) == 1);
@ -162,9 +217,7 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const
*/ */
if (IsCitusTable(parentRelationId)) if (IsCitusTable(parentRelationId))
{ {
bool missingOk = false; Oid relationId = RangeVarGetRelid(createStatement->relation, NoLock, missingOk);
Oid relationId = RangeVarGetRelid(createStatement->relation, NoLock,
missingOk);
Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId); Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId);
char parentDistributionMethod = DISTRIBUTE_BY_HASH; char parentDistributionMethod = DISTRIBUTE_BY_HASH;
char *parentRelationName = generate_qualified_relation_name(parentRelationId); char *parentRelationName = generate_qualified_relation_name(parentRelationId);
@ -174,9 +227,6 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const
parentDistributionMethod, parentRelationName, parentDistributionMethod, parentRelationName,
viaDeprecatedAPI); viaDeprecatedAPI);
} }
}
return NIL;
} }
@ -325,6 +375,15 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
leftRelationId = IndexGetRelation(leftRelationId, missingOk); 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); bool referencingIsLocalTable = !IsCitusTable(leftRelationId);
if (referencingIsLocalTable) if (referencingIsLocalTable)
{ {
@ -375,7 +434,7 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
* only subcommand of ALTER TABLE. It was already checked in * only subcommand of ALTER TABLE. It was already checked in
* ErrorIfUnsupportedAlterTableStmt. * ErrorIfUnsupportedAlterTableStmt.
*/ */
Assert(list_length(commandList) <= 1); Assert(list_length(commandList) == 1);
rightRelationId = RangeVarGetRelid(constraint->pktable, lockmode, rightRelationId = RangeVarGetRelid(constraint->pktable, lockmode,
alterTableStatement->missing_ok); alterTableStatement->missing_ok);
@ -390,6 +449,22 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
constraint->skip_validation = true; 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) else if (alterTableType == AT_AddColumn)
{ {
/* /*
@ -456,6 +531,18 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand)
rightRelationId = RangeVarGetRelid(partitionCommand->name, NoLock, false); 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 * 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. * PreprocessAlterTableStmt issues a warning.
* ALTER TABLE ALL IN TABLESPACE statements have their node type as * ALTER TABLE ALL IN TABLESPACE statements have their node type as
@ -868,6 +1080,7 @@ ErrorUnsupportedAlterTableAddColumn(Oid relationId, AlterTableCmd *command,
*/ */
void void
ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod, ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
char referencingReplicationModel,
Var *distributionColumn, uint32 colocationId) 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. * and if they are OK, we do not error out for other types of constraints.
*/ */
ErrorIfUnsupportedForeignConstraintExists(relation, distributionMethod, ErrorIfUnsupportedForeignConstraintExists(relation, distributionMethod,
referencingReplicationModel,
distributionColumn, distributionColumn,
colocationId); colocationId);
@ -987,12 +1201,17 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
* ALTER TABLE REPLICA IDENTITY * ALTER TABLE REPLICA IDENTITY
* ALTER TABLE SET () * ALTER TABLE SET ()
* ALTER TABLE RESET () * ALTER TABLE RESET ()
* ALTER TABLE ENABLE/DISABLE TRIGGER (only for citus local tables)
*/ */
static void static void
ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement) ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
{ {
/* error out if any of the subcommands are unsupported */
List *commandList = alterTableStatement->cmds; 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; AlterTableCmd *command = NULL;
foreach_ptr(command, commandList) foreach_ptr(command, commandList)
{ {
@ -1074,12 +1293,10 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
case AT_AttachPartition: case AT_AttachPartition:
{ {
Oid relationId = AlterTableLookupRelation(alterTableStatement,
NoLock);
PartitionCmd *partitionCommand = (PartitionCmd *) command->def; PartitionCmd *partitionCommand = (PartitionCmd *) command->def;
bool missingOK = false; bool missingOK = false;
Oid partitionRelationId = RangeVarGetRelid(partitionCommand->name, Oid partitionRelationId = RangeVarGetRelid(partitionCommand->name,
NoLock, missingOK); lockmode, missingOK);
/* we only allow partitioning commands if they are only subcommand */ /* we only allow partitioning commands if they are only subcommand */
if (commandList->length > 1) if (commandList->length > 1)
@ -1091,6 +1308,8 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
"separately."))); "separately.")));
} }
ErrorIfCitusLocalTablePartitionCommand(command, relationId);
if (IsCitusTable(partitionRelationId) && if (IsCitusTable(partitionRelationId) &&
!TablesColocated(relationId, partitionRelationId)) !TablesColocated(relationId, partitionRelationId))
{ {
@ -1115,14 +1334,13 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
"separately."))); "separately.")));
} }
ErrorIfCitusLocalTablePartitionCommand(command, relationId);
break; break;
} }
case AT_DropConstraint: case AT_DropConstraint:
{ {
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId)) if (!OidIsValid(relationId))
{ {
return; return;
@ -1136,17 +1354,41 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
break; 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_EnableTrigAll:
case AT_DisableTrigAll: 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_ReplicaIdentity:
case AT_ValidateConstraint: case AT_ValidateConstraint:
{ {
/* /*
* We will not perform any special check for ALTER TABLE DROP CONSTRAINT * We will not perform any special check for:
* , ALTER TABLE .. ALTER COLUMN .. SET NOT NULL and ALTER TABLE ENABLE/ * ALTER TABLE .. ALTER COLUMN .. SET NOT NULL
* DISABLE TRIGGER ALL, ALTER TABLE .. REPLICA IDENTITY .., ALTER TABLE * ALTER TABLE .. REPLICA IDENTITY ..
* .. VALIDATE CONSTRAINT .. * ALTER TABLE .. VALIDATE CONSTRAINT ..
*/ */
break; 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 * SetupExecutionModeForAlterTable is the function that is responsible
* for two things for practical purpose for not doing the same checks * 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 * the distributed tables, thus contradicting our purpose of using
* sequential mode. * sequential mode.
*/ */
if (executeSequentially && !IsCitusTableType(relationId, REFERENCE_TABLE) && if (executeSequentially &&
!IsCitusTableType(relationId, CITUS_TABLE_WITH_NO_DIST_KEY) &&
ParallelQueryExecutedInTransaction()) ParallelQueryExecutedInTransaction())
{ {
char *relationName = get_rel_name(relationId); char *relationName = get_rel_name(relationId);
@ -1320,65 +1609,40 @@ static List *
InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId, InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
const char *commandString) const char *commandString)
{ {
List *taskList = NIL;
List *leftShardList = LoadShardIntervalList(leftRelationId); 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); Oid leftSchemaId = get_rel_namespace(leftRelationId);
char *leftSchemaName = get_namespace_name(leftSchemaId); char *leftSchemaName = get_namespace_name(leftSchemaId);
char *escapedLeftSchemaName = quote_literal_cstr(leftSchemaName); char *escapedLeftSchemaName = quote_literal_cstr(leftSchemaName);
List *rightShardList = LoadShardIntervalList(rightRelationId);
ListCell *rightShardCell = NULL;
Oid rightSchemaId = get_rel_namespace(rightRelationId); Oid rightSchemaId = get_rel_namespace(rightRelationId);
char *rightSchemaName = get_namespace_name(rightSchemaId); char *rightSchemaName = get_namespace_name(rightSchemaId);
char *escapedRightSchemaName = quote_literal_cstr(rightSchemaName); char *escapedRightSchemaName = quote_literal_cstr(rightSchemaName);
char *escapedCommandString = quote_literal_cstr(commandString); char *escapedCommandString = quote_literal_cstr(commandString);
uint64 jobId = INVALID_JOB_ID;
int taskId = 1;
/* List *taskList = NIL;
* 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);
ListCell *leftShardCell = NULL;
ListCell *rightShardCell = NULL;
forboth(leftShardCell, leftShardList, rightShardCell, rightShardList) forboth(leftShardCell, leftShardList, rightShardCell, rightShardList)
{ {
ShardInterval *leftShardInterval = (ShardInterval *) lfirst(leftShardCell); 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); ShardInterval *rightShardInterval = (ShardInterval *) lfirst(rightShardCell);
uint64 leftShardId = leftShardInterval->shardId;
uint64 rightShardId = rightShardInterval->shardId; uint64 rightShardId = rightShardInterval->shardId;
leftRelationShard->relationId = leftRelationId; StringInfo applyCommand = makeStringInfo();
leftRelationShard->shardId = leftShardId;
rightRelationShard->relationId = rightRelationId;
rightRelationShard->shardId = rightShardId;
appendStringInfo(applyCommand, WORKER_APPLY_INTER_SHARD_DDL_COMMAND, appendStringInfo(applyCommand, WORKER_APPLY_INTER_SHARD_DDL_COMMAND,
leftShardId, escapedLeftSchemaName, rightShardId, leftShardId, escapedLeftSchemaName, rightShardId,
escapedRightSchemaName, escapedCommandString); escapedRightSchemaName, escapedCommandString);
@ -1391,8 +1655,9 @@ InterShardDDLTaskList(Oid leftRelationId, Oid rightRelationId,
task->dependentTaskList = NULL; task->dependentTaskList = NULL;
task->replicationModel = REPLICATION_MODEL_INVALID; task->replicationModel = REPLICATION_MODEL_INVALID;
task->anchorShardId = leftShardId; task->anchorShardId = leftShardId;
task->taskPlacementList = ActiveShardPlacementList(leftShardId); SetInterShardDDLTaskPlacementList(task, leftShardInterval, rightShardInterval);
task->relationShardList = list_make2(leftRelationShard, rightRelationShard); SetInterShardDDLTaskRelationShardList(task, leftShardInterval,
rightShardInterval);
taskList = lappend(taskList, task); 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 * AlterInvolvesPartitionColumn checks if the given alter table command
* involves relation's partition column. * involves relation's partition column.
@ -1450,12 +1796,14 @@ ErrorIfUnsupportedAlterAddConstraintStmt(AlterTableStmt *alterTableStatement)
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode); Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
char distributionMethod = PartitionMethod(relationId); char distributionMethod = PartitionMethod(relationId);
char referencingReplicationModel = TableReplicationModel(relationId);
Var *distributionColumn = DistPartitionKey(relationId); Var *distributionColumn = DistPartitionKey(relationId);
uint32 colocationId = TableColocationId(relationId); uint32 colocationId = TableColocationId(relationId);
Relation relation = relation_open(relationId, ExclusiveLock); Relation relation = relation_open(relationId, ExclusiveLock);
ErrorIfUnsupportedConstraint(relation, distributionMethod, distributionColumn, ErrorIfUnsupportedConstraint(relation, distributionMethod,
colocationId); referencingReplicationModel,
distributionColumn, colocationId);
relation_close(relation, NoLock); relation_close(relation, NoLock);
} }

View File

@ -17,17 +17,44 @@
#else #else
#include "access/heapam.h" #include "access/heapam.h"
#include "access/htup_details.h" #include "access/htup_details.h"
#include "access/sysattr.h"
#endif #endif
#include "catalog/indexing.h" #include "catalog/indexing.h"
#include "catalog/namespace.h" #include "catalog/namespace.h"
#include "catalog/pg_trigger.h" #include "catalog/pg_trigger.h"
#include "commands/trigger.h"
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/commands.h" #include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/deparser.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/namespace_utils.h" #include "distributed/namespace_utils.h"
#include "distributed/shard_utils.h"
#include "distributed/worker_protocol.h"
#include "utils/fmgroids.h" #include "utils/fmgroids.h"
#include "utils/lsyscache.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 * GetExplicitTriggerCommandList returns the list of DDL commands to create
@ -59,6 +86,54 @@ GetExplicitTriggerCommandList(Oid relationId)
} }
/*
* 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.
*/
HeapTuple
GetTriggerTupleById(Oid triggerId, bool missingOk)
{
Relation pgTrigger = table_open(TriggerRelationId, AccessShareLock);
int scanKeyCount = 1;
ScanKeyData scanKey[1];
#if PG_VERSION_NUM >= PG_VERSION_12
AttrNumber attrNumber = Anum_pg_trigger_oid;
#else
AttrNumber attrNumber = ObjectIdAttributeNumber;
#endif
ScanKeyInit(&scanKey[0], attrNumber, BTEqualStrategyNumber,
F_OIDEQ, ObjectIdGetDatum(triggerId));
bool useIndex = true;
SysScanDesc scanDescriptor = systable_beginscan(pgTrigger, TriggerOidIndexId,
useIndex, NULL, scanKeyCount,
scanKey);
HeapTuple targetHeapTuple = NULL;
HeapTuple heapTuple = systable_getnext(scanDescriptor);
if (HeapTupleIsValid(heapTuple))
{
targetHeapTuple = heap_copytuple(heapTuple);
}
systable_endscan(scanDescriptor);
table_close(pgTrigger, NoLock);
if (targetHeapTuple == NULL && missingOk == false)
{
ereport(ERROR, (errmsg("could not find heap tuple for trigger with "
"OID %d", triggerId)));
}
return targetHeapTuple;
}
/* /*
* GetExplicitTriggerIdList returns a list of OIDs corresponding to the triggers * GetExplicitTriggerIdList returns a list of OIDs corresponding to the triggers
* that are explicitly created on the relation with relationId. That means, * that are explicitly created on the relation with relationId. That means,
@ -135,45 +210,531 @@ get_relation_trigger_oid_compat(HeapTuple heapTuple)
/* /*
* ErrorIfUnsupportedCreateTriggerCommand errors out for the CREATE TRIGGER * PostprocessCreateTriggerStmt is called after a CREATE TRIGGER command has
* command that is run for a citus table if it is not citus_truncate_trigger. * 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 * Never returns NULL, but the objid in the address can be invalid if missingOk
* foreign key validation already wouldn't be executed via process utility, * was set to true.
* hence there is no need to check that case here. */
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 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; bool missingOk = true;
Oid relationId = RangeVarGetRelid(triggerRelation, AccessShareLock, missingOk); Oid relationId = RangeVarGetRelid(relation, DROP_TRIGGER_LOCK_MODE, missingOk);
if (!OidIsValid(relationId)) if (!OidIsValid(relationId))
{ {
/* /* let standard process utility to error out */
* standard_ProcessUtility would already error out if the given table return NIL;
* does not exist
*/
return;
} }
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)) if (!IsCitusTable(relationId))
{ {
return; return;
} }
char *functionName = makeRangeVarFromNameList(createTriggerStmt->funcname)->relname; EnsureCoordinator();
if (strncmp(functionName, CITUS_TRUNCATE_TRIGGER_NAME, NAMEDATALEN) == 0) 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; return;
} }
char *relationName = triggerRelation->relname; ereport(ERROR, (errmsg("triggers are only supported for citus local tables")));
}
Assert(relationName != NULL);
ereport(ERROR, (errmsg("cannot create trigger on relation \"%s\" because it "
"is either a distributed table or a reference table", /*
relationName))); * 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)); PG_RETURN_DATUM(PointerGetDatum(NULL));
} }
/* we might be truncating multiple relations */
UseCoordinatedTransaction();
if (IsCitusTableType(relationId, APPEND_DISTRIBUTED)) if (IsCitusTableType(relationId, APPEND_DISTRIBUTED))
{ {
Oid schemaId = get_rel_namespace(relationId); 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 * done before standard process utility is called for truncate
* command. * command.
*/ */
void void
PostprocessTruncateStatement(TruncateStmt *truncateStatement) PreprocessTruncateStatement(TruncateStmt *truncateStatement)
{ {
ErrorIfUnsupportedTruncateStmt(truncateStatement); ErrorIfUnsupportedTruncateStmt(truncateStatement);
EnsurePartitionTableNotReplicatedForTruncate(truncateStatement); EnsurePartitionTableNotReplicatedForTruncate(truncateStatement);
@ -316,7 +319,7 @@ ExecuteTruncateStmtSequentialIfNecessary(TruncateStmt *command)
{ {
Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK); Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK);
if (IsCitusTableType(relationId, REFERENCE_TABLE) && if (IsCitusTableType(relationId, CITUS_TABLE_WITH_NO_DIST_KEY) &&
TableReferenced(relationId)) TableReferenced(relationId))
{ {
char *relationName = get_rel_name(relationId); char *relationName = get_rel_name(relationId);

View File

@ -214,13 +214,6 @@ multi_ProcessUtility(PlannedStmt *pstmt,
parsetree = ProcessCreateSubscriptionStmt(createSubStmt); parsetree = ProcessCreateSubscriptionStmt(createSubStmt);
} }
if (IsA(parsetree, CreateTrigStmt))
{
CreateTrigStmt *createTriggerStmt = (CreateTrigStmt *) parsetree;
ErrorIfUnsupportedCreateTriggerCommand(createTriggerStmt);
}
if (IsA(parsetree, CallStmt)) if (IsA(parsetree, CallStmt))
{ {
CallStmt *callStmt = (CallStmt *) parsetree; CallStmt *callStmt = (CallStmt *) parsetree;
@ -378,7 +371,7 @@ multi_ProcessUtility(PlannedStmt *pstmt,
if (IsA(parsetree, TruncateStmt)) if (IsA(parsetree, TruncateStmt))
{ {
PostprocessTruncateStatement((TruncateStmt *) parsetree); PreprocessTruncateStatement((TruncateStmt *) parsetree);
} }
/* only generate worker DDLJobs if propagation is enabled */ /* 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)) if (IsA(parsetree, CreateStmt))
{ {
CreateStmt *createStatement = (CreateStmt *) parsetree; 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; 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) */ #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/indexing.h"
#include "catalog/pg_class.h" #include "catalog/pg_class.h"
#include "catalog/pg_depend.h" #include "catalog/pg_depend.h"
#include "catalog/pg_proc_d.h"
#include "catalog/pg_rewrite.h" #include "catalog/pg_rewrite.h"
#include "catalog/pg_rewrite_d.h" #include "catalog/pg_rewrite_d.h"
#include "catalog/pg_shdepend.h" #include "catalog/pg_shdepend.h"
@ -28,6 +29,7 @@
#if PG_VERSION_NUM >= PG_VERSION_13 #if PG_VERSION_NUM >= PG_VERSION_13
#include "common/hashfn.h" #include "common/hashfn.h"
#endif #endif
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h" #include "distributed/commands/utility_hook.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/metadata/dependency.h" #include "distributed/metadata/dependency.h"
@ -116,6 +118,8 @@ typedef struct ViewDependencyNode
}ViewDependencyNode; }ViewDependencyNode;
static List * GetRelationTriggerFunctionDepencyList(Oid relationId);
static DependencyDefinition * CreateObjectAddressDependencyDef(Oid classId, Oid objectId);
static ObjectAddress DependencyDefinitionObjectAddress(DependencyDefinition *definition); static ObjectAddress DependencyDefinitionObjectAddress(DependencyDefinition *definition);
/* forward declarations for functions to interact with the ObjectAddressCollector */ /* 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) if (get_typtype(target.objectId) == TYPTYPE_COMPOSITE)
{ {
DependencyDefinition *dependency = palloc0(sizeof(DependencyDefinition)); Oid typeRelationId = get_typ_typrelid(target.objectId);
dependency->mode = DependencyObjectAddress; DependencyDefinition *dependency =
ObjectAddressSet(dependency->data.address, CreateObjectAddressDependencyDef(RelationRelationId, typeRelationId);
RelationRelationId,
get_typ_typrelid(target.objectId));
result = lappend(result, dependency); result = lappend(result, dependency);
} }
@ -874,18 +875,32 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
*/ */
if (type_is_array(target.objectId)) if (type_is_array(target.objectId))
{ {
DependencyDefinition *dependency = palloc0(sizeof(DependencyDefinition)); Oid typeId = get_element_type(target.objectId);
dependency->mode = DependencyObjectAddress; DependencyDefinition *dependency =
ObjectAddressSet(dependency->data.address, CreateObjectAddressDependencyDef(TypeRelationId, typeId);
TypeRelationId,
get_element_type(target.objectId));
result = lappend(result, dependency); result = lappend(result, dependency);
} }
break; 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: default:
{ {
/* no expansion for unsupported types */ /* 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 * DependencyDefinitionObjectAddress returns the object address of the dependency defined
* by the dependency definition, irregardless what the source of the definition is * by the dependency definition, irregardless what the source of the definition is

View File

@ -211,7 +211,6 @@ static ScanKeyData DistObjectScanKey[3];
/* local function forward declarations */ /* local function forward declarations */
static bool IsCitusTableViaCatalog(Oid relationId); static bool IsCitusTableViaCatalog(Oid relationId);
static ShardIdCacheEntry * LookupShardIdCacheEntry(int64 shardId); static ShardIdCacheEntry * LookupShardIdCacheEntry(int64 shardId);
static CitusTableCacheEntry * LookupCitusTableCacheEntry(Oid relationId);
static CitusTableCacheEntry * BuildCitusTableCacheEntry(Oid relationId); static CitusTableCacheEntry * BuildCitusTableCacheEntry(Oid relationId);
static void BuildCachedShardList(CitusTableCacheEntry *cacheEntry); static void BuildCachedShardList(CitusTableCacheEntry *cacheEntry);
static void PrepareWorkerNodeCache(void); static void PrepareWorkerNodeCache(void);
@ -362,7 +361,14 @@ IsCitusTableTypeInternal(CitusTableCacheEntry *tableEntry, CitusTableType tableT
case REFERENCE_TABLE: case REFERENCE_TABLE:
{ {
return tableEntry->partitionMethod == DISTRIBUTE_BY_NONE; return tableEntry->partitionMethod == DISTRIBUTE_BY_NONE &&
tableEntry->replicationModel == REPLICATION_MODEL_2PC;
}
case CITUS_LOCAL_TABLE:
{
return tableEntry->partitionMethod == DISTRIBUTE_BY_NONE &&
tableEntry->replicationModel != REPLICATION_MODEL_2PC;
} }
case CITUS_TABLE_WITH_NO_DIST_KEY: case CITUS_TABLE_WITH_NO_DIST_KEY:
@ -426,6 +432,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 * CitusTableList returns a list that includes all the valid distributed table
* cache entries. * cache entries.
@ -923,7 +953,7 @@ GetCitusTableCacheEntry(Oid distributedRelationId)
* passed relationId. For efficiency it caches lookups. This function returns * passed relationId. For efficiency it caches lookups. This function returns
* NULL if the relation isn't a distributed table. * NULL if the relation isn't a distributed table.
*/ */
static CitusTableCacheEntry * CitusTableCacheEntry *
LookupCitusTableCacheEntry(Oid relationId) LookupCitusTableCacheEntry(Oid relationId)
{ {
bool foundInCache = false; bool foundInCache = false;
@ -3809,16 +3839,21 @@ ReferenceTableOidList()
while (HeapTupleIsValid(heapTuple)) while (HeapTupleIsValid(heapTuple))
{ {
bool isNull = false; bool isNull = false;
Datum relationIdDatum = heap_getattr(heapTuple,
Anum_pg_dist_partition_logicalrelid,
tupleDescriptor, &isNull);
Oid relationId = DatumGetObjectId(relationIdDatum);
char partitionMethod = heap_getattr(heapTuple, char partitionMethod = heap_getattr(heapTuple,
Anum_pg_dist_partition_partmethod, Anum_pg_dist_partition_partmethod,
tupleDescriptor, &isNull); tupleDescriptor, &isNull);
char replicationModel = heap_getattr(heapTuple,
Anum_pg_dist_partition_repmodel,
tupleDescriptor, &isNull);
if (partitionMethod == DISTRIBUTE_BY_NONE) if (IsReferenceTableByDistParams(partitionMethod, replicationModel))
{ {
Datum relationIdDatum = heap_getattr(heapTuple,
Anum_pg_dist_partition_logicalrelid,
tupleDescriptor, &isNull);
Oid relationId = DatumGetObjectId(relationIdDatum);
referenceTableOidList = lappend_oid(referenceTableOidList, relationId); referenceTableOidList = lappend_oid(referenceTableOidList, relationId);
} }

View File

@ -229,7 +229,7 @@ ShouldSyncTableMetadata(Oid relationId)
bool mxTable = (streamingReplicated && IsCitusTableTypeCacheEntry(tableEntry, bool mxTable = (streamingReplicated && IsCitusTableTypeCacheEntry(tableEntry,
HASH_DISTRIBUTED)); HASH_DISTRIBUTED));
if (mxTable || IsCitusTableTypeCacheEntry(tableEntry, REFERENCE_TABLE)) if (mxTable || IsCitusTableTypeCacheEntry(tableEntry, CITUS_TABLE_WITH_NO_DIST_KEY))
{ {
return true; return true;
} }
@ -1036,7 +1036,7 @@ List *
SequenceDDLCommandsForTable(Oid relationId) SequenceDDLCommandsForTable(Oid relationId)
{ {
List *sequenceDDLList = NIL; List *sequenceDDLList = NIL;
List *ownedSequences = getOwnedSequencesCompat(relationId, InvalidAttrNumber); List *ownedSequences = GetSequencesOwnedByRelation(relationId);
char *ownerName = TableOwner(relationId); char *ownerName = TableOwner(relationId);
Oid sequenceOid = InvalidOid; Oid sequenceOid = InvalidOid;

View File

@ -172,12 +172,12 @@ master_apply_delete_command(PG_FUNCTION_ARGS)
"are not supported with master_apply_delete_command."), "are not supported with master_apply_delete_command."),
errhint("Use the DELETE command instead."))); errhint("Use the DELETE command instead.")));
} }
else if (IsCitusTableType(relationId, REFERENCE_TABLE)) else if (IsCitusTableType(relationId, CITUS_TABLE_WITH_NO_DIST_KEY))
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot delete from reference table"), errmsg("cannot delete from table"),
errdetail("Delete statements on reference tables " errdetail("Delete statements on reference and citus "
"are not supported."))); "local tables are not supported.")));
} }

View File

@ -749,7 +749,7 @@ GetTableIndexAndConstraintCommands(Oid relationId)
/* /*
* IndexImpliedByAConstraint is an helper function to be used while scanning * IndexImpliedByAConstraint is a helper function to be used while scanning
* pg_index. It returns true if the index identified by the given indexForm is * pg_index. It returns true if the index identified by the given indexForm is
* implied by a constraint. Note that caller is responsible for passing a valid * implied by a constraint. Note that caller is responsible for passing a valid
* indexFrom, which means an alive heap tuple which is of form Form_pg_index. * indexFrom, which means an alive heap tuple which is of form Form_pg_index.
@ -851,3 +851,14 @@ WorkerNodeGetDatum(WorkerNode *workerNode, TupleDesc tupleDescriptor)
return workerNodeDatum; return workerNodeDatum;
} }
/*
* DistributedTableReplicationIsEnabled returns true if distributed table shards
* are replicated according to ShardReplicationFactor.
*/
bool
DistributedTableReplicationIsEnabled()
{
return (ShardReplicationFactor > 1);
}

View File

@ -204,6 +204,14 @@ ErrorIfTableCannotBeReplicated(Oid relationId)
CitusTableCacheEntry *tableEntry = GetCitusTableCacheEntry(relationId); CitusTableCacheEntry *tableEntry = GetCitusTableCacheEntry(relationId);
char *relationName = get_rel_name(relationId); char *relationName = get_rel_name(relationId);
if (IsCitusTableTypeCacheEntry(tableEntry, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
(errmsg("Table %s is a citus local table. Replicating "
"shard of a citus local table currently is not "
"supported", quote_literal_cstr(relationName)))));
}
/* /*
* ShouldSyncTableMetadata() returns true also for reference table, * ShouldSyncTableMetadata() returns true also for reference table,
* we don't want to error in that case since reference tables aren't * we don't want to error in that case since reference tables aren't
@ -809,13 +817,13 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
colocatedShardForeignConstraintCommandList, colocatedShardForeignConstraintCommandList,
List **referenceTableForeignConstraintList) 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 *schemaName = get_namespace_name(schemaId);
char *escapedSchemaName = quote_literal_cstr(schemaName); char *escapedSchemaName = quote_literal_cstr(schemaName);
int shardIndex = 0; int shardIndex = 0;
List *commandList = GetReferencingForeignConstaintCommands( List *commandList = GetReferencingForeignConstaintCommands(relationId);
shardInterval->relationId);
/* we will only use shardIndex if there is a foreign constraint */ /* we will only use shardIndex if there is a foreign constraint */
if (commandList != NIL) if (commandList != NIL)
@ -836,7 +844,7 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
StringInfo applyForeignConstraintCommand = makeStringInfo(); 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); Oid referencedRelationId = ForeignConstraintGetReferencedTableId(command);
if (referencedRelationId == InvalidOid) if (referencedRelationId == InvalidOid)
{ {
@ -849,10 +857,29 @@ CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
char *referencedSchemaName = get_namespace_name(referencedSchemaId); char *referencedSchemaName = get_namespace_name(referencedSchemaId);
char *escapedReferencedSchemaName = quote_literal_cstr(referencedSchemaName); char *escapedReferencedSchemaName = quote_literal_cstr(referencedSchemaName);
if (IsCitusTableType(referencedRelationId, CITUS_TABLE_WITH_NO_DIST_KEY)) if (IsCitusTableType(referencedRelationId, REFERENCE_TABLE))
{ {
referencedShardId = GetFirstShardId(referencedRelationId); 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 else
{ {
referencedShardId = ColocatedShardIdInRelation(referencedRelationId, referencedShardId = ColocatedShardIdInRelation(referencedRelationId,

View File

@ -143,13 +143,20 @@ master_create_empty_shard(PG_FUNCTION_ARGS)
errdetail("We currently don't support creating shards " errdetail("We currently don't support creating shards "
"on hash-partitioned tables"))); "on hash-partitioned tables")));
} }
else if (IsCitusTableType(relationId, CITUS_TABLE_WITH_NO_DIST_KEY)) else if (IsCitusTableType(relationId, REFERENCE_TABLE))
{ {
ereport(ERROR, (errmsg("relation \"%s\" is a reference table", ereport(ERROR, (errmsg("relation \"%s\" is a reference table",
relationName), relationName),
errdetail("We currently don't support creating shards " errdetail("We currently don't support creating shards "
"on reference tables"))); "on reference tables")));
} }
else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errmsg("relation \"%s\" is a citus local table",
relationName),
errdetail("We currently don't support creating shards "
"on citus local tables")));
}
char replicationModel = TableReplicationModel(relationId); char replicationModel = TableReplicationModel(relationId);
@ -257,7 +264,8 @@ master_append_table_to_shard(PG_FUNCTION_ARGS)
{ {
ereport(ERROR, (errmsg("cannot append to shardId " UINT64_FORMAT, shardId), ereport(ERROR, (errmsg("cannot append to shardId " UINT64_FORMAT, shardId),
errdetail("We currently don't support appending to shards " errdetail("We currently don't support appending to shards "
"in hash-partitioned or reference tables"))); "in hash-partitioned, reference and citus local "
"tables")));
} }
/* ensure that the shard placement metadata does not change during the append */ /* ensure that the shard placement metadata does not change during the append */

View File

@ -15,6 +15,7 @@
#include "miscadmin.h" #include "miscadmin.h"
#include "commands/dbcommands.h" #include "commands/dbcommands.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/hash_helpers.h" #include "distributed/hash_helpers.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
@ -404,6 +405,25 @@ NodeIsPrimaryWorker(WorkerNode *node)
} }
/*
* CoordinatorAddedAsWorkerNode returns true if coordinator is added to the
* 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.
*/
bool
CoordinatorAddedAsWorkerNode()
{
bool groupContainsNodes = false;
LockRelationOid(DistNodeRelationId(), ShareLock);
PrimaryNodeForGroup(COORDINATOR_GROUP_ID, &groupContainsNodes);
return groupContainsNodes;
}
/* /*
* ReferenceTablePlacementNodeList returns the set of nodes that should have * ReferenceTablePlacementNodeList returns the set of nodes that should have
* reference table placements. This includes all primaries, including the * reference table placements. This includes all primaries, including the
@ -417,6 +437,45 @@ ReferenceTablePlacementNodeList(LOCKMODE lockMode)
} }
/*
* CoordinatorNodeIfAddedAsWorkerOrError returns the WorkerNode object for
* coordinator node if it is added to pg_dist_node, otherwise errors out.
* Also, as CoordinatorAddedAsWorkerNode acquires AccessShareLock on pg_dist_node
* and doesn't release it, callers can safely assume coordinator won't be
* removed from metadata until the end of transaction when this function
* returns coordinator node.
*/
WorkerNode *
CoordinatorNodeIfAddedAsWorkerOrError()
{
ErrorIfCoordinatorNotAddedAsWorkerNode();
WorkerNode *coordinatorNode = LookupNodeForGroup(COORDINATOR_GROUP_ID);
WorkerNode *coordinatorNodeCopy = palloc0(sizeof(WorkerNode));
*coordinatorNodeCopy = *coordinatorNode;
return coordinatorNodeCopy;
}
/*
* ErrorIfCoordinatorNotAddedAsWorkerNode errors out if coordinator is not added
* to metadata.
*/
void
ErrorIfCoordinatorNotAddedAsWorkerNode()
{
if (CoordinatorAddedAsWorkerNode())
{
return;
}
ereport(ERROR, (errmsg("could not find the coordinator node in "
"metadata as it is not added as a worker")));
}
/* /*
* DistributedTablePlacementNodeList returns a list of all active, primary * DistributedTablePlacementNodeList returns a list of all active, primary
* worker nodes that can store new data, i.e shouldstoreshards is 'true' * worker nodes that can store new data, i.e shouldstoreshards is 'true'

View File

@ -77,7 +77,7 @@ RebuildQueryStrings(Job *workerJob)
RangeTblEntry *copiedSubqueryRte = ExtractSelectRangeTableEntry(query); RangeTblEntry *copiedSubqueryRte = ExtractSelectRangeTableEntry(query);
Query *copiedSubquery = copiedSubqueryRte->subquery; Query *copiedSubquery = copiedSubqueryRte->subquery;
/* there are no restrictions to add for reference tables */ /* there are no restrictions to add for reference and citus local tables */
if (IsCitusTableType(shardInterval->relationId, DISTRIBUTED_TABLE)) if (IsCitusTableType(shardInterval->relationId, DISTRIBUTED_TABLE))
{ {
AddShardIntervalRestrictionToSelect(copiedSubquery, shardInterval); AddShardIntervalRestrictionToSelect(copiedSubquery, shardInterval);

View File

@ -123,6 +123,7 @@ static PlannedStmt * PlanFastPathDistributedStmt(DistributedPlanningContext *pla
Node *distributionKeyValue); Node *distributionKeyValue);
static PlannedStmt * PlanDistributedStmt(DistributedPlanningContext *planContext, static PlannedStmt * PlanDistributedStmt(DistributedPlanningContext *planContext,
int rteIdCounter); int rteIdCounter);
static RTEListProperties * GetRTEListProperties(List *rangeTableList);
/* Distributed planner hook */ /* Distributed planner hook */
@ -2262,3 +2263,69 @@ HasUnresolvedExternParamsWalker(Node *expression, ParamListInfo boundParams)
boundParams); boundParams);
} }
} }
/*
* GetRTEListPropertiesForQuery is a wrapper around GetRTEListProperties that
* returns RTEListProperties for the rte list retrieved from query.
*/
RTEListProperties *
GetRTEListPropertiesForQuery(Query *query)
{
List *rteList = ExtractRangeTableEntryList(query);
return GetRTEListProperties(rteList);
}
/*
* GetRTEListProperties returns RTEListProperties struct processing the given
* rangeTableList.
*/
static RTEListProperties *
GetRTEListProperties(List *rangeTableList)
{
RTEListProperties *rteListProperties = palloc0(sizeof(RTEListProperties));
RangeTblEntry *rangeTableEntry = NULL;
foreach_ptr(rangeTableEntry, rangeTableList)
{
if (!(rangeTableEntry->rtekind == RTE_RELATION &&
rangeTableEntry->relkind == RELKIND_RELATION))
{
continue;
}
Oid relationId = rangeTableEntry->relid;
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
if (!cacheEntry)
{
rteListProperties->hasPostgresLocalTable = true;
}
else if (IsCitusTableTypeCacheEntry(cacheEntry, REFERENCE_TABLE))
{
rteListProperties->hasReferenceTable = true;
}
else if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_LOCAL_TABLE))
{
rteListProperties->hasCitusLocalTable = true;
}
else if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE))
{
rteListProperties->hasDistributedTable = true;
}
else
{
/* it's not expected, but let's do a bug catch here */
ereport(ERROR, (errcode(ERRCODE_INTERNAL_ERROR),
errmsg("encountered with an unexpected citus "
"table type while processing range table "
"entries of query")));
}
}
rteListProperties->hasCitusTable = (rteListProperties->hasDistributedTable ||
rteListProperties->hasReferenceTable ||
rteListProperties->hasCitusLocalTable);
return rteListProperties;
}

View File

@ -556,6 +556,40 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte,
NULL, NULL); NULL, NULL);
} }
RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery(subquery);
if (subqueryRteListProperties->hasDistributedTable &&
(subqueryRteListProperties->hasCitusLocalTable ||
subqueryRteListProperties->hasPostgresLocalTable))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"distributed INSERT ... SELECT cannot select from "
"distributed tables and local tables at the same time",
NULL, NULL);
}
if (subqueryRteListProperties->hasDistributedTable &&
IsCitusTableType(targetRelationId, CITUS_LOCAL_TABLE))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"distributed INSERT ... SELECT cannot insert into a "
"citus local table",
NULL, NULL);
}
/*
* In some cases, it might be possible to allow postgres local tables
* in distributed insert select. However, we want to behave consistent
* on all cases including Citus MX, and let insert select via coordinator
* to kick-in.
*/
if (subqueryRteListProperties->hasPostgresLocalTable)
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"distributed INSERT ... SELECT cannot select from "
"a local table", NULL, NULL);
return NULL;
}
/* we do not expect to see a view in modify target */ /* we do not expect to see a view in modify target */
foreach(rangeTableCell, queryTree->rtable) foreach(rangeTableCell, queryTree->rtable)
{ {
@ -584,12 +618,19 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte,
return error; return error;
} }
if (IsCitusTableType(targetRelationId, CITUS_LOCAL_TABLE))
{
/*
* If we're inserting into a citus local table, it is ok because we've
* checked the non-existence of distributed tables in the subquery.
*/
}
else if (IsCitusTableType(targetRelationId, REFERENCE_TABLE))
{
/* /*
* If we're inserting into a reference table, all participating tables * If we're inserting into a reference table, all participating tables
* should be reference tables as well. * should be reference tables as well.
*/ */
if (IsCitusTableType(targetRelationId, REFERENCE_TABLE))
{
if (!allReferenceTables) if (!allReferenceTables)
{ {
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
@ -726,7 +767,9 @@ RouterModifyTaskForShardInterval(Query *originalQuery,
* prevent shard pruning logic (i.e, namely UpdateRelationNames()) * prevent shard pruning logic (i.e, namely UpdateRelationNames())
* modifies range table entries, which makes hard to add the quals. * modifies range table entries, which makes hard to add the quals.
*/ */
if (!allReferenceTables) RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery(
copiedSubquery);
if (subqueryRteListProperties->hasDistributedTable)
{ {
AddShardIntervalRestrictionToSelect(copiedSubquery, shardInterval); AddShardIntervalRestrictionToSelect(copiedSubquery, shardInterval);
} }

View File

@ -311,7 +311,9 @@ NodeTryGetRteRelid(Node *node)
} }
RangeTblEntry *rangeTableEntry = (RangeTblEntry *) node; RangeTblEntry *rangeTableEntry = (RangeTblEntry *) node;
if (rangeTableEntry->rtekind != RTE_RELATION)
if (!(rangeTableEntry->rtekind == RTE_RELATION &&
rangeTableEntry->relkind == RELKIND_RELATION))
{ {
return InvalidOid; return InvalidOid;
} }
@ -332,6 +334,18 @@ IsCitusTableRTE(Node *node)
} }
/*
* IsPostgresLocalTableRte gets a node and returns true if the node is a
* range table relation entry that points to a postgres local table.
*/
bool
IsPostgresLocalTableRte(Node *node)
{
Oid relationId = NodeTryGetRteRelid(node);
return OidIsValid(relationId) && !IsCitusTable(relationId);
}
/* /*
* IsDistributedTableRTE gets a node and returns true if the node * IsDistributedTableRTE gets a node and returns true if the node
* is a range table relation entry that points to a distributed relation, * is a range table relation entry that points to a distributed relation,
@ -357,6 +371,18 @@ IsReferenceTableRTE(Node *node)
} }
/*
* IsCitusLocalTableRTE gets a node and returns true if the node
* is a range table relation entry that points to a citus local table.
*/
bool
IsCitusLocalTableRTE(Node *node)
{
Oid relationId = NodeTryGetRteRelid(node);
return OidIsValid(relationId) && IsCitusTableType(relationId, CITUS_LOCAL_TABLE);
}
/* /*
* FullCompositeFieldList gets a composite field list, and checks if all fields * FullCompositeFieldList gets a composite field list, and checks if all fields
* of composite type are used in the list. * of composite type are used in the list.
@ -926,6 +952,16 @@ DeferErrorIfQueryNotSupported(Query *queryTree)
errorMessage = "subquery in OFFSET is not supported in multi-shard queries"; errorMessage = "subquery in OFFSET is not supported in multi-shard queries";
} }
RTEListProperties *queryRteListProperties = GetRTEListPropertiesForQuery(queryTree);
if (queryRteListProperties->hasCitusLocalTable ||
queryRteListProperties->hasPostgresLocalTable)
{
preconditionsSatisfied = false;
errorMessage = "direct joins between distributed and local tables are "
"not supported";
errorHint = LOCAL_TABLE_SUBQUERY_CTE_HINT;
}
/* finally check and error out if not satisfied */ /* finally check and error out if not satisfied */
if (!preconditionsSatisfied) if (!preconditionsSatisfied)
{ {

View File

@ -130,6 +130,12 @@ static bool IsTidColumn(Node *node);
static DeferredErrorMessage * ModifyPartialQuerySupported(Query *queryTree, bool static DeferredErrorMessage * ModifyPartialQuerySupported(Query *queryTree, bool
multiShardQuery, multiShardQuery,
Oid *distributedTableId); Oid *distributedTableId);
static DeferredErrorMessage * DeferErrorIfUnsupportedModifyQueryWithLocalTable(
Query *query);
static DeferredErrorMessage * DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable(
RTEListProperties *rteListProperties, Oid targetRelationId);
static DeferredErrorMessage * DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable(
RTEListProperties *rteListProperties, Oid targetRelationId);
static DeferredErrorMessage * MultiShardModifyQuerySupported(Query *originalQuery, static DeferredErrorMessage * MultiShardModifyQuerySupported(Query *originalQuery,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext); plannerRestrictionContext);
@ -570,22 +576,14 @@ static DeferredErrorMessage *
ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery, ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery,
Oid *distributedTableIdOutput) Oid *distributedTableIdOutput)
{ {
uint32 rangeTableId = 1; DeferredErrorMessage *deferredError =
CmdType commandType = queryTree->commandType; DeferErrorIfUnsupportedModifyQueryWithLocalTable(queryTree);
if (deferredError != NULL)
Oid distributedTableId = ModifyQueryResultRelationId(queryTree);
*distributedTableIdOutput = distributedTableId;
if (!IsCitusTable(distributedTableId))
{ {
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, return deferredError;
"cannot plan modifications of local tables involving "
"distributed tables",
NULL, NULL);
} }
Var *partitionColumn = PartitionColumn(distributedTableId, rangeTableId); deferredError = DeferErrorIfModifyView(queryTree);
DeferredErrorMessage *deferredError = DeferErrorIfModifyView(queryTree);
if (deferredError != NULL) if (deferredError != NULL)
{ {
return deferredError; return deferredError;
@ -675,6 +673,11 @@ ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery,
} }
} }
Oid distributedTableId = ModifyQueryResultRelationId(queryTree);
uint32 rangeTableId = 1;
Var *partitionColumn = PartitionColumn(distributedTableId, rangeTableId);
CmdType commandType = queryTree->commandType;
if (commandType == CMD_INSERT || commandType == CMD_UPDATE || if (commandType == CMD_INSERT || commandType == CMD_UPDATE ||
commandType == CMD_DELETE) commandType == CMD_DELETE)
{ {
@ -788,6 +791,97 @@ ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery,
return deferredError; return deferredError;
} }
/* set it for caller to use when we don't return any errors */
*distributedTableIdOutput = distributedTableId;
return NULL;
}
/*
* DeferErrorIfUnsupportedModifyQueryWithLocalTable returns DeferredErrorMessage
* for unsupported modify queries that cannot be planned by router planner due to
* unsupported usage of postgres local or citus local tables.
*/
static DeferredErrorMessage *
DeferErrorIfUnsupportedModifyQueryWithLocalTable(Query *query)
{
RTEListProperties *rteListProperties = GetRTEListPropertiesForQuery(query);
Oid targetRelationId = ModifyQueryResultRelationId(query);
DeferredErrorMessage *deferredErrorMessage =
DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable(rteListProperties,
targetRelationId);
if (deferredErrorMessage)
{
return deferredErrorMessage;
}
deferredErrorMessage = DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable(
rteListProperties,
targetRelationId);
return deferredErrorMessage;
}
/*
* DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable is a helper function
* that takes RTEListProperties & targetRelationId and returns deferred error
* if query is not supported due to unsupported usage of citus local tables.
*/
static DeferredErrorMessage *
DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable(
RTEListProperties *rteListProperties, Oid targetRelationId)
{
if (rteListProperties->hasDistributedTable && rteListProperties->hasCitusLocalTable)
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"cannot plan modifications with citus local tables and "
"distributed tables", NULL,
LOCAL_TABLE_SUBQUERY_CTE_HINT);
}
if (IsCitusTableType(targetRelationId, REFERENCE_TABLE) &&
rteListProperties->hasCitusLocalTable)
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"cannot plan modifications of reference tables with citus "
"local tables", NULL,
LOCAL_TABLE_SUBQUERY_CTE_HINT);
}
return NULL;
}
/*
* DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable is a helper
* function that takes RTEListProperties & targetRelationId and returns
* deferred error if query is not supported due to unsupported usage of
* postgres local tables.
*/
static DeferredErrorMessage *
DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable(
RTEListProperties *rteListProperties, Oid targetRelationId)
{
if (rteListProperties->hasPostgresLocalTable &&
rteListProperties->hasCitusTable)
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"cannot plan modifications with local tables involving "
"citus tables", NULL,
LOCAL_TABLE_SUBQUERY_CTE_HINT);
}
if (!IsCitusTable(targetRelationId))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"cannot plan modifications of local tables involving "
"distributed tables",
NULL, NULL);
}
return NULL; return NULL;
} }
@ -2704,7 +2798,7 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError)
Assert(query->commandType == CMD_INSERT); Assert(query->commandType == CMD_INSERT);
/* reference tables can only have one shard */ /* reference tables and citus local tables can only have one shard */
if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY)) if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY))
{ {
List *shardIntervalList = LoadShardIntervalList(distributedTableId); List *shardIntervalList = LoadShardIntervalList(distributedTableId);
@ -2712,7 +2806,16 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError)
int shardCount = list_length(shardIntervalList); int shardCount = list_length(shardIntervalList);
if (shardCount != 1) if (shardCount != 1)
{ {
ereport(ERROR, (errmsg("reference table cannot have %d shards", shardCount))); if (IsCitusTableTypeCacheEntry(cacheEntry, REFERENCE_TABLE))
{
ereport(ERROR, (errmsg("reference table cannot have %d shards",
shardCount)));
}
else if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errmsg("citus local table cannot have %d shards",
shardCount)));
}
} }
ShardInterval *shardInterval = linitial(shardIntervalList); ShardInterval *shardInterval = linitial(shardIntervalList);
@ -3313,7 +3416,7 @@ MultiRouterPlannableQuery(Query *query)
NULL, NULL); NULL, NULL);
} }
bool hasLocalTable = false; bool hasPostgresOrCitusLocalTable = false;
bool hasDistributedTable = false; bool hasDistributedTable = false;
ExtractRangeTableRelationWalker((Node *) query, &rangeTableRelationList); ExtractRangeTableRelationWalker((Node *) query, &rangeTableRelationList);
@ -3327,7 +3430,13 @@ MultiRouterPlannableQuery(Query *query)
/* local tables are allowed if there are no distributed tables */ /* local tables are allowed if there are no distributed tables */
if (!IsCitusTable(distributedTableId)) if (!IsCitusTable(distributedTableId))
{ {
hasLocalTable = true; hasPostgresOrCitusLocalTable = true;
continue;
}
else if (IsCitusTableType(distributedTableId, CITUS_LOCAL_TABLE))
{
hasPostgresOrCitusLocalTable = true;
elog(DEBUG4, "Router planner finds a citus local table");
continue; continue;
} }
@ -3367,7 +3476,7 @@ MultiRouterPlannableQuery(Query *query)
} }
/* local tables are not allowed if there are distributed tables */ /* local tables are not allowed if there are distributed tables */
if (hasLocalTable && hasDistributedTable) if (hasPostgresOrCitusLocalTable && hasDistributedTable)
{ {
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"Local tables cannot be used in distributed queries.", "Local tables cannot be used in distributed queries.",

View File

@ -165,7 +165,7 @@ static bool ShouldRecursivelyPlanSetOperation(Query *query,
RecursivePlanningContext *context); RecursivePlanningContext *context);
static void RecursivelyPlanSetOperations(Query *query, Node *node, static void RecursivelyPlanSetOperations(Query *query, Node *node,
RecursivePlanningContext *context); RecursivePlanningContext *context);
static bool IsLocalTableRTE(Node *node); static bool IsLocalTableRteOrMatView(Node *node);
static void RecursivelyPlanSubquery(Query *subquery, static void RecursivelyPlanSubquery(Query *subquery,
RecursivePlanningContext *planningContext); RecursivePlanningContext *planningContext);
static DistributedSubPlan * CreateDistributedSubPlan(uint32 subPlanId, static DistributedSubPlan * CreateDistributedSubPlan(uint32 subPlanId,
@ -376,7 +376,8 @@ ShouldRecursivelyPlanNonColocatedSubqueries(Query *subquery,
} }
/* direct joins with local tables are not supported by any of Citus planners */ /* direct joins with local tables are not supported by any of Citus planners */
if (FindNodeMatchingCheckFunctionInRangeTableList(subquery->rtable, IsLocalTableRTE)) if (FindNodeMatchingCheckFunctionInRangeTableList(subquery->rtable,
IsLocalTableRteOrMatView))
{ {
return false; return false;
} }
@ -877,7 +878,8 @@ RecursivelyPlanSubqueryWalker(Node *node, RecursivePlanningContext *context)
static bool static bool
ShouldRecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *context) ShouldRecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *context)
{ {
if (FindNodeMatchingCheckFunctionInRangeTableList(subquery->rtable, IsLocalTableRTE)) if (FindNodeMatchingCheckFunctionInRangeTableList(subquery->rtable,
IsLocalTableRteOrMatView))
{ {
/* /*
* Postgres can always plan queries that don't require distributed planning. * Postgres can always plan queries that don't require distributed planning.
@ -1045,12 +1047,12 @@ RecursivelyPlanSetOperations(Query *query, Node *node,
/* /*
* IsLocalTableRTE gets a node and returns true if the node * IsLocalTableRteOrMatView gets a node and returns true if the node is a range
* is a range table relation entry that points to a local * table entry that points to a postgres local or citus local table or to a
* relation (i.e., not a distributed relation). * materialized view.
*/ */
static bool static bool
IsLocalTableRTE(Node *node) IsLocalTableRteOrMatView(Node *node)
{ {
if (node == NULL) if (node == NULL)
{ {
@ -1074,13 +1076,18 @@ IsLocalTableRTE(Node *node)
} }
Oid relationId = rangeTableEntry->relid; Oid relationId = rangeTableEntry->relid;
if (IsCitusTable(relationId)) if (!IsCitusTable(relationId))
{ {
return false; /* postgres local table or a materialized view */
return true;
}
else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
return true;
} }
/* local table found */ /* no local table found */
return true; return false;
} }

View File

@ -96,8 +96,8 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
case T_AlterTableStmt: case T_AlterTableStmt:
{ {
/* /*
* We append shardId to the very end of table and index names to * We append shardId to the very end of table and index, constraint
* avoid name collisions. We also append shardId to constraint names. * and trigger names to avoid name collisions.
*/ */
AlterTableStmt *alterTableStmt = (AlterTableStmt *) parseTree; AlterTableStmt *alterTableStmt = (AlterTableStmt *) parseTree;
@ -167,6 +167,14 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
AppendShardIdToName(indexName, 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; break;
@ -236,6 +244,33 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
break; 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: case T_DropStmt:
{ {
DropStmt *dropStmt = (DropStmt *) parseTree; DropStmt *dropStmt = (DropStmt *) parseTree;
@ -310,6 +345,10 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
{ {
DropPolicyEventExtendNames(dropStmt, schemaName, shardId); DropPolicyEventExtendNames(dropStmt, schemaName, shardId);
} }
else if (objectType == OBJECT_TRIGGER)
{
DropTriggerEventExtendNames(dropStmt, schemaName, shardId);
}
else else
{ {
ereport(WARNING, (errmsg("unsafe object type in drop statement"), ereport(WARNING, (errmsg("unsafe object type in drop statement"),
@ -456,7 +495,7 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
*newRelationName, NAMEDATALEN - 1))); *newRelationName, NAMEDATALEN - 1)));
} }
} }
else if (objectType == OBJECT_COLUMN || objectType == OBJECT_TRIGGER) else if (objectType == OBJECT_COLUMN)
{ {
char **relationName = &(renameStmt->relation->relname); char **relationName = &(renameStmt->relation->relname);
char **objectSchemaName = &(renameStmt->relation->schemaname); char **objectSchemaName = &(renameStmt->relation->schemaname);
@ -466,6 +505,10 @@ RelayEventExtendNames(Node *parseTree, char *schemaName, uint64 shardId)
AppendShardIdToName(relationName, shardId); AppendShardIdToName(relationName, shardId);
} }
else if (objectType == OBJECT_TRIGGER)
{
AlterTriggerRenameEventExtendNames(renameStmt, schemaName, shardId);
}
else if (objectType == OBJECT_POLICY) else if (objectType == OBJECT_POLICY)
{ {
RenamePolicyEventExtendNames(renameStmt, schemaName, shardId); RenamePolicyEventExtendNames(renameStmt, schemaName, shardId);

View File

@ -2,6 +2,7 @@
-- bump version to 9.5-1 -- bump version to 9.5-1
#include "udfs/undistribute_table/9.5-1.sql" #include "udfs/undistribute_table/9.5-1.sql"
#include "udfs/create_citus_local_table/9.5-1.sql"
SET search_path = 'pg_catalog'; SET search_path = 'pg_catalog';

View File

@ -2,6 +2,8 @@
SET search_path = 'pg_catalog'; SET search_path = 'pg_catalog';
DROP FUNCTION create_citus_local_table(table_name regclass);
-- task_tracker_* functions -- task_tracker_* functions
CREATE FUNCTION task_tracker_assign_task(bigint, integer, text) CREATE FUNCTION task_tracker_assign_task(bigint, integer, text)

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION pg_catalog.create_citus_local_table(table_name regclass)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$create_citus_local_table$$;
COMMENT ON FUNCTION pg_catalog.create_citus_local_table(table_name regclass)
IS 'create a citus local table';

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION pg_catalog.create_citus_local_table(table_name regclass)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$create_citus_local_table$$;
COMMENT ON FUNCTION pg_catalog.create_citus_local_table(table_name regclass)
IS 'create a citus local table';

View File

@ -173,7 +173,7 @@ RecordRelationAccessIfNonDistTable(Oid relationId, ShardPlacementAccessType acce
* recursively calling RecordRelationAccessBase(), so becareful about * recursively calling RecordRelationAccessBase(), so becareful about
* removing this check. * removing this check.
*/ */
if (!IsCitusTableType(relationId, REFERENCE_TABLE)) if (!IsCitusTableType(relationId, CITUS_TABLE_WITH_NO_DIST_KEY))
{ {
return; return;
} }
@ -691,9 +691,9 @@ ShouldRecordRelationAccess()
/* /*
* CheckConflictingRelationAccesses is mostly a wrapper around * CheckConflictingRelationAccesses is mostly a wrapper around
* HoldsConflictingLockWithReferencingRelations(). We're only interested in accesses * HoldsConflictingLockWithReferencingRelations(). We're only interested in
* to reference tables that are referenced via a foreign constraint by a * accesses to reference tables and citus local tables that are referenced via
* hash distributed tables. * a foreign constraint by a hash distributed table.
*/ */
static void static void
CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType accessType) CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType accessType)
@ -708,7 +708,7 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
if (!(IsCitusTableTypeCacheEntry(cacheEntry, REFERENCE_TABLE) && if (!(IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY) &&
cacheEntry->referencingRelationsViaForeignKey != NIL)) cacheEntry->referencingRelationsViaForeignKey != NIL))
{ {
return; return;
@ -734,8 +734,8 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
*/ */
if (relationName == NULL) if (relationName == NULL)
{ {
ereport(ERROR, (errmsg("cannot execute %s on reference table because " ereport(ERROR, (errmsg("cannot execute %s on table because there was "
"there was a parallel %s access to distributed table " "a parallel %s access to distributed table "
"\"%s\" in the same transaction", "\"%s\" in the same transaction",
accessTypeText, conflictingAccessTypeText, accessTypeText, conflictingAccessTypeText,
conflictingRelationName), conflictingRelationName),
@ -745,10 +745,9 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
} }
else else
{ {
ereport(ERROR, (errmsg( ereport(ERROR, (errmsg("cannot execute %s on table \"%s\" because "
"cannot execute %s on reference table \"%s\" because " "there was a parallel %s access to distributed "
"there was a parallel %s access to distributed table " "table \"%s\" in the same transaction",
"\"%s\" in the same transaction",
accessTypeText, relationName, accessTypeText, relationName,
conflictingAccessTypeText, conflictingAccessTypeText,
conflictingRelationName), conflictingRelationName),
@ -769,13 +768,13 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
* would still use the already opened parallel connections to the workers, * would still use the already opened parallel connections to the workers,
* thus contradicting our purpose of using sequential mode. * thus contradicting our purpose of using sequential mode.
*/ */
ereport(ERROR, (errmsg("cannot modify reference table \"%s\" because there " ereport(ERROR, (errmsg("cannot modify table \"%s\" because there was "
"was a parallel operation on a distributed table", "a parallel operation on a distributed table",
relationName), relationName),
errdetail("When there is a foreign key to a reference " errdetail("When there is a foreign key to a reference "
"table, Citus needs to perform all operations " "table or to a citus local table, Citus needs "
"over a single connection per node to ensure " "to perform all operations over a single "
"consistency."), "connection per node to ensure consistency."),
errhint("Try re-running the transaction with " errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO " "\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\""))); "\'sequential\';\"")));
@ -788,8 +787,8 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
*/ */
ereport(DEBUG1, (errmsg("switching to sequential query execution mode"), ereport(DEBUG1, (errmsg("switching to sequential query execution mode"),
errdetail( errdetail(
"Reference table \"%s\" is modified, which might lead " "Table \"%s\" is modified, which might lead to data "
"to data inconsistencies or distributed deadlocks via " "inconsistencies or distributed deadlocks via "
"parallel accesses to hash distributed tables due to " "parallel accesses to hash distributed tables due to "
"foreign keys. Any parallel modification to " "foreign keys. Any parallel modification to "
"those hash distributed tables in the same " "those hash distributed tables in the same "
@ -903,8 +902,11 @@ HoldsConflictingLockWithReferencedRelations(Oid relationId, ShardPlacementAccess
Oid referencedRelation = InvalidOid; Oid referencedRelation = InvalidOid;
foreach_oid(referencedRelation, cacheEntry->referencedRelationsViaForeignKey) foreach_oid(referencedRelation, cacheEntry->referencedRelationsViaForeignKey)
{ {
/* we're only interested in foreign keys to reference tables */ /*
if (!IsCitusTableType(referencedRelation, REFERENCE_TABLE)) * We're only interested in foreign keys to reference tables and citus
* local tables.
*/
if (!IsCitusTableType(referencedRelation, CITUS_TABLE_WITH_NO_DIST_KEY))
{ {
continue; continue;
} }
@ -966,7 +968,7 @@ HoldsConflictingLockWithReferencingRelations(Oid relationId, ShardPlacementAcces
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
bool holdsConflictingLocks = false; bool holdsConflictingLocks = false;
Assert(IsCitusTableTypeCacheEntry(cacheEntry, REFERENCE_TABLE)); Assert(IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY));
Oid referencingRelation = InvalidOid; Oid referencingRelation = InvalidOid;
foreach_oid(referencingRelation, cacheEntry->referencingRelationsViaForeignKey) foreach_oid(referencingRelation, cacheEntry->referencingRelationsViaForeignKey)

View File

@ -241,6 +241,13 @@ CreateColocationGroupForRelation(Oid sourceRelationId)
static void static void
MarkTablesColocated(Oid sourceRelationId, Oid targetRelationId) MarkTablesColocated(Oid sourceRelationId, Oid targetRelationId)
{ {
if (IsCitusTableType(sourceRelationId, CITUS_LOCAL_TABLE) ||
IsCitusTableType(targetRelationId, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errmsg("citus local tables cannot be colocated with "
"other tables")));
}
CheckReplicationModel(sourceRelationId, targetRelationId); CheckReplicationModel(sourceRelationId, targetRelationId);
CheckDistributionColumnType(sourceRelationId, targetRelationId); CheckDistributionColumnType(sourceRelationId, targetRelationId);

View File

@ -204,3 +204,40 @@ ListTake(List *pointerList, int size)
return result; 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 "postgres.h"
#include "catalog/namespace.h" #include "catalog/namespace.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/namespace_utils.h" #include "distributed/namespace_utils.h"
#include "utils/regproc.h"
/* /*
* PushOverrideEmptySearchPath pushes search_path to be NIL and sets addCatalog to * PushOverrideEmptySearchPath pushes search_path to be NIL and sets addCatalog to

View File

@ -351,6 +351,16 @@ upgrade_to_reference_table(PG_FUNCTION_ARGS)
errdetail("Relation \"%s\" is already a reference table", errdetail("Relation \"%s\" is already a reference table",
relationName))); relationName)));
} }
else if (IsCitusTableTypeCacheEntry(tableEntry, CITUS_LOCAL_TABLE))
{
char *relationName = get_rel_name(relationId);
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("cannot upgrade to reference table"),
errdetail("Relation \"%s\" is a citus local table and "
"currently it is not supported to upgrade "
"a citus local table to a reference table ",
relationName)));
}
if (tableEntry->replicationModel == REPLICATION_MODEL_STREAMING) if (tableEntry->replicationModel == REPLICATION_MODEL_STREAMING)
{ {

View File

@ -10,8 +10,8 @@
*/ */
#include "postgres.h" #include "postgres.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "distributed/metadata_cache.h"
#include "distributed/relay_utility.h" #include "distributed/relay_utility.h"
#include "distributed/shard_utils.h" #include "distributed/shard_utils.h"
@ -36,23 +36,3 @@ GetTableLocalShardOid(Oid citusTableOid, uint64 shardId)
return shardRelationOid; 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

@ -229,8 +229,8 @@ CompareRelationShards(const void *leftElement, const void *rightElement)
* *
* For hash partitioned tables, it calculates hash value of a number in its * For hash partitioned tables, it calculates hash value of a number in its
* range (e.g. min value) and finds which shard should contain the hashed * range (e.g. min value) and finds which shard should contain the hashed
* value. For reference tables, it simply returns 0. For distribution methods * value. For reference tables and citus local tables, it simply returns 0.
* other than hash and reference, the function errors out. * For the other table types, the function errors out.
*/ */
int int
ShardIndex(ShardInterval *shardInterval) ShardIndex(ShardInterval *shardInterval)
@ -247,17 +247,21 @@ ShardIndex(ShardInterval *shardInterval)
*/ */
if (!IsCitusTableTypeCacheEntry(cacheEntry, HASH_DISTRIBUTED) && if (!IsCitusTableTypeCacheEntry(cacheEntry, HASH_DISTRIBUTED) &&
!IsCitusTableTypeCacheEntry( !IsCitusTableTypeCacheEntry(
cacheEntry, REFERENCE_TABLE)) cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY))
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("finding index of a given shard is only supported for " errmsg("finding index of a given shard is only supported for "
"hash distributed and reference tables"))); "hash distributed tables, reference tables and citus "
"local tables")));
} }
/* short-circuit for reference tables */ /* short-circuit for reference tables */
if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY)) if (IsCitusTableTypeCacheEntry(cacheEntry, CITUS_TABLE_WITH_NO_DIST_KEY))
{ {
/* reference tables has only a single shard, so the index is fixed to 0 */ /*
* Reference tables and citus local tables have only a single shard,
* so the index is fixed to 0.
*/
shardIndex = 0; shardIndex = 0;
return shardIndex; return shardIndex;

View File

@ -97,6 +97,25 @@ citus_table_is_visible(PG_FUNCTION_ARGS)
} }
/*
* ErrorIfRelationIsAKnownShard errors out if the relation with relationId is
* a shard relation.
*/
void
ErrorIfRelationIsAKnownShard(Oid relationId)
{
/* search the relation in all schemas */
bool onlySearchPath = false;
if (!RelationIsAKnownShard(relationId, onlySearchPath))
{
return;
}
const char *relationName = get_rel_name(relationId);
ereport(ERROR, (errmsg("relation \"%s\" is a shard relation ", relationName)));
}
/* /*
* RelationIsAKnownShard gets a relationId, check whether it's a shard of * RelationIsAKnownShard gets a relationId, check whether it's a shard of
* any distributed table. If onlySearchPath is true, then it searches * any distributed table. If onlySearchPath is true, then it searches

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_relation_name(Oid relid, List *namespaces);
extern char * generate_qualified_relation_name(Oid relid); extern char * generate_qualified_relation_name(Oid relid);
extern char * generate_operator_name(Oid operid, Oid arg1, Oid arg2); 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 */ #endif /* CITUS_RULEUTILS_H */

View File

@ -48,6 +48,22 @@ typedef struct DistributeObjectOps
const DistributeObjectOps * GetDistributeObjectOps(Node *node); const DistributeObjectOps * GetDistributeObjectOps(Node *node);
/*
* Flags that can be passed to GetForeignKeyOids to indicate
* which foreign key constraint OIDs are to be extracted
*/
typedef enum ExtractForeignKeyConstraintsMode
{
/* extract the foreign key OIDs where the table is the referencing one */
INCLUDE_REFERENCING_CONSTRAINTS = 1 << 0,
/* extract the foreign key OIDs the table is the referenced one */
INCLUDE_REFERENCED_CONSTRAINTS = 1 << 1,
/* exclude the self-referencing foreign keys */
EXCLUDE_SELF_REFERENCES = 1 << 2
} ExtractForeignKeyConstraintMode;
/* cluster.c - forward declarations */ /* cluster.c - forward declarations */
extern List * PreprocessClusterStmt(Node *node, const char *clusterCommand); extern List * PreprocessClusterStmt(Node *node, const char *clusterCommand);
@ -98,16 +114,23 @@ extern ObjectAddress AlterExtensionUpdateStmtObjectAddress(Node *stmt,
extern bool ConstraintIsAForeignKeyToReferenceTable(char *constraintName, extern bool ConstraintIsAForeignKeyToReferenceTable(char *constraintName,
Oid leftRelationId); Oid leftRelationId);
extern void ErrorIfUnsupportedForeignConstraintExists(Relation relation, extern void ErrorIfUnsupportedForeignConstraintExists(Relation relation,
char referencingReplicationModel,
char distributionMethod, char distributionMethod,
Var *distributionColumn, Var *distributionColumn,
uint32 colocationId); uint32 colocationId);
extern void ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId);
extern bool ColumnAppearsInForeignKeyToReferenceTable(char *columnName, Oid extern bool ColumnAppearsInForeignKeyToReferenceTable(char *columnName, Oid
relationId); relationId);
extern List * GetReferencingForeignConstaintCommands(Oid relationOid); extern List * GetReferencingForeignConstaintCommands(Oid relationOid);
extern bool HasForeignKeyToCitusLocalTable(Oid relationId);
extern bool HasForeignKeyToReferenceTable(Oid relationOid); extern bool HasForeignKeyToReferenceTable(Oid relationOid);
extern bool TableReferenced(Oid relationOid); extern bool TableReferenced(Oid relationOid);
extern bool TableReferencing(Oid relationOid); extern bool TableReferencing(Oid relationOid);
extern bool ConstraintIsAForeignKey(char *inputConstaintName, 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 */ /* function.c - forward declarations */
@ -213,7 +236,7 @@ extern Node * ProcessCreateSubscriptionStmt(CreateSubscriptionStmt *createSubStm
/* table.c - forward declarations */ /* table.c - forward declarations */
extern List * PreprocessDropTableStmt(Node *stmt, const char *queryString); extern List * PreprocessDropTableStmt(Node *stmt, const char *queryString);
extern List * PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, extern void PostprocessCreateTableStmt(CreateStmt *createStatement,
const char *queryString); const char *queryString);
extern List * PostprocessAlterTableStmtAttachPartition( extern List * PostprocessAlterTableStmtAttachPartition(
AlterTableStmt *alterTableStatement, AlterTableStmt *alterTableStatement,
@ -230,6 +253,7 @@ extern void PostprocessAlterTableStmt(AlterTableStmt *pStmt);
extern void ErrorUnsupportedAlterTableAddColumn(Oid relationId, AlterTableCmd *command, extern void ErrorUnsupportedAlterTableAddColumn(Oid relationId, AlterTableCmd *command,
Constraint *constraint); Constraint *constraint);
extern void ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod, extern void ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod,
char referencingReplicationModel,
Var *distributionColumn, uint32 colocationId); Var *distributionColumn, uint32 colocationId);
extern ObjectAddress AlterTableSchemaStmtObjectAddress(Node *stmt, extern ObjectAddress AlterTableSchemaStmtObjectAddress(Node *stmt,
bool missing_ok); bool missing_ok);
@ -237,7 +261,7 @@ extern List * MakeNameListFromRangeVar(const RangeVar *rel);
/* truncate.c - forward declarations */ /* truncate.c - forward declarations */
extern void PostprocessTruncateStatement(TruncateStmt *truncateStatement); extern void PreprocessTruncateStatement(TruncateStmt *truncateStatement);
/* type.c - forward declarations */ /* type.c - forward declarations */
extern List * PreprocessCompositeTypeStmt(Node *stmt, const char *queryString); extern List * PreprocessCompositeTypeStmt(Node *stmt, const char *queryString);
@ -278,9 +302,27 @@ extern void PostprocessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumComm
/* trigger.c - forward declarations */ /* trigger.c - forward declarations */
extern List * GetExplicitTriggerCommandList(Oid relationId); extern List * GetExplicitTriggerCommandList(Oid relationId);
extern HeapTuple GetTriggerTupleById(Oid triggerId, bool missingOk);
extern List * GetExplicitTriggerIdList(Oid relationId); extern List * GetExplicitTriggerIdList(Oid relationId);
extern Oid get_relation_trigger_oid_compat(HeapTuple heapTuple); 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 bool ShouldPropagateSetCommand(VariableSetStmt *setStmt);
extern void PostprocessVariableSetStmt(VariableSetStmt *setStmt, const char *setCommand); 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 List * GetTableIndexAndConstraintCommands(Oid relationId);
extern bool IndexImpliedByAConstraint(Form_pg_index indexForm); extern bool IndexImpliedByAConstraint(Form_pg_index indexForm);
extern char ShardStorageType(Oid relationId); extern char ShardStorageType(Oid relationId);
extern bool DistributedTableReplicationIsEnabled(void);
extern void CheckDistributedTable(Oid relationId); extern void CheckDistributedTable(Oid relationId);
extern void CreateAppendDistributedShardPlacements(Oid relationId, int64 shardId, extern void CreateAppendDistributedShardPlacements(Oid relationId, int64 shardId,
List *workerNodeList, int List *workerNodeList, int

View File

@ -135,6 +135,24 @@ typedef struct RelationRowLock
} RelationRowLock; } RelationRowLock;
/*
* Parameters to be set according to range table entries of a query.
*/
typedef struct RTEListProperties
{
bool hasPostgresLocalTable;
bool hasReferenceTable;
bool hasCitusLocalTable;
/* includes hash, append and range partitioned tables */
bool hasDistributedTable;
/* union of above three */
bool hasCitusTable;
} RTEListProperties;
typedef struct DistributedPlanningContext typedef struct DistributedPlanningContext
{ {
/* The parsed query that is given to the planner. It is a slightly modified /* The parsed query that is given to the planner. It is a slightly modified
@ -195,6 +213,14 @@ extern PlannedStmt * distributed_planner(Query *parse,
#endif #endif
/*
* Common hint message to workaround using postgres local and citus local tables
* in distributed queries
*/
#define LOCAL_TABLE_SUBQUERY_CTE_HINT \
"Use CTE's or subqueries to select from local tables and use them in joins"
extern List * ExtractRangeTableEntryList(Query *query); extern List * ExtractRangeTableEntryList(Query *query);
extern bool NeedsDistributedPlanning(Query *query); extern bool NeedsDistributedPlanning(Query *query);
extern struct DistributedPlan * GetDistributedPlan(CustomScan *node); extern struct DistributedPlan * GetDistributedPlan(CustomScan *node);
@ -218,6 +244,7 @@ extern int32 BlessRecordExpression(Expr *expr);
extern void DissuadePlannerFromUsingPlan(PlannedStmt *plan); extern void DissuadePlannerFromUsingPlan(PlannedStmt *plan);
extern PlannedStmt * FinalizePlan(PlannedStmt *localPlan, extern PlannedStmt * FinalizePlan(PlannedStmt *localPlan,
struct DistributedPlan *distributedPlan); struct DistributedPlan *distributedPlan);
extern RTEListProperties * GetRTEListPropertiesForQuery(Query *query);
extern struct DistributedPlan * CreateDistributedPlan(uint64 planId, Query *originalQuery, extern struct DistributedPlan * CreateDistributedPlan(uint64 planId, Query *originalQuery,

View File

@ -90,5 +90,7 @@ extern ArrayType * DatumArrayToArrayType(Datum *datumArray, int datumCount,
extern HTAB * ListToHashSet(List *pointerList, Size keySize, bool isStringList); extern HTAB * ListToHashSet(List *pointerList, Size keySize, bool isStringList);
extern char * StringJoin(List *stringList, char delimiter); extern char * StringJoin(List *stringList, char delimiter);
extern List * ListTake(List *pointerList, int size); 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 */ #endif /* CITUS_LISTUTILS_H */

View File

@ -133,10 +133,12 @@ typedef enum
} CitusTableType; } CitusTableType;
extern bool IsCitusTableType(Oid relationId, CitusTableType tableType); extern bool IsCitusTableType(Oid relationId, CitusTableType tableType);
extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry, CitusTableType extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry,
tableType); CitusTableType tableType);
extern bool IsCitusTable(Oid relationId); extern bool IsCitusTable(Oid relationId);
extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel);
extern bool IsReferenceTableByDistParams(char partitionMethod, char replicationModel);
extern List * CitusTableList(void); extern List * CitusTableList(void);
extern ShardInterval * LoadShardInterval(uint64 shardId); extern ShardInterval * LoadShardInterval(uint64 shardId);
extern Oid RelationIdForShard(uint64 shardId); extern Oid RelationIdForShard(uint64 shardId);
@ -145,6 +147,7 @@ extern ShardPlacement * FindShardPlacementOnGroup(int32 groupId, uint64 shardId)
extern GroupShardPlacement * LoadGroupShardPlacement(uint64 shardId, uint64 placementId); extern GroupShardPlacement * LoadGroupShardPlacement(uint64 shardId, uint64 placementId);
extern ShardPlacement * LoadShardPlacement(uint64 shardId, uint64 placementId); extern ShardPlacement * LoadShardPlacement(uint64 shardId, uint64 placementId);
extern CitusTableCacheEntry * GetCitusTableCacheEntry(Oid distributedRelationId); extern CitusTableCacheEntry * GetCitusTableCacheEntry(Oid distributedRelationId);
extern CitusTableCacheEntry * LookupCitusTableCacheEntry(Oid relationId);
extern DistObjectCacheEntry * LookupDistObjectCacheEntry(Oid classid, Oid objid, int32 extern DistObjectCacheEntry * LookupDistObjectCacheEntry(Oid classid, Oid objid, int32
objsubid); objsubid);
extern int32 GetLocalGroupId(void); extern int32 GetLocalGroupId(void);

View File

@ -153,6 +153,7 @@ extern void EnsureHashDistributedTable(Oid relationId);
extern void EnsureSequenceOwner(Oid sequenceOid); extern void EnsureSequenceOwner(Oid sequenceOid);
extern void EnsureFunctionOwner(Oid functionId); extern void EnsureFunctionOwner(Oid functionId);
extern void EnsureSuperUser(void); extern void EnsureSuperUser(void);
extern void EnsureTableNotDistributed(Oid relationId);
extern void EnsureReplicationSettings(Oid relationId, char replicationModel); extern void EnsureReplicationSettings(Oid relationId, char replicationModel);
extern bool RegularTable(Oid relationId); extern bool RegularTable(Oid relationId);
extern char * ConstructQualifiedShardName(ShardInterval *shardInterval); extern char * ConstructQualifiedShardName(ShardInterval *shardInterval);

View File

@ -192,8 +192,10 @@ extern bool TargetListOnPartitionColumn(Query *query, List *targetEntryList);
extern bool FindNodeMatchingCheckFunctionInRangeTableList(List *rtable, bool (*check)( extern bool FindNodeMatchingCheckFunctionInRangeTableList(List *rtable, bool (*check)(
Node *)); Node *));
extern bool IsCitusTableRTE(Node *node); extern bool IsCitusTableRTE(Node *node);
extern bool IsPostgresLocalTableRte(Node *node);
extern bool IsDistributedTableRTE(Node *node); extern bool IsDistributedTableRTE(Node *node);
extern bool IsReferenceTableRTE(Node *node); extern bool IsReferenceTableRTE(Node *node);
extern bool IsCitusLocalTableRTE(Node *node);
extern bool QueryContainsDistributedTableRTE(Query *query); extern bool QueryContainsDistributedTableRTE(Query *query);
extern bool IsCitusExtraDataContainerRelation(RangeTblEntry *rte); extern bool IsCitusExtraDataContainerRelation(RangeTblEntry *rte);
extern bool ContainsReadIntermediateResultFunction(Node *node); extern bool ContainsReadIntermediateResultFunction(Node *node);

View File

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

View File

@ -34,7 +34,8 @@
#define pg_plan_query_compat(p, q, c, b) pg_plan_query(p, q, c, b) #define pg_plan_query_compat(p, q, c, b) pg_plan_query(p, q, c, b)
#define planner_compat(p, c, b) planner(p, NULL, c, b) #define planner_compat(p, c, b) planner(p, NULL, c, b)
#define standard_planner_compat(a, c, d) standard_planner(a, NULL, c, d) #define standard_planner_compat(a, c, d) standard_planner(a, NULL, c, d)
#define getOwnedSequencesCompat(a, b) getOwnedSequences(a) #define GetSequencesOwnedByRelation(a) getOwnedSequences(a)
#define GetSequencesOwnedByColumn(a, b) getOwnedSequences_internal(a, b, 0)
#define CMDTAG_SELECT_COMPAT CMDTAG_SELECT #define CMDTAG_SELECT_COMPAT CMDTAG_SELECT
#define ExplainOnePlanCompat(a, b, c, d, e, f, g, h) \ #define ExplainOnePlanCompat(a, b, c, d, e, f, g, h) \
ExplainOnePlan(a, b, c, d, e, f, g, h) ExplainOnePlan(a, b, c, d, e, f, g, h)
@ -48,7 +49,8 @@
#define planner_compat(p, c, b) planner(p, c, b) #define planner_compat(p, c, b) planner(p, c, b)
#define standard_planner_compat(a, c, d) standard_planner(a, c, d) #define standard_planner_compat(a, c, d) standard_planner(a, c, d)
#define CMDTAG_SELECT_COMPAT "SELECT" #define CMDTAG_SELECT_COMPAT "SELECT"
#define getOwnedSequencesCompat(a, b) getOwnedSequences(a, b) #define GetSequencesOwnedByRelation(a) getOwnedSequences(a, InvalidAttrNumber)
#define GetSequencesOwnedByColumn(a, b) getOwnedSequences(a, b)
#define ExplainOnePlanCompat(a, b, c, d, e, f, g, h) ExplainOnePlan(a, b, c, d, e, f, g) #define ExplainOnePlanCompat(a, b, c, d, e, f, g, h) ExplainOnePlan(a, b, c, d, e, f, g)
#define SetListCellPtr(a, b) ((a)->data.ptr_value = (b)) #define SetListCellPtr(a, b) ((a)->data.ptr_value = (b))
#define RangeTableEntryFromNSItem(a) (a) #define RangeTableEntryFromNSItem(a) (a)

View File

@ -73,7 +73,10 @@ extern WorkerNode * WorkerGetLocalFirstCandidateNode(List *currentNodeList);
extern uint32 ActivePrimaryNonCoordinatorNodeCount(void); extern uint32 ActivePrimaryNonCoordinatorNodeCount(void);
extern List * ActivePrimaryNonCoordinatorNodeList(LOCKMODE lockMode); extern List * ActivePrimaryNonCoordinatorNodeList(LOCKMODE lockMode);
extern List * ActivePrimaryNodeList(LOCKMODE lockMode); extern List * ActivePrimaryNodeList(LOCKMODE lockMode);
extern bool CoordinatorAddedAsWorkerNode(void);
extern List * ReferenceTablePlacementNodeList(LOCKMODE lockMode); extern List * ReferenceTablePlacementNodeList(LOCKMODE lockMode);
extern WorkerNode * CoordinatorNodeIfAddedAsWorkerOrError(void);
extern void ErrorIfCoordinatorNotAddedAsWorkerNode(void);
extern List * DistributedTablePlacementNodeList(LOCKMODE lockMode); extern List * DistributedTablePlacementNodeList(LOCKMODE lockMode);
extern bool NodeCanHaveDistTablePlacements(WorkerNode *node); extern bool NodeCanHaveDistTablePlacements(WorkerNode *node);
extern uint32 ActiveReadableNonCoordinatorNodeCount(void); extern uint32 ActiveReadableNonCoordinatorNodeCount(void);

View File

@ -17,6 +17,7 @@ extern bool OverrideTableVisibility;
extern void ReplaceTableVisibleFunction(Node *inputNode); extern void ReplaceTableVisibleFunction(Node *inputNode);
extern void ErrorIfRelationIsAKnownShard(Oid relationId);
extern bool RelationIsAKnownShard(Oid shardRelationId, bool onlySearchPath); extern bool RelationIsAKnownShard(Oid shardRelationId, bool onlySearchPath);

View File

@ -34,6 +34,18 @@ s/ keyval(1|2|ref)_[0-9]+ / keyval\1_xxxxxxx /g
# shard table names for custom_aggregate_support # shard table names for custom_aggregate_support
s/ daily_uniques_[0-9]+ / daily_uniques_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
# 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 # In foreign_key_restriction_enforcement, normalize shard names
s/"(on_update_fkey_table_|fkey_)[0-9]+"/"\1xxxxxxx"/g 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

@ -0,0 +1,494 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1504000;
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_test_schema;
SET search_path TO citus_local_tables_test_schema;
---------------------------------------------------------------------
------- citus local table creation -------
---------------------------------------------------------------------
-- 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_1 (a int);
-- this should work as coordinator is added to pg_dist_node
SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- try to remove coordinator and observe failure as there exist a citus local table
SELECT 1 FROM master_remove_node('localhost', :master_port);
ERROR: you cannot remove the primary node of a node group which has shard placements
DROP TABLE citus_local_table_1;
NOTICE: executing the command locally: DROP TABLE IF EXISTS citus_local_tables_test_schema.citus_local_table_1_xxxxx CASCADE
-- this should work now as the citus local table is dropped
SELECT 1 FROM master_remove_node('localhost', :master_port);
?column?
---------------------------------------------------------------------
1
(1 row)
CREATE TABLE citus_local_table_1 (a int primary key);
-- this should fail as coordinator is removed from pg_dist_node
SELECT create_citus_local_table('citus_local_table_1');
ERROR: could not find the coordinator node in metadata as it is not added as a worker
-- let coordinator have citus local tables again for next tests
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;
-- creating citus local table having no data initially would work
SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- creating citus local table having data in it would also work
CREATE TABLE citus_local_table_2(a int primary key);
INSERT INTO citus_local_table_2 VALUES(1);
SELECT create_citus_local_table('citus_local_table_2');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- also create indexes on them
CREATE INDEX citus_local_table_1_idx ON citus_local_table_1(a);
NOTICE: executing the command locally: CREATE INDEX citus_local_table_1_idx_1504001 ON citus_local_tables_test_schema.citus_local_table_1_1504001 USING btree (a )
CREATE INDEX citus_local_table_2_idx ON citus_local_table_2(a);
NOTICE: executing the command locally: CREATE INDEX citus_local_table_2_idx_1504002 ON citus_local_tables_test_schema.citus_local_table_2_1504002 USING btree (a )
-- drop them for next tests
DROP TABLE citus_local_table_1, citus_local_table_2;
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
-- create indexes before creating the citus local tables
-- .. for an initially empty table
CREATE TABLE citus_local_table_1(a int);
CREATE INDEX citus_local_table_1_idx ON citus_local_table_1(a);
SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- .. and for another table having data in it before creating citus local table
CREATE TABLE citus_local_table_2(a int);
INSERT INTO citus_local_table_2 VALUES(1);
CREATE INDEX citus_local_table_2_idx ON citus_local_table_2(a);
SELECT create_citus_local_table('citus_local_table_2');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE distributed_table (a int);
SELECT create_distributed_table('distributed_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- cannot create citus local table from an existing citus table
SELECT create_citus_local_table('distributed_table');
ERROR: table "distributed_table" is already distributed
-- partitioned table tests --
CREATE TABLE partitioned_table(a int, b int) PARTITION BY RANGE (a);
CREATE TABLE partitioned_table_1 PARTITION OF partitioned_table FOR VALUES FROM (0) TO (10);
CREATE TABLE partitioned_table_2 PARTITION OF partitioned_table FOR VALUES FROM (10) TO (20);
-- cannot create partitioned citus local tables
SELECT create_citus_local_table('partitioned_table');
ERROR: cannot create citus local table "partitioned_table", only regular tables and foreign tables are supported for citus local table creation
BEGIN;
CREATE TABLE citus_local_table PARTITION OF partitioned_table FOR VALUES FROM (20) TO (30);
-- cannot create citus local table as a partition of a local table
SELECT create_citus_local_table('citus_local_table');
ERROR: cannot create citus local table "citus_local_table", citus local tables cannot be partition of other tables
ROLLBACK;
BEGIN;
CREATE TABLE citus_local_table (a int, b int);
SELECT create_citus_local_table('citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- cannot create citus local table as a partition of a local table
-- via ALTER TABLE commands as well
ALTER TABLE partitioned_table ATTACH PARTITION citus_local_table FOR VALUES FROM (20) TO (30);
ERROR: non-distributed tables cannot have distributed partitions
ROLLBACK;
BEGIN;
SELECT create_distributed_table('partitioned_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE citus_local_table (a int, b int);
SELECT create_citus_local_table('citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- 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: 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);
CREATE TABLE child_table () INHERITS (parent_table);
-- both of below should error out
SELECT create_citus_local_table('parent_table');
ERROR: cannot create citus local table "parent_table", citus local tables cannot be involved in inheritance relationships
SELECT create_citus_local_table('child_table');
ERROR: cannot create citus local table "child_table", citus local tables cannot be involved in inheritance relationships
-- show that we support UNLOGGED tables --
CREATE UNLOGGED TABLE unlogged_table (a int primary key);
SELECT create_citus_local_table('unlogged_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- show that we allow triggers --
BEGIN;
CREATE TABLE citus_local_table_3 (value int);
-- create a simple function to be invoked by trigger
CREATE FUNCTION update_value() RETURNS trigger AS $update_value$
BEGIN
UPDATE citus_local_table_3 SET value=value+1;
RETURN NEW;
END;
$update_value$ LANGUAGE plpgsql;
CREATE TRIGGER insert_trigger
AFTER INSERT ON citus_local_table_3
FOR EACH STATEMENT EXECUTE FUNCTION update_value();
SELECT create_citus_local_table('citus_local_table_3');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
INSERT INTO citus_local_table_3 VALUES (1);
NOTICE: executing the command locally: INSERT INTO citus_local_tables_test_schema.citus_local_table_3_1504024 (value) VALUES (1)
NOTICE: executing the command locally: UPDATE citus_local_tables_test_schema.citus_local_table_3_1504024 citus_local_table_3 SET value = (value OPERATOR(pg_catalog.+) 1)
-- show that trigger is executed only once, we should see "2" (not "3")
SELECT * FROM citus_local_table_3;
NOTICE: executing the command locally: SELECT value FROM citus_local_tables_test_schema.citus_local_table_3_1504024 citus_local_table_3
value
---------------------------------------------------------------------
2
(1 row)
ROLLBACK;
-- show that we do not support policies in citus community --
BEGIN;
CREATE TABLE citus_local_table_3 (table_user text);
ALTER TABLE citus_local_table_3 ENABLE ROW LEVEL SECURITY;
CREATE ROLE table_users;
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
CREATE POLICY table_policy ON citus_local_table_3 TO table_users
USING (table_user = current_user);
-- this should error out
SELECT create_citus_local_table('citus_local_table_3');
ERROR: policies on distributed tables are only supported in Citus Enterprise
ROLLBACK;
-- show that we properly handle sequences on citus local tables --
BEGIN;
CREATE SEQUENCE col3_seq;
CREATE TABLE citus_local_table_3 (col1 serial, col2 int, col3 int DEFAULT nextval('col3_seq'));
SELECT create_citus_local_table('citus_local_table_3');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- print column default expressions
-- we should only see shell relation below
SELECT table_name, column_name, column_default
FROM information_schema.COLUMNS
WHERE table_name like 'citus_local_table_3%' and column_default != '' ORDER BY 1,2;
table_name | column_name | column_default
---------------------------------------------------------------------
citus_local_table_3 | col1 | nextval('citus_local_table_3_col1_seq'::regclass)
citus_local_table_3 | col3 | nextval('col3_seq'::regclass)
(2 rows)
-- print sequence ownerships
-- show that the only internal sequence is on col1 and it is owned by shell relation
SELECT s.relname as sequence_name, t.relname, a.attname
FROM pg_class s
JOIN pg_depend d on d.objid=s.oid and d.classid='pg_class'::regclass and d.refclassid='pg_class'::regclass
JOIN pg_class t on t.oid=d.refobjid
JOIN pg_attribute a on a.attrelid=t.oid and a.attnum=d.refobjsubid
WHERE s.relkind='S' and s.relname like 'citus_local_table_3%' ORDER BY 1,2;
sequence_name | relname | attname
---------------------------------------------------------------------
citus_local_table_3_col1_seq | citus_local_table_3 | col1
(1 row)
ROLLBACK;
-- test foreign tables using fake FDW --
CREATE FOREIGN TABLE foreign_table (
id bigint not null,
full_name text not null default ''
) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true');
-- observe that we do not create fdw server for shell table, both shard relation
-- & shell relation points to the same same server object
SELECT create_citus_local_table('foreign_table');
NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined
NOTICE: server "fake_fdw_server" already exists, skipping
NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- drop them for next tests
DROP TABLE citus_local_table_1, citus_local_table_2, distributed_table;
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
-- create test tables
CREATE TABLE citus_local_table_1 (a int primary key);
SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE citus_local_table_2 (a int primary key);
SELECT create_citus_local_table('citus_local_table_2');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE local_table (a int primary key);
CREATE TABLE distributed_table (a int primary key);
SELECT create_distributed_table('distributed_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table (a int primary key);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
-- show that colociation of citus local tables are not supported for now
-- between citus local tables
SELECT mark_tables_colocated('citus_local_table_1', ARRAY['citus_local_table_2']);
ERROR: citus local tables cannot be colocated with other tables
-- between citus local tables and reference tables
SELECT mark_tables_colocated('citus_local_table_1', ARRAY['reference_table']);
ERROR: citus local tables cannot be colocated with other tables
SELECT mark_tables_colocated('reference_table', ARRAY['citus_local_table_1']);
ERROR: citus local tables cannot be colocated with other tables
-- between citus local tables and distributed tables
SELECT mark_tables_colocated('citus_local_table_1', ARRAY['distributed_table']);
ERROR: citus local tables cannot be colocated with other tables
SELECT mark_tables_colocated('distributed_table', ARRAY['citus_local_table_1']);
ERROR: citus local tables cannot be colocated with other tables
-- upgrade_to_reference_table is not supported
SELECT upgrade_to_reference_table('citus_local_table_1');
ERROR: cannot upgrade to reference table
-- master_create_empty_shard is not supported
SELECT master_create_empty_shard('citus_local_table_1');
ERROR: relation "citus_local_table_1" is a citus local table
-- get_shard_id_for_distribution_column is supported
SELECT get_shard_id_for_distribution_column('citus_local_table_1', 'not_checking_this_arg_for_non_dist_tables');
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1504027
(1 row)
SELECT get_shard_id_for_distribution_column('citus_local_table_1');
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1504027
(1 row)
-- master_copy_shard_placement is not supported
SELECT master_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, true)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table_1'::regclass) as shardid;
ERROR: Table 'citus_local_table_1' is a citus local table. Replicating shard of a citus local table currently is not supported
-- undistribute_table is supported
BEGIN;
SELECT undistribute_table('citus_local_table_1');
NOTICE: Creating a new local table for citus_local_tables_test_schema.citus_local_table_1
NOTICE: Moving the data of citus_local_tables_test_schema.citus_local_table_1
NOTICE: executing the command locally: SELECT a FROM citus_local_tables_test_schema.citus_local_table_1_1504027 citus_local_table_1
NOTICE: Dropping the old citus_local_tables_test_schema.citus_local_table_1
NOTICE: executing the command locally: DROP TABLE IF EXISTS citus_local_tables_test_schema.citus_local_table_1_xxxxx CASCADE
NOTICE: Renaming the new table to citus_local_tables_test_schema.citus_local_table_1
undistribute_table
---------------------------------------------------------------------
(1 row)
ROLLBACK;
-- tests with citus local tables initially having foreign key relationships
CREATE TABLE local_table_1 (a int primary key);
CREATE TABLE local_table_2 (a int primary key references local_table_1(a));
CREATE TABLE local_table_3 (a int primary key, b int references local_table_3(a));
-- below two should fail as we do not allow foreign keys between
-- postgres local tables and citus local tables
SELECT create_citus_local_table('local_table_1');
ERROR: relation "local_table_1" is involved in a foreign key relationship with another table
SELECT create_citus_local_table('local_table_2');
ERROR: relation "local_table_2" is involved in a foreign key relationship with another table
-- below should work as we allow initial self references in citus local tables
SELECT create_citus_local_table('local_table_3');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
---------------------------------------------------------------------
----- tests for object names that should be escaped properly -----
---------------------------------------------------------------------
CREATE SCHEMA "CiTUS!LocalTables";
-- create table with weird names
CREATE TABLE "CiTUS!LocalTables"."LocalTabLE.1!?!"(id int, "TeNANt_Id" int);
-- should work
SELECT create_citus_local_table('"CiTUS!LocalTables"."LocalTabLE.1!?!"');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- drop the table before creating it when the search path is set
SET search_path to "CiTUS!LocalTables" ;
DROP TABLE "LocalTabLE.1!?!";
NOTICE: executing the command locally: DROP TABLE IF EXISTS "CiTUS!LocalTables"."LocalTabLE.1!?!_1504035" CASCADE
-- have a custom type in the local table
CREATE TYPE local_type AS (key int, value jsonb);
-- create btree_gist for GiST index
CREATE EXTENSION btree_gist;
CREATE TABLE "LocalTabLE.1!?!"(
id int PRIMARY KEY,
"TeNANt_Id" int,
"local_Type" local_type,
"jsondata" jsonb NOT NULL,
name text,
price numeric CHECK (price > 0),
serial_data bigserial, UNIQUE (id, price),
EXCLUDE USING GIST (name WITH =));
-- create some objects before create_citus_local_table
CREATE INDEX "my!Index1" ON "LocalTabLE.1!?!"(id) WITH ( fillfactor = 80 ) WHERE id > 10;
CREATE UNIQUE INDEX uniqueIndex ON "LocalTabLE.1!?!" (id);
-- ingest some data before create_citus_local_table
INSERT INTO "LocalTabLE.1!?!" VALUES (1, 1, (1, row_to_json(row(1,1)))::local_type, row_to_json(row(1,1), true)),
(2, 1, (2, row_to_json(row(2,2)))::local_type, row_to_json(row(2,2), 'false'));
-- create a replica identity before create_citus_local_table
ALTER TABLE "LocalTabLE.1!?!" REPLICA IDENTITY USING INDEX uniqueIndex;
-- this shouldn't give any syntax errors
SELECT create_citus_local_table('"LocalTabLE.1!?!"');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
-- create some objects after create_citus_local_table
CREATE INDEX "my!Index2" ON "LocalTabLE.1!?!"(id) WITH ( fillfactor = 90 ) WHERE id < 20;
NOTICE: executing the command locally: CREATE INDEX "my!Index2_1504036" ON "CiTUS!LocalTables"."LocalTabLE.1!?!_1504036" USING btree (id ) WITH (fillfactor = '90' )WHERE (id < 20)
CREATE UNIQUE INDEX uniqueIndex2 ON "LocalTabLE.1!?!"(id);
NOTICE: executing the command locally: CREATE UNIQUE INDEX uniqueindex2_1504036 ON "CiTUS!LocalTables"."LocalTabLE.1!?!_1504036" USING btree (id )
---------------------------------------------------------------------
---- utility command execution ----
---------------------------------------------------------------------
SET search_path TO citus_local_tables_test_schema;
-- 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: 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: 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
-- test vacuum
VACUUM citus_local_table_1;
VACUUM citus_local_table_1, distributed_table, local_table, reference_table;
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 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

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -1,12 +1,13 @@
-- This test file includes tests to show that we do not allow triggers -- 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 -- on distributed tables and reference tables. Note that in other
-- the successfull citus table creation cases. -- regression tests, we already test the successfull citus table
-- creation cases.
\set VERBOSITY terse \set VERBOSITY terse
SET citus.next_shard_id TO 1505000; SET citus.next_shard_id TO 1505000;
CREATE SCHEMA table_triggers_schema; CREATE SCHEMA table_triggers_schema;
SET search_path TO 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 a simple function to be invoked by triggers
CREATE FUNCTION update_value() RETURNS trigger AS $update_value$ CREATE FUNCTION update_value() RETURNS trigger AS $update_value$
@ -32,12 +33,65 @@ SELECT create_reference_table('reference_table');
-- below two should fail -- below two should fail
CREATE TRIGGER update_value_dist CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE PROCEDURE update_value(); FOR EACH ROW EXECUTE FUNCTION update_value();
ERROR: cannot create trigger on relation "distributed_table" because it is either a distributed table or a reference table ERROR: triggers are only supported for citus local tables
CREATE TRIGGER update_value_ref CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table AFTER INSERT ON reference_table
FOR EACH ROW EXECUTE PROCEDURE update_value(); FOR EACH ROW EXECUTE FUNCTION update_value();
ERROR: cannot create trigger on relation "reference_table" because it is either a distributed table or a reference table 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 -- show that we do not allow creating citus tables if the
-- table has already triggers -- 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 TABLE distributed_table_1 (value int);
CREATE TRIGGER update_value_dist CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table_1 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 TABLE reference_table_1 (value int);
CREATE TRIGGER update_value_ref CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table_1 AFTER INSERT ON reference_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value(); FOR EACH ROW EXECUTE FUNCTION update_value();
-- below two should fail -- below two should fail
SELECT create_distributed_table('distributed_table_1', 'value'); SELECT create_distributed_table('distributed_table_1', 'value');
ERROR: cannot distribute relation "distributed_table_1" because it has triggers 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 -- cleanup at exit
DROP SCHEMA table_triggers_schema CASCADE; DROP SCHEMA table_triggers_schema CASCADE;
NOTICE: drop cascades to 7 other objects NOTICE: drop cascades to 8 other objects

View File

@ -418,9 +418,11 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut
INSERT INTO dist_table VALUES(1); INSERT INTO dist_table VALUES(1);
NOTICE: executing the command locally: INSERT INTO coordinator_shouldhaveshards.dist_table_1503017 (a) VALUES (1) NOTICE: executing the command locally: INSERT INTO coordinator_shouldhaveshards.dist_table_1503017 (a) VALUES (1)
SELECT * FROM local JOIN dist_table ON (a = x); SELECT * FROM local JOIN dist_table ON (a = x);
ERROR: relation local is not distributed ERROR: direct joins between distributed and local tables are not supported
HINT: Use CTE's or subqueries to select from local tables and use them in joins
SELECT * FROM local JOIN dist_table ON (a = x) WHERE a = 1;; SELECT * FROM local JOIN dist_table ON (a = x) WHERE a = 1;;
ERROR: relation local is not distributed ERROR: direct joins between distributed and local tables are not supported
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- intermediate results are allowed -- intermediate results are allowed
WITH cte_1 AS (SELECT * FROM dist_table LIMIT 1) WITH cte_1 AS (SELECT * FROM dist_table LIMIT 1)
SELECT * FROM ref JOIN local ON (a = x) JOIN cte_1 ON (local.x = cte_1.a); SELECT * FROM ref JOIN local ON (a = x) JOIN cte_1 ON (local.x = cte_1.a);

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

@ -350,7 +350,8 @@ FROM
distributed_table distributed_table
WHERE WHERE
distributed_table.tenant_id = local_table.id; distributed_table.tenant_id = local_table.id;
ERROR: cannot plan modifications of local tables involving distributed tables ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
RESET client_min_messages; RESET client_min_messages;
DROP SCHEMA recursive_dml_queries CASCADE; DROP SCHEMA recursive_dml_queries CASCADE;
NOTICE: drop cascades to 5 other objects NOTICE: drop cascades to 5 other objects

View File

@ -351,7 +351,7 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99; SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -368,7 +368,7 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99; SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -386,7 +386,7 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 99; SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 99;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -415,7 +415,7 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 99; SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 101 AND id = 99;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -445,20 +445,20 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 15;
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 15;
ROLLBACK; ROLLBACK;
-- case 2.4: UPDATE to a reference table is followed by multiple router UPDATEs -- case 2.4: UPDATE to a reference table is followed by multiple router UPDATEs
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 1; UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 1;
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 2; UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 2;
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 3; UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 3;
@ -467,7 +467,7 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 1; UPDATE on_update_fkey_table SET value_1 = 101 WHERE id = 1;
ERROR: insert or update on table "on_update_fkey_table_xxxxxxx" violates foreign key constraint "fkey_xxxxxxx" ERROR: insert or update on table "on_update_fkey_table_xxxxxxx" violates foreign key constraint "fkey_xxxxxxx"
DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001". DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001".
@ -483,7 +483,7 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint; ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
DEBUG: rewriting table "on_update_fkey_table" DEBUG: rewriting table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey" DEBUG: validating foreign key constraint "fkey"
@ -491,7 +491,7 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint; ALTER TABLE on_update_fkey_table ALTER COLUMN value_1 SET DATA TYPE bigint;
DEBUG: rewriting table "on_update_fkey_table" DEBUG: rewriting table "on_update_fkey_table"
DEBUG: validating foreign key constraint "fkey" DEBUG: validating foreign key constraint "fkey"
@ -500,26 +500,26 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT; ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT;
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT; ALTER TABLE on_update_fkey_table ADD COLUMN value_1_X INT;
ROLLBACK; ROLLBACK;
-- case 2.7: UPDATE to a reference table is followed by COPY -- case 2.7: UPDATE to a reference table is followed by COPY
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
COPY on_update_fkey_table FROM STDIN WITH CSV; COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
COPY on_update_fkey_table FROM STDIN WITH CSV; COPY on_update_fkey_table FROM STDIN WITH CSV;
ERROR: insert or update on table "on_update_fkey_table_xxxxxxx" violates foreign key constraint "fkey_xxxxxxx" ERROR: insert or update on table "on_update_fkey_table_xxxxxxx" violates foreign key constraint "fkey_xxxxxxx"
DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001". DETAIL: Key (value_1)=(101) is not present in table "reference_table_2380001".
@ -528,20 +528,20 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
TRUNCATE on_update_fkey_table; TRUNCATE on_update_fkey_table;
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
TRUNCATE on_update_fkey_table; TRUNCATE on_update_fkey_table;
ROLLBACK; ROLLBACK;
-- case 3.1: an unrelated DDL to a reference table is followed by a real-time SELECT -- case 3.1: an unrelated DDL to a reference table is followed by a real-time SELECT
BEGIN; BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DEFAULT 1001; ALTER TABLE reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
SELECT count(*) FROM on_update_fkey_table; SELECT count(*) FROM on_update_fkey_table;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -552,7 +552,7 @@ ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001; ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
SELECT count(*) FROM on_update_fkey_table; SELECT count(*) FROM on_update_fkey_table;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -583,20 +583,20 @@ ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DEFAULT 1001; ALTER TABLE reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11; UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001; ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11; UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ROLLBACK; ROLLBACK;
-- case 3.4: DDL to a reference table followed by multiple router UPDATEs -- case 3.4: DDL to a reference table followed by multiple router UPDATEs
BEGIN; BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DEFAULT 1001; ALTER TABLE reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 1; UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 1;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 2; UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 2;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 3; UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 3;
@ -605,7 +605,7 @@ ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001; ALTER TABLE transitive_reference_table ALTER COLUMN id SET DEFAULT 1001;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 1; UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 1;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 2; UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 2;
UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 3; UPDATE on_update_fkey_table SET value_1 = 98 WHERE id = 3;
@ -645,26 +645,26 @@ ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE reference_table ADD COLUMN X int; ALTER TABLE reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
COPY on_update_fkey_table FROM STDIN WITH CSV; COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE transitive_reference_table ADD COLUMN X int; ALTER TABLE transitive_reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
COPY on_update_fkey_table FROM STDIN WITH CSV; COPY on_update_fkey_table FROM STDIN WITH CSV;
ROLLBACK; ROLLBACK;
-- case 3.8: DDL to a reference table is followed by TRUNCATE -- case 3.8: DDL to a reference table is followed by TRUNCATE
BEGIN; BEGIN;
ALTER TABLE reference_table ADD COLUMN X int; ALTER TABLE reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
TRUNCATE on_update_fkey_table; TRUNCATE on_update_fkey_table;
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE transitive_reference_table ADD COLUMN X int; ALTER TABLE transitive_reference_table ADD COLUMN X int;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "transitive_reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
TRUNCATE on_update_fkey_table; TRUNCATE on_update_fkey_table;
ROLLBACK; ROLLBACK;
-- case 3.9: DDL to a reference table is followed by TRUNCATE -- case 3.9: DDL to a reference table is followed by TRUNCATE
@ -721,8 +721,8 @@ BEGIN;
(1 row) (1 row)
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify reference table "reference_table" because there was a parallel operation on a distributed table ERROR: cannot modify table "reference_table" because there was a parallel operation on a distributed table
DETAIL: When there is a foreign key to a reference table, Citus needs to perform all operations over a single connection per node to ensure consistency. DETAIL: When there is a foreign key to a reference table or to a citus local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
@ -733,8 +733,8 @@ BEGIN;
(1 row) (1 row)
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify reference table "transitive_reference_table" because there was a parallel operation on a distributed table ERROR: cannot modify table "transitive_reference_table" because there was a parallel operation on a distributed table
DETAIL: When there is a foreign key to a reference table, Citus needs to perform all operations over a single connection per node to ensure consistency. DETAIL: When there is a foreign key to a reference table or to a citus local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 4.3: SELECT to a dist table is follwed by an unrelated DDL to a reference table -- case 4.3: SELECT to a dist table is follwed by an unrelated DDL to a reference table
@ -746,7 +746,7 @@ BEGIN;
(1 row) (1 row)
ALTER TABLE reference_table ADD COLUMN X INT; ALTER TABLE reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference table "reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
@ -757,7 +757,7 @@ BEGIN;
(1 row) (1 row)
ALTER TABLE transitive_reference_table ADD COLUMN X INT; ALTER TABLE transitive_reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference table "transitive_reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "transitive_reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 4.4: SELECT to a dist table is follwed by a DDL to a reference table -- case 4.4: SELECT to a dist table is follwed by a DDL to a reference table
@ -771,7 +771,7 @@ BEGIN;
ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint; ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "reference_table" DEBUG: rewriting table "reference_table"
DEBUG: validating foreign key constraint "fkey" DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference table "reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
@ -784,7 +784,7 @@ BEGIN;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint; ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
DEBUG: rewriting table "transitive_reference_table" DEBUG: rewriting table "transitive_reference_table"
DEBUG: validating foreign key constraint "fkey" DEBUG: validating foreign key constraint "fkey"
ERROR: cannot execute DDL on reference table "transitive_reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "transitive_reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 4.5: SELECT to a dist table is follwed by a TRUNCATE -- case 4.5: SELECT to a dist table is follwed by a TRUNCATE
@ -799,7 +799,7 @@ BEGIN;
TRUNCATE reference_table CASCADE; TRUNCATE reference_table CASCADE;
NOTICE: truncate cascades to table "on_update_fkey_table" NOTICE: truncate cascades to table "on_update_fkey_table"
ERROR: cannot execute DDL on reference table "reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99; SELECT count(*) FROM on_update_fkey_table WHERE value_1 = 99;
@ -811,7 +811,7 @@ BEGIN;
TRUNCATE transitive_reference_table CASCADE; TRUNCATE transitive_reference_table CASCADE;
NOTICE: truncate cascades to table "reference_table" NOTICE: truncate cascades to table "reference_table"
NOTICE: truncate cascades to table "on_update_fkey_table" NOTICE: truncate cascades to table "on_update_fkey_table"
ERROR: cannot execute DDL on reference table "transitive_reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "transitive_reference_table" because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction
ROLLBACK; ROLLBACK;
-- case 4.6: Router SELECT to a dist table is followed by a TRUNCATE -- case 4.6: Router SELECT to a dist table is followed by a TRUNCATE
BEGIN; BEGIN;
@ -849,7 +849,7 @@ BEGIN;
DROP TABLE reference_table CASCADE; DROP TABLE reference_table CASCADE;
NOTICE: drop cascades to constraint fkey on table on_update_fkey_table NOTICE: drop cascades to constraint fkey on table on_update_fkey_table
ERROR: cannot execute DDL on reference table because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table because there was a parallel SELECT access to distributed table "on_update_fkey_table" in the same transaction
ROLLBACK; ROLLBACK;
-- case 4.8: Router SELECT to a dist table is followed by a TRUNCATE -- case 4.8: Router SELECT to a dist table is followed by a TRUNCATE
-- No errors expected from below block as SELECT there is a router -- No errors expected from below block as SELECT there is a router
@ -889,13 +889,13 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
UPDATE reference_table SET id = 160 WHERE id = 15; UPDATE reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
UPDATE transitive_reference_table SET id = 160 WHERE id = 15; UPDATE transitive_reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
@ -916,7 +916,7 @@ BEGIN;
(10 rows) (10 rows)
UPDATE reference_table SET id = 160 WHERE id = 15; UPDATE reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
@ -937,46 +937,46 @@ BEGIN;
(10 rows) (10 rows)
UPDATE transitive_reference_table SET id = 160 WHERE id = 15; UPDATE transitive_reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 5.3: Parallel UPDATE on distributed table follow by an unrelated DDL on reference table -- case 5.3: Parallel UPDATE on distributed table follow by an unrelated DDL on reference table
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE reference_table ADD COLUMN X INT; ALTER TABLE reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_table ADD COLUMN X INT; ALTER TABLE transitive_reference_table ADD COLUMN X INT;
ERROR: cannot execute DDL on reference table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 5.4: Parallel UPDATE on distributed table follow by a related DDL on reference table -- case 5.4: Parallel UPDATE on distributed table follow by a related DDL on reference table
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint; ALTER TABLE reference_table ALTER COLUMN id SET DATA TYPE smallint;
ERROR: cannot execute DDL on reference table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15; UPDATE on_update_fkey_table SET value_1 = 16 WHERE value_1 = 15;
ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint; ALTER TABLE transitive_reference_table ALTER COLUMN id SET DATA TYPE smallint;
ERROR: cannot execute DDL on reference table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 6:1: Unrelated parallel DDL on distributed table followed by SELECT on ref. table -- case 6:1: Unrelated parallel DDL on distributed table followed by SELECT on ref. table
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM reference_table; SELECT count(*) FROM reference_table;
ERROR: cannot execute SELECT on reference table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute SELECT on table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_table; SELECT count(*) FROM transitive_reference_table;
ERROR: cannot execute SELECT on reference table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute SELECT on table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 6:2: Related parallel DDL on distributed table followed by SELECT on ref. table -- case 6:2: Related parallel DDL on distributed table followed by SELECT on ref. table
@ -992,39 +992,39 @@ ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM reference_table; SELECT count(*) FROM reference_table;
ERROR: cannot execute SELECT on reference table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute SELECT on table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
SELECT count(*) FROM transitive_reference_table; SELECT count(*) FROM transitive_reference_table;
ERROR: cannot execute SELECT on reference table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute SELECT on table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 6:4: Related parallel DDL on distributed table followed by SELECT on ref. table -- case 6:4: Related parallel DDL on distributed table followed by SELECT on ref. table
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
UPDATE reference_table SET id = 160 WHERE id = 15; UPDATE reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
UPDATE transitive_reference_table SET id = 160 WHERE id = 15; UPDATE transitive_reference_table SET id = 160 WHERE id = 15;
ERROR: cannot execute DML on reference table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 6:5: Unrelated parallel DDL on distributed table followed by unrelated DDL on ref. table -- case 6:5: Unrelated parallel DDL on distributed table followed by unrelated DDL on ref. table
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
ALTER TABLE reference_table ADD COLUMN X int; ALTER TABLE reference_table ADD COLUMN X int;
ERROR: cannot execute DDL on reference table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
ALTER TABLE on_update_fkey_table ADD COLUMN X int; ALTER TABLE on_update_fkey_table ADD COLUMN X int;
ALTER TABLE transitive_reference_table ADD COLUMN X int; ALTER TABLE transitive_reference_table ADD COLUMN X int;
ERROR: cannot execute DDL on reference table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DDL on table "transitive_reference_table" because there was a parallel DDL access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- case 6:6: Unrelated parallel DDL on distributed table followed by related DDL on ref. table -- case 6:6: Unrelated parallel DDL on distributed table followed by related DDL on ref. table
@ -1040,13 +1040,13 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11; UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
DELETE FROM reference_table WHERE id = 99; DELETE FROM reference_table WHERE id = 99;
ERROR: cannot execute DML on reference table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11; UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
DELETE FROM transitive_reference_table WHERE id = 99; DELETE FROM transitive_reference_table WHERE id = 99;
ERROR: cannot execute DML on reference table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- an unrelated update followed by update on dist table and update -- an unrelated update followed by update on dist table and update
@ -1055,14 +1055,14 @@ BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15; UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11; UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
ERROR: cannot execute DML on reference table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15; UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11; UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
UPDATE transitive_reference_table SET id = 101 WHERE id = 99; UPDATE transitive_reference_table SET id = 101 WHERE id = 99;
ERROR: cannot execute DML on reference table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction ERROR: cannot execute DML on table "transitive_reference_table" because there was a parallel DML access to distributed table "on_update_fkey_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK; ROLLBACK;
-- an unrelated update followed by update on the reference table and update -- an unrelated update followed by update on the reference table and update
@ -1073,8 +1073,8 @@ ROLLBACK;
BEGIN; BEGIN;
UPDATE unrelated_dist_table SET value_1 = 15; UPDATE unrelated_dist_table SET value_1 = 15;
UPDATE reference_table SET id = 101 WHERE id = 99; UPDATE reference_table SET id = 101 WHERE id = 99;
ERROR: cannot modify reference table "reference_table" because there was a parallel operation on a distributed table ERROR: cannot modify table "reference_table" because there was a parallel operation on a distributed table
DETAIL: When there is a foreign key to a reference table, Citus needs to perform all operations over a single connection per node to ensure consistency. DETAIL: When there is a foreign key to a reference table or to a citus local table, Citus needs to perform all operations over a single connection per node to ensure consistency.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11; UPDATE on_update_fkey_table SET value_1 = 5 WHERE id != 11;
ERROR: current transaction is aborted, commands ignored until end of transaction block ERROR: current transaction is aborted, commands ignored until end of transaction block
@ -1403,7 +1403,7 @@ INSERT INTO reference_table SELECT i FROM generate_series(0, 10) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
INSERT INTO distributed_table SELECT i, i % 10 FROM generate_series(0, 100) i; INSERT INTO distributed_table SELECT i, i % 10 FROM generate_series(0, 100) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator DEBUG: Collecting INSERT ... SELECT results on coordinator
@ -1414,7 +1414,7 @@ WITH t1 AS (DELETE FROM reference_table RETURNING id)
DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.distributed_table USING (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) RETURNING distributed_table.id, distributed_table.value_1, t1.id DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.distributed_table USING (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) RETURNING distributed_table.id, distributed_table.value_1, t1.id
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
id | value_1 | id id | value_1 | id
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -1424,7 +1424,7 @@ INSERT INTO reference_table SELECT i FROM generate_series(0, 10) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
INSERT INTO distributed_table SELECT i, i % 10 FROM generate_series(0, 100) i; INSERT INTO distributed_table SELECT i, i % 10 FROM generate_series(0, 100) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator DEBUG: Collecting INSERT ... SELECT results on coordinator
@ -1435,7 +1435,7 @@ WITH t1 AS (DELETE FROM reference_table RETURNING id)
DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1 WHERE (distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id)
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: Reference table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode DETAIL: Table "reference_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
@ -1449,7 +1449,7 @@ WITH t1 AS (DELETE FROM distributed_table RETURNING id),
DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: generating subplan XXX_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id DEBUG: generating subplan XXX_2 for CTE t2: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id)) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM test_fkey_to_ref_in_tx.distributed_table, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t1, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id integer)) t2 WHERE ((distributed_table.value_1 OPERATOR(pg_catalog.=) t1.id) AND (distributed_table.value_1 OPERATOR(pg_catalog.=) t2.id))
ERROR: cannot execute DML on reference table "reference_table" because there was a parallel DML access to distributed table "distributed_table" in the same transaction ERROR: cannot execute DML on table "reference_table" because there was a parallel DML access to distributed table "distributed_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
-- similarly this should fail since we first access to a distributed -- similarly this should fail since we first access to a distributed
-- table via t1, and then access to the reference table in the main query -- table via t1, and then access to the reference table in the main query
@ -1457,7 +1457,7 @@ WITH t1 AS (DELETE FROM distributed_table RETURNING id)
DELETE FROM reference_table RETURNING id; DELETE FROM reference_table RETURNING id;
DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id DEBUG: generating subplan XXX_1 for CTE t1: DELETE FROM test_fkey_to_ref_in_tx.distributed_table RETURNING id
DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM test_fkey_to_ref_in_tx.reference_table RETURNING id
ERROR: cannot execute DML on reference table "reference_table" because there was a parallel DML access to distributed table "distributed_table" in the same transaction ERROR: cannot execute DML on table "reference_table" because there was a parallel DML access to distributed table "distributed_table" in the same transaction
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';" HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
-- finally, make sure that we can execute the same queries -- finally, make sure that we can execute the same queries
-- in the sequential mode -- in the sequential mode

View File

@ -0,0 +1,215 @@
Parsed test spec with 2 sessions
starting permutation: s1-begin s2-begin s1-create-citus-local-table-1 s2-create-citus-local-table-1 s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-1: <... completed>
error in steps s1-commit s2-create-citus-local-table-1: ERROR: relation "citus_local_table_1_xxxxxxx" is a shard relation
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-create-citus-local-table-3 s2-create-citus-local-table-3 s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-3: SELECT create_citus_local_table('another_schema.citus_local_table_3');
create_citus_local_table
step s2-create-citus-local-table-3: SELECT create_citus_local_table('another_schema.citus_local_table_3'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-3: <... completed>
error in steps s1-commit s2-create-citus-local-table-3: ERROR: relation "citus_local_table_3_xxxxxxx" is a shard relation
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-create-citus-local-table-1 s2-create-citus-local-table-1 s1-rollback s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-rollback: ROLLBACK;
step s2-create-citus-local-table-1: <... completed>
create_citus_local_table
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-create-citus-local-table-1 s2-select s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
step s2-select: SELECT * FROM citus_local_table_1; <waiting ...>
step s1-commit: COMMIT;
step s2-select: <... completed>
a
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-create-citus-local-table-1 s2-update s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
step s2-update: UPDATE citus_local_table_1 SET a=1 WHERE a=2; <waiting ...>
step s1-commit: COMMIT;
step s2-update: <... completed>
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-create-citus-local-table-1 s2-truncate s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
step s2-truncate: TRUNCATE citus_local_table_1; <waiting ...>
step s1-commit: COMMIT;
step s2-truncate: <... completed>
step s2-commit: COMMIT;
master_remove_node
starting permutation: s2-create-citus-local-table-2 s1-begin s2-begin s1-create-citus-local-table-1 s2-fkey-to-another s1-commit s2-commit
step s2-create-citus-local-table-2: SELECT create_citus_local_table('citus_local_table_2');
create_citus_local_table
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
step s2-fkey-to-another: ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(a) REFERENCES citus_local_table_2(a); <waiting ...>
step s1-commit: COMMIT;
step s2-fkey-to-another: <... completed>
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-create-citus-local-table-1 s2-remove-coordinator s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1');
create_citus_local_table
step s2-remove-coordinator: SELECT master_remove_node('localhost', 57636); <waiting ...>
step s1-commit: COMMIT;
step s2-remove-coordinator: <... completed>
error in steps s1-commit s2-remove-coordinator: ERROR: you cannot remove the primary node of a node group which has shard placements
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-drop-table s2-create-citus-local-table-1 s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-drop-table: DROP TABLE citus_local_table_1;
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-1: <... completed>
error in steps s1-commit s2-create-citus-local-table-1: ERROR: cannot create citus local table, relation does not exist
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-delete s2-create-citus-local-table-1 s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-delete: DELETE FROM citus_local_table_1 WHERE a=2;
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-1: <... completed>
create_citus_local_table
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-select s2-create-citus-local-table-1 s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-select: SELECT * FROM citus_local_table_1;
a
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-1: <... completed>
create_citus_local_table
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-begin s2-begin s1-remove-coordinator s2-create-citus-local-table-1 s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-remove-coordinator: SELECT master_remove_node('localhost', 57636);
master_remove_node
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-1: <... completed>
error in steps s1-commit s2-create-citus-local-table-1: ERROR: could not find the coordinator node in metadata as it is not added as a worker
step s2-commit: COMMIT;
master_remove_node
starting permutation: s1-remove-coordinator s1-begin s2-begin s1-add-coordinator s2-create-citus-local-table-1 s1-commit s2-commit
step s1-remove-coordinator: SELECT master_remove_node('localhost', 57636);
master_remove_node
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-add-coordinator: SELECT 1 FROM master_add_node('localhost', 57636, 0);
?column?
1
step s2-create-citus-local-table-1: SELECT create_citus_local_table('citus_local_table_1'); <waiting ...>
step s1-commit: COMMIT;
step s2-create-citus-local-table-1: <... completed>
create_citus_local_table
step s2-commit: COMMIT;
master_remove_node

View File

@ -241,7 +241,7 @@ NOTICE: truncate cascades to table "dist_table_xxxxx"
NOTICE: truncate cascades to table "dist_table_xxxxx" NOTICE: truncate cascades to table "dist_table_xxxxx"
NOTICE: truncate cascades to table "dist_table_xxxxx" NOTICE: truncate cascades to table "dist_table_xxxxx"
NOTICE: truncate cascades to table "dist_table_xxxxx" NOTICE: truncate cascades to table "dist_table_xxxxx"
ERROR: cannot execute DDL on reference table "ref_table" because there was a parallel SELECT access to distributed table "dist_table" in the same transaction ERROR: cannot execute DDL on table "ref_table" because there was a parallel SELECT access to distributed table "dist_table" in the same transaction
COMMIT; COMMIT;
-- as we do not support local ANALYZE execution yet, below block would error out -- as we do not support local ANALYZE execution yet, below block would error out
BEGIN; BEGIN;

View File

@ -426,8 +426,9 @@ SELECT * FROM print_extension_changes();
function task_tracker_task_status(bigint,integer) | function task_tracker_task_status(bigint,integer) |
function worker_execute_sql_task(bigint,integer,text,boolean) | function worker_execute_sql_task(bigint,integer,text,boolean) |
function worker_merge_files_and_run_query(bigint,integer,text,text) | function worker_merge_files_and_run_query(bigint,integer,text,text) |
| function create_citus_local_table(regclass)
| function undistribute_table(regclass) | function undistribute_table(regclass)
(7 rows) (8 rows)
DROP TABLE prev_objects, extension_diff; DROP TABLE prev_objects, extension_diff;
-- show running version -- show running version

View File

@ -6,13 +6,43 @@ SELECT create_distributed_table('the_table', 'a');
(1 row) (1 row)
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
CREATE TABLE reference_table (a int, b int, z bigserial);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE citus_local_table (a int, b int, z bigserial);
SELECT create_citus_local_table('citus_local_table');
create_citus_local_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE local (a int, b int); CREATE TABLE local (a int, b int);
\c - - - :follower_master_port \c - - - :follower_master_port
-- inserts normally do not work on a standby coordinator -- inserts normally do not work on a standby coordinator
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2); INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
ERROR: writing to worker nodes is not currently allowed ERROR: writing to worker nodes is not currently allowed
DETAIL: the database is read-only DETAIL: the database is read-only
-- we can allow DML on a writable standby coordinator INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
ERROR: writing to worker nodes is not currently allowed
DETAIL: the database is read-only
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
ERROR: writing to worker nodes is not currently allowed
DETAIL: the database is read-only
-- We can allow DML on a writable standby coordinator.
-- Note that it doesn't help to enable writes for citus local tables
-- and coordinator replicated reference tables. This is because, the
-- data is in the coordinator and will hit read-only tranaction checks
-- on Postgres
SET citus.writable_standby_coordinator TO on; SET citus.writable_standby_coordinator TO on;
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2); INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
SELECT * FROM the_table; SELECT * FROM the_table;
@ -21,28 +51,82 @@ SELECT * FROM the_table;
1 | 2 | 2 1 | 2 | 2
(1 row) (1 row)
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
ERROR: cannot execute INSERT in a read-only transaction
SELECT * FROM reference_table;
a | b | z
---------------------------------------------------------------------
(0 rows)
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
ERROR: cannot execute INSERT in a read-only transaction
SELECT * FROM citus_local_table;
a | b | z
---------------------------------------------------------------------
(0 rows)
UPDATE the_table SET z = 3 WHERE a = 1; UPDATE the_table SET z = 3 WHERE a = 1;
UPDATE reference_table SET z = 3 WHERE a = 1;
ERROR: cannot execute UPDATE in a read-only transaction
UPDATE citus_local_table SET z = 3 WHERE a = 1;
ERROR: cannot execute UPDATE in a read-only transaction
SELECT * FROM the_table; SELECT * FROM the_table;
a | b | z a | b | z
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 | 2 | 3 1 | 2 | 3
(1 row) (1 row)
SELECT * FROM reference_table;
a | b | z
---------------------------------------------------------------------
(0 rows)
SELECT * FROM citus_local_table;
a | b | z
---------------------------------------------------------------------
(0 rows)
DELETE FROM the_table WHERE a = 1; DELETE FROM the_table WHERE a = 1;
DELETE FROM reference_table WHERE a = 1;
ERROR: cannot execute DELETE in a read-only transaction
DELETE FROM citus_local_table WHERE a = 1;
ERROR: cannot execute DELETE in a read-only transaction
SELECT * FROM the_table; SELECT * FROM the_table;
a | b | z a | b | z
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
SELECT * FROM reference_table;
a | b | z
---------------------------------------------------------------------
(0 rows)
SELECT * FROM citus_local_table;
a | b | z
---------------------------------------------------------------------
(0 rows)
-- drawing from a sequence is not possible -- drawing from a sequence is not possible
INSERT INTO the_table (a, b) VALUES (1, 2); INSERT INTO the_table (a, b) VALUES (1, 2);
ERROR: cannot assign TransactionIds during recovery ERROR: cannot assign TransactionIds during recovery
INSERT INTO reference_table (a, b) VALUES (1, 2);
ERROR: cannot assign TransactionIds during recovery
INSERT INTO citus_local_table (a, b) VALUES (1, 2);
ERROR: cannot assign TransactionIds during recovery
-- 2PC is not possible -- 2PC is not possible
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7); INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
ERROR: cannot assign TransactionIds during recovery ERROR: cannot assign TransactionIds during recovery
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
ERROR: cannot execute INSERT in a read-only transaction
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
ERROR: cannot execute INSERT in a read-only transaction
-- COPY is not possible in 2PC mode -- COPY is not possible in 2PC mode
COPY the_table (a, b, z) FROM STDIN WITH CSV; COPY the_table (a, b, z) FROM STDIN WITH CSV;
ERROR: cannot assign TransactionIds during recovery ERROR: cannot assign TransactionIds during recovery
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
ERROR: cannot assign TransactionIds during recovery
COPY citus_local_table (a, b, z) FROM STDIN WITH CSV;
ERROR: cannot assign TransactionIds during recovery
-- 1PC is possible -- 1PC is possible
SET citus.multi_shard_commit_protocol TO '1pc'; SET citus.multi_shard_commit_protocol TO '1pc';
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7); INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
@ -53,6 +137,20 @@ SELECT * FROM the_table ORDER BY a;
5 | 6 | 7 5 | 6 | 7
(2 rows) (2 rows)
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
ERROR: cannot execute INSERT in a read-only transaction
SELECT * FROM reference_table ORDER BY a;
a | b | z
---------------------------------------------------------------------
(0 rows)
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
ERROR: cannot execute INSERT in a read-only transaction
SELECT * FROM citus_local_table ORDER BY a;
a | b | z
---------------------------------------------------------------------
(0 rows)
-- modifying CTEs are possible -- modifying CTEs are possible
WITH del AS (DELETE FROM the_table RETURNING *) WITH del AS (DELETE FROM the_table RETURNING *)
SELECT * FROM del ORDER BY a; SELECT * FROM del ORDER BY a;
@ -62,8 +160,17 @@ SELECT * FROM del ORDER BY a;
5 | 6 | 7 5 | 6 | 7
(2 rows) (2 rows)
WITH del AS (DELETE FROM reference_table RETURNING *)
SELECT * FROM del ORDER BY a;
ERROR: cannot execute DELETE in a read-only transaction
WITH del AS (DELETE FROM citus_local_table RETURNING *)
SELECT * FROM del ORDER BY a;
ERROR: cannot execute DELETE in a read-only transaction
-- COPY is possible in 1PC mode -- COPY is possible in 1PC mode
COPY the_table (a, b, z) FROM STDIN WITH CSV; COPY the_table (a, b, z) FROM STDIN WITH CSV;
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
ERROR: cannot assign TransactionIds during recovery
COPY citus_local_table (a, b, z) FROM STDIN WITH CSV;
SELECT * FROM the_table ORDER BY a; SELECT * FROM the_table ORDER BY a;
a | b | z a | b | z
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -71,22 +178,68 @@ SELECT * FROM the_table ORDER BY a;
11 | 11 | 11 11 | 11 | 11
(2 rows) (2 rows)
SELECT * FROM reference_table ORDER BY a;
a | b | z
---------------------------------------------------------------------
(0 rows)
SELECT * FROM citus_local_table ORDER BY a;
a | b | z
---------------------------------------------------------------------
10 | 10 | 10
11 | 11 | 11
(2 rows)
DELETE FROM the_table; DELETE FROM the_table;
DELETE FROM reference_table;
ERROR: cannot execute DELETE in a read-only transaction
DELETE FROM citus_local_table;
ERROR: cannot execute DELETE in a read-only transaction
-- DDL is not possible -- DDL is not possible
TRUNCATE the_table; TRUNCATE the_table;
ERROR: cannot execute TRUNCATE TABLE in a read-only transaction ERROR: cannot execute TRUNCATE TABLE in a read-only transaction
TRUNCATE reference_table;
ERROR: cannot execute TRUNCATE TABLE in a read-only transaction
TRUNCATE citus_local_table;
ERROR: cannot execute TRUNCATE TABLE in a read-only transaction
ALTER TABLE the_table ADD COLUMN c int; ALTER TABLE the_table ADD COLUMN c int;
ERROR: cannot acquire lock mode AccessExclusiveLock on database objects while recovery is in progress ERROR: cannot acquire lock mode AccessExclusiveLock on database objects while recovery is in progress
HINT: Only RowExclusiveLock or less can be acquired on database objects during recovery. HINT: Only RowExclusiveLock or less can be acquired on database objects during recovery.
ALTER TABLE reference_table ADD COLUMN c int;
ERROR: cannot acquire lock mode AccessExclusiveLock on database objects while recovery is in progress
HINT: Only RowExclusiveLock or less can be acquired on database objects during recovery.
ALTER TABLE citus_local_table ADD COLUMN c int;
ERROR: cannot acquire lock mode AccessExclusiveLock on database objects while recovery is in progress
HINT: Only RowExclusiveLock or less can be acquired on database objects during recovery.
-- rollback is possible -- rollback is possible
BEGIN; BEGIN;
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2); INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
ROLLBACK; ROLLBACK;
BEGIN;
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
ERROR: cannot execute INSERT in a read-only transaction
ROLLBACK;
BEGIN;
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
ERROR: cannot execute INSERT in a read-only transaction
ROLLBACK;
SELECT * FROM the_table ORDER BY a; SELECT * FROM the_table ORDER BY a;
a | b | z a | b | z
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
SELECT * FROM reference_table ORDER BY a;
a | b | z
---------------------------------------------------------------------
(0 rows)
SELECT * FROM citus_local_table ORDER BY a;
a | b | z
---------------------------------------------------------------------
10 | 10 | 10
11 | 11 | 11
(2 rows)
-- we should still disallow writes to local tables -- we should still disallow writes to local tables
INSERT INTO local VALUES (1, 1); INSERT INTO local VALUES (1, 1);
ERROR: cannot execute INSERT in a read-only transaction ERROR: cannot execute INSERT in a read-only transaction
@ -106,5 +259,23 @@ SELECT * FROM the_table ORDER BY a;
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 3);
ERROR: writing to worker nodes is not currently allowed
DETAIL: citus.use_secondary_nodes is set to 'always'
SELECT * FROM reference_table ORDER BY a;
ERROR: there is a shard placement in node group 0 but there are no nodes in that group
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 3);
ERROR: writing to worker nodes is not currently allowed
DETAIL: citus.use_secondary_nodes is set to 'always'
SELECT * FROM citus_local_table ORDER BY a;
ERROR: there is a shard placement in node group 0 but there are no nodes in that group
\c - - - :master_port \c - - - :master_port
DROP TABLE the_table; DROP TABLE the_table;
DROP TABLE reference_table;
DROP TABLE citus_local_table;
SELECT master_remove_node('localhost', :master_port);
master_remove_node
---------------------------------------------------------------------
(1 row)

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)); CREATE TABLE reference_table(id int, referencing_column int REFERENCES referenced_by_reference_table(id));
SELECT create_reference_table('reference_table'); SELECT create_reference_table('reference_table');
ERROR: cannot create foreign key constraint since foreign keys from reference tables to distributed tables are not supported 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 -- test foreign key creation on CREATE TABLE from + to reference table
DROP TABLE reference_table; DROP TABLE reference_table;
CREATE TABLE reference_table(id int PRIMARY KEY, referencing_column int); 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); 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 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 -- test foreign key creation on ALTER TABLE to reference table
CREATE TABLE references_to_reference_table(id int, referencing_column int); CREATE TABLE references_to_reference_table(id int, referencing_column int);
SELECT create_distributed_table('references_to_reference_table', 'referencing_column'); SELECT create_distributed_table('references_to_reference_table', 'referencing_column');

View File

@ -270,7 +270,8 @@ CREATE TABLE bidders ( name text, id bigint );
DELETE FROM limit_orders USING bidders WHERE limit_orders.id = 246 AND DELETE FROM limit_orders USING bidders WHERE limit_orders.id = 246 AND
limit_orders.bidder_id = bidders.id AND limit_orders.bidder_id = bidders.id AND
bidders.name = 'Bernie Madoff'; bidders.name = 'Bernie Madoff';
ERROR: relation bidders is not distributed ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- commands containing a CTE are supported -- commands containing a CTE are supported
WITH new_orders AS (INSERT INTO limit_orders VALUES (411, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66)) WITH new_orders AS (INSERT INTO limit_orders VALUES (411, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66))
DELETE FROM limit_orders WHERE id < 0; DELETE FROM limit_orders WHERE id < 0;
@ -433,7 +434,8 @@ UPDATE limit_orders SET limit_price = 0.00 FROM bidders
WHERE limit_orders.id = 246 AND WHERE limit_orders.id = 246 AND
limit_orders.bidder_id = bidders.id AND limit_orders.bidder_id = bidders.id AND
bidders.name = 'Bernie Madoff'; bidders.name = 'Bernie Madoff';
ERROR: relation bidders is not distributed ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- should succeed with a CTE -- should succeed with a CTE
WITH deleted_orders AS (INSERT INTO limit_orders VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)) WITH deleted_orders AS (INSERT INTO limit_orders VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43))
UPDATE limit_orders SET symbol = 'GM'; UPDATE limit_orders SET symbol = 'GM';

View File

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

View File

@ -58,6 +58,11 @@ CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text);
CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
-- now create required stuff in the worker 1 -- now create required stuff in the worker 1
\c - - - :worker_1_port \c - - - :worker_1_port
-- show that we do not support creating citus local tables from mx workers for now
CREATE TABLE citus_local_table(a int);
SELECT create_citus_local_table('citus_local_table');
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
-- create schema to test schema support -- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2; CREATE SCHEMA citus_mx_test_schema_join_2;

View File

@ -158,7 +158,7 @@ CREATE TABLE bidders ( name text, id bigint );
DELETE FROM limit_orders_mx USING bidders WHERE limit_orders_mx.id = 246 AND DELETE FROM limit_orders_mx USING bidders WHERE limit_orders_mx.id = 246 AND
limit_orders_mx.bidder_id = bidders.id AND limit_orders_mx.bidder_id = bidders.id AND
bidders.name = 'Bernie Madoff'; bidders.name = 'Bernie Madoff';
ERROR: relation bidders is not distributed ERROR: cannot plan modifications with local tables involving citus tables
-- commands containing a CTE are supported -- commands containing a CTE are supported
WITH new_orders AS (INSERT INTO limit_orders_mx VALUES (411, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66)) WITH new_orders AS (INSERT INTO limit_orders_mx VALUES (411, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66))
DELETE FROM limit_orders_mx WHERE id < 0; DELETE FROM limit_orders_mx WHERE id < 0;
@ -225,7 +225,7 @@ UPDATE limit_orders_mx SET limit_price = 0.00 FROM bidders
WHERE limit_orders_mx.id = 246 AND WHERE limit_orders_mx.id = 246 AND
limit_orders_mx.bidder_id = bidders.id AND limit_orders_mx.bidder_id = bidders.id AND
bidders.name = 'Bernie Madoff'; bidders.name = 'Bernie Madoff';
ERROR: relation bidders is not distributed ERROR: cannot plan modifications with local tables involving citus tables
-- commands containing a CTE are supported -- commands containing a CTE are supported
WITH deleted_orders AS (INSERT INTO limit_orders_mx VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)) WITH deleted_orders AS (INSERT INTO limit_orders_mx VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43))
UPDATE limit_orders_mx SET symbol = 'GM'; UPDATE limit_orders_mx SET symbol = 'GM';

View File

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

View File

@ -1409,8 +1409,8 @@ ALTER TABLE reference_schema.reference_table_ddl_test RENAME TO reference_table_
-- now test reference tables against some helper UDFs that Citus provides -- now test reference tables against some helper UDFs that Citus provides
-- cannot delete / drop shards from a reference table -- cannot delete / drop shards from a reference table
SELECT master_apply_delete_command('DELETE FROM reference_schema.reference_table_ddl'); SELECT master_apply_delete_command('DELETE FROM reference_schema.reference_table_ddl');
ERROR: cannot delete from reference table ERROR: cannot delete from table
DETAIL: Delete statements on reference tables are not supported. DETAIL: Delete statements on reference and citus local tables are not supported.
-- cannot add shards -- cannot add shards
SELECT master_create_empty_shard('reference_schema.reference_table_ddl'); SELECT master_create_empty_shard('reference_schema.reference_table_ddl');
ERROR: relation "reference_schema.reference_table_ddl" is a reference table ERROR: relation "reference_schema.reference_table_ddl" is a reference table
@ -1434,7 +1434,7 @@ SELECT master_update_shard_statistics(:a_shard_id);
CREATE TABLE append_reference_tmp_table (id INT); CREATE TABLE append_reference_tmp_table (id INT);
SELECT master_append_table_to_shard(:a_shard_id, 'append_reference_tmp_table', 'localhost', :master_port); SELECT master_append_table_to_shard(:a_shard_id, 'append_reference_tmp_table', 'localhost', :master_port);
ERROR: cannot append to shardId 1250019 ERROR: cannot append to shardId 1250019
DETAIL: We currently don't support appending to shards in hash-partitioned or reference tables DETAIL: We currently don't support appending to shards in hash-partitioned, reference and citus local tables
SELECT master_get_table_ddl_events('reference_schema.reference_table_ddl'); SELECT master_get_table_ddl_events('reference_schema.reference_table_ddl');
master_get_table_ddl_events master_get_table_ddl_events
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -72,7 +72,8 @@ CREATE TABLE temp_nations(name text, key integer);
SELECT master_modify_multiple_shards('DELETE FROM multi_shard_modify_test USING temp_nations WHERE multi_shard_modify_test.t_value = temp_nations.key AND temp_nations.name = ''foobar'' '); SELECT master_modify_multiple_shards('DELETE FROM multi_shard_modify_test USING temp_nations WHERE multi_shard_modify_test.t_value = temp_nations.key AND temp_nations.name = ''foobar'' ');
WARNING: master_modify_multiple_shards is deprecated and will be removed in a future release. WARNING: master_modify_multiple_shards is deprecated and will be removed in a future release.
HINT: Run the command directly HINT: Run the command directly
ERROR: relation temp_nations is not distributed ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- commands with a USING clause are unsupported -- commands with a USING clause are unsupported
SELECT create_distributed_table('temp_nations', 'name', 'hash'); SELECT create_distributed_table('temp_nations', 'name', 'hash');
create_distributed_table create_distributed_table

View File

@ -729,12 +729,14 @@ UPDATE users_test_table
SET value_2 = 5 SET value_2 = 5
FROM events_test_table_local FROM events_test_table_local
WHERE users_test_table.user_id = events_test_table_local.user_id; WHERE users_test_table.user_id = events_test_table_local.user_id;
ERROR: relation events_test_table_local is not distributed ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
UPDATE events_test_table_local UPDATE events_test_table_local
SET value_2 = 5 SET value_2 = 5
FROM users_test_table FROM users_test_table
WHERE events_test_table_local.user_id = users_test_table.user_id; WHERE events_test_table_local.user_id = users_test_table.user_id;
ERROR: cannot plan modifications of local tables involving distributed tables ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- Local tables in a subquery are supported through recursive planning -- Local tables in a subquery are supported through recursive planning
UPDATE users_test_table UPDATE users_test_table
SET value_2 = 5 SET value_2 = 5

View File

@ -284,10 +284,12 @@ ERROR: could not run distributed query with subquery outside the FROM, WHERE an
HINT: Consider using an equality filter on the distributed table's partition column. HINT: Consider using an equality filter on the distributed table's partition column.
-- joins are not supported between local and distributed tables -- joins are not supported between local and distributed tables
SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id; SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id;
ERROR: relation authors is not distributed ERROR: direct joins between distributed and local tables are not supported
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- inner joins are not supported (I think) -- inner joins are not supported (I think)
SELECT * FROM (articles INNER JOIN authors ON articles.id = authors.id); SELECT * FROM (articles INNER JOIN authors ON articles.id = authors.id);
ERROR: relation authors is not distributed ERROR: direct joins between distributed and local tables are not supported
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- test use of EXECUTE statements within plpgsql -- test use of EXECUTE statements within plpgsql
DO $sharded_execute$ DO $sharded_execute$
BEGIN BEGIN

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

@ -17,7 +17,8 @@ FROM
users_table_local, (SELECT user_id FROM events_table) as evs users_table_local, (SELECT user_id FROM events_table) as evs
WHERE users_table_local.user_id = evs.user_id WHERE users_table_local.user_id = evs.user_id
) as foo; ) as foo;
ERROR: relation users_table_local is not distributed ERROR: direct joins between distributed and local tables are not supported
HINT: Use CTE's or subqueries to select from local tables and use them in joins
RESET client_min_messages; RESET client_min_messages;
-- we don't support subqueries with local tables when they are not leaf queries -- we don't support subqueries with local tables when they are not leaf queries
SELECT user_id FROM users_table WHERE user_id IN SELECT user_id FROM users_table WHERE user_id IN

View File

@ -307,7 +307,8 @@ WITH cte AS (
SELECT user_id FROM users_table SELECT user_id FROM users_table
) )
SELECT min(user_id) FROM cte JOIN local_table ON (user_id = id) JOIN events_table USING (user_id); SELECT min(user_id) FROM cte JOIN local_table ON (user_id = id) JOIN events_table USING (user_id);
ERROR: relation local_table is not distributed ERROR: direct joins between distributed and local tables are not supported
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- unless the distributed table is part of a recursively planned subquery -- unless the distributed table is part of a recursively planned subquery
WITH cte AS ( WITH cte AS (
SELECT user_id FROM users_table SELECT user_id FROM users_table
@ -325,7 +326,8 @@ WITH cte AS (
) )
SELECT count(*) FROM local_table JOIN ref_table USING (id) SELECT count(*) FROM local_table JOIN ref_table USING (id)
WHERE id IN (SELECT * FROM cte); WHERE id IN (SELECT * FROM cte);
ERROR: relation local_table is not distributed ERROR: direct joins between distributed and local tables are not supported
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- CTEs should be able to terminate a router query -- CTEs should be able to terminate a router query
WITH cte AS ( WITH cte AS (
WITH cte_1 AS ( WITH cte_1 AS (

View File

@ -671,14 +671,16 @@ SELECT * FROM raw_data ORDER BY val;
-- Test that local tables are barred -- Test that local tables are barred
UPDATE local_table lt SET val = mt.val UPDATE local_table lt SET val = mt.val
FROM modify_table mt WHERE mt.id = lt.id; FROM modify_table mt WHERE mt.id = lt.id;
ERROR: cannot plan modifications of local tables involving distributed tables ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- Including inside CTEs -- Including inside CTEs
WITH cte AS ( WITH cte AS (
UPDATE local_table lt SET val = mt.val UPDATE local_table lt SET val = mt.val
FROM modify_table mt WHERE mt.id = lt.id FROM modify_table mt WHERE mt.id = lt.id
RETURNING lt.id, lt.val RETURNING lt.id, lt.val
) SELECT * FROM cte JOIN modify_table mt ON mt.id = cte.id ORDER BY 1,2; ) SELECT * FROM cte JOIN modify_table mt ON mt.id = cte.id ORDER BY 1,2;
ERROR: cannot plan modifications of local tables involving distributed tables ERROR: cannot plan modifications with local tables involving citus tables
HINT: Use CTE's or subqueries to select from local tables and use them in joins
-- Make sure checks for volatile functions apply to CTEs too -- Make sure checks for volatile functions apply to CTEs too
WITH cte AS (UPDATE modify_table SET val = random() WHERE id = 3 RETURNING *) WITH cte AS (UPDATE modify_table SET val = random() WHERE id = 3 RETURNING *)
SELECT * FROM cte JOIN modify_table mt ON mt.id = 3 AND mt.id = cte.id ORDER BY 1,2; SELECT * FROM cte JOIN modify_table mt ON mt.id = 3 AND mt.id = cte.id ORDER BY 1,2;

View File

@ -33,6 +33,7 @@ test: isolation_dump_local_wait_edges
test: isolation_replace_wait_function test: isolation_replace_wait_function
test: isolation_distributed_deadlock_detection test: isolation_distributed_deadlock_detection
test: isolation_replicate_reference_tables_to_coordinator test: isolation_replicate_reference_tables_to_coordinator
test: isolation_create_citus_local_table
# creating a restore point briefly blocks all # creating a restore point briefly blocks all
# writes, run this test serially. # writes, run this test serially.

View File

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

View File

@ -14,18 +14,19 @@
# --- # ---
# Tests around schema changes, these are run first, so there's no preexisting objects. # 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_extension
test: multi_703_upgrade test: multi_703_upgrade
test: single_node test: single_node
test: multi_cluster_management 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: alter_role_propagation
test: propagate_extension_commands test: propagate_extension_commands
test: escape_extension_name test: escape_extension_name
test: ref_citus_local_fkeys
test: multi_test_helpers multi_test_helpers_superuser test: multi_test_helpers multi_test_helpers_superuser
test: multi_test_catalog_views test: multi_test_catalog_views
test: multi_table_ddl 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_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: multi_shard_update_delete recursive_dml_with_different_planners_executors
test: insert_select_repartition window_functions dml_recursive multi_insert_select_window 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 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 # following should not run in parallel because it relies on connection counts to workers
@ -299,11 +300,12 @@ test: multi_remove_node_reference_table
test: add_coordinator test: add_coordinator
test: multi_upgrade_reference_table test: multi_upgrade_reference_table
test: multi_replicate_reference_table test: multi_replicate_reference_table
test: multi_reference_table test: multi_reference_table citus_local_tables_queries
test: foreign_key_to_reference_table test: foreign_key_to_reference_table citus_local_table_triggers
test: replicate_reference_tables_to_coordinator test: replicate_reference_tables_to_coordinator
test: coordinator_shouldhaveshards test: coordinator_shouldhaveshards
test: local_shard_utility_command_execution test: local_shard_utility_command_execution
test: citus_local_tables
test: remove_coordinator test: remove_coordinator

View File

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

View File

@ -82,7 +82,7 @@ FROM
pg_dist_shard pg_dist_shard
WHERE 'multi_append_table_to_shard_right_reference_hash'::regclass::oid = logicalrelid; WHERE 'multi_append_table_to_shard_right_reference_hash'::regclass::oid = logicalrelid;
ERROR: cannot append to shardId 230001 ERROR: cannot append to shardId 230001
DETAIL: We currently don't support appending to shards in hash-partitioned or reference tables DETAIL: We currently don't support appending to shards in hash-partitioned, reference and citus local tables
-- Clean up after test -- Clean up after test
SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_left'); SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_left');
master_apply_delete_command master_apply_delete_command

View File

@ -0,0 +1,74 @@
setup
{
SELECT 1 FROM master_add_node('localhost', 57636, 0);
CREATE TABLE citus_local_table_1(a int);
CREATE TABLE citus_local_table_2(a int unique);
CREATE SCHEMA another_schema;
CREATE TABLE another_schema.citus_local_table_3(a int unique);
}
teardown
{
DROP TABLE IF EXISTS citus_local_table_1, citus_local_table_2 CASCADE;
DROP SCHEMA IF EXISTS another_schema CASCADE;
-- remove coordinator only if it is added to pg_dist_node
SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node WHERE nodeport=57636;
}
session "s1"
step "s1-begin" { BEGIN; }
step "s1-create-citus-local-table-1" { SELECT create_citus_local_table('citus_local_table_1'); }
step "s1-create-citus-local-table-3" { SELECT create_citus_local_table('another_schema.citus_local_table_3'); }
step "s1-drop-table" { DROP TABLE citus_local_table_1; }
step "s1-delete" { DELETE FROM citus_local_table_1 WHERE a=2; }
step "s1-select" { SELECT * FROM citus_local_table_1; }
step "s1-remove-coordinator" { SELECT master_remove_node('localhost', 57636); }
step "s1-add-coordinator" { SELECT 1 FROM master_add_node('localhost', 57636, 0); }
step "s1-commit" { COMMIT; }
step "s1-rollback" { ROLLBACK; }
session "s2"
step "s2-begin" { BEGIN; }
step "s2-create-citus-local-table-1" { SELECT create_citus_local_table('citus_local_table_1'); }
step "s2-create-citus-local-table-2" { SELECT create_citus_local_table('citus_local_table_2'); }
step "s2-create-citus-local-table-3" { SELECT create_citus_local_table('another_schema.citus_local_table_3'); }
step "s2-select" { SELECT * FROM citus_local_table_1; }
step "s2-update" { UPDATE citus_local_table_1 SET a=1 WHERE a=2; }
step "s2-truncate" { TRUNCATE citus_local_table_1; }
step "s2-fkey-to-another" { ALTER TABLE citus_local_table_1 ADD CONSTRAINT fkey_c_to_c FOREIGN KEY(a) REFERENCES citus_local_table_2(a); }
step "s2-remove-coordinator" { SELECT master_remove_node('localhost', 57636); }
step "s2-commit" { COMMIT; }
// create_citus_local_table vs command/query //
// Second session should error out as the table becomes a citus local table after the first session commits ..
permutation "s1-begin" "s2-begin" "s1-create-citus-local-table-1" "s2-create-citus-local-table-1" "s1-commit" "s2-commit"
// .. and it should error out even if we are in a different schema than the table
permutation "s1-begin" "s2-begin" "s1-create-citus-local-table-3" "s2-create-citus-local-table-3" "s1-commit" "s2-commit"
// Second session should be able to create citus local table as the first one rollbacks
permutation "s1-begin" "s2-begin" "s1-create-citus-local-table-1" "s2-create-citus-local-table-1" "s1-rollback" "s2-commit"
// Any modifying queries, DML commands and SELECT will block
permutation "s1-begin" "s2-begin" "s1-create-citus-local-table-1" "s2-select" "s1-commit" "s2-commit"
permutation "s1-begin" "s2-begin" "s1-create-citus-local-table-1" "s2-update" "s1-commit" "s2-commit"
permutation "s1-begin" "s2-begin" "s1-create-citus-local-table-1" "s2-truncate" "s1-commit" "s2-commit"
// Foreign key creation should succeed as it will be blocked until first session creates citus local table
permutation "s2-create-citus-local-table-2" "s1-begin" "s2-begin" "s1-create-citus-local-table-1" "s2-fkey-to-another" "s1-commit" "s2-commit"
// master_remove_node should first block and then fail
permutation "s1-begin" "s2-begin" "s1-create-citus-local-table-1" "s2-remove-coordinator" "s1-commit" "s2-commit"
// command/query vs create_citus_local_table //
// create_citus_local_table_1 should first block and then error out as the first session drops the table
permutation "s1-begin" "s2-begin" "s1-drop-table" "s2-create-citus-local-table-1" "s1-commit" "s2-commit"
// Any modifying queries, DML commands and SELECT will block
permutation "s1-begin" "s2-begin" "s1-delete" "s2-create-citus-local-table-1" "s1-commit" "s2-commit"
permutation "s1-begin" "s2-begin" "s1-select" "s2-create-citus-local-table-1" "s1-commit" "s2-commit"
// create_citus_local_table should block on master_remove_node and then fail
permutation "s1-begin" "s2-begin" "s1-remove-coordinator" "s2-create-citus-local-table-1" "s1-commit" "s2-commit"
// create_citus_local_table should block on master_add_node and then succeed
permutation "s1-remove-coordinator" "s1-begin" "s2-begin" "s1-add-coordinator" "s2-create-citus-local-table-1" "s1-commit" "s2-commit"

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

@ -0,0 +1,368 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1504000;
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_test_schema;
SET search_path TO citus_local_tables_test_schema;
------------------------------------------
------- citus local table creation -------
------------------------------------------
-- 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_1 (a int);
-- this should work as coordinator is added to pg_dist_node
SELECT create_citus_local_table('citus_local_table_1');
-- try to remove coordinator and observe failure as there exist a citus local table
SELECT 1 FROM master_remove_node('localhost', :master_port);
DROP TABLE citus_local_table_1;
-- this should work now as the citus local table is dropped
SELECT 1 FROM master_remove_node('localhost', :master_port);
CREATE TABLE citus_local_table_1 (a int primary key);
-- this should fail as coordinator is removed from pg_dist_node
SELECT create_citus_local_table('citus_local_table_1');
-- let coordinator have citus local tables again for next tests
set client_min_messages to ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
RESET client_min_messages;
-- creating citus local table having no data initially would work
SELECT create_citus_local_table('citus_local_table_1');
-- creating citus local table having data in it would also work
CREATE TABLE citus_local_table_2(a int primary key);
INSERT INTO citus_local_table_2 VALUES(1);
SELECT create_citus_local_table('citus_local_table_2');
-- also create indexes on them
CREATE INDEX citus_local_table_1_idx ON citus_local_table_1(a);
CREATE INDEX citus_local_table_2_idx ON citus_local_table_2(a);
-- drop them for next tests
DROP TABLE citus_local_table_1, citus_local_table_2;
-- create indexes before creating the citus local tables
-- .. for an initially empty table
CREATE TABLE citus_local_table_1(a int);
CREATE INDEX citus_local_table_1_idx ON citus_local_table_1(a);
SELECT create_citus_local_table('citus_local_table_1');
-- .. and for another table having data in it before creating citus local table
CREATE TABLE citus_local_table_2(a int);
INSERT INTO citus_local_table_2 VALUES(1);
CREATE INDEX citus_local_table_2_idx ON citus_local_table_2(a);
SELECT create_citus_local_table('citus_local_table_2');
CREATE TABLE distributed_table (a int);
SELECT create_distributed_table('distributed_table', 'a');
-- cannot create citus local table from an existing citus table
SELECT create_citus_local_table('distributed_table');
-- partitioned table tests --
CREATE TABLE partitioned_table(a int, b int) PARTITION BY RANGE (a);
CREATE TABLE partitioned_table_1 PARTITION OF partitioned_table FOR VALUES FROM (0) TO (10);
CREATE TABLE partitioned_table_2 PARTITION OF partitioned_table FOR VALUES FROM (10) TO (20);
-- cannot create partitioned citus local tables
SELECT create_citus_local_table('partitioned_table');
BEGIN;
CREATE TABLE citus_local_table PARTITION OF partitioned_table FOR VALUES FROM (20) TO (30);
-- cannot create citus local table as a partition of a local table
SELECT create_citus_local_table('citus_local_table');
ROLLBACK;
BEGIN;
CREATE TABLE citus_local_table (a int, b int);
SELECT create_citus_local_table('citus_local_table');
-- cannot create citus local table as a partition of a local table
-- via ALTER TABLE commands as well
ALTER TABLE partitioned_table ATTACH PARTITION citus_local_table FOR VALUES FROM (20) TO (30);
ROLLBACK;
BEGIN;
SELECT create_distributed_table('partitioned_table', 'a');
CREATE TABLE citus_local_table (a int, b int);
SELECT create_citus_local_table('citus_local_table');
-- 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);
ROLLBACK;
-- show that we do not support inheritance relationships --
CREATE TABLE parent_table (a int, b text);
CREATE TABLE child_table () INHERITS (parent_table);
-- both of below should error out
SELECT create_citus_local_table('parent_table');
SELECT create_citus_local_table('child_table');
-- show that we support UNLOGGED tables --
CREATE UNLOGGED TABLE unlogged_table (a int primary key);
SELECT create_citus_local_table('unlogged_table');
-- show that we allow triggers --
BEGIN;
CREATE TABLE citus_local_table_3 (value int);
-- create a simple function to be invoked by trigger
CREATE FUNCTION update_value() RETURNS trigger AS $update_value$
BEGIN
UPDATE citus_local_table_3 SET value=value+1;
RETURN NEW;
END;
$update_value$ LANGUAGE plpgsql;
CREATE TRIGGER insert_trigger
AFTER INSERT ON citus_local_table_3
FOR EACH STATEMENT EXECUTE FUNCTION update_value();
SELECT create_citus_local_table('citus_local_table_3');
INSERT INTO citus_local_table_3 VALUES (1);
-- show that trigger is executed only once, we should see "2" (not "3")
SELECT * FROM citus_local_table_3;
ROLLBACK;
-- show that we do not support policies in citus community --
BEGIN;
CREATE TABLE citus_local_table_3 (table_user text);
ALTER TABLE citus_local_table_3 ENABLE ROW LEVEL SECURITY;
CREATE ROLE table_users;
CREATE POLICY table_policy ON citus_local_table_3 TO table_users
USING (table_user = current_user);
-- this should error out
SELECT create_citus_local_table('citus_local_table_3');
ROLLBACK;
-- show that we properly handle sequences on citus local tables --
BEGIN;
CREATE SEQUENCE col3_seq;
CREATE TABLE citus_local_table_3 (col1 serial, col2 int, col3 int DEFAULT nextval('col3_seq'));
SELECT create_citus_local_table('citus_local_table_3');
-- print column default expressions
-- we should only see shell relation below
SELECT table_name, column_name, column_default
FROM information_schema.COLUMNS
WHERE table_name like 'citus_local_table_3%' and column_default != '' ORDER BY 1,2;
-- print sequence ownerships
-- show that the only internal sequence is on col1 and it is owned by shell relation
SELECT s.relname as sequence_name, t.relname, a.attname
FROM pg_class s
JOIN pg_depend d on d.objid=s.oid and d.classid='pg_class'::regclass and d.refclassid='pg_class'::regclass
JOIN pg_class t on t.oid=d.refobjid
JOIN pg_attribute a on a.attrelid=t.oid and a.attnum=d.refobjsubid
WHERE s.relkind='S' and s.relname like 'citus_local_table_3%' ORDER BY 1,2;
ROLLBACK;
-- test foreign tables using fake FDW --
CREATE FOREIGN TABLE foreign_table (
id bigint not null,
full_name text not null default ''
) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true');
-- observe that we do not create fdw server for shell table, both shard relation
-- & shell relation points to the same same server object
SELECT create_citus_local_table('foreign_table');
-- drop them for next tests
DROP TABLE citus_local_table_1, citus_local_table_2, distributed_table;
-- create test tables
CREATE TABLE citus_local_table_1 (a int primary key);
SELECT create_citus_local_table('citus_local_table_1');
CREATE TABLE citus_local_table_2 (a int primary key);
SELECT create_citus_local_table('citus_local_table_2');
CREATE TABLE local_table (a int primary key);
CREATE TABLE distributed_table (a int primary key);
SELECT create_distributed_table('distributed_table', 'a');
CREATE TABLE reference_table (a int primary key);
SELECT create_reference_table('reference_table');
-- show that colociation of citus local tables are not supported for now
-- between citus local tables
SELECT mark_tables_colocated('citus_local_table_1', ARRAY['citus_local_table_2']);
-- between citus local tables and reference tables
SELECT mark_tables_colocated('citus_local_table_1', ARRAY['reference_table']);
SELECT mark_tables_colocated('reference_table', ARRAY['citus_local_table_1']);
-- between citus local tables and distributed tables
SELECT mark_tables_colocated('citus_local_table_1', ARRAY['distributed_table']);
SELECT mark_tables_colocated('distributed_table', ARRAY['citus_local_table_1']);
-- upgrade_to_reference_table is not supported
SELECT upgrade_to_reference_table('citus_local_table_1');
-- master_create_empty_shard is not supported
SELECT master_create_empty_shard('citus_local_table_1');
-- get_shard_id_for_distribution_column is supported
SELECT get_shard_id_for_distribution_column('citus_local_table_1', 'not_checking_this_arg_for_non_dist_tables');
SELECT get_shard_id_for_distribution_column('citus_local_table_1');
-- master_copy_shard_placement is not supported
SELECT master_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, true)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table_1'::regclass) as shardid;
-- undistribute_table is supported
BEGIN;
SELECT undistribute_table('citus_local_table_1');
ROLLBACK;
-- tests with citus local tables initially having foreign key relationships
CREATE TABLE local_table_1 (a int primary key);
CREATE TABLE local_table_2 (a int primary key references local_table_1(a));
CREATE TABLE local_table_3 (a int primary key, b int references local_table_3(a));
-- below two should fail as we do not allow foreign keys between
-- postgres local tables and citus local tables
SELECT create_citus_local_table('local_table_1');
SELECT create_citus_local_table('local_table_2');
-- below should work as we allow initial self references in citus local tables
SELECT create_citus_local_table('local_table_3');
------------------------------------------------------------------
----- tests for object names that should be escaped properly -----
------------------------------------------------------------------
CREATE SCHEMA "CiTUS!LocalTables";
-- create table with weird names
CREATE TABLE "CiTUS!LocalTables"."LocalTabLE.1!?!"(id int, "TeNANt_Id" int);
-- should work
SELECT create_citus_local_table('"CiTUS!LocalTables"."LocalTabLE.1!?!"');
-- drop the table before creating it when the search path is set
SET search_path to "CiTUS!LocalTables" ;
DROP TABLE "LocalTabLE.1!?!";
-- have a custom type in the local table
CREATE TYPE local_type AS (key int, value jsonb);
-- create btree_gist for GiST index
CREATE EXTENSION btree_gist;
CREATE TABLE "LocalTabLE.1!?!"(
id int PRIMARY KEY,
"TeNANt_Id" int,
"local_Type" local_type,
"jsondata" jsonb NOT NULL,
name text,
price numeric CHECK (price > 0),
serial_data bigserial, UNIQUE (id, price),
EXCLUDE USING GIST (name WITH =));
-- create some objects before create_citus_local_table
CREATE INDEX "my!Index1" ON "LocalTabLE.1!?!"(id) WITH ( fillfactor = 80 ) WHERE id > 10;
CREATE UNIQUE INDEX uniqueIndex ON "LocalTabLE.1!?!" (id);
-- ingest some data before create_citus_local_table
INSERT INTO "LocalTabLE.1!?!" VALUES (1, 1, (1, row_to_json(row(1,1)))::local_type, row_to_json(row(1,1), true)),
(2, 1, (2, row_to_json(row(2,2)))::local_type, row_to_json(row(2,2), 'false'));
-- create a replica identity before create_citus_local_table
ALTER TABLE "LocalTabLE.1!?!" REPLICA IDENTITY USING INDEX uniqueIndex;
-- this shouldn't give any syntax errors
SELECT create_citus_local_table('"LocalTabLE.1!?!"');
-- create some objects after create_citus_local_table
CREATE INDEX "my!Index2" ON "LocalTabLE.1!?!"(id) WITH ( fillfactor = 90 ) WHERE id < 20;
CREATE UNIQUE INDEX uniqueIndex2 ON "LocalTabLE.1!?!"(id);
-----------------------------------
---- utility command execution ----
-----------------------------------
SET search_path TO citus_local_tables_test_schema;
-- 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);
ALTER TABLE distributed_table ADD CONSTRAINT fkey_dist_to_c FOREIGN KEY(a) references citus_local_table_1(a);
-- 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),
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;
-- test vacuum
VACUUM citus_local_table_1;
VACUUM citus_local_table_1, 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

@ -0,0 +1,635 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1509000;
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_queries;
SET search_path TO citus_local_table_queries;
-- 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(a int, b int);
SELECT create_citus_local_table('citus_local_table');
CREATE TABLE citus_local_table_2(a int, b int);
SELECT create_citus_local_table('citus_local_table_2');
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
CREATE TABLE distributed_table(a int, b int);
SELECT create_distributed_table('distributed_table', 'a');
CREATE TABLE postgres_local_table(a int, b int);
-- Define a helper function to truncate & insert some data into our test tables
-- We should call this function at some places in this test file to prevent
-- test to take a long time.
-- We shouldn't use LIMIT in INSERT SELECT queries to make the test faster as
-- LIMIT would force planner to wrap SELECT query in an intermediate result and
-- this might reduce the coverage of the test cases.
CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$
BEGIN
SET client_min_messages to ERROR;
TRUNCATE postgres_local_table, citus_local_table, reference_table, distributed_table;
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO citus_local_table_2 SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO distributed_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
RESET client_min_messages;
END;
$$ LANGUAGE plpgsql;
----------------
---- SELECT ----
----------------
SELECT clear_and_init_test_tables();
-- join between citus local tables and reference tables would succeed
SELECT count(*) FROM citus_local_table, reference_table WHERE citus_local_table.a = reference_table.a;
SELECT * FROM citus_local_table, reference_table WHERE citus_local_table.a = reference_table.a ORDER BY 1,2,3,4 FOR UPDATE;
-- should work
WITH cte_1 AS
(SELECT * FROM citus_local_table, reference_table WHERE citus_local_table.a = reference_table.a ORDER BY 1,2,3,4 FOR UPDATE)
SELECT count(*) FROM cte_1;
-- should work as joins are between ctes
WITH cte_citus_local_table AS
(SELECT * FROM citus_local_table),
cte_postgres_local_table AS
(SELECT * FROM postgres_local_table),
cte_distributed_table AS
(SELECT * FROM distributed_table)
SELECT count(*) FROM cte_distributed_table, cte_citus_local_table, cte_postgres_local_table
WHERE cte_citus_local_table.a = 1 AND cte_distributed_table.a = 1;
-- should fail as we don't support direct joins between distributed/local tables
SELECT count(*) FROM distributed_table d1, distributed_table d2, citus_local_table;
-- local table inside subquery should just work
SELECT count(*) FROM
(
SELECT * FROM (SELECT * FROM citus_local_table) as subquery_inner
) as subquery_top;
SELECT clear_and_init_test_tables();
-- join between citus/postgres local tables should just work
SELECT count(*) FROM
(
SELECT * FROM (SELECT count(*) FROM citus_local_table, postgres_local_table) as subquery_inner
) as subquery_top;
-- should fail as we don't support direct joins between distributed/local tables
SELECT count(*) FROM
(
SELECT *, random() FROM (SELECT *, random() FROM citus_local_table, distributed_table) as subquery_inner
) as subquery_top;
-- should fail as we don't support direct joins between distributed/local tables
SELECT count(*) FROM
(
SELECT *, random()
FROM (
WITH cte_1 AS (SELECT *, random() FROM citus_local_table, distributed_table) SELECT * FROM cte_1) as subquery_inner
) as subquery_top;
-- should be fine
SELECT count(*) FROM
(
SELECT *, random()
FROM (
WITH cte_1 AS (SELECT *, random() FROM citus_local_table), cte_2 AS (SELECT * FROM distributed_table) SELECT count(*) FROM cte_1, cte_2
) as subquery_inner
) as subquery_top;
SELECT clear_and_init_test_tables();
-- prepared statement
PREPARE citus_local_only AS SELECT count(*) FROM citus_local_table;
-- execute 6 times, local tables without params
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
-- execute 6 times, with param
PREPARE citus_local_only_p(int) AS SELECT count(*) FROM citus_local_table WHERE a = $1;
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
-- do not evalute the function
-- show the logs
EXECUTE citus_local_only_p(random());
EXECUTE citus_local_only_p(random());
PREPARE mixed_query(int, int, int) AS
WITH cte_citus_local_table AS
(SELECT * FROM citus_local_table WHERE a = $1),
cte_postgres_local_table AS
(SELECT * FROM postgres_local_table WHERE a = $2),
cte_distributed_table AS
(SELECT * FROM distributed_table WHERE a = $3),
cte_mixes AS (SELECT * FROM cte_distributed_table, cte_citus_local_table, cte_postgres_local_table)
SELECT count(*) FROM cte_mixes;
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
SELECT clear_and_init_test_tables();
-- anonymous columns
WITH a AS (SELECT a, '' FROM citus_local_table GROUP BY a) SELECT a.a FROM a ORDER BY 1 LIMIT 5;
WITH a AS (SELECT b, '' FROM citus_local_table WHERE a = 1) SELECT * FROM a, a b ORDER BY 1 LIMIT 5;
-- weird expression on citus/pg table joins should be fine
SELECT * FROM citus_local_table, postgres_local_table
WHERE citus_local_table.a - postgres_local_table.a = 0
ORDER BY 1,2,3,4
LIMIT 10;
-- set operations should just work
SELECT * FROM citus_local_table UNION SELECT * FROM postgres_local_table UNION SELECT * FROM distributed_table ORDER BY 1,2;
(SELECT * FROM citus_local_table ORDER BY 1,2 LIMIT 5) INTERSECT (SELECT i, i FROM generate_series(0, 100) i) ORDER BY 1, 2;
-- should just work as recursive planner kicks in
SELECT count(*) FROM distributed_table WHERE a IN (SELECT a FROM citus_local_table);
SELECT count(*) FROM citus_local_table WHERE a IN (SELECT a FROM distributed_table);
SELECT count(*) FROM reference_table WHERE a IN (SELECT a FROM citus_local_table);
SELECT count(*) FROM citus_local_table WHERE a IN (SELECT a FROM reference_table);
-- nested recursive queries should just work
SELECT count(*) FROM citus_local_table
WHERE a IN
(SELECT a FROM distributed_table WHERE a IN
(SELECT b FROM citus_local_table WHERE b IN (SELECT b FROM postgres_local_table)));
-- local outer joins
SELECT count(*) FROM citus_local_table LEFT JOIN reference_table ON (true);
SELECT count(*) FROM reference_table
LEFT JOIN citus_local_table ON (true)
LEFT JOIN postgres_local_table ON (true)
LEFT JOIN reference_table r2 ON (true);
-- not supported direct outer join
SELECT count(*) FROM citus_local_table LEFT JOIN distributed_table ON (true);
-- distinct in subquery on CTE
WITH one_row AS (
SELECT a from citus_local_table WHERE b = 1
)
SELECT
*
FROM
distributed_table
WHERE
b IN (SELECT DISTINCT a FROM one_row)
ORDER BY
1, 2
LIMIT
1;
WITH one_row_2 AS (
SELECT a from distributed_table WHERE b = 1
)
SELECT
*
FROM
citus_local_table
WHERE
b IN (SELECT DISTINCT a FROM one_row_2)
ORDER BY
1 ,2
LIMIT
1;
-- join between citus local tables and distributed tables would fail
SELECT count(*) FROM citus_local_table, distributed_table;
SELECT * FROM citus_local_table, distributed_table ORDER BY 1,2,3,4 FOR UPDATE;
-- join between citus local tables and postgres local tables are okey
SELECT count(citus_local_table.b), count(postgres_local_table.a)
FROM citus_local_table, postgres_local_table
WHERE citus_local_table.a = postgres_local_table.b;
-- select for update is just OK
SELECT * FROM citus_local_table ORDER BY 1,2 FOR UPDATE;
---------------------------
----- INSERT SELECT -----
---------------------------
-- simple INSERT SELECT is OK
SELECT clear_and_init_test_tables();
INSERT INTO citus_local_table
SELECT * from reference_table;
INSERT INTO reference_table
SELECT * from citus_local_table;
INSERT INTO citus_local_table
SELECT * from distributed_table;
INSERT INTO distributed_table
SELECT * from citus_local_table;
INSERT INTO citus_local_table
SELECT * from citus_local_table_2;
INSERT INTO citus_local_table
SELECT * from citus_local_table_2
ORDER BY 1,2
LIMIT 10;
INSERT INTO citus_local_table
SELECT * from postgres_local_table;
INSERT INTO postgres_local_table
SELECT * from citus_local_table;
-- INSERT SELECT with local joins are OK
SELECT clear_and_init_test_tables();
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
INSERT INTO reference_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
INSERT INTO reference_table
SELECT reference_table.* FROM reference_table, postgres_local_table
JOIN citus_local_table ON (true);
SELECT clear_and_init_test_tables();
INSERT INTO distributed_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
INSERT INTO distributed_table
SELECT reference_table.* FROM reference_table, postgres_local_table
JOIN citus_local_table ON (true);
INSERT INTO postgres_local_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
-- INSERT SELECT that joins reference and distributed tables is also OK
SELECT clear_and_init_test_tables();
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table
JOIN distributed_table ON (true);
INSERT INTO citus_local_table
SELECT reference_table.*
FROM reference_table, distributed_table;
-- INSERT SELECT that joins citus local and distributed table directly will fail ..
INSERT INTO citus_local_table
SELECT distributed_table.* FROM distributed_table
JOIN citus_local_table ON (true);
-- .. but when wrapped into a CTE, join works fine
INSERT INTO citus_local_table
SELECT distributed_table.* FROM distributed_table
JOIN (WITH cte AS (SELECT * FROM citus_local_table) SELECT * FROM cte) as foo ON (true);
-- multi row insert is OK
INSERT INTO citus_local_table VALUES (1, 2), (3, 4);
---------------------------
----- DELETE / UPDATE -----
---------------------------
-- modifications using citus local tables and postgres local tables
-- are not supported, see below four tests
SELECT clear_and_init_test_tables();
DELETE FROM citus_local_table
USING postgres_local_table
WHERE citus_local_table.b = postgres_local_table.b;
UPDATE citus_local_table
SET b = 5
FROM postgres_local_table
WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b;
DELETE FROM postgres_local_table
USING citus_local_table
WHERE citus_local_table.b = postgres_local_table.b;
UPDATE postgres_local_table
SET b = 5
FROM citus_local_table
WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b;
-- no direct joins supported
UPDATE distributed_table
SET b = 6
FROM citus_local_table
WHERE citus_local_table.a = distributed_table.a;
UPDATE reference_table
SET b = 6
FROM citus_local_table
WHERE citus_local_table.a = reference_table.a;
-- should not work, add HINT use CTEs
UPDATE citus_local_table
SET b = 6
FROM distributed_table
WHERE citus_local_table.a = distributed_table.a;
-- should work, add HINT use CTEs
UPDATE citus_local_table
SET b = 6
FROM reference_table
WHERE citus_local_table.a = reference_table.a;
-- should not work, add HINT use CTEs
DELETE FROM distributed_table
USING citus_local_table
WHERE citus_local_table.a = distributed_table.a;
-- should not work, add HINT use CTEs
DELETE FROM citus_local_table
USING distributed_table
WHERE citus_local_table.a = distributed_table.a;
DELETE FROM reference_table
USING citus_local_table
WHERE citus_local_table.a = reference_table.a;
-- should work, add HINT use CTEs
DELETE FROM citus_local_table
USING reference_table
WHERE citus_local_table.a = reference_table.a;
-- just works
DELETE FROM citus_local_table
WHERE citus_local_table.a IN (SELECT a FROM distributed_table);
-- just works
DELETE FROM citus_local_table
WHERE citus_local_table.a IN (SELECT a FROM reference_table);
-- just works
WITH distributed_table_cte AS (SELECT * FROM distributed_table)
UPDATE citus_local_table
SET b = 6
FROM distributed_table_cte
WHERE citus_local_table.a = distributed_table_cte.a;
-- just works
WITH reference_table_cte AS (SELECT * FROM reference_table)
UPDATE citus_local_table
SET b = 6
FROM reference_table_cte
WHERE citus_local_table.a = reference_table_cte.a;
------------------------
----- VIEW QUERIES -----
------------------------
CREATE MATERIALIZED VIEW mat_view_4 AS
SELECT count(*)
FROM citus_local_table
JOIN reference_table
USING (a);
-- ok
SELECT count(*) FROM mat_view_4;
-- should work
SELECT count(*) FROM distributed_table WHERE b in
(SELECT count FROM mat_view_4);
CREATE VIEW view_2 AS
SELECT count(*)
FROM citus_local_table
JOIN citus_local_table_2 USING (a)
JOIN distributed_table USING (a);
-- should fail as view contains direct local dist join
SELECT count(*) FROM view_2;
CREATE VIEW view_3
AS SELECT count(*)
FROM citus_local_table_2
JOIN reference_table
USING (a);
-- ok
SELECT count(*) FROM view_3;
-- view treated as subquery, so should work
SELECT count(*) FROM view_3, distributed_table;
----------------------------------------------
-- Some other tests with subqueries & CTE's --
----------------------------------------------
SELECT clear_and_init_test_tables();
SELECT count(*) AS a, count(*) AS b
FROM reference_table
JOIN (SELECT count(*) as a, count(*) as b
FROM citus_local_table_2
JOIN (SELECT count(*) as a, count(*) as b
FROM postgres_local_table
JOIN (SELECT count(*) as a, count(*) as b
FROM reference_table as table_4677) subquery5108
USING (a)) subquery7132
USING (b)) subquery7294
USING (a);
-- direct join inside CTE not supported
WITH cte AS (
UPDATE citus_local_table lt SET a = mt.a
FROM distributed_table mt WHERE mt.b = lt.b
RETURNING lt.b, lt.a
) SELECT * FROM cte JOIN distributed_table mt ON mt.b = cte.b ORDER BY 1,2,3,4;
-- join with CTE just works
UPDATE citus_local_table
SET a=5
FROM (SELECT avg(distributed_table.b) as avg_b
FROM distributed_table) as foo
WHERE
foo.avg_b = citus_local_table.b;
-- should work
UPDATE distributed_table
SET b = avg_a
FROM (SELECT avg(citus_local_table.a) as avg_a FROM citus_local_table) as foo
WHERE foo.avg_a = distributed_table.a
RETURNING distributed_table.*;
-- it is unfortunate that recursive planner cannot detect this
-- but expected to not work
UPDATE citus_local_table
SET a=5
FROM (SELECT b FROM distributed_table) AS foo
WHERE foo.b = citus_local_table.b;
------------------------------------
-- test different execution paths --
------------------------------------
-- a bit different explain output than for postgres local tables
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT * FROM distributed_table
ORDER BY distributed_table.*
LIMIT 10;
-- show that we do not pull to coordinator
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT * FROM citus_local_table;
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table;
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table, postgres_local_table;
-- show that we pull to coordinator when a distributed table is involved
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table, distributed_table;
-- truncate tables & add unique constraints to be able to define foreign keys
TRUNCATE reference_table, citus_local_table, distributed_table;
ALTER TABLE reference_table ADD CONSTRAINT pkey_ref PRIMARY KEY (a);
ALTER TABLE citus_local_table ADD CONSTRAINT pkey_c PRIMARY KEY (a);
-- define a foreign key chain distributed table -> reference table -> citus local table
-- to test sequential execution
ALTER TABLE distributed_table ADD CONSTRAINT fkey_dist_to_ref FOREIGN KEY(a) REFERENCES reference_table(a) ON DELETE RESTRICT;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(a) REFERENCES citus_local_table(a) ON DELETE RESTRICT;
INSERT INTO citus_local_table VALUES (1);
INSERT INTO reference_table VALUES (1);
BEGIN;
INSERT INTO citus_local_table VALUES (1) ON CONFLICT (a) DO NOTHING;
INSERT INTO distributed_table VALUES (1);
-- should show sequential as first inserting into citus local table
-- would force the xact block to use sequential execution
show citus.multi_shard_modify_mode;
ROLLBACK;
BEGIN;
TRUNCATE distributed_table;
-- should error out as we truncated distributed_table via parallel execution
TRUNCATE citus_local_table CASCADE;
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
TRUNCATE distributed_table;
-- should work fine as we already switched to sequential execution
-- before parallel truncate
TRUNCATE citus_local_table CASCADE;
ROLLBACK;
ALTER TABLE distributed_table DROP CONSTRAINT fkey_dist_to_ref;
BEGIN;
INSERT INTO citus_local_table VALUES (1) ON CONFLICT (a) DO NOTHING;
show citus.multi_shard_modify_mode;
ROLLBACK;
-- remove uniqueness constraint and dependent foreign key constraint for next tests
ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;
ALTER TABLE citus_local_table DROP CONSTRAINT pkey_c;
COPY citus_local_table(a) FROM PROGRAM 'seq 1';
-- should use local execution
BEGIN;
COPY citus_local_table(a) FROM PROGRAM 'seq 1';
COPY citus_local_table(a) FROM PROGRAM 'seq 1';
COMMIT;
COPY citus_local_table TO STDOUT;
COPY (SELECT * FROM citus_local_table) TO STDOUT;
BEGIN;
COPY citus_local_table TO STDOUT;
COMMIT;
BEGIN;
COPY (SELECT * FROM citus_local_table) TO STDOUT;
COMMIT;
-- truncate test tables for next test
TRUNCATE citus_local_table, reference_table, distributed_table;
BEGIN;
INSERT INTO citus_local_table VALUES (1), (2);
SAVEPOINT sp1;
INSERT INTO citus_local_table VALUES (3), (4);
ROLLBACK TO SAVEPOINT sp1;
SELECT * FROM citus_local_table ORDER BY 1,2;
SAVEPOINT sp2;
INSERT INTO citus_local_table VALUES (5), (6);
INSERT INTO distributed_table VALUES (5), (6);
ROLLBACK TO SAVEPOINT sp2;
SELECT * FROM citus_local_table ORDER BY 1,2;
SELECT * FROM distributed_table ORDER BY 1,2;
SAVEPOINT sp3;
INSERT INTO citus_local_table VALUES (7), (8);
INSERT INTO reference_table VALUES (7), (8);
ROLLBACK TO SAVEPOINT sp3;
SELECT * FROM citus_local_table ORDER BY 1,2;
SELECT * FROM reference_table ORDER BY 1,2;
COMMIT;
-- cleanup at exit
DROP SCHEMA citus_local_table_queries CASCADE;

View File

@ -0,0 +1,659 @@
\set VERBOSITY terse
SET citus.next_shard_id TO 1510000;
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_queries_mx;
SET search_path TO citus_local_table_queries_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;
-- start metadata sync to worker 1
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
SET citus.replication_model TO streaming;
CREATE TABLE citus_local_table(a int, b int);
SELECT create_citus_local_table('citus_local_table');
CREATE TABLE citus_local_table_2(a int, b int);
SELECT create_citus_local_table('citus_local_table_2');
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
CREATE TABLE distributed_table(a int, b int);
SELECT create_distributed_table('distributed_table', 'a');
\c - - - :worker_1_port
SET search_path TO citus_local_table_queries_mx;
CREATE TABLE postgres_local_table(a int, b int);
-- Define a helper function to truncate & insert some data into our test tables
-- We should call this function at some places in this test file to prevent
-- test to take a long time.
-- We shouldn't use LIMIT in INSERT SELECT queries to make the test faster as
-- LIMIT would force planner to wrap SELECT query in an intermediate result and
-- this might reduce the coverage of the test cases.
CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$
BEGIN
SET client_min_messages to ERROR;
TRUNCATE postgres_local_table, citus_local_table, reference_table, distributed_table;
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO citus_local_table_2 SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO distributed_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
RESET client_min_messages;
END;
$$ LANGUAGE plpgsql;
----------------
---- SELECT ----
----------------
SELECT clear_and_init_test_tables();
-- join between citus local tables and reference tables would succeed
SELECT count(*) FROM citus_local_table, reference_table WHERE citus_local_table.a = reference_table.a;
SELECT * FROM citus_local_table, reference_table WHERE citus_local_table.a = reference_table.a ORDER BY 1,2,3,4 FOR UPDATE;
-- should work
WITH cte_1 AS
(SELECT * FROM citus_local_table, reference_table WHERE citus_local_table.a = reference_table.a ORDER BY 1,2,3,4 FOR UPDATE)
SELECT count(*) FROM cte_1;
-- should work as joins are between ctes
WITH cte_citus_local_table AS
(SELECT * FROM citus_local_table),
cte_postgres_local_table AS
(SELECT * FROM postgres_local_table),
cte_distributed_table AS
(SELECT * FROM distributed_table)
SELECT count(*) FROM cte_distributed_table, cte_citus_local_table, cte_postgres_local_table
WHERE cte_citus_local_table.a = 1 AND cte_distributed_table.a = 1;
-- should fail as we don't support direct joins between distributed/local tables
SELECT count(*) FROM distributed_table d1, distributed_table d2, citus_local_table;
-- local table inside subquery should just work
SELECT count(*) FROM
(
SELECT * FROM (SELECT * FROM citus_local_table) as subquery_inner
) as subquery_top;
SELECT clear_and_init_test_tables();
-- join between citus/postgres local tables wouldn't work as citus local table is on the coordinator
SELECT count(*) FROM
(
SELECT * FROM (SELECT count(*) FROM citus_local_table, postgres_local_table) as subquery_inner
) as subquery_top;
-- should fail as we don't support direct joins between distributed/local tables
SELECT count(*) FROM
(
SELECT *, random() FROM (SELECT *, random() FROM citus_local_table, distributed_table) as subquery_inner
) as subquery_top;
-- should fail as we don't support direct joins between distributed/local tables
SELECT count(*) FROM
(
SELECT *, random()
FROM (
WITH cte_1 AS (SELECT *, random() FROM citus_local_table, distributed_table) SELECT * FROM cte_1) as subquery_inner
) as subquery_top;
-- should be fine
SELECT count(*) FROM
(
SELECT *, random()
FROM (
WITH cte_1 AS (SELECT *, random() FROM citus_local_table), cte_2 AS (SELECT * FROM distributed_table) SELECT count(*) FROM cte_1, cte_2
) as subquery_inner
) as subquery_top;
SELECT clear_and_init_test_tables();
-- prepared statement
PREPARE citus_local_only AS SELECT count(*) FROM citus_local_table;
-- execute 6 times, local tables without params
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
EXECUTE citus_local_only;
-- execute 6 times, with param
PREPARE citus_local_only_p(int) AS SELECT count(*) FROM citus_local_table WHERE a = $1;
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
EXECUTE citus_local_only_p(1);
-- do not evalute the function
-- show the logs
EXECUTE citus_local_only_p(random());
EXECUTE citus_local_only_p(random());
PREPARE mixed_query(int, int, int) AS
WITH cte_citus_local_table AS
(SELECT * FROM citus_local_table WHERE a = $1),
cte_postgres_local_table AS
(SELECT * FROM postgres_local_table WHERE a = $2),
cte_distributed_table AS
(SELECT * FROM distributed_table WHERE a = $3),
cte_mixes AS (SELECT * FROM cte_distributed_table, cte_citus_local_table, cte_postgres_local_table)
SELECT count(*) FROM cte_mixes;
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
EXECUTE mixed_query(1,2,3);
SELECT clear_and_init_test_tables();
-- anonymous columns
WITH a AS (SELECT a, '' FROM citus_local_table GROUP BY a) SELECT a.a FROM a ORDER BY 1 LIMIT 5;
WITH a AS (SELECT b, '' FROM citus_local_table WHERE a = 1) SELECT * FROM a, a b ORDER BY 1 LIMIT 5;
-- set operations should just work
SELECT * FROM citus_local_table UNION SELECT * FROM postgres_local_table UNION SELECT * FROM distributed_table ORDER BY 1,2;
(SELECT * FROM citus_local_table ORDER BY 1,2 LIMIT 5) INTERSECT (SELECT i, i FROM generate_series(0, 100) i) ORDER BY 1, 2;
-- should just work as recursive planner kicks in
SELECT count(*) FROM distributed_table WHERE a IN (SELECT a FROM citus_local_table);
SELECT count(*) FROM citus_local_table WHERE a IN (SELECT a FROM distributed_table);
SELECT count(*) FROM reference_table WHERE a IN (SELECT a FROM citus_local_table);
SELECT count(*) FROM citus_local_table WHERE a IN (SELECT a FROM reference_table);
-- nested recursive queries should just work
SELECT count(*) FROM citus_local_table
WHERE a IN
(SELECT a FROM distributed_table WHERE a IN
(SELECT b FROM citus_local_table WHERE b IN (SELECT b FROM postgres_local_table)));
-- local outer joins
SELECT count(*) FROM citus_local_table LEFT JOIN reference_table ON (true);
SELECT count(*) FROM reference_table
LEFT JOIN citus_local_table ON (true)
LEFT JOIN postgres_local_table ON (true)
LEFT JOIN reference_table r2 ON (true);
-- not supported direct outer join
SELECT count(*) FROM citus_local_table LEFT JOIN distributed_table ON (true);
-- distinct in subquery on CTE
WITH one_row AS (
SELECT a from citus_local_table WHERE b = 1
)
SELECT
*
FROM
distributed_table
WHERE
b IN (SELECT DISTINCT a FROM one_row)
ORDER BY
1, 2
LIMIT
1;
WITH one_row_2 AS (
SELECT a from distributed_table WHERE b = 1
)
SELECT
*
FROM
citus_local_table
WHERE
b IN (SELECT DISTINCT a FROM one_row_2)
ORDER BY
1 ,2
LIMIT
1;
-- join between citus local tables and distributed tables would fail
SELECT count(*) FROM citus_local_table, distributed_table;
SELECT * FROM citus_local_table, distributed_table ORDER BY 1,2,3,4 FOR UPDATE;
-- join between citus local table and postgres local table would fail
-- as citus local table is on the coordinator
SELECT count(citus_local_table.b), count(postgres_local_table.a)
FROM citus_local_table, postgres_local_table
WHERE citus_local_table.a = postgres_local_table.b;
-- select for update is just OK
SELECT * FROM citus_local_table ORDER BY 1,2 FOR UPDATE;
---------------------------
----- INSERT SELECT -----
---------------------------
-- simple INSERT SELECT is OK
SELECT clear_and_init_test_tables();
INSERT INTO citus_local_table
SELECT * from reference_table;
INSERT INTO reference_table
SELECT * from citus_local_table;
INSERT INTO citus_local_table
SELECT * from distributed_table;
INSERT INTO distributed_table
SELECT * from citus_local_table;
INSERT INTO citus_local_table
SELECT * from citus_local_table_2;
INSERT INTO citus_local_table
SELECT * from citus_local_table_2
ORDER BY 1,2
LIMIT 10;
INSERT INTO citus_local_table
SELECT * from postgres_local_table;
INSERT INTO postgres_local_table
SELECT * from citus_local_table;
-- INSERT SELECT with local joins are OK
SELECT clear_and_init_test_tables();
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
INSERT INTO reference_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
INSERT INTO reference_table
SELECT reference_table.* FROM reference_table, postgres_local_table
JOIN citus_local_table ON (true);
SELECT clear_and_init_test_tables();
INSERT INTO distributed_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
INSERT INTO distributed_table
SELECT reference_table.* FROM reference_table, postgres_local_table
JOIN citus_local_table ON (true);
INSERT INTO postgres_local_table
SELECT reference_table.* FROM reference_table
JOIN citus_local_table ON (true);
-- INSERT SELECT that joins reference and distributed tables is also OK
SELECT clear_and_init_test_tables();
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table
JOIN distributed_table ON (true);
INSERT INTO citus_local_table
SELECT reference_table.*
FROM reference_table, distributed_table;
-- INSERT SELECT that joins citus local and distributed table directly will fail ..
INSERT INTO citus_local_table
SELECT distributed_table.* FROM distributed_table
JOIN citus_local_table ON (true);
-- .. but when wrapped into a CTE, join works fine
INSERT INTO citus_local_table
SELECT distributed_table.* FROM distributed_table
JOIN (WITH cte AS (SELECT * FROM citus_local_table) SELECT * FROM cte) as foo ON (true);
-- multi row insert is OK
INSERT INTO citus_local_table VALUES (1, 2), (3, 4);
---------------------------
----- DELETE / UPDATE -----
---------------------------
-- modifications using citus local tables and postgres local tables
-- are not supported, see below four tests
SELECT clear_and_init_test_tables();
DELETE FROM citus_local_table
USING postgres_local_table
WHERE citus_local_table.b = postgres_local_table.b;
UPDATE citus_local_table
SET b = 5
FROM postgres_local_table
WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b;
DELETE FROM postgres_local_table
USING citus_local_table
WHERE citus_local_table.b = postgres_local_table.b;
UPDATE postgres_local_table
SET b = 5
FROM citus_local_table
WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b;
-- no direct joins supported
UPDATE distributed_table
SET b = 6
FROM citus_local_table
WHERE citus_local_table.a = distributed_table.a;
UPDATE reference_table
SET b = 6
FROM citus_local_table
WHERE citus_local_table.a = reference_table.a;
-- should not work, add HINT use CTEs
UPDATE citus_local_table
SET b = 6
FROM distributed_table
WHERE citus_local_table.a = distributed_table.a;
-- should work, add HINT use CTEs
UPDATE citus_local_table
SET b = 6
FROM reference_table
WHERE citus_local_table.a = reference_table.a;
-- should not work, add HINT use CTEs
DELETE FROM distributed_table
USING citus_local_table
WHERE citus_local_table.a = distributed_table.a;
-- should not work, add HINT use CTEs
DELETE FROM citus_local_table
USING distributed_table
WHERE citus_local_table.a = distributed_table.a;
DELETE FROM reference_table
USING citus_local_table
WHERE citus_local_table.a = reference_table.a;
-- should work, add HINT use CTEs
DELETE FROM citus_local_table
USING reference_table
WHERE citus_local_table.a = reference_table.a;
-- just works
DELETE FROM citus_local_table
WHERE citus_local_table.a IN (SELECT a FROM distributed_table);
-- just works
DELETE FROM citus_local_table
WHERE citus_local_table.a IN (SELECT a FROM reference_table);
-- just works
WITH distributed_table_cte AS (SELECT * FROM distributed_table)
UPDATE citus_local_table
SET b = 6
FROM distributed_table_cte
WHERE citus_local_table.a = distributed_table_cte.a;
-- just works
WITH reference_table_cte AS (SELECT * FROM reference_table)
UPDATE citus_local_table
SET b = 6
FROM reference_table_cte
WHERE citus_local_table.a = reference_table_cte.a;
------------------------
----- VIEW QUERIES -----
------------------------
CREATE MATERIALIZED VIEW mat_view_4 AS
SELECT count(*)
FROM citus_local_table
JOIN reference_table
USING (a);
-- ok
SELECT count(*) FROM mat_view_4;
-- should work
SELECT count(*) FROM distributed_table WHERE b in
(SELECT count FROM mat_view_4);
CREATE VIEW view_2 AS
SELECT count(*)
FROM citus_local_table
JOIN citus_local_table_2 USING (a)
JOIN distributed_table USING (a);
-- should fail as view contains direct local dist join
SELECT count(*) FROM view_2;
CREATE VIEW view_3
AS SELECT count(*)
FROM citus_local_table_2
JOIN reference_table
USING (a);
-- ok
SELECT count(*) FROM view_3;
-- view treated as subquery, so should work
SELECT count(*) FROM view_3, distributed_table;
----------------------------------------------
-- Some other tests with subqueries & CTE's --
----------------------------------------------
SELECT clear_and_init_test_tables();
SELECT count(*) AS a, count(*) AS b
FROM reference_table
JOIN (SELECT count(*) as a, count(*) as b
FROM citus_local_table_2
JOIN (SELECT count(*) as a, count(*) as b
FROM postgres_local_table
JOIN (SELECT count(*) as a, count(*) as b
FROM reference_table as table_4677) subquery5108
USING (a)) subquery7132
USING (b)) subquery7294
USING (a);
-- direct join inside CTE not supported
WITH cte AS (
UPDATE citus_local_table lt SET a = mt.a
FROM distributed_table mt WHERE mt.b = lt.b
RETURNING lt.b, lt.a
) SELECT * FROM cte JOIN distributed_table mt ON mt.b = cte.b ORDER BY 1,2,3,4;
-- join with CTE just works
UPDATE citus_local_table
SET a=5
FROM (SELECT avg(distributed_table.b) as avg_b
FROM distributed_table) as foo
WHERE
foo.avg_b = citus_local_table.b;
-- should work
UPDATE distributed_table
SET b = avg_a
FROM (SELECT avg(citus_local_table.a) as avg_a FROM citus_local_table) as foo
WHERE foo.avg_a = distributed_table.a
RETURNING distributed_table.*;
-- it is unfortunate that recursive planner cannot detect this
-- but expected to not work
UPDATE citus_local_table
SET a=5
FROM (SELECT b FROM distributed_table) AS foo
WHERE foo.b = citus_local_table.b;
------------------------------------
-- test different execution paths --
------------------------------------
-- a bit different explain output than for postgres local tables
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT * FROM distributed_table
ORDER BY distributed_table.*
LIMIT 10;
-- show that we do not pull to coordinator
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT * FROM citus_local_table;
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table;
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table, postgres_local_table;
-- show that we pull to coordinator when a distributed table is involved
EXPLAIN (COSTS FALSE)
INSERT INTO citus_local_table
SELECT reference_table.* FROM reference_table, distributed_table;
-- truncate tables & add unique constraints to be able to define foreign keys
TRUNCATE reference_table, citus_local_table, distributed_table;
\c - - - :master_port
SET search_path TO citus_local_table_queries_mx;
SET citus.replication_model TO streaming;
ALTER TABLE reference_table ADD CONSTRAINT pkey_ref PRIMARY KEY (a);
ALTER TABLE citus_local_table ADD CONSTRAINT pkey_c PRIMARY KEY (a);
-- define a foreign key chain distributed table -> reference table -> citus local table
-- to test sequential execution
ALTER TABLE distributed_table ADD CONSTRAINT fkey_dist_to_ref FOREIGN KEY(a) REFERENCES reference_table(a) ON DELETE RESTRICT;
ALTER TABLE reference_table ADD CONSTRAINT fkey_ref_to_local FOREIGN KEY(a) REFERENCES citus_local_table(a) ON DELETE RESTRICT;
\c - - - :worker_1_port
SET search_path TO citus_local_table_queries_mx;
INSERT INTO citus_local_table VALUES (1);
INSERT INTO reference_table VALUES (1);
BEGIN;
INSERT INTO citus_local_table VALUES (1) ON CONFLICT (a) DO NOTHING;
INSERT INTO distributed_table VALUES (1);
-- should show sequential as first inserting into citus local table
-- would force the xact block to use sequential execution
show citus.multi_shard_modify_mode;
ROLLBACK;
BEGIN;
TRUNCATE distributed_table;
-- should error out as we truncated distributed_table via parallel execution
TRUNCATE citus_local_table CASCADE;
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
TRUNCATE distributed_table;
-- should work fine as we already switched to sequential execution
-- before parallel truncate
TRUNCATE citus_local_table CASCADE;
ROLLBACK;
\c - - - :master_port
SET search_path TO citus_local_table_queries_mx;
SET citus.replication_model TO streaming;
ALTER TABLE distributed_table DROP CONSTRAINT fkey_dist_to_ref;
\c - - - :worker_1_port
SET search_path TO citus_local_table_queries_mx;
BEGIN;
INSERT INTO citus_local_table VALUES (1) ON CONFLICT (a) DO NOTHING;
show citus.multi_shard_modify_mode;
ROLLBACK;
\c - - - :master_port
SET search_path TO citus_local_table_queries_mx;
SET citus.replication_model TO streaming;
-- remove uniqueness constraint and dependent foreign key constraint for next tests
ALTER TABLE reference_table DROP CONSTRAINT fkey_ref_to_local;
ALTER TABLE citus_local_table DROP CONSTRAINT pkey_c;
\c - - - :worker_1_port
SET search_path TO citus_local_table_queries_mx;
COPY citus_local_table(a) FROM PROGRAM 'seq 1';
BEGIN;
COPY citus_local_table(a) FROM PROGRAM 'seq 1';
COPY citus_local_table(a) FROM PROGRAM 'seq 1';
COMMIT;
COPY citus_local_table TO STDOUT;
COPY (SELECT * FROM citus_local_table) TO STDOUT;
BEGIN;
COPY citus_local_table TO STDOUT;
COMMIT;
BEGIN;
COPY (SELECT * FROM citus_local_table) TO STDOUT;
COMMIT;
-- truncate test tables for next test
TRUNCATE citus_local_table, reference_table, distributed_table;
BEGIN;
INSERT INTO citus_local_table VALUES (1), (2);
SAVEPOINT sp1;
INSERT INTO citus_local_table VALUES (3), (4);
ROLLBACK TO SAVEPOINT sp1;
SELECT * FROM citus_local_table ORDER BY 1,2;
SAVEPOINT sp2;
INSERT INTO citus_local_table VALUES (5), (6);
INSERT INTO distributed_table VALUES (5), (6);
ROLLBACK TO SAVEPOINT sp2;
SELECT * FROM citus_local_table ORDER BY 1,2;
SELECT * FROM distributed_table ORDER BY 1,2;
SAVEPOINT sp3;
INSERT INTO citus_local_table VALUES (7), (8);
INSERT INTO reference_table VALUES (7), (8);
ROLLBACK TO SAVEPOINT sp3;
SELECT * FROM citus_local_table ORDER BY 1,2;
SELECT * FROM reference_table ORDER BY 1,2;
COMMIT;
\c - - - :master_port
-- cleanup at exit
DROP SCHEMA citus_local_table_queries_mx CASCADE;

View File

@ -1,6 +1,7 @@
-- This test file includes tests to show that we do not allow triggers -- 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 -- on distributed tables and reference tables. Note that in other
-- the successfull citus table creation cases. -- regression tests, we already test the successfull citus table
-- creation cases.
\set VERBOSITY terse \set VERBOSITY terse
@ -9,9 +10,9 @@ SET citus.next_shard_id TO 1505000;
CREATE SCHEMA table_triggers_schema; CREATE SCHEMA table_triggers_schema;
SET search_path TO 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 a simple function to be invoked by triggers
CREATE FUNCTION update_value() RETURNS trigger AS $update_value$ CREATE FUNCTION update_value() RETURNS trigger AS $update_value$
@ -30,11 +31,54 @@ SELECT create_reference_table('reference_table');
-- below two should fail -- below two should fail
CREATE TRIGGER update_value_dist CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table AFTER INSERT ON distributed_table
FOR EACH ROW EXECUTE PROCEDURE update_value(); FOR EACH ROW EXECUTE FUNCTION update_value();
CREATE TRIGGER update_value_ref CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table 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 -- 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 CREATE TRIGGER update_value_dist
AFTER INSERT ON distributed_table_1 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 TABLE reference_table_1 (value int);
CREATE TRIGGER update_value_ref CREATE TRIGGER update_value_ref
AFTER INSERT ON reference_table_1 AFTER INSERT ON reference_table_1
FOR EACH ROW EXECUTE PROCEDURE update_value(); FOR EACH ROW EXECUTE FUNCTION update_value();
-- below two should fail -- below two should fail
SELECT create_distributed_table('distributed_table_1', 'value'); SELECT create_distributed_table('distributed_table_1', 'value');

View File

@ -3,64 +3,135 @@
CREATE TABLE the_table (a int, b int, z bigserial); CREATE TABLE the_table (a int, b int, z bigserial);
SELECT create_distributed_table('the_table', 'a'); SELECT create_distributed_table('the_table', 'a');
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
CREATE TABLE reference_table (a int, b int, z bigserial);
SELECT create_reference_table('reference_table');
CREATE TABLE citus_local_table (a int, b int, z bigserial);
SELECT create_citus_local_table('citus_local_table');
CREATE TABLE local (a int, b int); CREATE TABLE local (a int, b int);
\c - - - :follower_master_port \c - - - :follower_master_port
-- inserts normally do not work on a standby coordinator -- inserts normally do not work on a standby coordinator
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2); INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
-- we can allow DML on a writable standby coordinator -- We can allow DML on a writable standby coordinator.
-- Note that it doesn't help to enable writes for citus local tables
-- and coordinator replicated reference tables. This is because, the
-- data is in the coordinator and will hit read-only tranaction checks
-- on Postgres
SET citus.writable_standby_coordinator TO on; SET citus.writable_standby_coordinator TO on;
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2); INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
SELECT * FROM the_table; SELECT * FROM the_table;
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
SELECT * FROM reference_table;
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
SELECT * FROM citus_local_table;
UPDATE the_table SET z = 3 WHERE a = 1; UPDATE the_table SET z = 3 WHERE a = 1;
UPDATE reference_table SET z = 3 WHERE a = 1;
UPDATE citus_local_table SET z = 3 WHERE a = 1;
SELECT * FROM the_table; SELECT * FROM the_table;
SELECT * FROM reference_table;
SELECT * FROM citus_local_table;
DELETE FROM the_table WHERE a = 1; DELETE FROM the_table WHERE a = 1;
DELETE FROM reference_table WHERE a = 1;
DELETE FROM citus_local_table WHERE a = 1;
SELECT * FROM the_table; SELECT * FROM the_table;
SELECT * FROM reference_table;
SELECT * FROM citus_local_table;
-- drawing from a sequence is not possible -- drawing from a sequence is not possible
INSERT INTO the_table (a, b) VALUES (1, 2); INSERT INTO the_table (a, b) VALUES (1, 2);
INSERT INTO reference_table (a, b) VALUES (1, 2);
INSERT INTO citus_local_table (a, b) VALUES (1, 2);
-- 2PC is not possible -- 2PC is not possible
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7); INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
-- COPY is not possible in 2PC mode -- COPY is not possible in 2PC mode
COPY the_table (a, b, z) FROM STDIN WITH CSV; COPY the_table (a, b, z) FROM STDIN WITH CSV;
10,10,10 10,10,10
11,11,11 11,11,11
\. \.
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
10,10,10
11,11,11
\.
COPY citus_local_table (a, b, z) FROM STDIN WITH CSV;
10,10,10
11,11,11
\.
-- 1PC is possible -- 1PC is possible
SET citus.multi_shard_commit_protocol TO '1pc'; SET citus.multi_shard_commit_protocol TO '1pc';
INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7); INSERT INTO the_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
SELECT * FROM the_table ORDER BY a; SELECT * FROM the_table ORDER BY a;
INSERT INTO reference_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
SELECT * FROM reference_table ORDER BY a;
INSERT INTO citus_local_table (a, b, z) VALUES (2, 3, 4), (5, 6, 7);
SELECT * FROM citus_local_table ORDER BY a;
-- modifying CTEs are possible -- modifying CTEs are possible
WITH del AS (DELETE FROM the_table RETURNING *) WITH del AS (DELETE FROM the_table RETURNING *)
SELECT * FROM del ORDER BY a; SELECT * FROM del ORDER BY a;
WITH del AS (DELETE FROM reference_table RETURNING *)
SELECT * FROM del ORDER BY a;
WITH del AS (DELETE FROM citus_local_table RETURNING *)
SELECT * FROM del ORDER BY a;
-- COPY is possible in 1PC mode -- COPY is possible in 1PC mode
COPY the_table (a, b, z) FROM STDIN WITH CSV; COPY the_table (a, b, z) FROM STDIN WITH CSV;
10,10,10 10,10,10
11,11,11 11,11,11
\. \.
COPY reference_table (a, b, z) FROM STDIN WITH CSV;
10,10,10
11,11,11
\.
COPY citus_local_table (a, b, z) FROM STDIN WITH CSV;
10,10,10
11,11,11
\.
SELECT * FROM the_table ORDER BY a; SELECT * FROM the_table ORDER BY a;
SELECT * FROM reference_table ORDER BY a;
SELECT * FROM citus_local_table ORDER BY a;
DELETE FROM the_table; DELETE FROM the_table;
DELETE FROM reference_table;
DELETE FROM citus_local_table;
-- DDL is not possible -- DDL is not possible
TRUNCATE the_table; TRUNCATE the_table;
TRUNCATE reference_table;
TRUNCATE citus_local_table;
ALTER TABLE the_table ADD COLUMN c int; ALTER TABLE the_table ADD COLUMN c int;
ALTER TABLE reference_table ADD COLUMN c int;
ALTER TABLE citus_local_table ADD COLUMN c int;
-- rollback is possible -- rollback is possible
BEGIN; BEGIN;
INSERT INTO the_table (a, b, z) VALUES (1, 2, 2); INSERT INTO the_table (a, b, z) VALUES (1, 2, 2);
ROLLBACK; ROLLBACK;
BEGIN;
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 2);
ROLLBACK;
BEGIN;
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 2);
ROLLBACK;
SELECT * FROM the_table ORDER BY a; SELECT * FROM the_table ORDER BY a;
SELECT * FROM reference_table ORDER BY a;
SELECT * FROM citus_local_table ORDER BY a;
-- we should still disallow writes to local tables -- we should still disallow writes to local tables
INSERT INTO local VALUES (1, 1); INSERT INTO local VALUES (1, 1);
@ -75,6 +146,13 @@ CREATE TEMP TABLE local_copy_of_the_table AS SELECT * FROM the_table;
SET citus.writable_standby_coordinator TO on; SET citus.writable_standby_coordinator TO on;
INSERT INTO the_table (a, b, z) VALUES (1, 2, 3); INSERT INTO the_table (a, b, z) VALUES (1, 2, 3);
SELECT * FROM the_table ORDER BY a; SELECT * FROM the_table ORDER BY a;
INSERT INTO reference_table (a, b, z) VALUES (1, 2, 3);
SELECT * FROM reference_table ORDER BY a;
INSERT INTO citus_local_table (a, b, z) VALUES (1, 2, 3);
SELECT * FROM citus_local_table ORDER BY a;
\c - - - :master_port \c - - - :master_port
DROP TABLE the_table; DROP TABLE the_table;
DROP TABLE reference_table;
DROP TABLE citus_local_table;
SELECT master_remove_node('localhost', :master_port);

View File

@ -59,6 +59,10 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
-- now create required stuff in the worker 1 -- now create required stuff in the worker 1
\c - - - :worker_1_port \c - - - :worker_1_port
-- show that we do not support creating citus local tables from mx workers for now
CREATE TABLE citus_local_table(a int);
SELECT create_citus_local_table('citus_local_table');
-- create schema to test schema support -- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2; CREATE SCHEMA citus_mx_test_schema_join_2;

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;