diff --git a/src/backend/distributed/commands/alter_table.c b/src/backend/distributed/commands/alter_table.c index 5364d49cc..6664b6c1c 100644 --- a/src/backend/distributed/commands/alter_table.c +++ b/src/backend/distributed/commands/alter_table.c @@ -1059,7 +1059,8 @@ CreateTableConversion(TableConversionParameters *params) } relation_close(relation, NoLock); con->distributionKey = - BuildDistributionKeyFromColumnName(relation, con->distributionColumn); + BuildDistributionKeyFromColumnName(con->relationId, con->distributionColumn, + NoLock); con->originalAccessMethod = NULL; if (!PartitionedTable(con->relationId) && !IsForeignTable(con->relationId)) @@ -1175,6 +1176,9 @@ CreateDistributedTableLike(TableConversionState *con) newShardCount = con->shardCount; } + char *distributionColumnName = + ColumnToColumnName(con->newRelationId, (Node *) newDistributionKey); + Oid originalRelationId = con->relationId; if (con->originalDistributionKey != NULL && PartitionTable(originalRelationId)) { @@ -1190,16 +1194,13 @@ CreateDistributedTableLike(TableConversionState *con) */ Oid parentRelationId = PartitionParentOid(originalRelationId); Var *parentDistKey = DistPartitionKeyOrError(parentRelationId); - char *parentDistKeyColumnName = - ColumnToColumnName(parentRelationId, nodeToString(parentDistKey)); - - newDistributionKey = - FindColumnWithNameOnTargetRelation(parentRelationId, parentDistKeyColumnName, - con->newRelationId); + distributionColumnName = + ColumnToColumnName(parentRelationId, (Node *) parentDistKey); } char partitionMethod = PartitionMethod(con->relationId); - CreateDistributedTable(con->newRelationId, newDistributionKey, partitionMethod, + + CreateDistributedTable(con->newRelationId, distributionColumnName, partitionMethod, newShardCount, true, newColocateWith, false); } diff --git a/src/backend/distributed/commands/citus_global_signal.c b/src/backend/distributed/commands/citus_global_signal.c new file mode 100644 index 000000000..fc7618159 --- /dev/null +++ b/src/backend/distributed/commands/citus_global_signal.c @@ -0,0 +1,127 @@ +/*------------------------------------------------------------------------- + * + * citus_global_signal.c + * Commands for Citus' overriden versions of pg_cancel_backend + * and pg_terminate_backend statements. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/pg_version_constants.h" + +#include "distributed/backend_data.h" +#include "distributed/metadata_cache.h" +#include "distributed/worker_manager.h" +#include "lib/stringinfo.h" +#include "signal.h" + +static bool CitusSignalBackend(uint64 globalPID, uint64 timeout, int sig); + +PG_FUNCTION_INFO_V1(pg_cancel_backend); +PG_FUNCTION_INFO_V1(pg_terminate_backend); + +/* + * pg_cancel_backend overrides the Postgres' pg_cancel_backend to cancel + * a query with a global pid so a query can be cancelled from another node. + * + * To cancel a query that is on another node, a pg_cancel_backend command is sent + * to that node. This new command is sent with pid instead of global pid, so original + * pg_cancel_backend function is used. + */ +Datum +pg_cancel_backend(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + uint64 pid = PG_GETARG_INT64(0); + + int sig = SIGINT; + uint64 timeout = 0; + bool success = CitusSignalBackend(pid, timeout, sig); + + PG_RETURN_BOOL(success); +} + + +/* + * pg_terminate_backend overrides the Postgres' pg_terminate_backend to terminate + * a query with a global pid so a query can be terminated from another node. + * + * To terminate a query that is on another node, a pg_terminate_backend command is sent + * to that node. This new command is sent with pid instead of global pid, so original + * pg_terminate_backend function is used. + */ +Datum +pg_terminate_backend(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + uint64 pid = PG_GETARG_INT64(0); + uint64 timeout = PG_GETARG_INT64(1); + + int sig = SIGTERM; + bool success = CitusSignalBackend(pid, timeout, sig); + + PG_RETURN_BOOL(success); +} + + +/* + * CitusSignalBackend gets a global pid and and ends the original query with the global pid + * that might have started in another node by connecting to that node and running either + * pg_cancel_backend or pg_terminate_backend based on the withTerminate argument. + */ +static bool +CitusSignalBackend(uint64 globalPID, uint64 timeout, int sig) +{ + Assert((sig == SIGINT) || (sig == SIGTERM)); + +#if PG_VERSION_NUM < PG_VERSION_14 + if (timeout != 0) + { + elog(ERROR, "timeout parameter is only supported on Postgres 14 or later"); + } +#endif + + int nodeId = ExtractNodeIdFromGlobalPID(globalPID); + int processId = ExtractProcessIdFromGlobalPID(globalPID); + + WorkerNode *workerNode = FindNodeWithNodeId(nodeId); + + StringInfo cancelQuery = makeStringInfo(); + + if (sig == SIGINT) + { + appendStringInfo(cancelQuery, "SELECT pg_cancel_backend(%d::integer)", processId); + } + else + { +#if PG_VERSION_NUM >= PG_VERSION_14 + appendStringInfo(cancelQuery, + "SELECT pg_terminate_backend(%d::integer, %lu::bigint)", + processId, timeout); +#else + appendStringInfo(cancelQuery, "SELECT pg_terminate_backend(%d::integer)", + processId); +#endif + } + + StringInfo queryResult = makeStringInfo(); + + bool reportResultError = true; + + bool success = ExecuteRemoteQueryOrCommand(workerNode->workerName, + workerNode->workerPort, cancelQuery->data, + queryResult, reportResultError); + + if (success && queryResult && strcmp(queryResult->data, "f") == 0) + { + success = false; + } + + return success; +} diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index eb51195bf..b96882649 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -159,30 +159,14 @@ master_create_distributed_table(PG_FUNCTION_ARGS) char *colocateWithTableName = NULL; bool viaDeprecatedAPI = true; - /* - * Lock target relation with an exclusive lock - there's no way to make - * sense of this table until we've committed, and we don't want multiple - * backends manipulating this relation. - */ - Relation relation = try_relation_open(relationId, ExclusiveLock); - - if (relation == NULL) - { - ereport(ERROR, (errmsg("could not create distributed table: " - "relation does not exist"))); - } - char *distributionColumnName = text_to_cstring(distributionColumnText); - Var *distributionColumn = BuildDistributionKeyFromColumnName(relation, - distributionColumnName); - Assert(distributionColumn != NULL); + Assert(distributionColumnName != NULL); + char distributionMethod = LookupDistributionMethod(distributionMethodOid); - CreateDistributedTable(relationId, distributionColumn, distributionMethod, + CreateDistributedTable(relationId, distributionColumnName, distributionMethod, ShardCount, false, colocateWithTableName, viaDeprecatedAPI); - relation_close(relation, NoLock); - PG_RETURN_VOID(); } @@ -249,9 +233,8 @@ create_distributed_table(PG_FUNCTION_ARGS) relation_close(relation, NoLock); char *distributionColumnName = text_to_cstring(distributionColumnText); - Var *distributionColumn = BuildDistributionKeyFromColumnName(relation, - distributionColumnName); - Assert(distributionColumn != NULL); + Assert(distributionColumnName != NULL); + char distributionMethod = LookupDistributionMethod(distributionMethodOid); if (shardCount < 1 || shardCount > MAX_SHARD_COUNT) @@ -261,7 +244,7 @@ create_distributed_table(PG_FUNCTION_ARGS) shardCount, MAX_SHARD_COUNT))); } - CreateDistributedTable(relationId, distributionColumn, distributionMethod, + CreateDistributedTable(relationId, distributionColumnName, distributionMethod, shardCount, shardCountIsStrict, colocateWithTableName, viaDeprecatedAPI); @@ -281,7 +264,7 @@ create_reference_table(PG_FUNCTION_ARGS) Oid relationId = PG_GETARG_OID(0); char *colocateWithTableName = NULL; - Var *distributionColumn = NULL; + char *distributionColumnName = NULL; bool viaDeprecatedAPI = false; @@ -317,7 +300,7 @@ create_reference_table(PG_FUNCTION_ARGS) errdetail("There are no active worker nodes."))); } - CreateDistributedTable(relationId, distributionColumn, DISTRIBUTE_BY_NONE, + CreateDistributedTable(relationId, distributionColumnName, DISTRIBUTE_BY_NONE, ShardCount, false, colocateWithTableName, viaDeprecatedAPI); PG_RETURN_VOID(); } @@ -385,9 +368,10 @@ EnsureRelationExists(Oid relationId) * day, once we deprecate master_create_distribute_table completely. */ void -CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributionMethod, - int shardCount, bool shardCountIsStrict, - char *colocateWithTableName, bool viaDeprecatedAPI) +CreateDistributedTable(Oid relationId, char *distributionColumnName, + char distributionMethod, int shardCount, + bool shardCountIsStrict, char *colocateWithTableName, + bool viaDeprecatedAPI) { /* * EnsureTableNotDistributed errors out when relation is a citus table but @@ -443,6 +427,8 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio DropFKeysRelationInvolvedWithTableType(relationId, INCLUDE_LOCAL_TABLES); } + LockRelationOid(relationId, ExclusiveLock); + /* * Ensure that the sequences used in column defaults of the table * have proper types @@ -463,22 +449,9 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio colocateWithTableName, viaDeprecatedAPI); - /* - * Due to dropping columns, the parent's distribution key may not match the - * partition's distribution key. The input distributionColumn belongs to - * the parent. That's why we override the distribution column of partitions - * here. See issue #5123 for details. - */ - if (PartitionTable(relationId)) - { - Oid parentRelationId = PartitionParentOid(relationId); - char *distributionColumnName = - ColumnToColumnName(parentRelationId, nodeToString(distributionColumn)); - - distributionColumn = - FindColumnWithNameOnTargetRelation(parentRelationId, distributionColumnName, - relationId); - } + Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId, + distributionColumnName, + ExclusiveLock); /* * ColocationIdForNewTable assumes caller acquires lock on relationId. In our case, @@ -567,7 +540,7 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio foreach_oid(partitionRelationId, partitionList) { - CreateDistributedTable(partitionRelationId, distributionColumn, + CreateDistributedTable(partitionRelationId, distributionColumnName, distributionMethod, shardCount, false, parentRelationName, viaDeprecatedAPI); } @@ -1033,7 +1006,13 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn, EnsureTableNotDistributed(relationId); EnsureLocalTableEmptyIfNecessary(relationId, distributionMethod, viaDeprecatedAPI); - EnsureRelationHasNoTriggers(relationId); + + /* user really wants triggers? */ + if (!EnableUnsafeTriggers) + { + EnsureRelationHasNoTriggers(relationId); + } + /* we assume callers took necessary locks */ Relation relation = relation_open(relationId, NoLock); @@ -1355,11 +1334,11 @@ EnsureRelationHasNoTriggers(Oid relationId) Assert(relationName != NULL); ereport(ERROR, (errmsg("cannot distribute relation \"%s\" because it has " - "triggers ", relationName), - errdetail("Citus does not support distributing tables with " - "triggers."), - errhint("Drop all the triggers on \"%s\" and retry.", - relationName))); + "triggers and \"citus.enable_unsafe_triggers\" is " + "set to \"false\"", relationName), + errhint("Consider setting \"citus.enable_unsafe_triggers\" " + "to \"true\", or drop all the triggers on \"%s\" " + "and retry.", relationName))); } } diff --git a/src/backend/distributed/commands/distribute_object_ops.c b/src/backend/distributed/commands/distribute_object_ops.c index 37758aec3..f1aa1fa1e 100644 --- a/src/backend/distributed/commands/distribute_object_ops.c +++ b/src/backend/distributed/commands/distribute_object_ops.c @@ -564,7 +564,7 @@ static DistributeObjectOps TextSearchConfig_Rename = { static DistributeObjectOps Trigger_AlterObjectDepends = { .deparse = NULL, .qualify = NULL, - .preprocess = NULL, + .preprocess = PreprocessAlterTriggerDependsStmt, .postprocess = PostprocessAlterTriggerDependsStmt, .address = NULL, .markDistributed = false, diff --git a/src/backend/distributed/commands/schema.c b/src/backend/distributed/commands/schema.c index e0c0f2b11..77f3fcc32 100644 --- a/src/backend/distributed/commands/schema.c +++ b/src/backend/distributed/commands/schema.c @@ -86,13 +86,6 @@ PreprocessDropSchemaStmt(Node *node, const char *queryString, DropStmt *dropStatement = castNode(DropStmt, node); Assert(dropStatement->removeType == OBJECT_SCHEMA); - if (!ShouldPropagate()) - { - return NIL; - } - - EnsureCoordinator(); - List *distributedSchemas = FilterDistributedSchemas(dropStatement->objects); if (list_length(distributedSchemas) < 1) @@ -100,6 +93,13 @@ PreprocessDropSchemaStmt(Node *node, const char *queryString, return NIL; } + if (!ShouldPropagate()) + { + return NIL; + } + + EnsureCoordinator(); + EnsureSequentialMode(OBJECT_SCHEMA); Value *schemaVal = NULL; diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index 9236f0e1a..4bf1ff373 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -378,6 +378,8 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const } Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId); + char *distributionColumnName = + ColumnToColumnName(parentRelationId, (Node *) parentDistributionColumn); char parentDistributionMethod = DISTRIBUTE_BY_HASH; char *parentRelationName = generate_qualified_relation_name(parentRelationId); bool viaDeprecatedAPI = false; @@ -385,7 +387,7 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(parentRelationId, relationId); - CreateDistributedTable(relationId, parentDistributionColumn, + CreateDistributedTable(relationId, distributionColumnName, parentDistributionMethod, ShardCount, false, parentRelationName, viaDeprecatedAPI); } @@ -573,13 +575,8 @@ static void DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationId) { Var *distributionColumn = DistPartitionKeyOrError(parentCitusRelationId); - char *distributionColumnName = - ColumnToColumnName(parentCitusRelationId, - nodeToString(distributionColumn)); - distributionColumn = - FindColumnWithNameOnTargetRelation(parentCitusRelationId, - distributionColumnName, - partitionRelationId); + char *distributionColumnName = ColumnToColumnName(parentCitusRelationId, + (Node *) distributionColumn); char distributionMethod = DISTRIBUTE_BY_HASH; char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId); @@ -588,7 +585,7 @@ DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationI SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong( parentCitusRelationId, partitionRelationId); - CreateDistributedTable(partitionRelationId, distributionColumn, + CreateDistributedTable(partitionRelationId, distributionColumnName, distributionMethod, ShardCount, false, parentRelationName, viaDeprecatedAPI); } @@ -1085,15 +1082,9 @@ PreprocessAlterTableStmt(Node *node, const char *alterTableCommand, } 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); + return CitusCreateTriggerCommandDDLJob(leftRelationId, triggerName, + alterTableCommand); } /* @@ -2573,7 +2564,7 @@ ErrorIfUnsupportedConstraint(Relation relation, char distributionMethod, * ALTER TABLE REPLICA IDENTITY * ALTER TABLE SET () * ALTER TABLE RESET () - * ALTER TABLE ENABLE/DISABLE TRIGGER (only for citus local tables) + * ALTER TABLE ENABLE/DISABLE TRIGGER (if enable_unsafe_triggers is not set, we only support triggers for citus local tables) */ static void ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement) @@ -2907,7 +2898,7 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement) errhint("You can issue each subcommand separately"))); } - ErrorOutForTriggerIfNotCitusLocalTable(relationId); + ErrorOutForTriggerIfNotSupported(relationId); break; } diff --git a/src/backend/distributed/commands/trigger.c b/src/backend/distributed/commands/trigger.c index 15ef208aa..a277cb372 100644 --- a/src/backend/distributed/commands/trigger.c +++ b/src/backend/distributed/commands/trigger.c @@ -16,13 +16,16 @@ #include "catalog/indexing.h" #include "catalog/namespace.h" #include "catalog/pg_trigger.h" +#include "commands/extension.h" #include "commands/trigger.h" #include "distributed/commands.h" #include "distributed/commands/utility_hook.h" #include "distributed/coordinator_protocol.h" #include "distributed/deparser.h" #include "distributed/listutils.h" +#include "distributed/metadata/distobject.h" #include "distributed/metadata_cache.h" +#include "distributed/metadata_sync.h" #include "distributed/namespace_utils.h" #include "distributed/shard_utils.h" #include "distributed/worker_protocol.h" @@ -52,6 +55,10 @@ static void ErrorIfDropStmtDropsMultipleTriggers(DropStmt *dropTriggerStmt); static int16 GetTriggerTypeById(Oid triggerId); +/* GUC that overrides trigger checks for distributed tables and reference tables */ +bool EnableUnsafeTriggers = false; + + /* * GetExplicitTriggerCommandList returns the list of DDL commands to create * triggers that are explicitly created for the table with relationId. See @@ -215,20 +222,14 @@ PostprocessCreateTriggerStmt(Node *node, const char *queryString) } EnsureCoordinator(); + ErrorOutForTriggerIfNotSupported(relationId); - ErrorOutForTriggerIfNotCitusLocalTable(relationId); + ObjectAddress objectAddress = GetObjectAddressFromParseTree(node, missingOk); + EnsureDependenciesExistOnAllNodes(&objectAddress); - if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) - { - ObjectAddress objectAddress = GetObjectAddressFromParseTree(node, missingOk); - EnsureDependenciesExistOnAllNodes(&objectAddress); - - char *triggerName = createTriggerStmt->trigname; - return CitusLocalTableTriggerCommandDDLJob(relationId, triggerName, - queryString); - } - - return NIL; + char *triggerName = createTriggerStmt->trigname; + return CitusCreateTriggerCommandDDLJob(relationId, triggerName, + queryString); } @@ -328,17 +329,12 @@ PostprocessAlterTriggerRenameStmt(Node *node, const char *queryString) } EnsureCoordinator(); - ErrorOutForTriggerIfNotCitusLocalTable(relationId); + ErrorOutForTriggerIfNotSupported(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; + /* use newname as standard process utility already renamed it */ + char *triggerName = renameTriggerStmt->newname; + return CitusCreateTriggerCommandDDLJob(relationId, triggerName, + queryString); } @@ -368,6 +364,70 @@ AlterTriggerRenameEventExtendNames(RenameStmt *renameTriggerStmt, char *schemaNa } +/* + * PreprocessAlterTriggerDependsStmt is called during the planning phase of an + * ALTER TRIGGER ... DEPENDS ON EXTENSION ... statement. Since triggers depending on + * extensions are assumed to be Owned by an extension we assume the extension to keep + * the trigger in sync. + * + * If we would allow users to create a dependency between a distributed trigger and an + * extension, our pruning logic for which objects to distribute as dependencies of other + * objects will change significantly, which could cause issues adding new workers. Hence + * we don't allow this dependency to be created. + */ +List * +PreprocessAlterTriggerDependsStmt(Node *node, const char *queryString, + ProcessUtilityContext processUtilityContext) +{ + AlterObjectDependsStmt *alterTriggerDependsStmt = + castNode(AlterObjectDependsStmt, node); + Assert(alterTriggerDependsStmt->objectType == OBJECT_TRIGGER); + + if (creating_extension) + { + /* + * extensions should be created separately on the workers, triggers cascading + * from an extension should therefore not be propagated here. + */ + return NIL; + } + + if (!EnableMetadataSync) + { + /* + * we are configured to disable object propagation, should not propagate anything + */ + return NIL; + } + + RangeVar *relation = alterTriggerDependsStmt->relation; + + bool missingOk = false; + Oid relationId = RangeVarGetRelid(relation, ALTER_TRIGGER_LOCK_MODE, missingOk); + + if (!IsCitusTable(relationId)) + { + return NIL; + } + + /* + * Distributed objects should not start depending on an extension, this will break + * the dependency resolving mechanism we use to replicate distributed objects to new + * workers + */ + + Value *triggerNameValue = + GetAlterTriggerDependsTriggerNameValue(alterTriggerDependsStmt); + ereport(ERROR, (errmsg( + "Triggers \"%s\" on distributed tables and local tables added to metadata " + "are not allowed to depend on an extension", strVal( + triggerNameValue)), + errdetail( + "Triggers from extensions are expected to be created on the workers " + "by the extension they depend on."))); +} + + /* * PostprocessAlterTriggerDependsStmt is called after a ALTER TRIGGER DEPENDS ON * command has been executed by standard process utility. This function errors out @@ -392,17 +452,12 @@ PostprocessAlterTriggerDependsStmt(Node *node, const char *queryString) } EnsureCoordinator(); - ErrorOutForTriggerIfNotCitusLocalTable(relationId); + ErrorOutForTriggerIfNotSupported(relationId); - if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) - { - Value *triggerNameValue = - GetAlterTriggerDependsTriggerNameValue(alterTriggerDependsStmt); - return CitusLocalTableTriggerCommandDDLJob(relationId, strVal(triggerNameValue), - queryString); - } - - return NIL; + Value *triggerNameValue = + GetAlterTriggerDependsTriggerNameValue(alterTriggerDependsStmt); + return CitusCreateTriggerCommandDDLJob(relationId, strVal(triggerNameValue), + queryString); } @@ -459,7 +514,7 @@ GetAlterTriggerDependsTriggerNameValue(AlterObjectDependsStmt *alterTriggerDepen * 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 + * type in CitusCreateTriggerCommandDDLJob but trigger wouldn't exist after * standard process utility. */ List * @@ -487,15 +542,10 @@ PreprocessDropTriggerStmt(Node *node, const char *queryString, ErrorIfUnsupportedDropTriggerCommand(dropTriggerStmt); - if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) - { - char *triggerName = NULL; - ExtractDropStmtTriggerAndRelationName(dropTriggerStmt, &triggerName, NULL); - return CitusLocalTableTriggerCommandDDLJob(relationId, triggerName, - queryString); - } - - return NIL; + char *triggerName = NULL; + ExtractDropStmtTriggerAndRelationName(dropTriggerStmt, &triggerName, NULL); + return CitusCreateTriggerCommandDDLJob(relationId, triggerName, + queryString); } @@ -517,24 +567,34 @@ ErrorIfUnsupportedDropTriggerCommand(DropStmt *dropTriggerStmt) } EnsureCoordinator(); - ErrorOutForTriggerIfNotCitusLocalTable(relationId); + ErrorOutForTriggerIfNotSupported(relationId); } /* - * ErrorOutForTriggerIfNotCitusLocalTable is a helper function to error + * ErrorOutForTriggerIfNotSupported is a helper function to error * out for unsupported trigger commands depending on the citus table type. */ void -ErrorOutForTriggerIfNotCitusLocalTable(Oid relationId) +ErrorOutForTriggerIfNotSupported(Oid relationId) { - if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) + if (EnableUnsafeTriggers) { + /* user really wants triggers */ return; } + else if (IsCitusTableType(relationId, REFERENCE_TABLE)) + { + ereport(ERROR, (errmsg("triggers are not supported on reference tables"))); + } + else if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) + { + ereport(ERROR, (errmsg("triggers are not supported on distributed tables " + "when \"citus.enable_unsafe_triggers\" is set to " + "\"false\""))); + } - ereport(ERROR, (errmsg("triggers are only supported for local tables added " - "to metadata"))); + /* we always support triggers on citus local tables */ } @@ -643,13 +703,13 @@ ErrorIfDropStmtDropsMultipleTriggers(DropStmt *dropTriggerStmt) /* - * CitusLocalTableTriggerCommandDDLJob creates a ddl job to execute given + * CitusCreateTriggerCommandDDLJob 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) +CitusCreateTriggerCommandDDLJob(Oid relationId, char *triggerName, + const char *queryString) { DDLJob *ddlJob = palloc0(sizeof(DDLJob)); ddlJob->targetRelationId = relationId; diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 161a56942..2265ca691 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -31,6 +31,7 @@ #include "commands/dbcommands.h" #include "commands/extension.h" #include "commands/trigger.h" +#include "distributed/backend_data.h" #include "distributed/colocation_utils.h" #include "distributed/connection_management.h" #include "distributed/citus_ruleutils.h" @@ -3666,9 +3667,10 @@ GetLocalNodeId(void) /* * This is expected if the coordinator is not added to the metadata. - * We'll return 0 for this case and for all cases so views can function almost normally + * We'll return GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA for this case and + * for all cases so views can function almost normally */ - nodeId = 0; + nodeId = GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA; } LocalNodeId = nodeId; diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index e43bf4e43..490eeb4ca 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -1022,7 +1022,6 @@ DistributionCreateCommand(CitusTableCacheEntry *cacheEntry) StringInfo insertDistributionCommand = makeStringInfo(); Oid relationId = cacheEntry->relationId; char distributionMethod = cacheEntry->partitionMethod; - char *partitionKeyString = cacheEntry->partitionKeyString; char *qualifiedRelationName = generate_qualified_relation_name(relationId); uint32 colocationId = cacheEntry->colocationId; @@ -1036,7 +1035,7 @@ DistributionCreateCommand(CitusTableCacheEntry *cacheEntry) else { char *partitionKeyColumnName = - ColumnToColumnName(relationId, partitionKeyString); + ColumnToColumnName(relationId, (Node *) cacheEntry->partitionColumn); appendStringInfo(tablePartitionKeyNameString, "%s", quote_literal_cstr(partitionKeyColumnName)); } @@ -2445,12 +2444,10 @@ citus_internal_add_partition_metadata(PG_FUNCTION_ARGS) distributionColumnText = PG_GETARG_TEXT_P(2); distributionColumnString = text_to_cstring(distributionColumnText); - Relation relation = relation_open(relationId, AccessShareLock); distributionColumnVar = - BuildDistributionKeyFromColumnName(relation, distributionColumnString); + BuildDistributionKeyFromColumnName(relationId, distributionColumnString, + AccessShareLock); Assert(distributionColumnVar != NULL); - - relation_close(relation, NoLock); } if (!ShouldSkipMetadataChecks()) diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index a32e0aa20..706f000cb 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -1539,6 +1539,31 @@ FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort) } +/* + * FindNodeWithNodeId searches pg_dist_node and returns the node with the nodeId. + * If the node cannot be found this functions errors. + */ +WorkerNode * +FindNodeWithNodeId(int nodeId) +{ + List *workerList = ActiveReadableNodeList(); + WorkerNode *workerNode = NULL; + + foreach_ptr(workerNode, workerList) + { + if (workerNode->nodeId == nodeId) + { + return workerNode; + } + } + + /* there isn't any node with nodeId in pg_dist_node */ + elog(ERROR, "worker node with node id %d could not be found", nodeId); + + return NULL; +} + + /* * ReadDistNode iterates over pg_dist_node table, converts each row * into it's memory representation (i.e., WorkerNode) and adds them into diff --git a/src/backend/distributed/operations/citus_tools.c b/src/backend/distributed/operations/citus_tools.c index cb87e70b1..b7905d9f8 100644 --- a/src/backend/distributed/operations/citus_tools.c +++ b/src/backend/distributed/operations/citus_tools.c @@ -14,6 +14,7 @@ #include "access/htup_details.h" #include "catalog/pg_type.h" +#include "distributed/backend_data.h" #include "distributed/connection_management.h" #include "distributed/metadata_cache.h" #include "distributed/multi_client_executor.h" @@ -50,8 +51,6 @@ static void ExecuteCommandsAndStoreResults(StringInfo *nodeNameArray, bool *statusArray, StringInfo *resultStringArray, int commandCount); -static bool ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort, - char *queryString, StringInfo queryResult); static Tuplestorestate * CreateTupleStore(TupleDesc tupleDescriptor, StringInfo *nodeNameArray, int *nodePortArray, bool *statusArray, @@ -474,9 +473,10 @@ ExecuteCommandsAndStoreResults(StringInfo *nodeNameArray, int *nodePortArray, int32 nodePort = nodePortArray[commandIndex]; char *queryString = commandStringArray[commandIndex]->data; StringInfo queryResultString = resultStringArray[commandIndex]; + bool reportResultError = false; bool success = ExecuteRemoteQueryOrCommand(nodeName, nodePort, queryString, - queryResultString); + queryResultString, reportResultError); statusArray[commandIndex] = success; @@ -491,9 +491,9 @@ ExecuteCommandsAndStoreResults(StringInfo *nodeNameArray, int *nodePortArray, * (success/failure), and query result. The query is expected to return a single * target containing zero or one rows. */ -static bool +bool ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort, char *queryString, - StringInfo queryResultString) + StringInfo queryResultString, bool reportResultError) { int connectionFlags = FORCE_NEW_CONNECTION; MultiConnection *connection = @@ -517,6 +517,11 @@ ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort, char *queryString, PGresult *queryResult = GetRemoteCommandResult(connection, raiseInterrupts); bool success = EvaluateQueryResult(connection, queryResult, queryResultString); + if (!success && reportResultError) + { + ReportResultError(connection, queryResult, ERROR); + } + PQclear(queryResult); /* close the connection */ diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 60d3facc2..017b46149 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -1013,9 +1013,9 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer StringInfo errorHint = makeStringInfo(); CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry( distributedTableId); - char *partitionKeyString = cacheEntry->partitionKeyString; - char *partitionColumnName = ColumnToColumnName(distributedTableId, - partitionKeyString); + char *partitionColumnName = + ColumnToColumnName(distributedTableId, + (Node *) cacheEntry->partitionColumn); appendStringInfo(errorHint, "Consider using an equality filter on " "partition column \"%s\" to target a single shard.", @@ -3053,8 +3053,8 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError) if (prunedShardIntervalCount != 1) { char *partitionKeyString = cacheEntry->partitionKeyString; - char *partitionColumnName = ColumnToColumnName(distributedTableId, - partitionKeyString); + char *partitionColumnName = + ColumnToColumnName(distributedTableId, stringToNode(partitionKeyString)); StringInfo errorMessage = makeStringInfo(); StringInfo errorHint = makeStringInfo(); const char *targetCountType = NULL; diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index ea0b9f2db..6f4d630f1 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -985,6 +985,17 @@ RegisterCitusConfigVariables(void) GUC_SUPERUSER_ONLY | GUC_NO_SHOW_ALL, NULL, NULL, NULL); + DefineCustomBoolVariable( + "citus.enable_unsafe_triggers", + gettext_noop("Enables arbitrary triggers on distributed tables which may cause " + "visibility and deadlock issues. Use at your own risk."), + NULL, + &EnableUnsafeTriggers, + false, + PGC_USERSET, + GUC_STANDARD, + NULL, NULL, NULL); + DefineCustomBoolVariable( "citus.enable_version_checks", gettext_noop("Enables version checks during CREATE/ALTER EXTENSION commands"), diff --git a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql index d81b2c719..849b28761 100644 --- a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql +++ b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql @@ -17,8 +17,13 @@ #include "udfs/get_all_active_transactions/11.0-1.sql" #include "udfs/get_global_active_transactions/11.0-1.sql" +#include "udfs/citus_internal_local_blocked_processes/11.0-1.sql" +#include "udfs/citus_internal_global_blocked_processes/11.0-1.sql" + #include "udfs/citus_worker_stat_activity/11.0-1.sql" #include "udfs/worker_create_or_replace_object/11.0-1.sql" +#include "udfs/citus_isolation_test_session_is_blocked/11.0-1.sql" +#include "udfs/citus_blocking_pids/11.0-1.sql" CREATE VIEW citus.citus_worker_stat_activity AS SELECT * FROM pg_catalog.citus_worker_stat_activity(); @@ -35,6 +40,9 @@ GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; -- we have to recreate this view because recreated citus_dist_stat_activity that this view depends #include "udfs/citus_lock_waits/11.0-1.sql" +#include "udfs/pg_cancel_backend/11.0-1.sql" +#include "udfs/pg_terminate_backend/11.0-1.sql" + DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text); DROP FUNCTION pg_catalog.master_get_table_metadata(text); DROP FUNCTION pg_catalog.master_append_table_to_shard(bigint, text, text, integer); diff --git a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql index 204548c51..ba13b134a 100644 --- a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql +++ b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql @@ -113,6 +113,9 @@ CREATE FUNCTION get_global_active_transactions(OUT datid oid, OUT process_id int RESET search_path; +DROP FUNCTION citus_internal_local_blocked_processes CASCADE; +DROP FUNCTION citus_internal_global_blocked_processes CASCADE; + DROP FUNCTION pg_catalog.citus_dist_stat_activity CASCADE; CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, @@ -139,6 +142,162 @@ ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; SET search_path = 'pg_catalog'; +DROP FUNCTION citus_worker_stat_activity CASCADE; + +CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, + OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, + OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, + OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, + OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, + OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text) +RETURNS SETOF RECORD +LANGUAGE C STRICT AS 'MODULE_PATHNAME', +$$citus_worker_stat_activity$$; + +COMMENT ON FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, + OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, + OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, + OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, + OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, + OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text) +IS 'returns distributed transaction activity on shards of distributed tables'; + +DROP FUNCTION pg_catalog.worker_create_or_replace_object(text[]); +#include "../udfs/worker_create_or_replace_object/9.0-1.sql" + +DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(bigint) CASCADE; +DROP FUNCTION IF EXISTS pg_catalog.pg_terminate_backend(bigint, bigint) CASCADE; + +DROP FUNCTION pg_catalog.dump_local_wait_edges CASCADE; +CREATE FUNCTION pg_catalog.dump_local_wait_edges( + OUT waiting_pid int4, + OUT waiting_node_id int4, + OUT waiting_transaction_num int8, + OUT waiting_transaction_stamp timestamptz, + OUT blocking_pid int4, + OUT blocking_node_id int4, + OUT blocking_transaction_num int8, + OUT blocking_transaction_stamp timestamptz, + OUT blocking_transaction_waiting bool) +RETURNS SETOF RECORD +LANGUAGE C STRICT +AS $$MODULE_PATHNAME$$, $$dump_local_wait_edges$$; +COMMENT ON FUNCTION pg_catalog.dump_local_wait_edges() +IS 'returns all local lock wait chains, that start from distributed transactions'; + +DROP FUNCTION pg_catalog.dump_global_wait_edges CASCADE; +CREATE FUNCTION pg_catalog.dump_global_wait_edges( + OUT waiting_pid int4, + OUT waiting_node_id int4, + OUT waiting_transaction_num int8, + OUT waiting_transaction_stamp timestamptz, + OUT blocking_pid int4, + OUT blocking_node_id int4, + OUT blocking_transaction_num int8, + OUT blocking_transaction_stamp timestamptz, + OUT blocking_transaction_waiting bool) +RETURNS SETOF RECORD +LANGUAGE 'c' STRICT +AS $$MODULE_PATHNAME$$, $$dump_global_wait_edges$$; +COMMENT ON FUNCTION pg_catalog.dump_global_wait_edges() +IS 'returns a global list of blocked transactions originating from this node'; + +DROP FUNCTION pg_catalog.citus_isolation_test_session_is_blocked(pBlockedPid integer, pInterestingPids integer[]); +CREATE FUNCTION pg_catalog.citus_isolation_test_session_is_blocked(pBlockedPid integer, pInterestingPids integer[]) +RETURNS boolean AS $$ + DECLARE + mBlockedTransactionNum int8; + workerProcessId integer := current_setting('citus.isolation_test_session_remote_process_id'); + coordinatorProcessId integer := current_setting('citus.isolation_test_session_process_id'); + BEGIN + IF pg_catalog.old_pg_isolation_test_session_is_blocked(pBlockedPid, pInterestingPids) THEN + RETURN true; + END IF; + + -- pg says we're not blocked locally; check whether we're blocked globally. + -- Note that worker process may be blocked or waiting for a lock. So we need to + -- get transaction number for both of them. Following IF provides the transaction + -- number when the worker process waiting for other session. + IF EXISTS (SELECT transaction_number FROM get_global_active_transactions() + WHERE process_id = workerProcessId AND pBlockedPid = coordinatorProcessId) THEN + SELECT transaction_number INTO mBlockedTransactionNum FROM get_global_active_transactions() + WHERE process_id = workerProcessId AND pBlockedPid = coordinatorProcessId; + ELSE + -- Check whether transactions initiated from the coordinator get locked + SELECT transaction_number INTO mBlockedTransactionNum + FROM get_all_active_transactions() WHERE process_id = pBlockedPid; + END IF; + + RETURN EXISTS ( + SELECT 1 FROM dump_global_wait_edges() + WHERE waiting_transaction_num = mBlockedTransactionNum + ) OR EXISTS ( + -- Check on the workers if any logical replication job spawned by the + -- current PID is blocked, by checking it's application name + -- Query is heavily based on: https://wiki.postgresql.org/wiki/Lock_Monitoring + SELECT result FROM run_command_on_workers($two$ + SELECT blocked_activity.application_name AS blocked_application + FROM pg_catalog.pg_locks blocked_locks + JOIN pg_catalog.pg_stat_activity blocked_activity ON blocked_activity.pid = blocked_locks.pid + JOIN pg_catalog.pg_locks blocking_locks + ON blocking_locks.locktype = blocked_locks.locktype + AND blocking_locks.DATABASE IS NOT DISTINCT FROM blocked_locks.DATABASE + AND blocking_locks.relation IS NOT DISTINCT FROM blocked_locks.relation + AND blocking_locks.page IS NOT DISTINCT FROM blocked_locks.page + AND blocking_locks.tuple IS NOT DISTINCT FROM blocked_locks.tuple + AND blocking_locks.virtualxid IS NOT DISTINCT FROM blocked_locks.virtualxid + AND blocking_locks.transactionid IS NOT DISTINCT FROM blocked_locks.transactionid + AND blocking_locks.classid IS NOT DISTINCT FROM blocked_locks.classid + AND blocking_locks.objid IS NOT DISTINCT FROM blocked_locks.objid + AND blocking_locks.objsubid IS NOT DISTINCT FROM blocked_locks.objsubid + AND blocking_locks.pid != blocked_locks.pid + JOIN pg_catalog.pg_stat_activity blocking_activity ON blocking_activity.pid = blocking_locks.pid + WHERE NOT blocked_locks.GRANTED AND blocked_activity.application_name LIKE 'citus_shard_move_subscription_%' + $two$) where result='citus_shard_move_subscription_' || pBlockedPid); + + END; +$$ LANGUAGE plpgsql; + +REVOKE ALL ON FUNCTION citus_isolation_test_session_is_blocked(integer,integer[]) FROM PUBLIC; + +DROP FUNCTION pg_catalog.citus_blocking_pids(pBlockedPid integer); +CREATE FUNCTION pg_catalog.citus_blocking_pids(pBlockedPid integer) +RETURNS int4[] AS $$ + DECLARE + mLocalBlockingPids int4[]; + mRemoteBlockingPids int4[]; + mLocalTransactionNum int8; + BEGIN + SELECT pg_catalog.old_pg_blocking_pids(pBlockedPid) INTO mLocalBlockingPids; + + IF (array_length(mLocalBlockingPids, 1) > 0) THEN + RETURN mLocalBlockingPids; + END IF; + + -- pg says we're not blocked locally; check whether we're blocked globally. + SELECT transaction_number INTO mLocalTransactionNum + FROM get_all_active_transactions() WHERE process_id = pBlockedPid; + + SELECT array_agg(process_id) INTO mRemoteBlockingPids FROM ( + WITH activeTransactions AS ( + SELECT process_id, transaction_number FROM get_all_active_transactions() + ), blockingTransactions AS ( + SELECT blocking_transaction_num AS txn_num FROM dump_global_wait_edges() + WHERE waiting_transaction_num = mLocalTransactionNum + ) + SELECT activeTransactions.process_id FROM activeTransactions, blockingTransactions + WHERE activeTransactions.transaction_number = blockingTransactions.txn_num + ) AS sub; + + RETURN mRemoteBlockingPids; + END; +$$ LANGUAGE plpgsql; +REVOKE ALL ON FUNCTION citus_blocking_pids(integer) FROM PUBLIC; + +CREATE VIEW citus.citus_worker_stat_activity AS +SELECT * FROM pg_catalog.citus_worker_stat_activity(); +ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC; -- we have to recreate this view because we drop citus_dist_stat_activity that this view depends CREATE VIEW citus.citus_lock_waits AS @@ -150,7 +309,7 @@ citus_dist_stat_activity AS ), unique_global_wait_edges AS ( - SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges() + SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges() ), citus_dist_stat_activity_with_node_id AS ( @@ -183,32 +342,4 @@ JOIN ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; -DROP FUNCTION citus_worker_stat_activity CASCADE; - -CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text) -RETURNS SETOF RECORD -LANGUAGE C STRICT AS 'MODULE_PATHNAME', -$$citus_worker_stat_activity$$; - -COMMENT ON FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text) -IS 'returns distributed transaction activity on shards of distributed tables'; - -CREATE VIEW citus.citus_worker_stat_activity AS -SELECT * FROM pg_catalog.citus_worker_stat_activity(); -ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog; -GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC; - -DROP FUNCTION pg_catalog.worker_create_or_replace_object(text[]); -#include "../udfs/worker_create_or_replace_object/9.0-1.sql" - RESET search_path; diff --git a/src/backend/distributed/sql/udfs/citus_blocking_pids/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_blocking_pids/11.0-1.sql new file mode 100644 index 000000000..c7e607c1c --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_blocking_pids/11.0-1.sql @@ -0,0 +1,34 @@ +DROP FUNCTION pg_catalog.citus_blocking_pids; +CREATE FUNCTION pg_catalog.citus_blocking_pids(pBlockedPid integer) +RETURNS int4[] AS $$ + DECLARE + mLocalBlockingPids int4[]; + mRemoteBlockingPids int4[]; + mLocalGlobalPid int8; + BEGIN + SELECT pg_catalog.old_pg_blocking_pids(pBlockedPid) INTO mLocalBlockingPids; + + IF (array_length(mLocalBlockingPids, 1) > 0) THEN + RETURN mLocalBlockingPids; + END IF; + + -- pg says we're not blocked locally; check whether we're blocked globally. + SELECT global_pid INTO mLocalGlobalPid + FROM get_all_active_transactions() WHERE process_id = pBlockedPid; + + SELECT array_agg(global_pid) INTO mRemoteBlockingPids FROM ( + WITH activeTransactions AS ( + SELECT global_pid FROM get_all_active_transactions() + ), blockingTransactions AS ( + SELECT blocking_global_pid FROM citus_internal_global_blocked_processes() + WHERE waiting_global_pid = mLocalGlobalPid + ) + SELECT activeTransactions.global_pid FROM activeTransactions, blockingTransactions + WHERE activeTransactions.global_pid = blockingTransactions.blocking_global_pid + ) AS sub; + + RETURN mRemoteBlockingPids; + END; +$$ LANGUAGE plpgsql; + +REVOKE ALL ON FUNCTION citus_blocking_pids(integer) FROM PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_blocking_pids/latest.sql b/src/backend/distributed/sql/udfs/citus_blocking_pids/latest.sql new file mode 100644 index 000000000..c7e607c1c --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_blocking_pids/latest.sql @@ -0,0 +1,34 @@ +DROP FUNCTION pg_catalog.citus_blocking_pids; +CREATE FUNCTION pg_catalog.citus_blocking_pids(pBlockedPid integer) +RETURNS int4[] AS $$ + DECLARE + mLocalBlockingPids int4[]; + mRemoteBlockingPids int4[]; + mLocalGlobalPid int8; + BEGIN + SELECT pg_catalog.old_pg_blocking_pids(pBlockedPid) INTO mLocalBlockingPids; + + IF (array_length(mLocalBlockingPids, 1) > 0) THEN + RETURN mLocalBlockingPids; + END IF; + + -- pg says we're not blocked locally; check whether we're blocked globally. + SELECT global_pid INTO mLocalGlobalPid + FROM get_all_active_transactions() WHERE process_id = pBlockedPid; + + SELECT array_agg(global_pid) INTO mRemoteBlockingPids FROM ( + WITH activeTransactions AS ( + SELECT global_pid FROM get_all_active_transactions() + ), blockingTransactions AS ( + SELECT blocking_global_pid FROM citus_internal_global_blocked_processes() + WHERE waiting_global_pid = mLocalGlobalPid + ) + SELECT activeTransactions.global_pid FROM activeTransactions, blockingTransactions + WHERE activeTransactions.global_pid = blockingTransactions.blocking_global_pid + ) AS sub; + + RETURN mRemoteBlockingPids; + END; +$$ LANGUAGE plpgsql; + +REVOKE ALL ON FUNCTION citus_blocking_pids(integer) FROM PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_internal_global_blocked_processes/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_internal_global_blocked_processes/11.0-1.sql new file mode 100644 index 000000000..510cdf93d --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_global_blocked_processes/11.0-1.sql @@ -0,0 +1,17 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_global_blocked_processes( + OUT waiting_global_pid int8, + OUT waiting_pid int4, + OUT waiting_node_id int4, + OUT waiting_transaction_num int8, + OUT waiting_transaction_stamp timestamptz, + OUT blocking_global_pid int8, + OUT blocking_pid int4, + OUT blocking_node_id int4, + OUT blocking_transaction_num int8, + OUT blocking_transaction_stamp timestamptz, + OUT blocking_transaction_waiting bool) +RETURNS SETOF RECORD +LANGUAGE C STRICT +AS $$MODULE_PATHNAME$$, $$citus_internal_global_blocked_processes$$; +COMMENT ON FUNCTION pg_catalog.citus_internal_global_blocked_processes() +IS 'returns a global list of blocked backends originating from this node'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_global_blocked_processes/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_global_blocked_processes/latest.sql new file mode 100644 index 000000000..510cdf93d --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_global_blocked_processes/latest.sql @@ -0,0 +1,17 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_global_blocked_processes( + OUT waiting_global_pid int8, + OUT waiting_pid int4, + OUT waiting_node_id int4, + OUT waiting_transaction_num int8, + OUT waiting_transaction_stamp timestamptz, + OUT blocking_global_pid int8, + OUT blocking_pid int4, + OUT blocking_node_id int4, + OUT blocking_transaction_num int8, + OUT blocking_transaction_stamp timestamptz, + OUT blocking_transaction_waiting bool) +RETURNS SETOF RECORD +LANGUAGE C STRICT +AS $$MODULE_PATHNAME$$, $$citus_internal_global_blocked_processes$$; +COMMENT ON FUNCTION pg_catalog.citus_internal_global_blocked_processes() +IS 'returns a global list of blocked backends originating from this node'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_local_blocked_processes/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_internal_local_blocked_processes/11.0-1.sql new file mode 100644 index 000000000..3157a9aad --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_local_blocked_processes/11.0-1.sql @@ -0,0 +1,17 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_local_blocked_processes( + OUT waiting_global_pid int8, + OUT waiting_pid int4, + OUT waiting_node_id int4, + OUT waiting_transaction_num int8, + OUT waiting_transaction_stamp timestamptz, + OUT blocking_global_pid int8, + OUT blocking_pid int4, + OUT blocking_node_id int4, + OUT blocking_transaction_num int8, + OUT blocking_transaction_stamp timestamptz, + OUT blocking_transaction_waiting bool) +RETURNS SETOF RECORD +LANGUAGE C STRICT +AS $$MODULE_PATHNAME$$, $$citus_internal_local_blocked_processes$$; +COMMENT ON FUNCTION pg_catalog.citus_internal_local_blocked_processes() +IS 'returns all local lock wait chains, that start from any citus backend'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_local_blocked_processes/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_local_blocked_processes/latest.sql new file mode 100644 index 000000000..3157a9aad --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_local_blocked_processes/latest.sql @@ -0,0 +1,17 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_local_blocked_processes( + OUT waiting_global_pid int8, + OUT waiting_pid int4, + OUT waiting_node_id int4, + OUT waiting_transaction_num int8, + OUT waiting_transaction_stamp timestamptz, + OUT blocking_global_pid int8, + OUT blocking_pid int4, + OUT blocking_node_id int4, + OUT blocking_transaction_num int8, + OUT blocking_transaction_stamp timestamptz, + OUT blocking_transaction_waiting bool) +RETURNS SETOF RECORD +LANGUAGE C STRICT +AS $$MODULE_PATHNAME$$, $$citus_internal_local_blocked_processes$$; +COMMENT ON FUNCTION pg_catalog.citus_internal_local_blocked_processes() +IS 'returns all local lock wait chains, that start from any citus backend'; diff --git a/src/backend/distributed/sql/udfs/citus_isolation_test_session_is_blocked/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_isolation_test_session_is_blocked/11.0-1.sql new file mode 100644 index 000000000..64b89ec0e --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_isolation_test_session_is_blocked/11.0-1.sql @@ -0,0 +1,56 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_isolation_test_session_is_blocked(pBlockedPid integer, pInterestingPids integer[]) +RETURNS boolean AS $$ + DECLARE + mBlockedGlobalPid int8; + workerProcessId integer := current_setting('citus.isolation_test_session_remote_process_id'); + coordinatorProcessId integer := current_setting('citus.isolation_test_session_process_id'); + BEGIN + IF pg_catalog.old_pg_isolation_test_session_is_blocked(pBlockedPid, pInterestingPids) THEN + RETURN true; + END IF; + + -- pg says we're not blocked locally; check whether we're blocked globally. + -- Note that worker process may be blocked or waiting for a lock. So we need to + -- get transaction number for both of them. Following IF provides the transaction + -- number when the worker process waiting for other session. + IF EXISTS (SELECT 1 FROM get_global_active_transactions() + WHERE process_id = workerProcessId AND pBlockedPid = coordinatorProcessId) THEN + SELECT global_pid INTO mBlockedGlobalPid FROM get_global_active_transactions() + WHERE process_id = workerProcessId AND pBlockedPid = coordinatorProcessId; + ELSE + -- Check whether transactions initiated from the coordinator get locked + SELECT global_pid INTO mBlockedGlobalPid + FROM get_all_active_transactions() WHERE process_id = pBlockedPid; + END IF; + + RETURN EXISTS ( + SELECT 1 FROM citus_internal_global_blocked_processes() + WHERE waiting_global_pid = mBlockedGlobalPid + ) OR EXISTS ( + -- Check on the workers if any logical replication job spawned by the + -- current PID is blocked, by checking it's application name + -- Query is heavily based on: https://wiki.postgresql.org/wiki/Lock_Monitoring + SELECT result FROM run_command_on_workers($two$ + SELECT blocked_activity.application_name AS blocked_application + FROM pg_catalog.pg_locks blocked_locks + JOIN pg_catalog.pg_stat_activity blocked_activity ON blocked_activity.pid = blocked_locks.pid + JOIN pg_catalog.pg_locks blocking_locks + ON blocking_locks.locktype = blocked_locks.locktype + AND blocking_locks.DATABASE IS NOT DISTINCT FROM blocked_locks.DATABASE + AND blocking_locks.relation IS NOT DISTINCT FROM blocked_locks.relation + AND blocking_locks.page IS NOT DISTINCT FROM blocked_locks.page + AND blocking_locks.tuple IS NOT DISTINCT FROM blocked_locks.tuple + AND blocking_locks.virtualxid IS NOT DISTINCT FROM blocked_locks.virtualxid + AND blocking_locks.transactionid IS NOT DISTINCT FROM blocked_locks.transactionid + AND blocking_locks.classid IS NOT DISTINCT FROM blocked_locks.classid + AND blocking_locks.objid IS NOT DISTINCT FROM blocked_locks.objid + AND blocking_locks.objsubid IS NOT DISTINCT FROM blocked_locks.objsubid + AND blocking_locks.pid != blocked_locks.pid + JOIN pg_catalog.pg_stat_activity blocking_activity ON blocking_activity.pid = blocking_locks.pid + WHERE NOT blocked_locks.GRANTED AND blocked_activity.application_name LIKE 'citus_shard_move_subscription_%' + $two$) where result='citus_shard_move_subscription_' || pBlockedPid); + + END; +$$ LANGUAGE plpgsql; + +REVOKE ALL ON FUNCTION citus_isolation_test_session_is_blocked(integer,integer[]) FROM PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_isolation_test_session_is_blocked/latest.sql b/src/backend/distributed/sql/udfs/citus_isolation_test_session_is_blocked/latest.sql index 0b91cc37c..64b89ec0e 100644 --- a/src/backend/distributed/sql/udfs/citus_isolation_test_session_is_blocked/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_isolation_test_session_is_blocked/latest.sql @@ -1,7 +1,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.citus_isolation_test_session_is_blocked(pBlockedPid integer, pInterestingPids integer[]) RETURNS boolean AS $$ DECLARE - mBlockedTransactionNum int8; + mBlockedGlobalPid int8; workerProcessId integer := current_setting('citus.isolation_test_session_remote_process_id'); coordinatorProcessId integer := current_setting('citus.isolation_test_session_process_id'); BEGIN @@ -13,19 +13,19 @@ RETURNS boolean AS $$ -- Note that worker process may be blocked or waiting for a lock. So we need to -- get transaction number for both of them. Following IF provides the transaction -- number when the worker process waiting for other session. - IF EXISTS (SELECT transaction_number FROM get_global_active_transactions() + IF EXISTS (SELECT 1 FROM get_global_active_transactions() WHERE process_id = workerProcessId AND pBlockedPid = coordinatorProcessId) THEN - SELECT transaction_number INTO mBlockedTransactionNum FROM get_global_active_transactions() + SELECT global_pid INTO mBlockedGlobalPid FROM get_global_active_transactions() WHERE process_id = workerProcessId AND pBlockedPid = coordinatorProcessId; ELSE -- Check whether transactions initiated from the coordinator get locked - SELECT transaction_number INTO mBlockedTransactionNum + SELECT global_pid INTO mBlockedGlobalPid FROM get_all_active_transactions() WHERE process_id = pBlockedPid; END IF; RETURN EXISTS ( - SELECT 1 FROM dump_global_wait_edges() - WHERE waiting_transaction_num = mBlockedTransactionNum + SELECT 1 FROM citus_internal_global_blocked_processes() + WHERE waiting_global_pid = mBlockedGlobalPid ) OR EXISTS ( -- Check on the workers if any logical replication job spawned by the -- current PID is blocked, by checking it's application name diff --git a/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql index 2ae40374a..779341657 100644 --- a/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql +++ b/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql @@ -8,7 +8,7 @@ citus_dist_stat_activity AS ), unique_global_wait_edges AS ( - SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges() + SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() ), citus_dist_stat_activity_with_node_id AS ( @@ -21,6 +21,8 @@ citus_dist_stat_activity_with_node_id AS citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport ) SELECT + waiting.global_pid as waiting_gpid, + blocking.global_pid as blocking_gpid, waiting.pid AS waiting_pid, blocking.pid AS blocking_pid, waiting.query AS blocked_statement, @@ -34,9 +36,9 @@ SELECT FROM unique_global_wait_edges JOIN - citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id) + citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) JOIN - citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id); + citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql b/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql index 2ae40374a..779341657 100644 --- a/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql @@ -8,7 +8,7 @@ citus_dist_stat_activity AS ), unique_global_wait_edges AS ( - SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges() + SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() ), citus_dist_stat_activity_with_node_id AS ( @@ -21,6 +21,8 @@ citus_dist_stat_activity_with_node_id AS citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport ) SELECT + waiting.global_pid as waiting_gpid, + blocking.global_pid as blocking_gpid, waiting.pid AS waiting_pid, blocking.pid AS blocking_pid, waiting.query AS blocked_statement, @@ -34,9 +36,9 @@ SELECT FROM unique_global_wait_edges JOIN - citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id) + citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) JOIN - citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id); + citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/pg_cancel_backend/11.0-1.sql b/src/backend/distributed/sql/udfs/pg_cancel_backend/11.0-1.sql new file mode 100644 index 000000000..3a355aa6d --- /dev/null +++ b/src/backend/distributed/sql/udfs/pg_cancel_backend/11.0-1.sql @@ -0,0 +1,9 @@ +DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(global_pid bigint) CASCADE; + +CREATE OR REPLACE FUNCTION pg_catalog.pg_cancel_backend(global_pid bigint) + RETURNS BOOL + LANGUAGE C +AS 'MODULE_PATHNAME', $$pg_cancel_backend$$; + +COMMENT ON FUNCTION pg_catalog.pg_cancel_backend(global_pid bigint) + IS 'cancels a Citus query which might be on any node in the Citus cluster'; diff --git a/src/backend/distributed/sql/udfs/pg_cancel_backend/latest.sql b/src/backend/distributed/sql/udfs/pg_cancel_backend/latest.sql new file mode 100644 index 000000000..3a355aa6d --- /dev/null +++ b/src/backend/distributed/sql/udfs/pg_cancel_backend/latest.sql @@ -0,0 +1,9 @@ +DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(global_pid bigint) CASCADE; + +CREATE OR REPLACE FUNCTION pg_catalog.pg_cancel_backend(global_pid bigint) + RETURNS BOOL + LANGUAGE C +AS 'MODULE_PATHNAME', $$pg_cancel_backend$$; + +COMMENT ON FUNCTION pg_catalog.pg_cancel_backend(global_pid bigint) + IS 'cancels a Citus query which might be on any node in the Citus cluster'; diff --git a/src/backend/distributed/sql/udfs/pg_terminate_backend/11.0-1.sql b/src/backend/distributed/sql/udfs/pg_terminate_backend/11.0-1.sql new file mode 100644 index 000000000..9b9798b76 --- /dev/null +++ b/src/backend/distributed/sql/udfs/pg_terminate_backend/11.0-1.sql @@ -0,0 +1,9 @@ +DROP FUNCTION IF EXISTS pg_catalog.pg_terminate_backend(global_pid bigint, timeout bigint) CASCADE; + +CREATE OR REPLACE FUNCTION pg_catalog.pg_terminate_backend(global_pid bigint, timeout bigint DEFAULT 0) + RETURNS BOOL + LANGUAGE C +AS 'MODULE_PATHNAME', $$pg_terminate_backend$$; + +COMMENT ON FUNCTION pg_catalog.pg_terminate_backend(global_pid bigint, timeout bigint) + IS 'terminates a Citus query which might be on any node in the Citus cluster'; diff --git a/src/backend/distributed/sql/udfs/pg_terminate_backend/latest.sql b/src/backend/distributed/sql/udfs/pg_terminate_backend/latest.sql new file mode 100644 index 000000000..9b9798b76 --- /dev/null +++ b/src/backend/distributed/sql/udfs/pg_terminate_backend/latest.sql @@ -0,0 +1,9 @@ +DROP FUNCTION IF EXISTS pg_catalog.pg_terminate_backend(global_pid bigint, timeout bigint) CASCADE; + +CREATE OR REPLACE FUNCTION pg_catalog.pg_terminate_backend(global_pid bigint, timeout bigint DEFAULT 0) + RETURNS BOOL + LANGUAGE C +AS 'MODULE_PATHNAME', $$pg_terminate_backend$$; + +COMMENT ON FUNCTION pg_catalog.pg_terminate_backend(global_pid bigint, timeout bigint) + IS 'terminates a Citus query which might be on any node in the Citus cluster'; diff --git a/src/backend/distributed/test/distributed_deadlock_detection.c b/src/backend/distributed/test/distributed_deadlock_detection.c index 448228158..d3fa34db2 100644 --- a/src/backend/distributed/test/distributed_deadlock_detection.c +++ b/src/backend/distributed/test/distributed_deadlock_detection.c @@ -50,7 +50,10 @@ get_adjacency_list_wait_graph(PG_FUNCTION_ARGS) bool isNulls[2]; Tuplestorestate *tupleStore = SetupTuplestore(fcinfo, &tupleDescriptor); - WaitGraph *waitGraph = BuildGlobalWaitGraph(); + + /* distributed deadlock detection only considers distributed txs */ + bool onlyDistributedTx = true; + WaitGraph *waitGraph = BuildGlobalWaitGraph(onlyDistributedTx); HTAB *adjacencyList = BuildAdjacencyListsForWaitGraph(waitGraph); /* iterate on all nodes */ diff --git a/src/backend/distributed/test/run_from_same_connection.c b/src/backend/distributed/test/run_from_same_connection.c index 3b5f804b4..ad39664bf 100644 --- a/src/backend/distributed/test/run_from_same_connection.c +++ b/src/backend/distributed/test/run_from_same_connection.c @@ -58,6 +58,7 @@ static int64 GetRemoteProcessId(void); PG_FUNCTION_INFO_V1(start_session_level_connection_to_node); PG_FUNCTION_INFO_V1(run_commands_on_session_level_connection_to_node); PG_FUNCTION_INFO_V1(stop_session_level_connection_to_node); +PG_FUNCTION_INFO_V1(override_backend_data_command_originator); /* @@ -119,6 +120,17 @@ start_session_level_connection_to_node(PG_FUNCTION_ARGS) ExecuteCriticalRemoteCommand(singleConnection, setAppName); + /* + * We are hackily overriding the remote processes' worker_query to be false + * such that relevant observibility UDFs work fine. + */ + StringInfo overrideBackendDataCommandOriginator = makeStringInfo(); + appendStringInfo(overrideBackendDataCommandOriginator, + "SELECT override_backend_data_command_originator(true);"); + ExecuteCriticalRemoteCommand(singleConnection, + overrideBackendDataCommandOriginator->data); + + PG_RETURN_VOID(); } @@ -174,6 +186,23 @@ run_commands_on_session_level_connection_to_node(PG_FUNCTION_ARGS) } +/* + * override_backend_data_command_originator is a wrapper around + * OverrideBackendDataDistributedCommandOriginator(). + */ +Datum +override_backend_data_command_originator(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + bool distributedCommandOriginator = PG_GETARG_BOOL(0); + + OverrideBackendDataDistributedCommandOriginator(distributedCommandOriginator); + + PG_RETURN_VOID(); +} + + /* * stop_session_level_connection_to_node closes the connection opened by the * start_session_level_connection_to_node and set the flag to false which diff --git a/src/backend/distributed/transaction/backend_data.c b/src/backend/distributed/transaction/backend_data.c index bc54e1da9..5ca517199 100644 --- a/src/backend/distributed/transaction/backend_data.c +++ b/src/backend/distributed/transaction/backend_data.c @@ -33,6 +33,7 @@ #include "distributed/shared_connection_stats.h" #include "distributed/transaction_identifier.h" #include "distributed/tuplestore.h" +#include "distributed/worker_manager.h" #include "nodes/execnodes.h" #include "postmaster/autovacuum.h" /* to access autovacuum_max_workers */ #include "replication/walsender.h" @@ -47,6 +48,7 @@ #define GET_ACTIVE_TRANSACTION_QUERY "SELECT * FROM get_all_active_transactions();" #define ACTIVE_TRANSACTION_COLUMN_COUNT 7 +#define GLOBAL_PID_NODE_ID_MULTIPLIER 10000000000 /* * Each backend's data reside in the shared memory @@ -152,7 +154,6 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS) MyBackendData->citusBackend.initiatorNodeIdentifier = MyBackendData->transactionId.initiatorNodeIdentifier; - MyBackendData->citusBackend.transactionOriginator = false; SpinLockRelease(&MyBackendData->mutex); @@ -410,15 +411,12 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto initiatorNodeIdentifier = currentBackend->citusBackend.initiatorNodeIdentifier; /* - * We prefer to use worker_query instead of transactionOriginator in the user facing - * functions since its more intuitive. Thus, we negate the result before returning. - * - * We prefer to use citusBackend's transactionOriginator field over transactionId's - * field with the same name. The reason is that it also covers backends that are not - * inside a distributed transaction. + * We prefer to use worker_query instead of distributedCommandOriginator in + * the user facing functions since its more intuitive. Thus, + * we negate the result before returning. */ - bool coordinatorOriginatedQuery = - currentBackend->citusBackend.transactionOriginator; + bool distributedCommandOriginator = + currentBackend->distributedCommandOriginator; transactionNumber = currentBackend->transactionId.transactionNumber; TimestampTz transactionIdTimestamp = currentBackend->transactionId.timestamp; @@ -428,7 +426,7 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto values[0] = ObjectIdGetDatum(databaseId); values[1] = Int32GetDatum(backendPid); values[2] = Int32GetDatum(initiatorNodeIdentifier); - values[3] = !coordinatorOriginatedQuery; + values[3] = !distributedCommandOriginator; values[4] = UInt64GetDatum(transactionNumber); values[5] = TimestampTzGetDatum(transactionIdTimestamp); values[6] = UInt64GetDatum(currentBackend->globalPID); @@ -663,7 +661,6 @@ UnSetDistributedTransactionId(void) MyBackendData->transactionId.timestamp = 0; MyBackendData->citusBackend.initiatorNodeIdentifier = -1; - MyBackendData->citusBackend.transactionOriginator = false; SpinLockRelease(&MyBackendData->mutex); } @@ -776,7 +773,6 @@ AssignDistributedTransactionId(void) MyBackendData->transactionId.timestamp = currentTimestamp; MyBackendData->citusBackend.initiatorNodeIdentifier = localGroupId; - MyBackendData->citusBackend.transactionOriginator = true; SpinLockRelease(&MyBackendData->mutex); } @@ -798,7 +794,6 @@ MarkCitusInitiatedCoordinatorBackend(void) SpinLockAcquire(&MyBackendData->mutex); MyBackendData->citusBackend.initiatorNodeIdentifier = localGroupId; - MyBackendData->citusBackend.transactionOriginator = true; SpinLockRelease(&MyBackendData->mutex); } @@ -814,10 +809,12 @@ void AssignGlobalPID(void) { uint64 globalPID = INVALID_CITUS_INTERNAL_BACKEND_GPID; + bool distributedCommandOriginator = false; if (!IsCitusInternalBackend()) { globalPID = GenerateGlobalPID(); + distributedCommandOriginator = true; } else { @@ -826,6 +823,21 @@ AssignGlobalPID(void) SpinLockAcquire(&MyBackendData->mutex); MyBackendData->globalPID = globalPID; + MyBackendData->distributedCommandOriginator = distributedCommandOriginator; + SpinLockRelease(&MyBackendData->mutex); +} + + +/* + * OverrideBackendDataDistributedCommandOriginator should only be used for isolation testing. + * See how it is used in the relevant functions. + */ +void +OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator) +{ + SpinLockAcquire(&MyBackendData->mutex); + MyBackendData->distributedCommandOriginator = + distributedCommandOriginator; SpinLockRelease(&MyBackendData->mutex); } @@ -864,7 +876,7 @@ GenerateGlobalPID(void) * node ids might cause overflow. But even for the applications that scale around 50 nodes every * day it'd take about 100K years. So we are not worried. */ - return (((uint64) GetLocalNodeId()) * 10000000000) + getpid(); + return (((uint64) GetLocalNodeId()) * GLOBAL_PID_NODE_ID_MULTIPLIER) + getpid(); } @@ -907,6 +919,42 @@ ExtractGlobalPID(char *applicationName) } +/* + * ExtractNodeIdFromGlobalPID extracts the node id from the global pid. + * Global pid is constructed by multiplying node id with GLOBAL_PID_NODE_ID_MULTIPLIER + * and adding process id. So integer division of global pid by GLOBAL_PID_NODE_ID_MULTIPLIER + * gives us the node id. + */ +int +ExtractNodeIdFromGlobalPID(uint64 globalPID) +{ + int nodeId = (int) (globalPID / GLOBAL_PID_NODE_ID_MULTIPLIER); + + if (nodeId == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) + { + ereport(ERROR, (errmsg("originator node of the query with the global pid " + "%lu is not in Citus' metadata", globalPID), + errhint("connect to the node directly run pg_cancel_backend(pid) " + "or pg_terminate_backend(pid)"))); + } + + return nodeId; +} + + +/* + * ExtractProcessIdFromGlobalPID extracts the process id from the global pid. + * Global pid is constructed by multiplying node id with GLOBAL_PID_NODE_ID_MULTIPLIER + * and adding process id. So global pid mod GLOBAL_PID_NODE_ID_MULTIPLIER gives us the + * process id. + */ +int +ExtractProcessIdFromGlobalPID(uint64 globalPID) +{ + return (int) (globalPID % GLOBAL_PID_NODE_ID_MULTIPLIER); +} + + /* * CurrentDistributedTransactionNumber returns the transaction number of the * current distributed transaction. The caller must make sure a distributed diff --git a/src/backend/distributed/transaction/citus_dist_stat_activity.c b/src/backend/distributed/transaction/citus_dist_stat_activity.c index 3fb15295f..ec4f6e8a3 100644 --- a/src/backend/distributed/transaction/citus_dist_stat_activity.c +++ b/src/backend/distributed/transaction/citus_dist_stat_activity.c @@ -157,10 +157,9 @@ FROM \ WHERE \ backend_type = 'client backend' \ AND \ - pg_stat_activity.query NOT ILIKE '%stat_activity%' \ + worker_query = False \ AND \ - pg_stat_activity.application_name NOT SIMILAR TO 'citus_internal gpid=\\d+'; \ -" + pg_stat_activity.query NOT ILIKE '%stat_activity%';" #define CITUS_WORKER_STAT_ACTIVITY_QUERY \ "\ @@ -195,7 +194,7 @@ FROM \ get_all_active_transactions() AS dist_txs(database_id, process_id, initiator_node_identifier, worker_query, transaction_number, transaction_stamp, global_id) \ ON pg_stat_activity.pid = dist_txs.process_id \ WHERE \ - pg_stat_activity.application_name SIMILAR TO 'citus_internal gpid=\\d+' \ + worker_query = True \ AND \ pg_stat_activity.query NOT ILIKE '%stat_activity%';" diff --git a/src/backend/distributed/transaction/distributed_deadlock_detection.c b/src/backend/distributed/transaction/distributed_deadlock_detection.c index f9e4adca0..82c274661 100644 --- a/src/backend/distributed/transaction/distributed_deadlock_detection.c +++ b/src/backend/distributed/transaction/distributed_deadlock_detection.c @@ -119,7 +119,9 @@ CheckForDistributedDeadlocks(void) return false; } - WaitGraph *waitGraph = BuildGlobalWaitGraph(); + /* distributed deadlock detection only considers distributed txs */ + bool onlyDistributedTx = true; + WaitGraph *waitGraph = BuildGlobalWaitGraph(onlyDistributedTx); HTAB *adjacencyLists = BuildAdjacencyListsForWaitGraph(waitGraph); int edgeCount = waitGraph->edgeCount; diff --git a/src/backend/distributed/transaction/lock_graph.c b/src/backend/distributed/transaction/lock_graph.c index aa37e4371..62b5e4e04 100644 --- a/src/backend/distributed/transaction/lock_graph.c +++ b/src/backend/distributed/transaction/lock_graph.c @@ -47,7 +47,10 @@ typedef struct PROCStack static void AddWaitEdgeFromResult(WaitGraph *waitGraph, PGresult *result, int rowIndex); static void ReturnWaitGraph(WaitGraph *waitGraph, FunctionCallInfo fcinfo); -static WaitGraph * BuildLocalWaitGraph(void); +static void AddWaitEdgeFromBlockedProcessResult(WaitGraph *waitGraph, PGresult *result, + int rowIndex); +static void ReturnBlockedProcessGraph(WaitGraph *waitGraph, FunctionCallInfo fcinfo); +static WaitGraph * BuildLocalWaitGraph(bool onlyDistributedTx); static bool IsProcessWaitingForSafeOperations(PGPROC *proc); static void LockLockData(void); static void UnlockLockData(void); @@ -62,10 +65,30 @@ static void AddProcToVisit(PROCStack *remaining, PGPROC *proc); static bool IsSameLockGroup(PGPROC *leftProc, PGPROC *rightProc); static bool IsConflictingLockMask(int holdMask, int conflictMask); - +/* + * We almost have 2 sets of identical functions. The first set (e.g., dump_wait_edges) + * functions are intended for distributed deadlock detection purposes. + * + * The second set of functions (e.g., citus_internal_local_blocked_processes) are + * intended for citus_lock_waits view. + * + * The main difference is that the former functions only show processes that are blocked + * inside a distributed transaction (e.g., see AssignDistributedTransactionId()). + * The latter functions return a superset, where any blocked process is returned. + * + * We kept two different set of functions for two purposes. First, the deadlock detection + * is a performance critical code-path happening very frequently and we don't add any + * performance overhead. Secondly, to be able to do rolling upgrades, we cannot change + * the API of dump_global_wait_edges/dump_local_wait_edges such that they take a boolean + * parameter. If we do that, until all nodes are upgraded, the deadlock detection would fail, + * which is not acceptable. + */ PG_FUNCTION_INFO_V1(dump_local_wait_edges); PG_FUNCTION_INFO_V1(dump_global_wait_edges); +PG_FUNCTION_INFO_V1(citus_internal_local_blocked_processes); +PG_FUNCTION_INFO_V1(citus_internal_global_blocked_processes); + /* * dump_global_wait_edges returns global wait edges for distributed transactions @@ -74,7 +97,9 @@ PG_FUNCTION_INFO_V1(dump_global_wait_edges); Datum dump_global_wait_edges(PG_FUNCTION_ARGS) { - WaitGraph *waitGraph = BuildGlobalWaitGraph(); + bool onlyDistributedTx = true; + + WaitGraph *waitGraph = BuildGlobalWaitGraph(onlyDistributedTx); ReturnWaitGraph(waitGraph, fcinfo); @@ -82,20 +107,44 @@ dump_global_wait_edges(PG_FUNCTION_ARGS) } +/* + * citus_internal_global_blocked_processes returns global wait edges + * including all processes running on the cluster. + */ +Datum +citus_internal_global_blocked_processes(PG_FUNCTION_ARGS) +{ + bool onlyDistributedTx = false; + + WaitGraph *waitGraph = BuildGlobalWaitGraph(onlyDistributedTx); + + ReturnBlockedProcessGraph(waitGraph, fcinfo); + + return (Datum) 0; +} + + /* * BuildGlobalWaitGraph builds a wait graph for distributed transactions * that originate from this node, including edges from all (other) worker * nodes. + * + * + * If onlyDistributedTx is true, we only return distributed transactions + * (e.g., AssignDistributedTransaction() or assign_distributed_transactions()) + * has been called for the process. Distributed deadlock detection only + * interested in these processes. */ WaitGraph * -BuildGlobalWaitGraph(void) +BuildGlobalWaitGraph(bool onlyDistributedTx) { List *workerNodeList = ActiveReadableNodeList(); char *nodeUser = CitusExtensionOwnerName(); List *connectionList = NIL; int32 localGroupId = GetLocalGroupId(); - WaitGraph *waitGraph = BuildLocalWaitGraph(); + /* deadlock detection is only interested in distributed transactions */ + WaitGraph *waitGraph = BuildLocalWaitGraph(onlyDistributedTx); /* open connections in parallel */ WorkerNode *workerNode = NULL; @@ -124,9 +173,28 @@ BuildGlobalWaitGraph(void) MultiConnection *connection = NULL; foreach_ptr(connection, connectionList) { - const char *command = "SELECT * FROM dump_local_wait_edges()"; + StringInfo queryString = makeStringInfo(); - int querySent = SendRemoteCommand(connection, command); + if (onlyDistributedTx) + { + appendStringInfo(queryString, + "SELECT waiting_pid, waiting_node_id, " + "waiting_transaction_num, waiting_transaction_stamp, " + "blocking_pid, blocking_node_id, blocking_transaction_num, " + "blocking_transaction_stamp, blocking_transaction_waiting " + "FROM dump_local_wait_edges()"); + } + else + { + appendStringInfo(queryString, + "SELECT waiting_global_pid, waiting_pid, " + "waiting_node_id, waiting_transaction_num, waiting_transaction_stamp, " + "blocking_global_pid,blocking_pid, blocking_node_id, " + "blocking_transaction_num, blocking_transaction_stamp, blocking_transaction_waiting " + "FROM citus_internal_local_blocked_processes()"); + } + + int querySent = SendRemoteCommand(connection, queryString->data); if (querySent == 0) { ReportConnectionError(connection, WARNING); @@ -148,16 +216,29 @@ BuildGlobalWaitGraph(void) int64 rowCount = PQntuples(result); int64 colCount = PQnfields(result); - if (colCount != 9) + if (onlyDistributedTx && colCount != 9) { ereport(WARNING, (errmsg("unexpected number of columns from " "dump_local_wait_edges"))); continue; } + else if (!onlyDistributedTx && colCount != 11) + { + ereport(WARNING, (errmsg("unexpected number of columns from " + "citus_internal_local_blocked_processes"))); + continue; + } for (int64 rowIndex = 0; rowIndex < rowCount; rowIndex++) { - AddWaitEdgeFromResult(waitGraph, result, rowIndex); + if (onlyDistributedTx) + { + AddWaitEdgeFromResult(waitGraph, result, rowIndex); + } + else + { + AddWaitEdgeFromBlockedProcessResult(waitGraph, result, rowIndex); + } } PQclear(result); @@ -177,10 +258,12 @@ AddWaitEdgeFromResult(WaitGraph *waitGraph, PGresult *result, int rowIndex) { WaitEdge *waitEdge = AllocWaitEdge(waitGraph); + waitEdge->waitingGPid = 0; /* not requested for deadlock detection */ waitEdge->waitingPid = ParseIntField(result, rowIndex, 0); waitEdge->waitingNodeId = ParseIntField(result, rowIndex, 1); waitEdge->waitingTransactionNum = ParseIntField(result, rowIndex, 2); waitEdge->waitingTransactionStamp = ParseTimestampTzField(result, rowIndex, 3); + waitEdge->blockingGPid = 0; /* not requested for deadlock detection */ waitEdge->blockingPid = ParseIntField(result, rowIndex, 4); waitEdge->blockingNodeId = ParseIntField(result, rowIndex, 5); waitEdge->blockingTransactionNum = ParseIntField(result, rowIndex, 6); @@ -189,6 +272,29 @@ AddWaitEdgeFromResult(WaitGraph *waitGraph, PGresult *result, int rowIndex) } +/* + * AddWaitEdgeFromBlockedProcessResult adds an edge to the wait graph that + * is read from a PGresult. + */ +static void +AddWaitEdgeFromBlockedProcessResult(WaitGraph *waitGraph, PGresult *result, int rowIndex) +{ + WaitEdge *waitEdge = AllocWaitEdge(waitGraph); + + waitEdge->waitingGPid = ParseIntField(result, rowIndex, 0); + waitEdge->waitingPid = ParseIntField(result, rowIndex, 1); + waitEdge->waitingNodeId = ParseIntField(result, rowIndex, 2); + waitEdge->waitingTransactionNum = ParseIntField(result, rowIndex, 3); + waitEdge->waitingTransactionStamp = ParseTimestampTzField(result, rowIndex, 4); + waitEdge->blockingGPid = ParseIntField(result, rowIndex, 5); + waitEdge->blockingPid = ParseIntField(result, rowIndex, 6); + waitEdge->blockingNodeId = ParseIntField(result, rowIndex, 7); + waitEdge->blockingTransactionNum = ParseIntField(result, rowIndex, 8); + waitEdge->blockingTransactionStamp = ParseTimestampTzField(result, rowIndex, 9); + waitEdge->isBlockingXactWaiting = ParseBoolField(result, rowIndex, 10); +} + + /* * ParseIntField parses a int64 from a remote result or returns 0 if the * result is NULL. @@ -256,13 +362,31 @@ ParseTimestampTzField(PGresult *result, int rowIndex, int colIndex) Datum dump_local_wait_edges(PG_FUNCTION_ARGS) { - WaitGraph *waitGraph = BuildLocalWaitGraph(); + bool onlyDistributedTx = true; + + WaitGraph *waitGraph = BuildLocalWaitGraph(onlyDistributedTx); ReturnWaitGraph(waitGraph, fcinfo); return (Datum) 0; } +/* + * citus_internal_local_blocked_processes returns global wait edges + * including all processes running on the node. + */ +Datum +citus_internal_local_blocked_processes(PG_FUNCTION_ARGS) +{ + bool onlyDistributedTx = false; + + WaitGraph *waitGraph = BuildLocalWaitGraph(onlyDistributedTx); + ReturnBlockedProcessGraph(waitGraph, fcinfo); + + return (Datum) 0; +} + + /* * ReturnWaitGraph returns a wait graph for a set returning function. */ @@ -325,12 +449,83 @@ ReturnWaitGraph(WaitGraph *waitGraph, FunctionCallInfo fcinfo) } +/* + * ReturnBlockedProcessGraph returns a wait graph for a set returning function. + */ +static void +ReturnBlockedProcessGraph(WaitGraph *waitGraph, FunctionCallInfo fcinfo) +{ + TupleDesc tupleDesc; + Tuplestorestate *tupleStore = SetupTuplestore(fcinfo, &tupleDesc); + + /* + * Columns: + * 00: waiting_global_pid + * 01: waiting_pid + * 02: waiting_node_id + * 03: waiting_transaction_num + * 04: waiting_transaction_stamp + * 05: blocking_global_pid + * 06: blocking_pid + * 07: blocking__node_id + * 08: blocking_transaction_num + * 09: blocking_transaction_stamp + * 10: blocking_transaction_waiting + */ + for (size_t curEdgeNum = 0; curEdgeNum < waitGraph->edgeCount; curEdgeNum++) + { + Datum values[11]; + bool nulls[11]; + WaitEdge *curEdge = &waitGraph->edges[curEdgeNum]; + + memset(values, 0, sizeof(values)); + memset(nulls, 0, sizeof(nulls)); + + values[0] = UInt64GetDatum(curEdge->waitingGPid); + values[1] = Int32GetDatum(curEdge->waitingPid); + values[2] = Int32GetDatum(curEdge->waitingNodeId); + if (curEdge->waitingTransactionNum != 0) + { + values[3] = Int64GetDatum(curEdge->waitingTransactionNum); + values[4] = TimestampTzGetDatum(curEdge->waitingTransactionStamp); + } + else + { + nulls[3] = true; + nulls[4] = true; + } + + values[5] = UInt64GetDatum(curEdge->blockingGPid); + values[6] = Int32GetDatum(curEdge->blockingPid); + values[7] = Int32GetDatum(curEdge->blockingNodeId); + if (curEdge->blockingTransactionNum != 0) + { + values[8] = Int64GetDatum(curEdge->blockingTransactionNum); + values[9] = TimestampTzGetDatum(curEdge->blockingTransactionStamp); + } + else + { + nulls[8] = true; + nulls[9] = true; + } + values[10] = BoolGetDatum(curEdge->isBlockingXactWaiting); + + tuplestore_putvalues(tupleStore, tupleDesc, values, nulls); + } +} + + /* * BuildLocalWaitGraph builds a wait graph for distributed transactions * that originate from the local node. + * + * If onlyDistributedTx is true, we only return distributed transactions + * (e.g., AssignDistributedTransaction() or assign_distributed_transactions()) + * has been called for the process. Distributed deadlock detection only + * interested in these processes. */ static WaitGraph * -BuildLocalWaitGraph(void) +BuildLocalWaitGraph(bool onlyDistributedTx) { PROCStack remaining; int totalProcs = TotalProcCount(); @@ -379,7 +574,8 @@ BuildLocalWaitGraph(void) * care about distributed transactions for the purpose of distributed * deadlock detection. */ - if (!IsInDistributedTransaction(¤tBackendData)) + if (onlyDistributedTx && + !IsInDistributedTransaction(¤tBackendData)) { continue; } @@ -627,6 +823,7 @@ AddWaitEdge(WaitGraph *waitGraph, PGPROC *waitingProc, PGPROC *blockingProc, } curEdge->waitingPid = waitingProc->pid; + curEdge->waitingGPid = waitingBackendData.globalPID; if (IsInDistributedTransaction(&waitingBackendData)) { @@ -645,6 +842,7 @@ AddWaitEdge(WaitGraph *waitGraph, PGPROC *waitingProc, PGPROC *blockingProc, } curEdge->blockingPid = blockingProc->pid; + curEdge->blockingGPid = blockingBackendData.globalPID; if (IsInDistributedTransaction(&blockingBackendData)) { diff --git a/src/backend/distributed/utils/distribution_column.c b/src/backend/distributed/utils/distribution_column.c index a4dd647bd..5b59bd529 100644 --- a/src/backend/distributed/utils/distribution_column.c +++ b/src/backend/distributed/utils/distribution_column.c @@ -56,15 +56,12 @@ column_name_to_column(PG_FUNCTION_ARGS) text *columnText = PG_GETARG_TEXT_P(1); char *columnName = text_to_cstring(columnText); - Relation relation = relation_open(relationId, AccessShareLock); - - Var *column = BuildDistributionKeyFromColumnName(relation, columnName); + Var *column = BuildDistributionKeyFromColumnName(relationId, columnName, + AccessShareLock); Assert(column != NULL); char *columnNodeString = nodeToString(column); text *columnNodeText = cstring_to_text(columnNodeString); - relation_close(relation, AccessShareLock); - PG_RETURN_TEXT_P(columnNodeText); } @@ -81,13 +78,10 @@ column_name_to_column_id(PG_FUNCTION_ARGS) Oid distributedTableId = PG_GETARG_OID(0); char *columnName = PG_GETARG_CSTRING(1); - Relation relation = relation_open(distributedTableId, AccessExclusiveLock); - - Var *column = BuildDistributionKeyFromColumnName(relation, columnName); + Var *column = BuildDistributionKeyFromColumnName(distributedTableId, columnName, + AccessExclusiveLock); Assert(column != NULL); - relation_close(relation, NoLock); - PG_RETURN_INT16((int16) column->varattno); } @@ -107,8 +101,9 @@ column_to_column_name(PG_FUNCTION_ARGS) text *columnNodeText = PG_GETARG_TEXT_P(1); char *columnNodeString = text_to_cstring(columnNodeText); + Node *columnNode = stringToNode(columnNodeString); - char *columnName = ColumnToColumnName(relationId, columnNodeString); + char *columnName = ColumnToColumnName(relationId, columnNode); text *columnText = cstring_to_text(columnName); @@ -116,53 +111,6 @@ column_to_column_name(PG_FUNCTION_ARGS) } -/* - * FindColumnWithNameOnTargetRelation gets a source table and - * column name. The function returns the the column with the - * same name on the target table. - * - * Note that due to dropping columns, the parent's distribution key may not - * match the partition's distribution key. See issue #5123. - * - * The function throws error if the input or output is not valid or does - * not exist. - */ -Var * -FindColumnWithNameOnTargetRelation(Oid sourceRelationId, char *sourceColumnName, - Oid targetRelationId) -{ - if (sourceColumnName == NULL || sourceColumnName[0] == '\0') - { - ereport(ERROR, (errcode(ERRCODE_UNDEFINED_COLUMN), - errmsg("cannot find the given column on table \"%s\"", - generate_qualified_relation_name(sourceRelationId)))); - } - - AttrNumber attributeNumberOnTarget = get_attnum(targetRelationId, sourceColumnName); - if (attributeNumberOnTarget == InvalidAttrNumber) - { - ereport(ERROR, (errmsg("Column \"%s\" does not exist on " - "relation \"%s\"", sourceColumnName, - get_rel_name(targetRelationId)))); - } - - Index varNo = 1; - Oid targetTypeId = InvalidOid; - int32 targetTypMod = 0; - Oid targetCollation = InvalidOid; - Index varlevelsup = 0; - - /* this function throws error in case anything goes wrong */ - get_atttypetypmodcoll(targetRelationId, attributeNumberOnTarget, - &targetTypeId, &targetTypMod, &targetCollation); - Var *targetColumn = - makeVar(varNo, attributeNumberOnTarget, targetTypeId, targetTypMod, - targetCollation, varlevelsup); - - return targetColumn; -} - - /* * BuildDistributionKeyFromColumnName builds a simple distribution key consisting * only out of a reference to the column of name columnName. Errors out if the @@ -173,9 +121,18 @@ FindColumnWithNameOnTargetRelation(Oid sourceRelationId, char *sourceColumnName, * corresponds to reference tables. */ Var * -BuildDistributionKeyFromColumnName(Relation distributedRelation, char *columnName) +BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lockMode) { - char *tableName = RelationGetRelationName(distributedRelation); + Relation relation = try_relation_open(relationId, ExclusiveLock); + + if (relation == NULL) + { + ereport(ERROR, (errmsg("relation does not exist"))); + } + + relation_close(relation, NoLock); + + char *tableName = get_rel_name(relationId); /* short circuit for reference tables */ if (columnName == NULL) @@ -187,8 +144,7 @@ BuildDistributionKeyFromColumnName(Relation distributedRelation, char *columnNam truncate_identifier(columnName, strlen(columnName), true); /* lookup column definition */ - HeapTuple columnTuple = SearchSysCacheAttName(RelationGetRelid(distributedRelation), - columnName); + HeapTuple columnTuple = SearchSysCacheAttName(relationId, columnName); if (!HeapTupleIsValid(columnTuple)) { ereport(ERROR, (errcode(ERRCODE_UNDEFINED_COLUMN), @@ -218,15 +174,13 @@ BuildDistributionKeyFromColumnName(Relation distributedRelation, char *columnNam /* * ColumnToColumnName returns the human-readable name of a column given a - * relation identifier and the column's internal textual (Var) representation. + * relation identifier and the column's internal (Var) representation. * This function will raise an ERROR if no such column can be found or if the * provided Var refers to a system column. */ char * -ColumnToColumnName(Oid relationId, char *columnNodeString) +ColumnToColumnName(Oid relationId, Node *columnNode) { - Node *columnNode = stringToNode(columnNodeString); - if (columnNode == NULL || !IsA(columnNode, Var)) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), diff --git a/src/include/distributed/backend_data.h b/src/include/distributed/backend_data.h index 7f3a81e88..b463b89f5 100644 --- a/src/include/distributed/backend_data.h +++ b/src/include/distributed/backend_data.h @@ -29,7 +29,6 @@ typedef struct CitusInitiatedBackend { int initiatorNodeIdentifier; - bool transactionOriginator; } CitusInitiatedBackend; @@ -51,6 +50,7 @@ typedef struct BackendData slock_t mutex; bool cancelledDueToDeadlock; uint64 globalPID; + bool distributedCommandOriginator; CitusInitiatedBackend citusBackend; DistributedTransactionId transactionId; } BackendData; @@ -67,7 +67,11 @@ extern void AssignDistributedTransactionId(void); extern void MarkCitusInitiatedCoordinatorBackend(void); extern void AssignGlobalPID(void); extern uint64 GetGlobalPID(void); +extern void OverrideBackendDataDistributedCommandOriginator(bool + distributedCommandOriginator); extern uint64 ExtractGlobalPID(char *applicationName); +extern int ExtractNodeIdFromGlobalPID(uint64 globalPID); +extern int ExtractProcessIdFromGlobalPID(uint64 globalPID); extern void GetBackendDataForProc(PGPROC *proc, BackendData *result); extern void CancelTransactionDueToDeadlock(PGPROC *proc); extern bool MyBackendGotCancelledDueToDeadlock(bool clearState); @@ -78,6 +82,11 @@ extern int GetAllActiveClientBackendCount(void); extern void IncrementClientBackendCounter(void); extern void DecrementClientBackendCounter(void); +extern bool ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort, + char *queryString, StringInfo queryResultString, + bool reportResultError); + #define INVALID_CITUS_INTERNAL_BACKEND_GPID 0 +#define GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA 99999999 #endif /* BACKEND_DATA_H */ diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index 1b2b92590..c2bf66d5b 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -26,6 +26,9 @@ extern bool AddAllLocalTablesToMetadata; /* controlled via GUC, should be accessed via EnableLocalReferenceForeignKeys() */ extern bool EnableLocalReferenceForeignKeys; +extern bool EnableUnsafeTriggers; + + extern void SwitchToSequentialAndLocalExecutionIfRelationNameTooLong(Oid relationId, char * finalRelationName); @@ -580,16 +583,19 @@ extern List * PostprocessAlterTriggerRenameStmt(Node *node, const char *queryStr extern void AlterTriggerRenameEventExtendNames(RenameStmt *renameTriggerStmt, char *schemaName, uint64 shardId); extern List * PostprocessAlterTriggerDependsStmt(Node *node, const char *queryString); +extern List * PreprocessAlterTriggerDependsStmt(Node *node, const char *queryString, + ProcessUtilityContext + processUtilityContext); extern void AlterTriggerDependsEventExtendNames( AlterObjectDependsStmt *alterTriggerDependsStmt, char *schemaName, uint64 shardId); +extern void ErrorOutForTriggerIfNotSupported(Oid relationId); extern List * PreprocessDropTriggerStmt(Node *node, const char *queryString, ProcessUtilityContext processUtilityContext); -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 List * CitusCreateTriggerCommandDDLJob(Oid relationId, char *triggerName, + const char *queryString); extern Oid GetTriggerFunctionId(Oid triggerId); /* cascade_table_operation_for_connected_relations.c */ diff --git a/src/include/distributed/distribution_column.h b/src/include/distributed/distribution_column.h index 0bad0fe52..ced1be9a7 100644 --- a/src/include/distributed/distribution_column.h +++ b/src/include/distributed/distribution_column.h @@ -19,11 +19,9 @@ /* Remaining metadata utility functions */ -extern Var * FindColumnWithNameOnTargetRelation(Oid sourceRelationId, - char *sourceColumnName, - Oid targetRelationId); -extern Var * BuildDistributionKeyFromColumnName(Relation distributedRelation, - char *columnName); -extern char * ColumnToColumnName(Oid relationId, char *columnNodeString); +extern Var * BuildDistributionKeyFromColumnName(Oid relationId, + char *columnName, + LOCKMODE lockMode); +extern char * ColumnToColumnName(Oid relationId, Node *columnNode); #endif /* DISTRIBUTION_COLUMN_H */ diff --git a/src/include/distributed/lock_graph.h b/src/include/distributed/lock_graph.h index 46fec1dee..f204ebb03 100644 --- a/src/include/distributed/lock_graph.h +++ b/src/include/distributed/lock_graph.h @@ -31,11 +31,13 @@ */ typedef struct WaitEdge { + uint64 waitingGPid; int waitingPid; int waitingNodeId; int64 waitingTransactionNum; TimestampTz waitingTransactionStamp; + uint64 blockingGPid; int blockingPid; int blockingNodeId; int64 blockingTransactionNum; @@ -58,7 +60,7 @@ typedef struct WaitGraph } WaitGraph; -extern WaitGraph * BuildGlobalWaitGraph(void); +extern WaitGraph * BuildGlobalWaitGraph(bool onlyDistributedTx); extern bool IsProcessWaitingForLock(PGPROC *proc); extern bool IsInDistributedTransaction(BackendData *backendData); extern TimestampTz ParseTimestampTzField(PGresult *result, int rowIndex, int colIndex); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index d1db0f2fe..3e7a3b6f3 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -238,7 +238,7 @@ extern void DeleteShardRow(uint64 shardId); extern void UpdateShardPlacementState(uint64 placementId, char shardState); extern void UpdatePlacementGroupId(uint64 placementId, int groupId); extern void DeleteShardPlacementRow(uint64 placementId); -extern void CreateDistributedTable(Oid relationId, Var *distributionColumn, +extern void CreateDistributedTable(Oid relationId, char *distributionColumnName, char distributionMethod, int shardCount, bool shardCountIsStrict, char *colocateWithTableName, bool viaDeprecatedAPI); diff --git a/src/include/distributed/worker_manager.h b/src/include/distributed/worker_manager.h index 0a6b637b3..db8adaedb 100644 --- a/src/include/distributed/worker_manager.h +++ b/src/include/distributed/worker_manager.h @@ -86,6 +86,7 @@ extern List * ActiveReadableNodeList(void); extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNodeOrError(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort); +extern WorkerNode * FindNodeWithNodeId(int nodeId); extern List * ReadDistNode(bool includeNodesFromOtherClusters); extern void EnsureCoordinator(void); extern void InsertCoordinatorIfClusterEmpty(void); diff --git a/src/test/regress/bin/normalize.sed b/src/test/regress/bin/normalize.sed index 25f4388e1..916d62afa 100644 --- a/src/test/regress/bin/normalize.sed +++ b/src/test/regress/bin/normalize.sed @@ -256,3 +256,7 @@ s/CREATE TABLESPACE test_tablespace LOCATION.*/CREATE TABLESPACE test_tablespace s/(.*absolute correlation \()([0,1]\.[0-9]+)(\) of var attribute [0-9]+ is smaller than.*)/\1X\.YZ\3/g s/NOTICE: issuing WITH placement_data\(shardid, shardstate, shardlength, groupid, placementid\) AS \(VALUES \([0-9]+, [0-9]+, [0-9]+, [0-9]+, [0-9]+\)\)/NOTICE: issuing WITH placement_data\(shardid, shardstate, shardlength, groupid, placementid\) AS \(VALUES \(xxxxxx, xxxxxx, xxxxxx, xxxxxx, xxxxxx\)\)/g + +# global_pid when pg_cancel_backend is sent to workers +s/pg_cancel_backend\('[0-9]+'::bigint\)/pg_cancel_backend('xxxxx'::bigint)/g +s/issuing SELECT pg_cancel_backend\([0-9]+::integer\)/issuing SELECT pg_cancel_backend(xxxxx::integer)/g diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index f3091b359..52b0714c8 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -161,6 +161,7 @@ class CitusDefaultClusterConfig(CitusBaseClusterConfig): new_settings = { "client_min_messages": "WARNING", "citus.sort_returning": True, + "citus.use_citus_managed_tables": True, } self.settings.update(new_settings) self.add_coordinator_to_metadata = True @@ -291,6 +292,7 @@ class CitusUnusualQuerySettingsConfig(CitusMXBaseClusterConfig): def __init__(self, arguments): super().__init__(arguments) self.new_settings = { + "citus.use_citus_managed_tables": False, "citus.task_assignment_policy": "first-replica", "citus.enable_fast_path_router_planner": False, "citus.enable_local_execution": False, diff --git a/src/test/regress/expected/citus_local_table_triggers.out b/src/test/regress/expected/citus_local_table_triggers.out index 007986fab..fa89b78d4 100644 --- a/src/test/regress/expected/citus_local_table_triggers.out +++ b/src/test/regress/expected/citus_local_table_triggers.out @@ -194,29 +194,29 @@ NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1 BEFORE INSERT ON "interesting!schema"."citus_local!_table" FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();') CREATE EXTENSION seg; +-- ALTER TRIGGER DEPENDS ON +ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg; +ERROR: Triggers "trigger\'name" on distributed tables and local tables added to metadata are not allowed to depend on an extension BEGIN; - -- 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 (1507008, '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 + -- show that triggers on both shell relation and shard relation are not 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 + tgname --------------------------------------------------------------------- - trigger\'name - trigger\'name_1507008 -(2 rows) +(0 rows) DROP EXTENSION seg; - -- show that dropping extension drops the triggers automatically + -- show that dropping extension doesn't drop the triggers automatically SELECT * FROM citus_local_table_triggers WHERE tgname NOT LIKE 'RI_ConstraintTrigger%'; - tgname | tgrelid | tgenabled + tgname | tgrelid | tgenabled --------------------------------------------------------------------- - truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O -(1 row) + trigger\'name | "interesting!schema"."citus_local!_table" | O + trigger\'name_1507008 | "interesting!schema"."citus_local!_table_1507008" | O + truncate_trigger_xxxxxxx | "interesting!schema"."citus_local!_table" | O +(3 rows) ROLLBACK; -- ALTER TRIGGER RENAME diff --git a/src/test/regress/expected/citus_local_tables_mx.out b/src/test/regress/expected/citus_local_tables_mx.out index 8ea410829..0a50232ba 100644 --- a/src/test/regress/expected/citus_local_tables_mx.out +++ b/src/test/regress/expected/citus_local_tables_mx.out @@ -51,25 +51,22 @@ SELECT 1 FROM citus_activate_node('localhost', :worker_1_port); 1 (1 row) +-- show that the trigger is not allowed to depend(explicitly) on an extension. 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;') +ERROR: Triggers "dummy_function_trigger" on distributed tables and local tables added to metadata are not allowed to depend on an extension 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. +-- show that update trigger mx relation is renamed and disabled. -- both workers should should print 1. SELECT run_command_on_workers( $$ -SELECT COUNT(*) FROM pg_depend, pg_trigger, pg_extension +SELECT COUNT(*) FROM pg_trigger 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' + pg_trigger.tgenabled='D' $$); run_command_on_workers --------------------------------------------------------------------- @@ -724,6 +721,58 @@ $$); (localhost,57638,t,0) (2 rows) +-- verify that partitioned citus local tables with dropped columns can be distributed. issue: #5577 +CREATE TABLE parent_dropped_col(a int, eventtime date) PARTITION BY RANGE ( eventtime); +SELECT citus_add_local_table_to_metadata('parent_dropped_col'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE parent_dropped_col DROP column a; +CREATE TABLE parent_dropped_col_1 PARTITION OF parent_dropped_col for VALUES FROM ('2000-01-01') TO ('2001-01-01'); +SELECT create_distributed_table('parent_dropped_col', 'eventtime'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- another example to test +CREATE TABLE parent_dropped_col_2( + col_to_drop_0 text, + col_to_drop_1 text, + col_to_drop_2 date, + col_to_drop_3 inet, + col_to_drop_4 date, + measureid integer, + eventdatetime date, + measure_data jsonb, + PRIMARY KEY (measureid, eventdatetime, measure_data)) + PARTITION BY RANGE(eventdatetime); +select citus_add_local_table_to_metadata('parent_dropped_col_2'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE parent_dropped_col_2 DROP COLUMN col_to_drop_1; +CREATE TABLE parent_dropped_col_2_2000 PARTITION OF parent_dropped_col_2 FOR VALUES FROM ('2000-01-01') TO ('2001-01-01'); +SELECT create_distributed_table('parent_dropped_col_2', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- verify that the partitioned tables are distributed with the correct distribution column +SELECT logicalrelid, partmethod, partkey FROM pg_dist_partition + WHERE logicalrelid IN ('parent_dropped_col'::regclass, 'parent_dropped_col_2'::regclass) + ORDER BY logicalrelid; + logicalrelid | partmethod | partkey +--------------------------------------------------------------------- + parent_dropped_col | h | {VAR :varno 1 :varattno 1 :vartype 1082 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} + parent_dropped_col_2 | h | {VAR :varno 1 :varattno 5 :vartype 23 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 5 :location -1} +(2 rows) + -- cleanup at exit set client_min_messages to error; DROP SCHEMA citus_local_tables_mx CASCADE; diff --git a/src/test/regress/expected/citus_table_triggers.out b/src/test/regress/expected/citus_table_triggers.out index f9252c166..e5d14b69e 100644 --- a/src/test/regress/expected/citus_table_triggers.out +++ b/src/test/regress/expected/citus_table_triggers.out @@ -34,11 +34,11 @@ SELECT create_reference_table('reference_table'); CREATE TRIGGER update_value_dist AFTER INSERT ON distributed_table FOR EACH ROW EXECUTE FUNCTION update_value(); -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" CREATE TRIGGER update_value_ref AFTER INSERT ON reference_table FOR EACH ROW EXECUTE FUNCTION update_value(); -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables --------------------------------------------------------------------- -- show that we error out for trigger commands on distributed & reference tables --------------------------------------------------------------------- @@ -56,42 +56,42 @@ SET citus.enable_ddl_propagation to ON; 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 local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" ALTER TRIGGER update_value_dist ON distributed_table DEPENDS ON EXTENSION seg; -ERROR: triggers are only supported for local tables added to metadata +ERROR: Triggers "update_value_dist" on distributed tables and local tables added to metadata are not allowed to depend on an extension DROP TRIGGER update_value_dist ON distributed_table; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" ALTER TABLE distributed_table DISABLE TRIGGER ALL; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" ALTER TABLE distributed_table DISABLE TRIGGER USER; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" ALTER TABLE distributed_table DISABLE TRIGGER update_value_dist; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" ALTER TABLE distributed_table ENABLE TRIGGER ALL; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" ALTER TABLE distributed_table ENABLE TRIGGER USER; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" ALTER TABLE distributed_table ENABLE TRIGGER update_value_dist; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" -- below all should error out ALTER TRIGGER update_value_ref ON reference_table RENAME TO update_value_ref1; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables ALTER TRIGGER update_value_ref ON reference_table DEPENDS ON EXTENSION seg; -ERROR: triggers are only supported for local tables added to metadata +ERROR: Triggers "update_value_ref" on distributed tables and local tables added to metadata are not allowed to depend on an extension DROP TRIGGER update_value_ref ON reference_table; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables ALTER TABLE reference_table DISABLE TRIGGER ALL; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables ALTER TABLE reference_table DISABLE TRIGGER USER; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables ALTER TABLE reference_table DISABLE TRIGGER update_value_ref; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables ALTER TABLE reference_table ENABLE TRIGGER ALL; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables ALTER TABLE reference_table ENABLE TRIGGER USER; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables ALTER TABLE reference_table ENABLE TRIGGER update_value_ref; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on reference tables --------------------------------------------------------------------- -- show that we do not allow creating citus tables if the -- table has already triggers @@ -106,9 +106,9 @@ AFTER INSERT ON reference_table_1 FOR EACH ROW EXECUTE FUNCTION update_value(); -- below two should fail SELECT create_distributed_table('distributed_table_1', 'value'); -ERROR: cannot distribute relation "distributed_table_1" because it has triggers +ERROR: cannot distribute relation "distributed_table_1" because it has triggers and "citus.enable_unsafe_triggers" is set to "false" SELECT create_reference_table('reference_table_1'); -ERROR: cannot distribute relation "reference_table_1" because it has triggers +ERROR: cannot distribute relation "reference_table_1" because it has triggers and "citus.enable_unsafe_triggers" is set to "false" --------------------------------------------------------------------- -- test deparse logic for CREATE TRIGGER commands -- via master_get_table_ddl_events diff --git a/src/test/regress/expected/create_ref_dist_from_citus_local.out b/src/test/regress/expected/create_ref_dist_from_citus_local.out index 7f6821b1f..14fffed22 100644 --- a/src/test/regress/expected/create_ref_dist_from_citus_local.out +++ b/src/test/regress/expected/create_ref_dist_from_citus_local.out @@ -346,7 +346,7 @@ BEGIN; FOR EACH ROW EXECUTE PROCEDURE update_value(); -- show that we error out as we don't supprt triggers on distributed tables SELECT create_distributed_table('citus_local_table_6', 'col_1'); -ERROR: cannot distribute relation "citus_local_table_6" because it has triggers +ERROR: cannot distribute relation "citus_local_table_6" because it has triggers and "citus.enable_unsafe_triggers" is set to "false" ROLLBACK; -- make sure that creating append / range distributed tables is also ok BEGIN; diff --git a/src/test/regress/expected/distributed_triggers.out b/src/test/regress/expected/distributed_triggers.out new file mode 100644 index 000000000..ac038be51 --- /dev/null +++ b/src/test/regress/expected/distributed_triggers.out @@ -0,0 +1,921 @@ +SET citus.log_remote_commands TO OFF; +DROP SCHEMA IF EXISTS distributed_triggers CASCADE; +NOTICE: schema "distributed_triggers" does not exist, skipping +CREATE SCHEMA distributed_triggers; +SET search_path TO 'distributed_triggers'; +SET citus.shard_replication_factor = 1; +SET citus.shard_count = 32; +SET citus.next_shard_id TO 800000; +-- +-- Test citus.enable_unsafe_triggers +-- Enables arbitrary triggers on distributed tables +-- +CREATE TABLE data ( + shard_key_value text not null, + object_id text not null, + value jsonb not null +); +ALTER TABLE data +ADD CONSTRAINT data_pk +PRIMARY KEY (shard_key_value, object_id); +/* table of changes */ +CREATE TABLE data_changes ( + shard_key_value text not null, + object_id text not null, + change_id bigint not null, + change_time timestamptz default now(), + operation_type text not null, + new_value jsonb +); +ALTER TABLE data_changes +ADD CONSTRAINT data_changes_pk +PRIMARY KEY (shard_key_value, object_id, change_id); +SELECT create_distributed_table('data', 'shard_key_value'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('data_changes', 'shard_key_value', colocate_with := 'data'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET citus.enable_unsafe_triggers TO true; +SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO true;'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"ALTER SYSTEM") + (localhost,57638,t,"ALTER SYSTEM") +(2 rows) + +SELECT run_command_on_workers('SELECT pg_reload_conf();'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,t) + (localhost,57638,t,t) +(2 rows) + +/* trigger function that is called after any change */ +CREATE OR REPLACE FUNCTION record_change() +RETURNS trigger +AS $$ +DECLARE + last_change_id bigint; +BEGIN + IF (TG_OP = 'DELETE') THEN + /* get the last change ID for object key in OLD via index(-only) scan */ + SELECT change_id INTO last_change_id + FROM distributed_triggers.data_changes + WHERE shard_key_value = OLD.shard_key_value AND object_id = OLD.object_id + ORDER BY change_id DESC LIMIT 1; + + /* insert a change record for the delete */ + INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type) + VALUES (OLD.shard_key_value, OLD.object_id, COALESCE(last_change_id + 1, 1), TG_OP); + ELSE + /* get the last change ID for object key in NEW via index(-only) scan */ + SELECT change_id INTO last_change_id + FROM distributed_triggers.data_changes + WHERE shard_key_value = NEW.shard_key_value AND object_id = NEW.object_id + ORDER BY change_id DESC LIMIT 1; + + /* insert a change record for the insert/update */ + INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type, new_value) + VALUES (NEW.shard_key_value, NEW.object_id, COALESCE(last_change_id + 1, 1), TG_OP, NEW.value); + END IF; + + RETURN NULL; +END; +$$ LANGUAGE plpgsql; +SELECT proname from pg_proc WHERE oid='distributed_triggers.record_change'::regproc; + proname +--------------------------------------------------------------------- + record_change +(1 row) + +SELECT run_command_on_workers($$SELECT count(*) FROM pg_proc WHERE oid='distributed_triggers.record_change'::regproc$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +CREATE TRIGGER record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.record_change(); +-- Trigger function should appear on workers +SELECT proname from pg_proc WHERE oid='distributed_triggers.record_change'::regproc; + proname +--------------------------------------------------------------------- + record_change +(1 row) + +SELECT run_command_on_workers($$SELECT count(*) FROM pg_proc WHERE oid='distributed_triggers.record_change'::regproc$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +INSERT INTO data VALUES ('hello','world','{"hello":"world"}'); +INSERT INTO data VALUES ('hello2','world2','{"hello2":"world2"}'); +INSERT INTO data VALUES ('hello3','world3','{"hello3":"world3"}'); +DELETE FROM data where shard_key_value = 'hello'; +BEGIN; +UPDATE data SET value = '{}' where shard_key_value = 'hello3'; +END; +DELETE FROM data where shard_key_value = 'hello3'; +SELECT * FROM data +ORDER BY shard_key_value, object_id; + shard_key_value | object_id | value +--------------------------------------------------------------------- + hello2 | world2 | {"hello2": "world2"} +(1 row) + +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; + shard_key_value | object_id | change_id | operation_type | new_value +--------------------------------------------------------------------- + hello | world | 1 | INSERT | {"hello": "world"} + hello | world | 2 | DELETE | + hello2 | world2 | 1 | INSERT | {"hello2": "world2"} + hello3 | world3 | 1 | INSERT | {"hello3": "world3"} + hello3 | world3 | 2 | UPDATE | {} + hello3 | world3 | 3 | DELETE | +(6 rows) + +CREATE FUNCTION insert_delete_document(key text, id text) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO distributed_triggers.data VALUES (key, id, '{"id1":"id2"}'); + DELETE FROM distributed_triggers.data where shard_key_value = key; +END; +$fn$; +SELECT create_distributed_function( + 'insert_delete_document(text, text)', 'key', + colocate_with := 'data', + force_delegation := true +); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT insert_delete_document('hello4', 'world4'); + insert_delete_document +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT insert_delete_document('hello4', 'world4'); + insert_delete_document +--------------------------------------------------------------------- + +(1 row) + +COMMIT; +SELECT * FROM data +ORDER BY shard_key_value, object_id; + shard_key_value | object_id | value +--------------------------------------------------------------------- + hello2 | world2 | {"hello2": "world2"} +(1 row) + +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; + shard_key_value | object_id | change_id | operation_type | new_value +--------------------------------------------------------------------- + hello | world | 1 | INSERT | {"hello": "world"} + hello | world | 2 | DELETE | + hello2 | world2 | 1 | INSERT | {"hello2": "world2"} + hello3 | world3 | 1 | INSERT | {"hello3": "world3"} + hello3 | world3 | 2 | UPDATE | {} + hello3 | world3 | 3 | DELETE | + hello4 | world4 | 1 | INSERT | {"id1": "id2"} + hello4 | world4 | 2 | DELETE | + hello4 | world4 | 3 | INSERT | {"id1": "id2"} + hello4 | world4 | 4 | DELETE | +(10 rows) + +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_change_trigger%' ORDER BY 1,2; + tgrelid | tgname +--------------------------------------------------------------------- + data | record_change_trigger +(1 row) + +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_change_trigger%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,17) + (localhost,57638,t,17) +(2 rows) + +ALTER TRIGGER "record_change_trigger" ON "distributed_triggers"."data" RENAME TO "new_record_change_trigger"; +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_change_trigger%' ORDER BY 1,2; + tgrelid | tgname +--------------------------------------------------------------------- +(0 rows) + +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'new_record_change_trigger%' ORDER BY 1,2; + tgrelid | tgname +--------------------------------------------------------------------- + data | new_record_change_trigger +(1 row) + +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_change_trigger%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,0) + (localhost,57638,t,0) +(2 rows) + +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'new_record_change_trigger%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,17) + (localhost,57638,t,17) +(2 rows) + +--This should fail +DROP TRIGGER record_change_trigger ON data; +ERROR: trigger "record_change_trigger" for table "data" does not exist +DROP TRIGGER new_record_change_trigger ON data; +--Trigger should go away +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'new_record_change_trigger%' ORDER BY 1,2; + tgrelid | tgname +--------------------------------------------------------------------- +(0 rows) + +-- +-- Run bad triggers +-- +CREATE OR REPLACE FUNCTION bad_shardkey_record_change() +RETURNS trigger +AS $$ +DECLARE + last_change_id bigint; +BEGIN + INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type, new_value) + VALUES ('BAD', NEW.object_id, COALESCE(last_change_id + 1, 1), TG_OP, NEW.value); + RETURN NULL; +END; +$$ LANGUAGE plpgsql; +CREATE TRIGGER bad_shardkey_record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.bad_shardkey_record_change(); +-- Bad trigger fired from an individual SQL +-- Query-on-distributed table exception should catch this +INSERT INTO data VALUES ('hello6','world6','{"hello6":"world6"}'); +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type, new_value) + VALUES ('BAD', NEW.object_id, COALESCE(last_change_id + 1, 1), TG_OP, NEW.value)" +PL/pgSQL function bad_shardkey_record_change() line XX at SQL statement +while executing command on localhost:xxxxx +-- Bad trigger fired from SQL inside a force-delegated function +-- Incorrect distribution key exception should catch this +SELECT insert_delete_document('hello6', 'world6'); +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type, new_value) + VALUES ('BAD', NEW.object_id, COALESCE(last_change_id + 1, 1), TG_OP, NEW.value)" +PL/pgSQL function distributed_triggers.bad_shardkey_record_change() line XX at SQL statement +SQL statement "INSERT INTO distributed_triggers.data VALUES (key, id, '{"id1":"id2"}')" +PL/pgSQL function distributed_triggers.insert_delete_document(text,text) line XX at SQL statement +while executing command on localhost:xxxxx +SELECT * FROM data +ORDER BY shard_key_value, object_id; + shard_key_value | object_id | value +--------------------------------------------------------------------- + hello2 | world2 | {"hello2": "world2"} +(1 row) + +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; + shard_key_value | object_id | change_id | operation_type | new_value +--------------------------------------------------------------------- + hello | world | 1 | INSERT | {"hello": "world"} + hello | world | 2 | DELETE | + hello2 | world2 | 1 | INSERT | {"hello2": "world2"} + hello3 | world3 | 1 | INSERT | {"hello3": "world3"} + hello3 | world3 | 2 | UPDATE | {} + hello3 | world3 | 3 | DELETE | + hello4 | world4 | 1 | INSERT | {"id1": "id2"} + hello4 | world4 | 2 | DELETE | + hello4 | world4 | 3 | INSERT | {"id1": "id2"} + hello4 | world4 | 4 | DELETE | +(10 rows) + +DROP TRIGGER bad_shardkey_record_change_trigger ON data; +CREATE OR REPLACE FUNCTION remote_shardkey_record_change() +RETURNS trigger +AS $$ +DECLARE + last_change_id bigint; +BEGIN + UPDATE distributed_triggers.data_changes SET operation_type = TG_OP; + RETURN NULL; +END; +$$ LANGUAGE plpgsql; +CREATE TRIGGER remote_shardkey_record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.remote_shardkey_record_change(); +CREATE FUNCTION insert_document(key text, id text) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO distributed_triggers.data VALUES (key, id, '{"id1":"id2"}'); + DELETE FROM distributed_triggers.data where shard_key_value = key; +END; +$fn$; +SELECT create_distributed_function( + 'insert_document(text, text)', 'key', + colocate_with := 'data', + force_delegation := false +); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT insert_document('hello7', 'world7'); +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "UPDATE distributed_triggers.data_changes SET operation_type = TG_OP" +PL/pgSQL function remote_shardkey_record_change() line XX at SQL statement +while executing command on localhost:xxxxx +SQL statement "INSERT INTO distributed_triggers.data VALUES (key, id, '{"id1":"id2"}')" +PL/pgSQL function insert_document(text,text) line XX at SQL statement +END; +SELECT insert_document('hello7', 'world7'); +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "UPDATE distributed_triggers.data_changes SET operation_type = TG_OP" +PL/pgSQL function remote_shardkey_record_change() line XX at SQL statement +SQL statement "INSERT INTO distributed_triggers.data VALUES (key, id, '{"id1":"id2"}')" +PL/pgSQL function insert_document(text,text) line XX at SQL statement +while executing command on localhost:xxxxx +SELECT * FROM data +ORDER BY shard_key_value, object_id; + shard_key_value | object_id | value +--------------------------------------------------------------------- + hello2 | world2 | {"hello2": "world2"} +(1 row) + +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; + shard_key_value | object_id | change_id | operation_type | new_value +--------------------------------------------------------------------- + hello | world | 1 | INSERT | {"hello": "world"} + hello | world | 2 | DELETE | + hello2 | world2 | 1 | INSERT | {"hello2": "world2"} + hello3 | world3 | 1 | INSERT | {"hello3": "world3"} + hello3 | world3 | 2 | UPDATE | {} + hello3 | world3 | 3 | DELETE | + hello4 | world4 | 1 | INSERT | {"id1": "id2"} + hello4 | world4 | 2 | DELETE | + hello4 | world4 | 3 | INSERT | {"id1": "id2"} + hello4 | world4 | 4 | DELETE | +(10 rows) + +-- +-- Triggers (tables) which are not colocated +-- +CREATE TABLE emptest ( + empname text NOT NULL, + salary integer +); +CREATE TABLE emptest_audit( + operation char(1) NOT NULL, + stamp timestamp NOT NULL, + userid text NOT NULL, + empname text NOT NULL, + salary integer +); +SELECT create_distributed_table('emptest','empname',colocate_with :='none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('emptest_audit','empname',colocate_with :='none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION process_emp_audit() RETURNS TRIGGER AS $emp_audit$ + BEGIN + -- + -- Create a row in emp_audit to reflect the operation performed on emp, + -- making use of the special variable TG_OP to work out the operation. + -- + IF (TG_OP = 'DELETE') THEN + INSERT INTO distributed_triggers.emptest_audit SELECT 'D', now(), user, OLD.*; + ELSIF (TG_OP = 'UPDATE') THEN + INSERT INTO distributed_triggers.emptest_audit SELECT 'U', now(), user, NEW.*; + ELSIF (TG_OP = 'INSERT') THEN + INSERT INTO distributed_triggers.emptest_audit SELECT 'I', now(), user, NEW.*; + END IF; + RETURN NULL; -- result is ignored since this is an AFTER trigger + END; +$emp_audit$ LANGUAGE plpgsql; +CREATE TRIGGER emptest_audit +AFTER INSERT OR UPDATE OR DELETE ON emptest + FOR EACH ROW EXECUTE FUNCTION distributed_triggers.process_emp_audit(); +INSERT INTO emptest VALUES ('test1', 1); +INSERT INTO emptest VALUES ('test2', 1); +INSERT INTO emptest VALUES ('test3', 1); +INSERT INTO emptest VALUES ('test4', 1); +SELECT operation, userid, empname, salary +FROM emptest_audit +ORDER BY 3,1; + operation | userid | empname | salary +--------------------------------------------------------------------- + I | postgres | test1 | 1 + I | postgres | test2 | 1 + I | postgres | test3 | 1 + I | postgres | test4 | 1 +(4 rows) + +DELETE from emptest; +SELECT operation, userid, empname, salary +FROM emptest_audit +ORDER BY 3,1; + operation | userid | empname | salary +--------------------------------------------------------------------- + D | postgres | test1 | 1 + I | postgres | test1 | 1 + D | postgres | test2 | 1 + I | postgres | test2 | 1 + D | postgres | test3 | 1 + I | postgres | test3 | 1 + D | postgres | test4 | 1 + I | postgres | test4 | 1 +(8 rows) + +CREATE VIEW emp_triggers AS + SELECT tgname, tgrelid::regclass, tgenabled + FROM pg_trigger + WHERE tgrelid::regclass::text like 'emptest%' + ORDER BY 1, 2; +SELECT * FROM emp_triggers ORDER BY 1,2; + tgname | tgrelid | tgenabled +--------------------------------------------------------------------- + emptest_audit | emptest | O + truncate_trigger_xxxxxxx | emptest | O + truncate_trigger_xxxxxxx | emptest_audit | O +(3 rows) + +-- Triggers "FOR EACH STATEMENT" +CREATE TABLE record_op ( + empname text NOT NULL, + operation_type text not null, + stamp timestamp NOT NULL +); +SELECT create_distributed_table('record_op', 'empname', colocate_with := 'emptest'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION record_emp() RETURNS TRIGGER AS $rec_audit$ + BEGIN + INSERT INTO distributed_triggers.record_op SELECT 'dummy', TG_OP, now(); + RETURN NULL; -- result is ignored since this is an AFTER trigger + END; +$rec_audit$ LANGUAGE plpgsql; +CREATE TRIGGER record_emp_trig +AFTER INSERT OR UPDATE OR DELETE ON emptest + FOR EACH STATEMENT EXECUTE FUNCTION distributed_triggers.record_emp(); +INSERT INTO emptest VALUES ('test5', 1); +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO distributed_triggers.record_op SELECT 'dummy', TG_OP, now()" +PL/pgSQL function record_emp() line XX at SQL statement +while executing command on localhost:xxxxx +DELETE FROM emptest; +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO distributed_triggers.record_op SELECT 'dummy', TG_OP, now()" +PL/pgSQL function distributed_triggers.record_emp() line XX at SQL statement +while executing command on localhost:xxxxx +SELECT * FROM emptest; + empname | salary +--------------------------------------------------------------------- +(0 rows) + +SELECT operation_type FROM record_op; + operation_type +--------------------------------------------------------------------- +(0 rows) + +-- +-- Triggers on reference tables +-- +CREATE TABLE data_ref_table ( + shard_key_value text not null, + object_id text not null, + value jsonb not null +); +ALTER TABLE data_ref_table +ADD CONSTRAINT data_ref_pk +PRIMARY KEY (shard_key_value, object_id); +SELECT create_reference_table('data_ref_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +-- Trigger function record_change operates on data_changes which is *not* colocated with the reference table +CREATE TRIGGER record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data_ref_table +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.record_change(); +TRUNCATE TABLE data_changes; +INSERT INTO data_ref_table VALUES ('hello','world','{"ref":"table"}'); +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "SELECT change_id FROM distributed_triggers.data_changes + WHERE shard_key_value = NEW.shard_key_value AND object_id = NEW.object_id + ORDER BY change_id DESC LIMIT 1" +PL/pgSQL function record_change() line XX at SQL statement +while executing command on localhost:xxxxx +INSERT INTO data_ref_table VALUES ('hello2','world2','{"ref":"table"}'); +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "SELECT change_id FROM distributed_triggers.data_changes + WHERE shard_key_value = NEW.shard_key_value AND object_id = NEW.object_id + ORDER BY change_id DESC LIMIT 1" +PL/pgSQL function record_change() line XX at SQL statement +while executing command on localhost:xxxxx +DELETE FROM data_ref_table where shard_key_value = 'hello'; +BEGIN; +UPDATE data_ref_table SET value = '{}' where shard_key_value = 'hello2'; +END; +TABLE data_changes ORDER BY shard_key_value, object_id, change_id; + shard_key_value | object_id | change_id | change_time | operation_type | new_value +--------------------------------------------------------------------- +(0 rows) + +TABLE data_ref_table ORDER BY shard_key_value, object_id; + shard_key_value | object_id | value +--------------------------------------------------------------------- +(0 rows) + +-- Colocate data_changes table with reference table +SELECT undistribute_table('data_changes'); +NOTICE: creating a new table for distributed_triggers.data_changes +NOTICE: moving the data of distributed_triggers.data_changes +NOTICE: dropping the old distributed_triggers.data_changes +NOTICE: renaming the new table to distributed_triggers.data_changes + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_reference_table('data_changes'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO data_ref_table VALUES ('hello','world','{"ref":"table"}'); +ERROR: cannot execute a distributed query from a query on a shard +CONTEXT: SQL statement "INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type, new_value) + VALUES (NEW.shard_key_value, NEW.object_id, COALESCE(last_change_id + 1, 1), TG_OP, NEW.value)" +PL/pgSQL function record_change() line XX at SQL statement +while executing command on localhost:xxxxx +TABLE data_changes ORDER BY shard_key_value, object_id, change_id; + shard_key_value | object_id | change_id | change_time | operation_type | new_value +--------------------------------------------------------------------- +(0 rows) + +TABLE data_ref_table ORDER BY shard_key_value, object_id; + shard_key_value | object_id | value +--------------------------------------------------------------------- +(0 rows) + +-- Create data_changes table locally with reference table +DROP TABLE data_changes; +/* table of changes local to each placement of the reference table */ +CREATE TABLE data_changes ( + shard_key_value text not null, + object_id text not null, + change_id bigint not null, + change_time timestamptz default now(), + operation_type text not null, + new_value jsonb +); +SELECT run_command_on_workers($$CREATE TABLE distributed_triggers.data_changes( + shard_key_value text not null, + object_id text not null, + change_id bigint not null, + change_time timestamptz default now(), + operation_type text not null, + new_value jsonb); +$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"CREATE TABLE") + (localhost,57638,t,"CREATE TABLE") +(2 rows) + +SELECT run_command_on_workers('SELECT count(*) FROM distributed_triggers.data_changes;'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,0) + (localhost,57638,t,0) +(2 rows) + +INSERT INTO data_ref_table VALUES ('hello','world','{"ref":"table"}'); +INSERT INTO data_ref_table VALUES ('hello2','world2','{"ref":"table"}'); +BEGIN; +UPDATE data_ref_table SET value = '{}'; +END; +SELECT run_command_on_workers('SELECT count(*) FROM distributed_triggers.data_changes;'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,4) + (localhost,57638,t,4) +(2 rows) + +TABLE data_ref_table ORDER BY shard_key_value, object_id; + shard_key_value | object_id | value +--------------------------------------------------------------------- + hello | world | {} + hello2 | world2 | {} +(2 rows) + +-- +--Triggers on partitioned tables +-- +CREATE TABLE sale(sale_date date not null, state_code text, product_sku text, units integer) +PARTITION BY list (state_code); +ALTER TABLE sale ADD CONSTRAINT sale_pk PRIMARY KEY (state_code, sale_date); +CREATE TABLE sale_newyork PARTITION OF sale FOR VALUES IN ('NY'); +CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA'); +CREATE TABLE record_sale(operation_type text not null, product_sku text, state_code text); +SELECT create_distributed_table('sale', 'state_code'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('record_sale', 'state_code', colocate_with := 'sale'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION record_sale() +RETURNS trigger +AS $$ +BEGIN + INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code) + VALUES (TG_OP, NEW.product_sku, NEW.state_code); + RETURN NULL; +END; +$$ LANGUAGE plpgsql; +CREATE TRIGGER record_sale_trigger +AFTER INSERT OR UPDATE OR DELETE ON sale +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.record_sale(); +INSERT INTO sale(sale_date,state_code,product_sku,units) VALUES +('2019-01-01', 'CA', 'AZ-000A1', 85), +('2019-01-02', 'CA', 'AZ-000A1', 6), +('2019-01-03', 'NY', 'AZ-000A2', 83), +('2019-02-01', 'CA', 'AZ-000A2', 59), +('2019-02-02', 'CA', 'AZ-000A1', 9), +('2019-02-03', 'NY', 'AZ-000A1', 47); +TABLE sale ORDER BY state_code, sale_date; + sale_date | state_code | product_sku | units +--------------------------------------------------------------------- + 01-01-2019 | CA | AZ-000A1 | 85 + 01-02-2019 | CA | AZ-000A1 | 6 + 02-01-2019 | CA | AZ-000A2 | 59 + 02-02-2019 | CA | AZ-000A1 | 9 + 01-03-2019 | NY | AZ-000A2 | 83 + 02-03-2019 | NY | AZ-000A1 | 47 +(6 rows) + +TABLE record_sale ORDER BY 1,2,3; + operation_type | product_sku | state_code +--------------------------------------------------------------------- + INSERT | AZ-000A1 | CA + INSERT | AZ-000A1 | CA + INSERT | AZ-000A1 | CA + INSERT | AZ-000A1 | NY + INSERT | AZ-000A2 | CA + INSERT | AZ-000A2 | NY +(6 rows) + +-- +--Test ALTER TRIGGER +-- +CREATE VIEW sale_triggers AS + SELECT tgname, tgrelid::regclass, tgenabled + FROM pg_trigger + WHERE tgrelid::regclass::text like 'sale%' + ORDER BY 1, 2; +SELECT * FROM sale_triggers ORDER BY 1,2; + tgname | tgrelid | tgenabled +--------------------------------------------------------------------- + record_sale_trigger | sale | O + record_sale_trigger | sale_newyork | O + record_sale_trigger | sale_california | O + truncate_trigger_xxxxxxx | sale | O + truncate_trigger_xxxxxxx | sale_california | O + truncate_trigger_xxxxxxx | sale_newyork | O +(6 rows) + +ALTER TRIGGER "record_sale_trigger" ON "distributed_triggers"."sale" RENAME TO "new_record_sale_trigger"; +SELECT * FROM sale_triggers ORDER BY 1,2; + tgname | tgrelid | tgenabled +--------------------------------------------------------------------- + new_record_sale_trigger | sale | O + record_sale_trigger | sale_newyork | O + record_sale_trigger | sale_california | O + truncate_trigger_xxxxxxx | sale | O + truncate_trigger_xxxxxxx | sale_california | O + truncate_trigger_xxxxxxx | sale_newyork | O +(6 rows) + +CREATE EXTENSION seg; +ALTER TRIGGER "emptest_audit" ON "emptest" DEPENDS ON EXTENSION seg; +ERROR: Triggers "emptest_audit" on distributed tables and local tables added to metadata are not allowed to depend on an extension +DETAIL: Triggers from extensions are expected to be created on the workers by the extension they depend on. +DROP TABLE data_ref_table; +-- +--Triggers with add/remove node +-- +SELECT * FROM master_drain_node('localhost', :worker_2_port); +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... +NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ... + master_drain_node +--------------------------------------------------------------------- + +(1 row) + +SELECT 1 from master_remove_node('localhost', :worker_2_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +CREATE TABLE distributed_table(value int); +CREATE TABLE distributed_table_change(value int); +SELECT create_distributed_table('distributed_table', 'value', colocate_with => 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('distributed_table_change', 'value', colocate_with => 'distributed_table'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION insert_99() RETURNS trigger AS $insert_99$ +BEGIN + INSERT INTO distributed_triggers.distributed_table_change VALUES (99); + RETURN NEW; +END; +$insert_99$ LANGUAGE plpgsql; +CREATE TRIGGER insert_99_trigger +AFTER DELETE ON distributed_table +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.insert_99(); +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'insert_99_trigger%' ORDER BY 1,2; + tgrelid | tgname +--------------------------------------------------------------------- + distributed_table | insert_99_trigger +(1 row) + +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'insert_99_trigger%'$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,33) +(1 row) + +INSERT INTO distributed_table VALUES (99); +DELETE FROM distributed_table; +SELECT * FROM distributed_table_change; + value +--------------------------------------------------------------------- + 99 +(1 row) + +-- add the node back +SELECT 1 from master_add_node('localhost', :worker_2_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +INSERT INTO distributed_table VALUES (99); +DELETE FROM distributed_table; +SELECT * FROM distributed_table_change; + value +--------------------------------------------------------------------- + 99 + 99 +(2 rows) + +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'insert_99_trigger%' ORDER BY 1,2; + tgrelid | tgname +--------------------------------------------------------------------- + distributed_table | insert_99_trigger +(1 row) + +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'insert_99_trigger%'$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,33) + (localhost,57638,t,1) +(2 rows) + +RESET client_min_messages; +SET citus.enable_unsafe_triggers TO false; +SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO false;'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"ALTER SYSTEM") + (localhost,57638,t,"ALTER SYSTEM") +(2 rows) + +SELECT run_command_on_workers('SELECT pg_reload_conf();'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,t) + (localhost,57638,t,t) +(2 rows) + +SET citus.log_remote_commands TO off; +DROP SCHEMA distributed_triggers CASCADE; +NOTICE: drop cascades to 21 other objects +DETAIL: drop cascades to table data +drop cascades to function record_change() +drop cascades to function insert_delete_document(text,text) +drop cascades to function bad_shardkey_record_change() +drop cascades to function remote_shardkey_record_change() +drop cascades to function insert_document(text,text) +drop cascades to table emptest +drop cascades to table emptest_audit +drop cascades to function process_emp_audit() +drop cascades to view emp_triggers +drop cascades to table record_op +drop cascades to function record_emp() +drop cascades to table data_changes +drop cascades to table sale +drop cascades to table record_sale +drop cascades to function record_sale() +drop cascades to view sale_triggers +drop cascades to extension seg +drop cascades to table distributed_table +drop cascades to table distributed_table_change +drop cascades to function insert_99() diff --git a/src/test/regress/expected/follower_single_node.out b/src/test/regress/expected/follower_single_node.out index 7fbc40280..de2f88b6e 100644 --- a/src/test/regress/expected/follower_single_node.out +++ b/src/test/regress/expected/follower_single_node.out @@ -239,7 +239,7 @@ ERROR: node group 0 does not have a secondary node -- should work this time \c -reuse-previous=off regression - - :master_port SET search_path TO single_node; -SELECT 1 FROM master_add_node('localhost', :follower_master_port, groupid => 0, noderole => 'secondary'); +SELECT 1 FROM master_add_node('localhost', :follower_master_port, groupid => 0, noderole => 'secondary', nodecluster => 'second-cluster'); ?column? --------------------------------------------------------------------- 1 @@ -251,7 +251,7 @@ SELECT 1 FROM master_set_node_property('localhost', :master_port, 'shouldhavesha 1 (1 row) -\c "port=9070 dbname=regression options='-c\ citus.use_secondary_nodes=always'" +\c "port=9070 dbname=regression options='-c\ citus.use_secondary_nodes=always\ -c\ citus.cluster_name=second-cluster'" SET search_path TO single_node; SELECT * FROM test WHERE x = 1; x | y @@ -387,6 +387,40 @@ SELECT * FROM columnar_test ORDER BY 1,2; 1 | 8 (6 rows) +\c -reuse-previous=off regression - - :master_port +SET citus.shard_replication_factor TO 1; +SET search_path TO single_node; +CREATE TABLE dist_table (a INT, b INT); +SELECT create_distributed_table ('dist_table', 'a', shard_count:=4); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO dist_table VALUES (1, 1); +\c "port=9070 dbname=regression options='-c\ citus.use_secondary_nodes=always\ -c\ citus.cluster_name=second-cluster'" +SET search_path TO single_node; +SELECT * FROM dist_table; + a | b +--------------------------------------------------------------------- + 1 | 1 +(1 row) + +SELECT global_pid AS follower_coordinator_gpid FROM get_all_active_transactions() WHERE process_id = pg_backend_pid() \gset +SELECT pg_typeof(:follower_coordinator_gpid); + pg_typeof +--------------------------------------------------------------------- + bigint +(1 row) + +SELECT pg_cancel_backend(:follower_coordinator_gpid); +ERROR: canceling statement due to user request +SET citus.log_remote_commands TO ON; +SELECT pg_cancel_backend(:follower_coordinator_gpid) FROM dist_table WHERE a = 1; +NOTICE: executing the command locally: SELECT pg_cancel_backend('xxxxx'::bigint) AS pg_cancel_backend FROM single_node.dist_table_102008 dist_table WHERE (a OPERATOR(pg_catalog.=) 1) +NOTICE: issuing SELECT pg_cancel_backend(xxxxx::integer) +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +ERROR: canceling statement due to user request -- Cleanup \c -reuse-previous=off regression - - :master_port SET search_path TO single_node; diff --git a/src/test/regress/expected/global_cancel.out b/src/test/regress/expected/global_cancel.out new file mode 100644 index 000000000..5ebc4098d --- /dev/null +++ b/src/test/regress/expected/global_cancel.out @@ -0,0 +1,83 @@ +CREATE SCHEMA global_cancel; +SET search_path TO global_cancel; +SET citus.next_shard_id TO 56789000; +CREATE TABLE dist_table (a INT, b INT); +SELECT create_distributed_table ('dist_table', 'a', shard_count:=4); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO dist_table VALUES (1, 1); +SELECT global_pid AS coordinator_gpid FROM get_all_active_transactions() WHERE process_id = pg_backend_pid() \gset +SELECT pg_typeof(:coordinator_gpid); + pg_typeof +--------------------------------------------------------------------- + bigint +(1 row) + +SELECT pg_cancel_backend(:coordinator_gpid); +ERROR: canceling statement due to user request +SET citus.log_remote_commands TO ON; +SELECT pg_cancel_backend(:coordinator_gpid) FROM dist_table WHERE a = 1; +NOTICE: issuing SELECT pg_cancel_backend('xxxxx'::bigint) AS pg_cancel_backend FROM global_cancel.dist_table_56789000 dist_table WHERE (a OPERATOR(pg_catalog.=) 1) +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +ERROR: canceling statement due to user request +BEGIN; +SELECT pg_cancel_backend(:coordinator_gpid) FROM dist_table WHERE a = 1; +NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx'); +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +NOTICE: issuing SELECT pg_cancel_backend('xxxxx'::bigint) AS pg_cancel_backend FROM global_cancel.dist_table_56789000 dist_table WHERE (a OPERATOR(pg_catalog.=) 1) +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +ERROR: canceling statement due to user request +END; +SET citus.log_remote_commands TO OFF; +SELECT global_pid AS maintenance_daemon_gpid +FROM pg_stat_activity psa JOIN get_all_active_transactions() gaat ON psa.pid = gaat.process_id +WHERE application_name = 'Citus Maintenance Daemon' \gset +SET client_min_messages TO ERROR; +CREATE USER global_cancel_user; +SELECT 1 FROM run_command_on_workers('CREATE USER global_cancel_user'); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +RESET client_min_messages; +SET ROLE global_cancel_user; +SELECT pg_typeof(:maintenance_daemon_gpid); + pg_typeof +--------------------------------------------------------------------- + bigint +(1 row) + +SELECT pg_cancel_backend(:maintenance_daemon_gpid); +ERROR: must be a superuser to cancel superuser query +CONTEXT: while executing command on localhost:xxxxx +SELECT pg_terminate_backend(:maintenance_daemon_gpid); +ERROR: must be a superuser to terminate superuser process +CONTEXT: while executing command on localhost:xxxxx +RESET ROLE; +SELECT nodeid AS coordinator_node_id FROM pg_dist_node WHERE nodeport = :master_port \gset +SET client_min_messages TO DEBUG; +-- 10000000000 is the node id multiplier for global pid +SELECT pg_cancel_backend(10000000000 * :coordinator_node_id + 0); +DEBUG: PID 0 is not a PostgreSQL server process +DETAIL: from localhost:xxxxx + pg_cancel_backend +--------------------------------------------------------------------- + f +(1 row) + +SELECT pg_terminate_backend(10000000000 * :coordinator_node_id + 0); +DEBUG: PID 0 is not a PostgreSQL server process +DETAIL: from localhost:xxxxx + pg_terminate_backend +--------------------------------------------------------------------- + f +(1 row) + +RESET client_min_messages; +DROP SCHEMA global_cancel CASCADE; +NOTICE: drop cascades to table dist_table diff --git a/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out b/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out index 27e13263d..cae2222ed 100644 --- a/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out +++ b/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out @@ -2,13 +2,13 @@ Parsed test spec with 4 sessions starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-commit-worker s2-stop-connection step s1-begin: - BEGIN; + BEGIN; step s1-update-ref-table-from-coordinator: - UPDATE ref_table SET value_1 = 15; + UPDATE ref_table SET value_1 = 15; step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -16,7 +16,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -24,20 +24,20 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-update-ref-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| - UPDATE ref_table SET value_1 = 15; + UPDATE ref_table SET value_1 = 15; |localhost |coordinator_host | 57638| 57636 (1 row) step s1-commit: - COMMIT; + COMMIT; step s2-update-ref-table: <... completed> run_commands_on_session_level_connection_to_node @@ -54,7 +54,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -69,7 +69,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-ref-table s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -77,7 +77,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -85,7 +85,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-update-ref-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -93,7 +93,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -101,7 +101,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -109,10 +109,10 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-update-ref-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -142,7 +142,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -150,7 +150,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -165,7 +165,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-dist-table s2-start-session-level-connection s2-begin-on-worker s2-update-dist-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -173,7 +173,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -181,7 +181,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-update-dist-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -189,7 +189,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -197,7 +197,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -205,10 +205,10 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-update-dist-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -238,7 +238,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -246,7 +246,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -261,7 +261,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-delete-from-ref-table s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -269,7 +269,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -277,7 +277,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-delete-from-ref-table: - SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('DELETE FROM ref_table WHERE user_id = 1'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -285,7 +285,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -293,7 +293,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -301,10 +301,10 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-update-ref-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -334,7 +334,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -342,7 +342,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -357,7 +357,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-ref-table s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -365,7 +365,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -373,7 +373,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-insert-into-ref-table: - SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -381,7 +381,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -389,7 +389,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -397,10 +397,10 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-update-ref-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -430,7 +430,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -438,7 +438,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -453,7 +453,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-ref-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -461,7 +461,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -469,7 +469,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-insert-into-ref-table: - SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -477,7 +477,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -485,7 +485,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -493,7 +493,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-insert-into-ref-table: - SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -501,7 +501,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -524,7 +524,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -532,7 +532,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -547,7 +547,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-to-ref-table s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -555,7 +555,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -563,7 +563,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-copy-to-ref-table: - SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); + SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -571,7 +571,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -579,7 +579,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -587,10 +587,10 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-update-ref-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -620,7 +620,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -628,7 +628,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -643,7 +643,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-to-ref-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -651,7 +651,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -659,7 +659,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-copy-to-ref-table: - SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); + SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -667,7 +667,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -675,7 +675,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -683,7 +683,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-insert-into-ref-table: - SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -691,7 +691,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -714,7 +714,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -722,7 +722,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -737,7 +737,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-to-ref-table s2-start-session-level-connection s2-begin-on-worker s2-copy-to-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -745,7 +745,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -753,7 +753,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-copy-to-ref-table: - SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); + SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -761,7 +761,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -769,7 +769,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -777,7 +777,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-copy-to-ref-table: - SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); + SELECT run_commands_on_session_level_connection_to_node('COPY ref_table FROM PROGRAM ''echo 10, 101 && echo 11, 111'' WITH CSV'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -785,7 +785,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -808,7 +808,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -816,7 +816,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -831,7 +831,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-select-for-update s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -839,7 +839,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -847,7 +847,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-select-for-update: - SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM ref_table FOR UPDATE'); + SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM ref_table FOR UPDATE'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -855,7 +855,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -863,7 +863,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -871,10 +871,10 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-update-ref-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -904,7 +904,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -912,7 +912,7 @@ stop_session_level_connection_to_node (1 row) step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -927,7 +927,7 @@ restore_isolation_tester_func starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-insert-into-ref-table s1-begin s1-alter-table s3-select-distributed-waiting-queries s2-commit-worker s1-commit s2-stop-connection step s2-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57638); + SELECT start_session_level_connection_to_node('localhost', 57638); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -935,7 +935,7 @@ start_session_level_connection_to_node (1 row) step s2-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -943,7 +943,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s2-insert-into-ref-table: - SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO ref_table VALUES(8,81),(9,91)'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -951,18 +951,18 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-begin: - BEGIN; + BEGIN; step s1-alter-table: - ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); + ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- - ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); + ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); |INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |localhost | 57636| 57638 (1 row) @@ -976,10 +976,10 @@ run_commands_on_session_level_connection_to_node step s1-alter-table: <... completed> step s1-commit: - COMMIT; + COMMIT; step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -994,28 +994,28 @@ restore_isolation_tester_func starting permutation: s1-begin s1-update-on-the-coordinator s2-update-on-the-coordinator s3-select-distributed-waiting-queries s1-commit step s1-begin: - BEGIN; + BEGIN; step s1-update-on-the-coordinator: - UPDATE tt1 SET value_1 = 4; + UPDATE tt1 SET value_1 = 4; step s2-update-on-the-coordinator: - UPDATE tt1 SET value_1 = 4; + UPDATE tt1 SET value_1 = 4; step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- - UPDATE tt1 SET value_1 = 4; + UPDATE tt1 SET value_1 = 4; | - UPDATE tt1 SET value_1 = 4; + UPDATE tt1 SET value_1 = 4; |coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit: - COMMIT; + COMMIT; step s2-update-on-the-coordinator: <... completed> restore_isolation_tester_func @@ -1026,7 +1026,7 @@ restore_isolation_tester_func starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-dist-table s4-start-session-level-connection s4-begin-on-worker s4-update-dist-table s3-select-distributed-waiting-queries s1-commit-worker s4-commit-worker s1-stop-connection s4-stop-connection step s1-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -1034,7 +1034,7 @@ start_session_level_connection_to_node (1 row) step s1-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -1042,7 +1042,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-update-dist-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -1050,7 +1050,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s4-start-session-level-connection: - SELECT start_session_level_connection_to_node('localhost', 57637); + SELECT start_session_level_connection_to_node('localhost', 57637); start_session_level_connection_to_node --------------------------------------------------------------------- @@ -1058,7 +1058,7 @@ start_session_level_connection_to_node (1 row) step s4-begin-on-worker: - SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); run_commands_on_session_level_connection_to_node --------------------------------------------------------------------- @@ -1066,10 +1066,10 @@ run_commands_on_session_level_connection_to_node (1 row) step s4-update-dist-table: - SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- @@ -1099,7 +1099,7 @@ run_commands_on_session_level_connection_to_node (1 row) step s1-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -1107,7 +1107,138 @@ stop_session_level_connection_to_node (1 row) step s4-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-dist-table-id-1 s2-start-session-level-connection s2-update-dist-table-id-1 s3-select-distributed-waiting-queries s1-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-dist-table-id-1: + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4 WHERE user_id = 1'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-dist-table-id-1: + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4 WHERE user_id = 1'); + +step s3-select-distributed-waiting-queries: + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + +blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +--------------------------------------------------------------------- +UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|localhost |localhost | 57638| 57637 +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-dist-table-id-1: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-stop-connection +step s1-begin: + BEGIN; + +step s1-update-ref-table-from-coordinator: + UPDATE ref_table SET value_1 = 15; + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-ref-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); + +step s3-select-distributed-waiting-queries: + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + +blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +--------------------------------------------------------------------- +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| + UPDATE ref_table SET value_1 = 15; +|localhost |coordinator_host | 57638| 57636 +(1 row) + +step s1-commit: + COMMIT; + +step s2-update-ref-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- diff --git a/src/test/regress/expected/isolation_rebalancer_deferred_drop.out b/src/test/regress/expected/isolation_rebalancer_deferred_drop.out index d89a71ece..6963e9122 100644 --- a/src/test/regress/expected/isolation_rebalancer_deferred_drop.out +++ b/src/test/regress/expected/isolation_rebalancer_deferred_drop.out @@ -96,7 +96,7 @@ step s1-commit: COMMIT; step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- @@ -148,7 +148,8 @@ step s1-move-placement-back: SET client_min_messages to NOTICE; SHOW log_error_verbosity; SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57638, 'localhost', 57637); - + +step s1-move-placement-back: <... completed> log_error_verbosity --------------------------------------------------------------------- verbose @@ -159,7 +160,7 @@ step s1-commit: COMMIT; step s2-stop-connection: - SELECT stop_session_level_connection_to_node(); + SELECT stop_session_level_connection_to_node(); stop_session_level_connection_to_node --------------------------------------------------------------------- diff --git a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out index e83e71919..2d49f8586 100644 --- a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out +++ b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out @@ -101,13 +101,14 @@ step s2-view-worker: FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND - query NOT ILIKE '%dump_local_wait_edges%' + query NOT ILIKE '%dump_local_%' AND + query NOT ILIKE '%citus_internal_local_blocked_processes%' ORDER BY query, query_hostport DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -UPDATE public.ref_table_1500767 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -UPDATE public.ref_table_1500767 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression (2 rows) step s2-end: diff --git a/src/test/regress/expected/multi_distribution_metadata.out b/src/test/regress/expected/multi_distribution_metadata.out index ba004974b..c76b45a5b 100644 --- a/src/test/regress/expected/multi_distribution_metadata.out +++ b/src/test/regress/expected/multi_distribution_metadata.out @@ -220,7 +220,7 @@ SELECT column_to_column_name('pg_dist_node'::regclass,'{FROMEXPR :fromlist ({RAN ERROR: not a valid column -- test column_name_to_column with illegal arguments SELECT column_name_to_column(1204127312,''); -ERROR: could not open relation with OID 1204127312 +ERROR: relation does not exist SELECT column_name_to_column('customers','notacolumn'); ERROR: column "notacolumn" of relation "customers" does not exist -- make one huge shard and manually inspect shard row diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 0b185659b..e5cbcd994 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -428,20 +428,20 @@ SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDE ALTER EXTENSION citus UPDATE TO '9.4-2'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -469,20 +469,20 @@ SELECT * FROM multi_extension.print_extension_changes(); ALTER EXTENSION citus UPDATE TO '9.4-1'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -578,20 +578,20 @@ SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDE ALTER EXTENSION citus UPDATE TO '9.5-2'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -619,20 +619,20 @@ SELECT * FROM multi_extension.print_extension_changes(); ALTER EXTENSION citus UPDATE TO '9.5-1'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -1001,7 +1001,7 @@ SELECT * FROM multi_extension.print_extension_changes(); -- Snapshot of state at 11.0-1 ALTER EXTENSION citus UPDATE TO '11.0-1'; SELECT * FROM multi_extension.print_extension_changes(); - previous_object | current_object + previous_object | current_object --------------------------------------------------------------------- function citus_disable_node(text,integer) void | function create_distributed_function(regprocedure,text,text) void | @@ -1012,14 +1012,18 @@ SELECT * FROM multi_extension.print_extension_changes(); | function citus_check_connection_to_node(text,integer) boolean | function citus_disable_node(text,integer,boolean) void | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void + | function citus_internal_global_blocked_processes() SETOF record + | function citus_internal_local_blocked_processes() SETOF record | function citus_run_local_command(text) void | function citus_shard_indexes_on_worker() SETOF record | function citus_shards_on_worker() SETOF record | function create_distributed_function(regprocedure,text,text,boolean) void + | function pg_cancel_backend(bigint) boolean + | function pg_terminate_backend(bigint,bigint) boolean | function worker_create_or_replace_object(text[]) boolean | function worker_drop_sequence_dependency(text) void | function worker_drop_shell_table(text) void -(16 rows) +(20 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version diff --git a/src/test/regress/expected/multi_mx_schema_support.out b/src/test/regress/expected/multi_mx_schema_support.out index 71dfde3fd..6c7173c1f 100644 --- a/src/test/regress/expected/multi_mx_schema_support.out +++ b/src/test/regress/expected/multi_mx_schema_support.out @@ -515,12 +515,30 @@ SELECT table_schema AS "Shards' Schema" mx_new_schema (1 row) +-- check that we can drop a user-defined schema from workers +SET citus.enable_ddl_propagation TO OFF; +CREATE SCHEMA localschema; +RESET citus.enable_ddl_propagation; +DROP SCHEMA localschema; \c - - - :master_port SELECT * FROM mx_new_schema.table_set_schema; id --------------------------------------------------------------------- (0 rows) +-- verify local schema does not exist on the worker +-- worker errors out as "schema does not exist" +SET citus.enable_ddl_propagation TO OFF; +CREATE SCHEMA localschema; +-- should error out +SELECT run_command_on_workers($$DROP SCHEMA localschema;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,f,"ERROR: schema ""localschema"" does not exist") + (localhost,57638,f,"ERROR: schema ""localschema"" does not exist") +(2 rows) + +RESET citus.enable_ddl_propagation; DROP SCHEMA mx_old_schema CASCADE; DROP SCHEMA mx_new_schema CASCADE; NOTICE: drop cascades to table mx_new_schema.table_set_schema diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index 7ae524a3b..185bf19c5 100644 --- a/src/test/regress/expected/upgrade_list_citus_objects.out +++ b/src/test/regress/expected/upgrade_list_citus_objects.out @@ -74,6 +74,8 @@ ORDER BY 1; function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint) function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text) function citus_internal_delete_shard_metadata(bigint) + function citus_internal_global_blocked_processes() + function citus_internal_local_blocked_processes() function citus_internal_update_placement_metadata(bigint,integer,integer) function citus_internal_update_relation_colocation(oid,integer) function citus_isolation_test_session_is_blocked(integer,integer[]) @@ -172,6 +174,8 @@ ORDER BY 1; function master_update_shard_statistics(bigint) function master_update_table_statistics(regclass) function notify_constraint_dropped() + function pg_cancel_backend(bigint) + function pg_terminate_backend(bigint, bigint) function poolinfo_valid(text) function read_intermediate_result(text,citus_copy_format) function read_intermediate_results(text[],citus_copy_format) @@ -268,5 +272,5 @@ ORDER BY 1; view citus_worker_stat_activity view pg_dist_shard_placement view time_partitions -(252 rows) +(256 rows) diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 4d7f68e73..da7eb9c79 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -30,6 +30,7 @@ test: propagate_extension_commands test: escape_extension_name test: ref_citus_local_fkeys test: alter_database_owner +test: distributed_triggers test: multi_test_catalog_views test: multi_table_ddl @@ -282,6 +283,7 @@ test: create_citus_local_table_cascade test: fkeys_between_local_ref test: auto_undist_citus_local test: mx_regular_user +test: global_cancel test: remove_coordinator # ---------- diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index 2e26ea425..2e01b0518 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -1007,7 +1007,7 @@ SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value; (1 row) ALTER TABLE trigger_table DISABLE TRIGGER ALL; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" INSERT INTO trigger_table VALUES (1, 'trigger disabled'); SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value; value | count @@ -1016,7 +1016,7 @@ SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value; (1 row) ALTER TABLE trigger_table ENABLE TRIGGER ALL; -ERROR: triggers are only supported for local tables added to metadata +ERROR: triggers are not supported on distributed tables when "citus.enable_unsafe_triggers" is set to "false" INSERT INTO trigger_table VALUES (1, 'trigger disabled'); SELECT value, count(*) FROM trigger_table GROUP BY value ORDER BY value; value | count diff --git a/src/test/regress/spec/isolation_get_distributed_wait_queries_mx.spec b/src/test/regress/spec/isolation_get_distributed_wait_queries_mx.spec index b43708ab1..84daaf792 100644 --- a/src/test/regress/spec/isolation_get_distributed_wait_queries_mx.spec +++ b/src/test/regress/spec/isolation_get_distributed_wait_queries_mx.spec @@ -73,6 +73,11 @@ step "s1-select-for-update" SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM ref_table FOR UPDATE'); } +step "s1-update-dist-table-id-1" +{ + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4 WHERE user_id = 1'); +} + step "s1-commit-worker" { SELECT run_commands_on_session_level_connection_to_node('COMMIT'); @@ -115,6 +120,11 @@ step "s2-update-dist-table" SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); } +step "s2-update-dist-table-id-1" +{ + SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4 WHERE user_id = 1'); +} + step "s2-update-ref-table" { SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); @@ -149,7 +159,7 @@ session "s3" step "s3-select-distributed-waiting-queries" { - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits; + SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; } // session s1 and s4 executes the commands on the same worker node @@ -196,3 +206,9 @@ permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert- // blocked on the same node permutation "s1-begin" "s1-update-on-the-coordinator" "s2-update-on-the-coordinator" "s3-select-distributed-waiting-queries" "s1-commit" permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table" "s4-start-session-level-connection" "s4-begin-on-worker" "s4-update-dist-table" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s4-commit-worker" "s1-stop-connection" "s4-stop-connection" + + +// show that even if the commands are not in a transaction block +// we can find the blocking relationship +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table-id-1" "s2-start-session-level-connection" "s2-update-dist-table-id-1" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-begin" "s1-update-ref-table-from-coordinator" "s2-start-session-level-connection" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit" "s2-stop-connection" diff --git a/src/test/regress/spec/isolation_mx_common.include.spec b/src/test/regress/spec/isolation_mx_common.include.spec index 497d57fc4..4749e334c 100644 --- a/src/test/regress/spec/isolation_mx_common.include.spec +++ b/src/test/regress/spec/isolation_mx_common.include.spec @@ -7,6 +7,16 @@ setup LANGUAGE C STRICT VOLATILE AS 'citus', $$start_session_level_connection_to_node$$; + CREATE OR REPLACE FUNCTION override_backend_data_command_originator(bool) + RETURNS void + LANGUAGE C STRICT IMMUTABLE + AS 'citus', $$override_backend_data_command_originator$$; + + SELECT run_command_on_workers($$SET citus.enable_metadata_sync TO off;CREATE OR REPLACE FUNCTION override_backend_data_command_originator(bool) + RETURNS void + LANGUAGE C STRICT IMMUTABLE + AS 'citus'$$); + CREATE OR REPLACE FUNCTION run_commands_on_session_level_connection_to_node(text) RETURNS void LANGUAGE C STRICT VOLATILE diff --git a/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec b/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec index 0defcf549..fa2079ba5 100644 --- a/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec +++ b/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec @@ -93,7 +93,8 @@ step "s2-view-worker" FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND - query NOT ILIKE '%dump_local_wait_edges%' + query NOT ILIKE '%dump_local_%' AND + query NOT ILIKE '%citus_internal_local_blocked_processes%' ORDER BY query, query_hostport DESC; } diff --git a/src/test/regress/sql/citus_local_table_triggers.sql b/src/test/regress/sql/citus_local_table_triggers.sql index 9bb1ee81d..57394b306 100644 --- a/src/test/regress/sql/citus_local_table_triggers.sql +++ b/src/test/regress/sql/citus_local_table_triggers.sql @@ -161,12 +161,11 @@ BEFORE INSERT ON "interesting!schema"."citus_local!_table" FOR EACH STATEMENT EXECUTE FUNCTION dummy_function(); CREATE EXTENSION seg; +-- ALTER TRIGGER DEPENDS ON +ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg; BEGIN; - -- 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 + -- show that triggers on both shell relation and shard relation are not 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' @@ -174,7 +173,7 @@ BEGIN; DROP EXTENSION seg; - -- show that dropping extension drops the triggers automatically + -- show that dropping extension doesn't drop the triggers automatically SELECT * FROM citus_local_table_triggers WHERE tgname NOT LIKE 'RI_ConstraintTrigger%'; ROLLBACK; diff --git a/src/test/regress/sql/citus_local_tables_mx.sql b/src/test/regress/sql/citus_local_tables_mx.sql index bc6f84feb..4e72c6c13 100644 --- a/src/test/regress/sql/citus_local_tables_mx.sql +++ b/src/test/regress/sql/citus_local_tables_mx.sql @@ -37,22 +37,19 @@ FOR EACH ROW EXECUTE FUNCTION dummy_function(); -- the function that trigger needs in mx workers too. SELECT 1 FROM citus_activate_node('localhost', :worker_1_port); +-- show that the trigger is not allowed to depend(explicitly) on an extension. 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. +-- show that update trigger mx relation is renamed and disabled. -- both workers should should print 1. SELECT run_command_on_workers( $$ -SELECT COUNT(*) FROM pg_depend, pg_trigger, pg_extension +SELECT COUNT(*) FROM pg_trigger 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' + pg_trigger.tgenabled='D' $$); CREATE FUNCTION another_dummy_function() RETURNS trigger AS $another_dummy_function$ @@ -382,6 +379,36 @@ $$ SELECT count(*) FROM pg_catalog.pg_tables WHERE tablename='citus_local_table_4' $$); +-- verify that partitioned citus local tables with dropped columns can be distributed. issue: #5577 +CREATE TABLE parent_dropped_col(a int, eventtime date) PARTITION BY RANGE ( eventtime); +SELECT citus_add_local_table_to_metadata('parent_dropped_col'); +ALTER TABLE parent_dropped_col DROP column a; +CREATE TABLE parent_dropped_col_1 PARTITION OF parent_dropped_col for VALUES FROM ('2000-01-01') TO ('2001-01-01'); +SELECT create_distributed_table('parent_dropped_col', 'eventtime'); +-- another example to test +CREATE TABLE parent_dropped_col_2( + col_to_drop_0 text, + col_to_drop_1 text, + col_to_drop_2 date, + col_to_drop_3 inet, + col_to_drop_4 date, + measureid integer, + eventdatetime date, + measure_data jsonb, + PRIMARY KEY (measureid, eventdatetime, measure_data)) + PARTITION BY RANGE(eventdatetime); + +select citus_add_local_table_to_metadata('parent_dropped_col_2'); +ALTER TABLE parent_dropped_col_2 DROP COLUMN col_to_drop_1; +CREATE TABLE parent_dropped_col_2_2000 PARTITION OF parent_dropped_col_2 FOR VALUES FROM ('2000-01-01') TO ('2001-01-01'); + +SELECT create_distributed_table('parent_dropped_col_2', 'measureid'); + +-- verify that the partitioned tables are distributed with the correct distribution column +SELECT logicalrelid, partmethod, partkey FROM pg_dist_partition + WHERE logicalrelid IN ('parent_dropped_col'::regclass, 'parent_dropped_col_2'::regclass) + ORDER BY logicalrelid; + -- cleanup at exit set client_min_messages to error; DROP SCHEMA citus_local_tables_mx CASCADE; diff --git a/src/test/regress/sql/distributed_triggers.sql b/src/test/regress/sql/distributed_triggers.sql new file mode 100644 index 000000000..9f8a2e3b7 --- /dev/null +++ b/src/test/regress/sql/distributed_triggers.sql @@ -0,0 +1,470 @@ +SET citus.log_remote_commands TO OFF; +DROP SCHEMA IF EXISTS distributed_triggers CASCADE; +CREATE SCHEMA distributed_triggers; +SET search_path TO 'distributed_triggers'; +SET citus.shard_replication_factor = 1; +SET citus.shard_count = 32; +SET citus.next_shard_id TO 800000; + +-- +-- Test citus.enable_unsafe_triggers +-- Enables arbitrary triggers on distributed tables +-- +CREATE TABLE data ( + shard_key_value text not null, + object_id text not null, + value jsonb not null +); +ALTER TABLE data +ADD CONSTRAINT data_pk +PRIMARY KEY (shard_key_value, object_id); + +/* table of changes */ +CREATE TABLE data_changes ( + shard_key_value text not null, + object_id text not null, + change_id bigint not null, + change_time timestamptz default now(), + operation_type text not null, + new_value jsonb +); +ALTER TABLE data_changes +ADD CONSTRAINT data_changes_pk +PRIMARY KEY (shard_key_value, object_id, change_id); + +SELECT create_distributed_table('data', 'shard_key_value'); +SELECT create_distributed_table('data_changes', 'shard_key_value', colocate_with := 'data'); + +SET citus.enable_unsafe_triggers TO true; +SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO true;'); +SELECT run_command_on_workers('SELECT pg_reload_conf();'); + +/* trigger function that is called after any change */ +CREATE OR REPLACE FUNCTION record_change() +RETURNS trigger +AS $$ +DECLARE + last_change_id bigint; +BEGIN + IF (TG_OP = 'DELETE') THEN + /* get the last change ID for object key in OLD via index(-only) scan */ + SELECT change_id INTO last_change_id + FROM distributed_triggers.data_changes + WHERE shard_key_value = OLD.shard_key_value AND object_id = OLD.object_id + ORDER BY change_id DESC LIMIT 1; + + /* insert a change record for the delete */ + INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type) + VALUES (OLD.shard_key_value, OLD.object_id, COALESCE(last_change_id + 1, 1), TG_OP); + ELSE + /* get the last change ID for object key in NEW via index(-only) scan */ + SELECT change_id INTO last_change_id + FROM distributed_triggers.data_changes + WHERE shard_key_value = NEW.shard_key_value AND object_id = NEW.object_id + ORDER BY change_id DESC LIMIT 1; + + /* insert a change record for the insert/update */ + INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type, new_value) + VALUES (NEW.shard_key_value, NEW.object_id, COALESCE(last_change_id + 1, 1), TG_OP, NEW.value); + END IF; + + RETURN NULL; +END; +$$ LANGUAGE plpgsql; + +SELECT proname from pg_proc WHERE oid='distributed_triggers.record_change'::regproc; +SELECT run_command_on_workers($$SELECT count(*) FROM pg_proc WHERE oid='distributed_triggers.record_change'::regproc$$); + +CREATE TRIGGER record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.record_change(); + +-- Trigger function should appear on workers +SELECT proname from pg_proc WHERE oid='distributed_triggers.record_change'::regproc; +SELECT run_command_on_workers($$SELECT count(*) FROM pg_proc WHERE oid='distributed_triggers.record_change'::regproc$$); + +INSERT INTO data VALUES ('hello','world','{"hello":"world"}'); +INSERT INTO data VALUES ('hello2','world2','{"hello2":"world2"}'); +INSERT INTO data VALUES ('hello3','world3','{"hello3":"world3"}'); +DELETE FROM data where shard_key_value = 'hello'; +BEGIN; +UPDATE data SET value = '{}' where shard_key_value = 'hello3'; +END; +DELETE FROM data where shard_key_value = 'hello3'; + +SELECT * FROM data +ORDER BY shard_key_value, object_id; +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; + +CREATE FUNCTION insert_delete_document(key text, id text) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO distributed_triggers.data VALUES (key, id, '{"id1":"id2"}'); + DELETE FROM distributed_triggers.data where shard_key_value = key; +END; +$fn$; + +SELECT create_distributed_function( + 'insert_delete_document(text, text)', 'key', + colocate_with := 'data', + force_delegation := true +); + +SELECT insert_delete_document('hello4', 'world4'); +BEGIN; +SELECT insert_delete_document('hello4', 'world4'); +COMMIT; + +SELECT * FROM data +ORDER BY shard_key_value, object_id; +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; + +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_change_trigger%' ORDER BY 1,2; +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_change_trigger%';$$); + +ALTER TRIGGER "record_change_trigger" ON "distributed_triggers"."data" RENAME TO "new_record_change_trigger"; + +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_change_trigger%' ORDER BY 1,2; +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'new_record_change_trigger%' ORDER BY 1,2; +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_change_trigger%';$$); +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'new_record_change_trigger%';$$); + +--This should fail +DROP TRIGGER record_change_trigger ON data; +DROP TRIGGER new_record_change_trigger ON data; +--Trigger should go away +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'new_record_change_trigger%' ORDER BY 1,2; + +-- +-- Run bad triggers +-- +CREATE OR REPLACE FUNCTION bad_shardkey_record_change() +RETURNS trigger +AS $$ +DECLARE + last_change_id bigint; +BEGIN + INSERT INTO distributed_triggers.data_changes (shard_key_value, object_id, change_id, operation_type, new_value) + VALUES ('BAD', NEW.object_id, COALESCE(last_change_id + 1, 1), TG_OP, NEW.value); + RETURN NULL; +END; +$$ LANGUAGE plpgsql; + +CREATE TRIGGER bad_shardkey_record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.bad_shardkey_record_change(); + +-- Bad trigger fired from an individual SQL +-- Query-on-distributed table exception should catch this +INSERT INTO data VALUES ('hello6','world6','{"hello6":"world6"}'); + +-- Bad trigger fired from SQL inside a force-delegated function +-- Incorrect distribution key exception should catch this +SELECT insert_delete_document('hello6', 'world6'); + +SELECT * FROM data +ORDER BY shard_key_value, object_id; +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; +DROP TRIGGER bad_shardkey_record_change_trigger ON data; + +CREATE OR REPLACE FUNCTION remote_shardkey_record_change() +RETURNS trigger +AS $$ +DECLARE + last_change_id bigint; +BEGIN + UPDATE distributed_triggers.data_changes SET operation_type = TG_OP; + RETURN NULL; +END; +$$ LANGUAGE plpgsql; + +CREATE TRIGGER remote_shardkey_record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.remote_shardkey_record_change(); + +CREATE FUNCTION insert_document(key text, id text) +RETURNS void LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO distributed_triggers.data VALUES (key, id, '{"id1":"id2"}'); + DELETE FROM distributed_triggers.data where shard_key_value = key; +END; +$fn$; + +SELECT create_distributed_function( + 'insert_document(text, text)', 'key', + colocate_with := 'data', + force_delegation := false +); + +BEGIN; +SELECT insert_document('hello7', 'world7'); +END; + +SELECT insert_document('hello7', 'world7'); + +SELECT * FROM data +ORDER BY shard_key_value, object_id; +SELECT shard_key_value, object_id, change_id, operation_type, new_value +FROM data_changes +ORDER BY shard_key_value, object_id, change_id; + +-- +-- Triggers (tables) which are not colocated +-- +CREATE TABLE emptest ( + empname text NOT NULL, + salary integer +); + +CREATE TABLE emptest_audit( + operation char(1) NOT NULL, + stamp timestamp NOT NULL, + userid text NOT NULL, + empname text NOT NULL, + salary integer +); + +SELECT create_distributed_table('emptest','empname',colocate_with :='none'); +SELECT create_distributed_table('emptest_audit','empname',colocate_with :='none'); + +CREATE OR REPLACE FUNCTION process_emp_audit() RETURNS TRIGGER AS $emp_audit$ + BEGIN + -- + -- Create a row in emp_audit to reflect the operation performed on emp, + -- making use of the special variable TG_OP to work out the operation. + -- + IF (TG_OP = 'DELETE') THEN + INSERT INTO distributed_triggers.emptest_audit SELECT 'D', now(), user, OLD.*; + ELSIF (TG_OP = 'UPDATE') THEN + INSERT INTO distributed_triggers.emptest_audit SELECT 'U', now(), user, NEW.*; + ELSIF (TG_OP = 'INSERT') THEN + INSERT INTO distributed_triggers.emptest_audit SELECT 'I', now(), user, NEW.*; + END IF; + RETURN NULL; -- result is ignored since this is an AFTER trigger + END; +$emp_audit$ LANGUAGE plpgsql; + +CREATE TRIGGER emptest_audit +AFTER INSERT OR UPDATE OR DELETE ON emptest + FOR EACH ROW EXECUTE FUNCTION distributed_triggers.process_emp_audit(); + +INSERT INTO emptest VALUES ('test1', 1); +INSERT INTO emptest VALUES ('test2', 1); +INSERT INTO emptest VALUES ('test3', 1); +INSERT INTO emptest VALUES ('test4', 1); + +SELECT operation, userid, empname, salary +FROM emptest_audit +ORDER BY 3,1; + +DELETE from emptest; + +SELECT operation, userid, empname, salary +FROM emptest_audit +ORDER BY 3,1; + +CREATE VIEW emp_triggers AS + SELECT tgname, tgrelid::regclass, tgenabled + FROM pg_trigger + WHERE tgrelid::regclass::text like 'emptest%' + ORDER BY 1, 2; +SELECT * FROM emp_triggers ORDER BY 1,2; + +-- Triggers "FOR EACH STATEMENT" +CREATE TABLE record_op ( + empname text NOT NULL, + operation_type text not null, + stamp timestamp NOT NULL +); +SELECT create_distributed_table('record_op', 'empname', colocate_with := 'emptest'); +CREATE OR REPLACE FUNCTION record_emp() RETURNS TRIGGER AS $rec_audit$ + BEGIN + INSERT INTO distributed_triggers.record_op SELECT 'dummy', TG_OP, now(); + RETURN NULL; -- result is ignored since this is an AFTER trigger + END; +$rec_audit$ LANGUAGE plpgsql; + +CREATE TRIGGER record_emp_trig +AFTER INSERT OR UPDATE OR DELETE ON emptest + FOR EACH STATEMENT EXECUTE FUNCTION distributed_triggers.record_emp(); + +INSERT INTO emptest VALUES ('test5', 1); +DELETE FROM emptest; +SELECT * FROM emptest; +SELECT operation_type FROM record_op; + +-- +-- Triggers on reference tables +-- +CREATE TABLE data_ref_table ( + shard_key_value text not null, + object_id text not null, + value jsonb not null +); +ALTER TABLE data_ref_table +ADD CONSTRAINT data_ref_pk +PRIMARY KEY (shard_key_value, object_id); +SELECT create_reference_table('data_ref_table'); + +-- Trigger function record_change operates on data_changes which is *not* colocated with the reference table +CREATE TRIGGER record_change_trigger +AFTER INSERT OR UPDATE OR DELETE ON data_ref_table +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.record_change(); + +TRUNCATE TABLE data_changes; +INSERT INTO data_ref_table VALUES ('hello','world','{"ref":"table"}'); +INSERT INTO data_ref_table VALUES ('hello2','world2','{"ref":"table"}'); +DELETE FROM data_ref_table where shard_key_value = 'hello'; +BEGIN; +UPDATE data_ref_table SET value = '{}' where shard_key_value = 'hello2'; +END; +TABLE data_changes ORDER BY shard_key_value, object_id, change_id; +TABLE data_ref_table ORDER BY shard_key_value, object_id; + +-- Colocate data_changes table with reference table +SELECT undistribute_table('data_changes'); +SELECT create_reference_table('data_changes'); + +INSERT INTO data_ref_table VALUES ('hello','world','{"ref":"table"}'); +TABLE data_changes ORDER BY shard_key_value, object_id, change_id; +TABLE data_ref_table ORDER BY shard_key_value, object_id; + +-- Create data_changes table locally with reference table +DROP TABLE data_changes; + +/* table of changes local to each placement of the reference table */ +CREATE TABLE data_changes ( + shard_key_value text not null, + object_id text not null, + change_id bigint not null, + change_time timestamptz default now(), + operation_type text not null, + new_value jsonb +); +SELECT run_command_on_workers($$CREATE TABLE distributed_triggers.data_changes( + shard_key_value text not null, + object_id text not null, + change_id bigint not null, + change_time timestamptz default now(), + operation_type text not null, + new_value jsonb); +$$); + +SELECT run_command_on_workers('SELECT count(*) FROM distributed_triggers.data_changes;'); + +INSERT INTO data_ref_table VALUES ('hello','world','{"ref":"table"}'); +INSERT INTO data_ref_table VALUES ('hello2','world2','{"ref":"table"}'); +BEGIN; +UPDATE data_ref_table SET value = '{}'; +END; + +SELECT run_command_on_workers('SELECT count(*) FROM distributed_triggers.data_changes;'); +TABLE data_ref_table ORDER BY shard_key_value, object_id; + +-- +--Triggers on partitioned tables +-- +CREATE TABLE sale(sale_date date not null, state_code text, product_sku text, units integer) +PARTITION BY list (state_code); +ALTER TABLE sale ADD CONSTRAINT sale_pk PRIMARY KEY (state_code, sale_date); +CREATE TABLE sale_newyork PARTITION OF sale FOR VALUES IN ('NY'); +CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA'); + +CREATE TABLE record_sale(operation_type text not null, product_sku text, state_code text); + +SELECT create_distributed_table('sale', 'state_code'); +SELECT create_distributed_table('record_sale', 'state_code', colocate_with := 'sale'); + +CREATE OR REPLACE FUNCTION record_sale() +RETURNS trigger +AS $$ +BEGIN + INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code) + VALUES (TG_OP, NEW.product_sku, NEW.state_code); + RETURN NULL; +END; +$$ LANGUAGE plpgsql; +CREATE TRIGGER record_sale_trigger +AFTER INSERT OR UPDATE OR DELETE ON sale +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.record_sale(); + +INSERT INTO sale(sale_date,state_code,product_sku,units) VALUES +('2019-01-01', 'CA', 'AZ-000A1', 85), +('2019-01-02', 'CA', 'AZ-000A1', 6), +('2019-01-03', 'NY', 'AZ-000A2', 83), +('2019-02-01', 'CA', 'AZ-000A2', 59), +('2019-02-02', 'CA', 'AZ-000A1', 9), +('2019-02-03', 'NY', 'AZ-000A1', 47); + +TABLE sale ORDER BY state_code, sale_date; +TABLE record_sale ORDER BY 1,2,3; + +-- +--Test ALTER TRIGGER +-- +CREATE VIEW sale_triggers AS + SELECT tgname, tgrelid::regclass, tgenabled + FROM pg_trigger + WHERE tgrelid::regclass::text like 'sale%' + ORDER BY 1, 2; + +SELECT * FROM sale_triggers ORDER BY 1,2; +ALTER TRIGGER "record_sale_trigger" ON "distributed_triggers"."sale" RENAME TO "new_record_sale_trigger"; +SELECT * FROM sale_triggers ORDER BY 1,2; + +CREATE EXTENSION seg; +ALTER TRIGGER "emptest_audit" ON "emptest" DEPENDS ON EXTENSION seg; + +DROP TABLE data_ref_table; +-- +--Triggers with add/remove node +-- +SELECT * FROM master_drain_node('localhost', :worker_2_port); +SELECT 1 from master_remove_node('localhost', :worker_2_port); + +CREATE TABLE distributed_table(value int); +CREATE TABLE distributed_table_change(value int); + +SELECT create_distributed_table('distributed_table', 'value', colocate_with => 'none'); +SELECT create_distributed_table('distributed_table_change', 'value', colocate_with => 'distributed_table'); + +CREATE FUNCTION insert_99() RETURNS trigger AS $insert_99$ +BEGIN + INSERT INTO distributed_triggers.distributed_table_change VALUES (99); + RETURN NEW; +END; +$insert_99$ LANGUAGE plpgsql; + +CREATE TRIGGER insert_99_trigger +AFTER DELETE ON distributed_table +FOR EACH ROW EXECUTE FUNCTION distributed_triggers.insert_99(); + +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'insert_99_trigger%' ORDER BY 1,2; +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'insert_99_trigger%'$$); + +INSERT INTO distributed_table VALUES (99); +DELETE FROM distributed_table; +SELECT * FROM distributed_table_change; + +-- add the node back +SELECT 1 from master_add_node('localhost', :worker_2_port); +INSERT INTO distributed_table VALUES (99); +DELETE FROM distributed_table; +SELECT * FROM distributed_table_change; + +SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'insert_99_trigger%' ORDER BY 1,2; +SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'insert_99_trigger%'$$); + +RESET client_min_messages; +SET citus.enable_unsafe_triggers TO false; +SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO false;'); +SELECT run_command_on_workers('SELECT pg_reload_conf();'); +SET citus.log_remote_commands TO off; + +DROP SCHEMA distributed_triggers CASCADE; diff --git a/src/test/regress/sql/follower_single_node.sql b/src/test/regress/sql/follower_single_node.sql index 4cd7d9d0a..482c0b575 100644 --- a/src/test/regress/sql/follower_single_node.sql +++ b/src/test/regress/sql/follower_single_node.sql @@ -107,10 +107,10 @@ SELECT * FROM test WHERE x = 1; \c -reuse-previous=off regression - - :master_port SET search_path TO single_node; -SELECT 1 FROM master_add_node('localhost', :follower_master_port, groupid => 0, noderole => 'secondary'); +SELECT 1 FROM master_add_node('localhost', :follower_master_port, groupid => 0, noderole => 'secondary', nodecluster => 'second-cluster'); SELECT 1 FROM master_set_node_property('localhost', :master_port, 'shouldhaveshards', true); -\c "port=9070 dbname=regression options='-c\ citus.use_secondary_nodes=always'" +\c "port=9070 dbname=regression options='-c\ citus.use_secondary_nodes=always\ -c\ citus.cluster_name=second-cluster'" SET search_path TO single_node; SELECT * FROM test WHERE x = 1; @@ -169,6 +169,29 @@ INSERT INTO columnar_test(a, b) VALUES (1, 8); \c - - - :follower_master_port SELECT * FROM columnar_test ORDER BY 1,2; + +\c -reuse-previous=off regression - - :master_port +SET citus.shard_replication_factor TO 1; +SET search_path TO single_node; + +CREATE TABLE dist_table (a INT, b INT); +SELECT create_distributed_table ('dist_table', 'a', shard_count:=4); +INSERT INTO dist_table VALUES (1, 1); + +\c "port=9070 dbname=regression options='-c\ citus.use_secondary_nodes=always\ -c\ citus.cluster_name=second-cluster'" +SET search_path TO single_node; + +SELECT * FROM dist_table; + +SELECT global_pid AS follower_coordinator_gpid FROM get_all_active_transactions() WHERE process_id = pg_backend_pid() \gset +SELECT pg_typeof(:follower_coordinator_gpid); + +SELECT pg_cancel_backend(:follower_coordinator_gpid); + +SET citus.log_remote_commands TO ON; +SELECT pg_cancel_backend(:follower_coordinator_gpid) FROM dist_table WHERE a = 1; + + -- Cleanup \c -reuse-previous=off regression - - :master_port SET search_path TO single_node; diff --git a/src/test/regress/sql/global_cancel.sql b/src/test/regress/sql/global_cancel.sql new file mode 100644 index 000000000..edf380771 --- /dev/null +++ b/src/test/regress/sql/global_cancel.sql @@ -0,0 +1,51 @@ +CREATE SCHEMA global_cancel; +SET search_path TO global_cancel; +SET citus.next_shard_id TO 56789000; + +CREATE TABLE dist_table (a INT, b INT); +SELECT create_distributed_table ('dist_table', 'a', shard_count:=4); +INSERT INTO dist_table VALUES (1, 1); + +SELECT global_pid AS coordinator_gpid FROM get_all_active_transactions() WHERE process_id = pg_backend_pid() \gset +SELECT pg_typeof(:coordinator_gpid); + +SELECT pg_cancel_backend(:coordinator_gpid); + +SET citus.log_remote_commands TO ON; +SELECT pg_cancel_backend(:coordinator_gpid) FROM dist_table WHERE a = 1; + +BEGIN; +SELECT pg_cancel_backend(:coordinator_gpid) FROM dist_table WHERE a = 1; +END; + +SET citus.log_remote_commands TO OFF; + +SELECT global_pid AS maintenance_daemon_gpid +FROM pg_stat_activity psa JOIN get_all_active_transactions() gaat ON psa.pid = gaat.process_id +WHERE application_name = 'Citus Maintenance Daemon' \gset + +SET client_min_messages TO ERROR; +CREATE USER global_cancel_user; +SELECT 1 FROM run_command_on_workers('CREATE USER global_cancel_user'); +RESET client_min_messages; + +SET ROLE global_cancel_user; + +SELECT pg_typeof(:maintenance_daemon_gpid); + +SELECT pg_cancel_backend(:maintenance_daemon_gpid); +SELECT pg_terminate_backend(:maintenance_daemon_gpid); + +RESET ROLE; + +SELECT nodeid AS coordinator_node_id FROM pg_dist_node WHERE nodeport = :master_port \gset + +SET client_min_messages TO DEBUG; + +-- 10000000000 is the node id multiplier for global pid +SELECT pg_cancel_backend(10000000000 * :coordinator_node_id + 0); +SELECT pg_terminate_backend(10000000000 * :coordinator_node_id + 0); + +RESET client_min_messages; + +DROP SCHEMA global_cancel CASCADE; diff --git a/src/test/regress/sql/multi_mx_schema_support.sql b/src/test/regress/sql/multi_mx_schema_support.sql index 555794f92..80cc61311 100644 --- a/src/test/regress/sql/multi_mx_schema_support.sql +++ b/src/test/regress/sql/multi_mx_schema_support.sql @@ -333,8 +333,25 @@ SELECT table_schema AS "Shards' Schema" FROM information_schema.tables WHERE table_name LIKE 'table\_set\_schema\_%' GROUP BY table_schema; + +-- check that we can drop a user-defined schema from workers +SET citus.enable_ddl_propagation TO OFF; +CREATE SCHEMA localschema; +RESET citus.enable_ddl_propagation; +DROP SCHEMA localschema; + \c - - - :master_port SELECT * FROM mx_new_schema.table_set_schema; +-- verify local schema does not exist on the worker +-- worker errors out as "schema does not exist" +SET citus.enable_ddl_propagation TO OFF; +CREATE SCHEMA localschema; + +-- should error out +SELECT run_command_on_workers($$DROP SCHEMA localschema;$$); + +RESET citus.enable_ddl_propagation; + DROP SCHEMA mx_old_schema CASCADE; DROP SCHEMA mx_new_schema CASCADE;