diff --git a/src/backend/distributed/Makefile b/src/backend/distributed/Makefile index 0f1afbb94..2d2625d19 100644 --- a/src/backend/distributed/Makefile +++ b/src/backend/distributed/Makefile @@ -10,7 +10,7 @@ EXTENSION = citus DATA = $(patsubst $(citus_abs_srcdir)/%.sql,%.sql,$(wildcard $(citus_abs_srcdir)/$(EXTENSION)--*.sql)) # directories with source files -SUBDIRS = . commands connection ddl executor master metadata planner progress relay test transaction utils worker +SUBDIRS = . commands connection ddl deparser executor master metadata planner progress relay test transaction utils worker # That patsubst rule searches all directories listed in SUBDIRS for .c # files, and adds the corresponding .o files to OBJS diff --git a/src/backend/distributed/citus--8.3-1--8.4-1.sql b/src/backend/distributed/citus--8.3-1--8.4-1.sql index 34d317fbe..04265f323 100644 --- a/src/backend/distributed/citus--8.3-1--8.4-1.sql +++ b/src/backend/distributed/citus--8.3-1--8.4-1.sql @@ -42,6 +42,13 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_unmark_object_distributed(classid o COMMENT ON FUNCTION pg_catalog.master_unmark_object_distributed(classid oid, objid oid, objsubid int) IS 'remove an object address from citus.pg_dist_object once the object has been deleted'; +CREATE OR REPLACE FUNCTION pg_catalog.worker_create_or_replace_object(statement text) + RETURNS bool + LANGUAGE C STRICT + AS 'MODULE_PATHNAME', $$worker_create_or_replace_object$$; +COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statement text) + IS 'takes a sql CREATE statement, before executing the create it will check if an object with that name already exists and safely replaces that named object with the new object'; + CREATE TABLE citus.pg_dist_object ( classid oid NOT NULL, objid oid NOT NULL, diff --git a/src/backend/distributed/commands/dependencies.c b/src/backend/distributed/commands/dependencies.c index 18c336b5a..29b4a6cfa 100644 --- a/src/backend/distributed/commands/dependencies.c +++ b/src/backend/distributed/commands/dependencies.c @@ -19,7 +19,9 @@ #include "distributed/metadata_sync.h" #include "distributed/remote_commands.h" #include "distributed/worker_manager.h" +#include "distributed/worker_transaction.h" #include "storage/lmgr.h" +#include "utils/lsyscache.h" static List * GetDependencyCreateDDLCommands(const ObjectAddress *dependency); @@ -42,10 +44,9 @@ static List * GetDependencyCreateDDLCommands(const ObjectAddress *dependency); void EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target) { - const uint32 connectionFlag = FORCE_NEW_CONNECTION; - /* local variables to work with dependencies */ List *dependencies = NIL; + List *dependenciesWithCommands = NIL; ListCell *dependencyCell = NULL; /* local variables to collect ddl commands */ @@ -54,8 +55,6 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target) /* local variables to work with worker nodes */ List *workerNodeList = NULL; ListCell *workerNodeCell = NULL; - List *connections = NULL; - ListCell *connectionCell = NULL; /* * collect all dependencies in creation order and get their ddl commands @@ -64,8 +63,14 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target) foreach(dependencyCell, dependencies) { ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); - ddlCommands = list_concat(ddlCommands, - GetDependencyCreateDDLCommands(dependency)); + List *dependencyCommands = GetDependencyCreateDDLCommands(dependency); + ddlCommands = list_concat(ddlCommands, dependencyCommands); + + /* create a new list with dependencies that actually created commands */ + if (list_length(dependencyCommands) > 0) + { + dependenciesWithCommands = lappend(dependenciesWithCommands, dependency); + } } if (list_length(ddlCommands) <= 0) { @@ -73,6 +78,9 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target) return; } + /* since we are executing ddl commands lets disable propagation, primarily for mx */ + ddlCommands = list_concat(list_make1(DISABLE_DDL_PROPAGATION), ddlCommands); + /* * Make sure that no new nodes are added after this point until the end of the * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the @@ -93,7 +101,7 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target) * to the nodes before marking the objects as distributed these objects would never be * created on the workers when they get added, causing shards to fail to create. */ - foreach(dependencyCell, dependencies) + foreach(dependencyCell, dependenciesWithCommands) { ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); MarkObjectDistributed(dependency); @@ -108,37 +116,18 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target) /* no nodes to execute on */ return; } + + foreach(workerNodeCell, workerNodeList) { WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); - MultiConnection *connection = NULL; - char *nodeName = workerNode->workerName; + const char *nodeName = workerNode->workerName; uint32 nodePort = workerNode->workerPort; - connection = StartNodeUserDatabaseConnection(connectionFlag, nodeName, nodePort, - CitusExtensionOwnerName(), NULL); - - connections = lappend(connections, connection); - } - FinishConnectionListEstablishment(connections); - - /* - * create dependency on all nodes - */ - foreach(connectionCell, connections) - { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - ExecuteCriticalRemoteCommandList(connection, ddlCommands); - } - - /* - * disconnect from nodes - */ - foreach(connectionCell, connections) - { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - CloseConnection(connection); + SendCommandListToWorkerInSingleTransaction(nodeName, nodePort, + CitusExtensionOwnerName(), + ddlCommands); } } @@ -165,6 +154,26 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency) return list_make1((void *) schemaDDLCommand); } + case OCLASS_TYPE: + { + return CreateTypeDDLCommandsIdempotent(dependency); + } + + case OCLASS_CLASS: + { + /* + * types have an intermediate dependency on a relation (aka class), so we do + * support classes when the relkind is composite + */ + if (get_rel_relkind(dependency->objectId) == RELKIND_COMPOSITE_TYPE) + { + return NIL; + } + + /* if this relation is not supported, break to the error at the end */ + break; + } + default: { break; @@ -181,7 +190,6 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency) errdetail( "citus tries to recreate an unsupported object on its workers"), errhint("please report a bug as this should not be happening"))); - return NIL; } @@ -191,11 +199,9 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency) void ReplicateAllDependenciesToNode(const char *nodeName, int nodePort) { - const uint32 connectionFlag = FORCE_NEW_CONNECTION; ListCell *dependencyCell = NULL; List *dependencies = NIL; List *ddlCommands = NIL; - MultiConnection *connection = NULL; /* * collect all dependencies in creation order and get their ddl commands @@ -230,11 +236,9 @@ ReplicateAllDependenciesToNode(const char *nodeName, int nodePort) return; } - /* - * connect to the new host and create all applicable dependencies - */ - connection = GetNodeUserDatabaseConnection(connectionFlag, nodeName, nodePort, - CitusExtensionOwnerName(), NULL); - ExecuteCriticalRemoteCommandList(connection, ddlCommands); - CloseConnection(connection); + /* since we are executing ddl commands lets disable propagation, primarily for mx */ + ddlCommands = list_concat(list_make1(DISABLE_DDL_PROPAGATION), ddlCommands); + + SendCommandListToWorkerInSingleTransaction(nodeName, nodePort, + CitusExtensionOwnerName(), ddlCommands); } diff --git a/src/backend/distributed/commands/schema.c b/src/backend/distributed/commands/schema.c index a1b8dc258..72dbd195b 100644 --- a/src/backend/distributed/commands/schema.c +++ b/src/backend/distributed/commands/schema.c @@ -104,25 +104,58 @@ ProcessDropSchemaStmt(DropStmt *dropStatement) /* - * PlanAlterObjectSchemaStmt determines whether a given ALTER ... SET SCHEMA + * PlanAlterObjectSchemaStmt is called by citus' utility hook for AlterObjectSchemaStmt + * parsetrees. It dispatches the statement based on the object type for which the schema + * is being altered. + * + * A (potentially empty) list of DDLJobs is being returned with the jobs on how to + * distribute the change into the cluster. + */ +List * +PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + return PlanAlterTypeSchemaStmt(stmt, queryString); + } + + default: + { + /* do nothing for unsupported objects */ + break; + } + } + + /* + * old behaviour, needs to be reconciled to the above switch statement for all + * objectType's relating to tables. Maybe it is as easy to support + * ALTER TABLE ... SET SCHEMA + */ + return PlanAlterTableSchemaStmt(stmt, queryString); +} + + +/* + * PlanAlterTableSchemaStmt determines whether a given ALTER ... SET SCHEMA * statement involves a distributed table and issues a warning if so. Because * we do not support distributed ALTER ... SET SCHEMA, this function always * returns NIL. */ List * -PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, - const char *alterObjectSchemaCommand) +PlanAlterTableSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString) { Oid relationId = InvalidOid; - if (alterObjectSchemaStmt->relation == NULL) + if (stmt->relation == NULL) { return NIL; } - relationId = RangeVarGetRelid(alterObjectSchemaStmt->relation, + relationId = RangeVarGetRelid(stmt->relation, AccessExclusiveLock, - alterObjectSchemaStmt->missing_ok); + stmt->missing_ok); /* first check whether a distributed relation is affected */ if (!OidIsValid(relationId) || !IsDistributedTable(relationId)) @@ -138,3 +171,29 @@ PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, return NIL; } + + +/* + * ProcessAlterObjectSchemaStmt is called by multi_ProcessUtility _after_ the command has + * been applied to the local postgres. It is useful to create potentially new dependencies + * of this object (the new schema) on the workers before the command gets applied to the + * remote objects. + */ +void +ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + ProcessAlterTypeSchemaStmt(stmt, queryString); + return; + } + + default: + { + /* do nothing for unsupported objects */ + return; + } + } +} diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index 8093a0c9a..f1068e853 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -613,6 +613,19 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement) { List *commandList = alterTableStatement->cmds; ListCell *commandCell = NULL; + LOCKMODE lockmode = NoLock; + Oid relationId = InvalidOid; + + lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); + relationId = AlterTableLookupRelation(alterTableStatement, lockmode); + + if (relationId != InvalidOid) + { + /* changing a relation could introduce new dependencies */ + ObjectAddress tableAddress = { 0 }; + ObjectAddressSet(tableAddress, RelationRelationId, relationId); + EnsureDependenciesExistsOnAllNodes(&tableAddress); + } foreach(commandCell, commandList) { @@ -621,17 +634,12 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement) if (alterTableType == AT_AddConstraint) { - LOCKMODE lockmode = NoLock; - Oid relationId = InvalidOid; Constraint *constraint = NULL; Assert(list_length(commandList) == 1); ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement); - lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); - relationId = AlterTableLookupRelation(alterTableStatement, lockmode); - if (!OidIsValid(relationId)) { continue; @@ -647,8 +655,6 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement) { List *columnConstraints = NIL; ListCell *columnConstraint = NULL; - Oid relationId = InvalidOid; - LOCKMODE lockmode = NoLock; ColumnDef *columnDefinition = (ColumnDef *) command->def; columnConstraints = columnDefinition->constraints; @@ -657,8 +663,6 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement) ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement); } - lockmode = AlterTableGetLockLevel(alterTableStatement->cmds); - relationId = AlterTableLookupRelation(alterTableStatement, lockmode); if (!OidIsValid(relationId)) { continue; diff --git a/src/backend/distributed/commands/type.c b/src/backend/distributed/commands/type.c new file mode 100644 index 000000000..c95a9c2fa --- /dev/null +++ b/src/backend/distributed/commands/type.c @@ -0,0 +1,1415 @@ +/*------------------------------------------------------------------------- + * + * type.c + * Commands for TYPE statements. + * The following types are supported in citus + * - Composite Types + * - Enum Types + * - Array Types + * + * Types that are currently not supporter: + * - Range Types + * - Base Types + * + * Range types have a dependency on functions. We can only support Range + * types after we have function distribution sorted. + * + * Base types are more complex and often involve c code from extensions. + * These types should be created by creating the extension on all the + * workers as well. Therefore types created during the creation of an + * extension are not propagated to the worker nodes. + * + * Types will be created on the workers during the following situations: + * - on type creation (except if called in a transaction) + * By not distributing types directly when in a transaction allows + * the type to be used in a newly created table that will be + * distributed in the same transaction. In that case the type will be + * created just-in-time to allow citus' parallelism to work. + * - just-in-time + * When the type is not already distributed but used in an object + * that will distribute now. This allows distributed tables to use + * types that have not yet been propagated, either due to the + * transaction case abvove, or due to a type predating the citus + * extension. + * - node activation + * Together with all objects that are marked as distributed in citus + * types will be created during the activation of a new node to allow + * reference tables to use this type. + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "access/genam.h" +#include "access/htup_details.h" +#include "access/xact.h" +#include "catalog/namespace.h" +#include "catalog/pg_enum.h" +#include "catalog/pg_type.h" +#include "commands/extension.h" +#include "distributed/commands.h" +#include "distributed/commands/utility_hook.h" +#include "distributed/deparser.h" +#include "distributed/metadata/distobject.h" +#include "distributed/metadata/namespace.h" +#include "distributed/metadata_sync.h" +#include "distributed/multi_executor.h" +#include "distributed/relation_access_tracking.h" +#include "distributed/remote_commands.h" +#include "distributed/transaction_management.h" +#include "distributed/worker_manager.h" +#include "distributed/worker_transaction.h" +#include "miscadmin.h" +#include "nodes/makefuncs.h" +#include "parser/parse_type.h" +#include "storage/lmgr.h" +#include "utils/builtins.h" +#include "utils/fmgroids.h" +#include "utils/lsyscache.h" +#include "utils/regproc.h" +#include "utils/syscache.h" +#include "utils/typcache.h" + +#define AlterEnumIsRename(stmt) (stmt->oldVal != NULL) +#define AlterEnumIsAddValue(stmt) (stmt->oldVal == NULL) + + +#define ALTER_TYPE_OWNER_COMMAND "ALTER TYPE %s OWNER TO %s;" +#define CREATE_OR_REPLACE_COMMAND "SELECT worker_create_or_replace_object(%s);" + + +/* forward declaration for helper functions*/ +static List * FilterNameListForDistributedTypes(List *objects, bool missing_ok); +static List * TypeNameListToObjectAddresses(List *objects); +static TypeName * MakeTypeNameFromRangeVar(const RangeVar *relation); +static void EnsureSequentialModeForTypeDDL(void); +static Oid GetTypeOwner(Oid typeOid); +static const char * WrapCreateOrReplace(const char *sql); + +/* recreate functions */ +static CompositeTypeStmt * RecreateCompositeTypeStmt(Oid typeOid); +static List * CompositeTypeColumnDefList(Oid typeOid); +static CreateEnumStmt * RecreateEnumStmt(Oid typeOid); +static List * EnumValsList(Oid typeOid); + +static bool ShouldPropagateTypeCreate(void); + + +/* + * PlanCompositeTypeStmt is called during the creation of a composite type. It is executed + * before the statement is applied locally. + * + * We decide if the compisite type needs to be replicated to the worker, and if that is + * the case return a list of DDLJob's that describe how and where the type needs to be + * created. + * + * Since the planning happens before the statement has been applied locally we do not have + * access to the ObjectAddress of the new type. + */ +List * +PlanCompositeTypeStmt(CompositeTypeStmt *stmt, const char *queryString) +{ + const char *compositeTypeStmtSql = NULL; + List *commands = NIL; + + if (!ShouldPropagateTypeCreate()) + { + return NIL; + } + + /* + * managing types can only be done on the coordinator if ddl propagation is on. when + * it is off we will never get here + */ + EnsureCoordinator(); + + /* + * Make sure that no new nodes are added after this point until the end of the + * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the + * ExclusiveLock taken by master_add_node. + * This guarantees that all active nodes will have the object, because they will + * either get it now, or get it in master_add_node after this transaction finishes and + * the pg_dist_object record becomes visible. + */ + LockRelationOid(DistNodeRelationId(), RowShareLock); + + /* fully qualify before lookup and later deparsing */ + QualifyTreeNode((Node *) stmt); + + /* + * reconstruct creation statement in a portable fashion. The create_or_replace helper + * function will be used to create the type in an idempotent manner on the workers. + * + * Types could exist on the worker prior to being created on the coordinator when the + * type previously has been attempted to be created in a transaction which did not + * commit on the coordinator. + */ + compositeTypeStmtSql = DeparseCompositeTypeStmt(stmt); + compositeTypeStmtSql = WrapCreateOrReplace(compositeTypeStmtSql); + + /* + * when we allow propagation within a transaction block we should make sure to only + * allow this in sequential mode + */ + EnsureSequentialModeForTypeDDL(); + + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) compositeTypeStmtSql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * ProcessCompositeTypeStmt is executed after the type has been created locally and before + * we create it on the remote servers. Here we have access to the ObjectAddress of the new + * type which we use to make sure the type's dependencies are on all nodes. + */ +void +ProcessCompositeTypeStmt(CompositeTypeStmt *stmt, const char *queryString) +{ + const ObjectAddress *typeAddress = NULL; + + /* same check we perform during planning of the statement */ + if (!ShouldPropagateTypeCreate()) + { + return; + } + + /* + * find object address of the just created object, because the type has been created + * locally it can't be missing + */ + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + EnsureDependenciesExistsOnAllNodes(typeAddress); + + MarkObjectDistributed(typeAddress); +} + + +/* + * PlanAlterTypeStmt is invoked for alter type statements for composite types. + * + * Normally we would have a process step as well to re-ensure dependencies exists, however + * this is already implemented by the post processing for adding columns to tables. + */ +List * +PlanAlterTypeStmt(AlterTableStmt *stmt, const char *queryString) +{ + const char *alterTypeStmtSql = NULL; + const ObjectAddress *typeAddress = NULL; + List *commands = NIL; + + Assert(stmt->relkind == OBJECT_TYPE); + + if (creating_extension) + { + /* + * extensions should be created separately on the workers, types cascading from an + * extension should therefor not be propagated. + */ + return NIL; + } + + /* + * Only distributed types should be propagated + */ + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(typeAddress)) + { + return NIL; + } + + EnsureCoordinator(); + + /* reconstruct alter statement in a portable fashion */ + QualifyTreeNode((Node *) stmt); + alterTypeStmtSql = DeparseTreeNode((Node *) stmt); + + /* + * all types that are distributed will need their alter statements propagated + * regardless if in a transaction or not. If we would not propagate the alter + * statement the types would be different on worker and coordinator. + */ + EnsureSequentialModeForTypeDDL(); + + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) alterTypeStmtSql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * PlanCreateEnumStmt is called before the statement gets applied locally. + * + * It decides if the create statement will be applied to the workers and if that is the + * case returns a list of DDLJobs that will be executed _after_ the statement has been + * applied locally. + * + * Since planning is done before we have created the object locally we do not have an + * ObjectAddress for the new type just yet. + */ +List * +PlanCreateEnumStmt(CreateEnumStmt *stmt, const char *queryString) +{ + const char *createEnumStmtSql = NULL; + List *commands = NIL; + + if (!ShouldPropagateTypeCreate()) + { + return NIL; + } + + /* + * managing types can only be done on the coordinator if ddl propagation is on. when + * it is off we will never get here + */ + EnsureCoordinator(); + + /* enforce fully qualified typeName for correct deparsing and lookup */ + QualifyTreeNode((Node *) stmt); + + /* reconstruct creation statement in a portable fashion */ + createEnumStmtSql = DeparseCreateEnumStmt(stmt); + createEnumStmtSql = WrapCreateOrReplace(createEnumStmtSql); + + /* + * when we allow propagation within a transaction block we should make sure to only + * allow this in sequential mode + */ + EnsureSequentialModeForTypeDDL(); + + /* to prevent recursion with mx we disable ddl propagation */ + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) createEnumStmtSql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * ProcessCreateEnumStmt is called after the statement has been applied locally, but + * before the plan on how to create the types on the workers has been executed. + * + * We apply the same checks to verify if the type should be distributed, if that is the + * case we resolve the ObjectAddress for the just created object, distribute its + * dependencies to all the nodes, and mark the object as distributed. + */ +void +ProcessCreateEnumStmt(CreateEnumStmt *stmt, const char *queryString) +{ + const ObjectAddress *typeAddress = NULL; + + if (!ShouldPropagateTypeCreate()) + { + return; + } + + /* lookup type address of just created type */ + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + EnsureDependenciesExistsOnAllNodes(typeAddress); + + /* + * now that the object has been created and distributed to the workers we mark them as + * distributed so we know to keep them up to date and recreate on a new node in the + * future + */ + MarkObjectDistributed(typeAddress); +} + + +/* + * PlanAlterEnumStmt handles ALTER TYPE ... ADD VALUE for enum based types. Planning + * happens before the statement has been applied locally. + * + * Since it is an alter of an existing type we actually have the ObjectAddress. This is + * used to check if the type is distributed, if so the alter will be executed on the + * workers directly to keep the types in sync accross the cluster. + */ +List * +PlanAlterEnumStmt(AlterEnumStmt *stmt, const char *queryString) +{ + const char *alterEnumStmtSql = NULL; + const ObjectAddress *typeAddress = NULL; + List *commands = NIL; + + if (creating_extension) + { + /* + * extensions should be created separately on the workers, types cascading from an + * extension should therefor not be propagated here. + */ + return NIL; + } + + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(typeAddress)) + { + return NIL; + } + + /* + * alter enum will run for all distributed enums, regardless if in a transaction or + * not since the enum will be different on the coordinator and workers if we didn't. + * (adding values to an enum can not run in a transaction anyway and would error by + * postgres already). + */ + EnsureSequentialModeForTypeDDL(); + + /* + * managing types can only be done on the coordinator if ddl propagation is on. when + * it is off we will never get here + */ + EnsureCoordinator(); + + QualifyTreeNode((Node *) stmt); + alterEnumStmtSql = DeparseTreeNode((Node *) stmt); + + /* TODO this is not needed anymore for pg12, alter enum can actually run in a xact */ + if (AlterEnumIsAddValue(stmt)) + { + /* + * a plan cannot be made as it will be committed via 2PC when ran through the + * executor, instead we directly distributed during processing phase + */ + return NIL; + } + + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) alterEnumStmtSql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * ProcessAlterEnumStmt is called after the AlterEnumStmt has been applied locally. + * + * This function is used for ALTER ENUM ... ADD VALUE for postgres versions lower then 12 + * to distribute the call. Before pg12 these statements could not be called in a + * transaction. If we would plan the distirbution of these statements the same as we do + * with the other statements they would get executed in a transaction to perform 2PC, that + * would error out. + * + * If it would error on some workers we provide a warning to the user that the statement + * failed to distributed with some detail on what to call after the cluster has been + * repaired. + * + * For pg12 the statements can be called in a transaction but will only become visible + * when the transaction commits. This is behaviour that is ok to perform in a 2PC. + */ +void +ProcessAlterEnumStmt(AlterEnumStmt *stmt, const char *queryString) +{ + const ObjectAddress *typeAddress = NULL; + + if (creating_extension) + { + /* + * extensions should be created separately on the workers, types cascading from an + * extension should therefor not be propagated here. + */ + return; + } + + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(typeAddress)) + { + return; + } + + /* TODO this is not needed anymore for pg12, alter enum can actually run in a xact */ + if (AlterEnumIsAddValue(stmt)) + { + /* + * ADD VALUE can't be executed in a transaction, we will execute optimistically + * and on an error we will advise to fix the issue with the worker and rerun the + * query with the IF NOT EXTISTS modifier. The modifier is needed as the value + * might already be added to some nodes, but not all. + */ + + int result = 0; + List *commands = NIL; + const char *alterEnumStmtSql = NULL; + + /* qualification of the stmt happened during planning */ + alterEnumStmtSql = DeparseTreeNode((Node *) stmt); + + commands = list_make2(DISABLE_DDL_PROPAGATION, (void *) alterEnumStmtSql); + + result = SendBareOptionalCommandListToWorkersAsUser(ALL_WORKERS, commands, NULL); + + if (result != RESPONSE_OKAY) + { + const char *alterEnumStmtIfNotExistsSql = NULL; + bool oldSkipIfNewValueExists = stmt->skipIfNewValExists; + + /* deparse the query with IF NOT EXISTS */ + stmt->skipIfNewValExists = true; + alterEnumStmtIfNotExistsSql = DeparseTreeNode((Node *) stmt); + stmt->skipIfNewValExists = oldSkipIfNewValueExists; + + ereport(WARNING, (errmsg("not all workers applied change to enum"), + errdetail("retry with: %s", alterEnumStmtIfNotExistsSql), + errhint("make sure the coordinators can communicate with " + "all workers"))); + } + } +} + + +/* + * PlanDropTypeStmt is called for all DROP TYPE statements. For all types in the list that + * citus has distributed to the workers it will drop the type on the workers as well. If + * no types in the drop list are distributed no calls will be made to the workers. + */ +List * +PlanDropTypeStmt(DropStmt *stmt, const char *queryString) +{ + /* + * We swap the list of objects to remove during deparse so we need a reference back to + * the old list to put back + */ + List *oldTypes = stmt->objects; + List *distributedTypes = NIL; + const char *dropStmtSql = NULL; + ListCell *addressCell = NULL; + List *distributedTypeAddresses = NIL; + List *commands = NIL; + + if (creating_extension) + { + /* + * extensions should be created separately on the workers, types cascading from an + * extension should therefor not be propagated here. + */ + return NIL; + } + + distributedTypes = FilterNameListForDistributedTypes(oldTypes, stmt->missing_ok); + if (list_length(distributedTypes) <= 0) + { + /* no distributed types to drop */ + return NIL; + } + + /* + * managing types can only be done on the coordinator if ddl propagation is on. when + * it is off we will never get here. MX workers don't have a notion of distributed + * types, so we block the call. + */ + EnsureCoordinator(); + + /* + * remove the entries for the distributed objects on dropping + */ + distributedTypeAddresses = TypeNameListToObjectAddresses(distributedTypes); + foreach(addressCell, distributedTypeAddresses) + { + ObjectAddress *address = (ObjectAddress *) lfirst(addressCell); + UnmarkObjectDistributed(address); + } + + /* + * temporary swap the lists of objects to delete with the distributed objects and + * deparse to an executable sql statement for the workers + */ + stmt->objects = distributedTypes; + dropStmtSql = DeparseTreeNode((Node *) stmt); + stmt->objects = oldTypes; + + /* to prevent recursion with mx we disable ddl propagation */ + EnsureSequentialModeForTypeDDL(); + + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) dropStmtSql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * PlanRenameTypeStmt is called when the user is renaming the type. The invocation happens + * before the statement is applied locally. + * + * As the type already exists we have access to the ObjectAddress for the type, this is + * used to check if the type is distributed. If the type is distributed the rename is + * executed on all the workers to keep the types in sync across the cluster. + */ +List * +PlanRenameTypeStmt(RenameStmt *stmt, const char *queryString) +{ + const char *renameStmtSql = NULL; + const ObjectAddress *typeAddress = NULL; + List *commands = NIL; + + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(typeAddress)) + { + return NIL; + } + + /* + * we should not get to a point where an alter happens on a distributed type during an + * extension statement, but better safe then sorry. + */ + if (creating_extension) + { + /* + * extensions should be created separately on the workers, types cascading from an + * extension should therefor not be propagated here. + */ + return NIL; + } + + + /* fully qualify */ + QualifyTreeNode((Node *) stmt); + + /* deparse sql*/ + renameStmtSql = DeparseTreeNode((Node *) stmt); + + /* to prevent recursion with mx we disable ddl propagation */ + EnsureSequentialModeForTypeDDL(); + + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) renameStmtSql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * PlanRenameTypeAttributeStmt is called for changes of attribute names for composite + * types. Planning is called before the statement is applied locally. + * + * For distributed types we apply the attribute renames directly on all the workers to + * keep the type in sync across the cluster. + */ +List * +PlanRenameTypeAttributeStmt(RenameStmt *stmt, const char *queryString) +{ + const char *sql = NULL; + const ObjectAddress *address = NULL; + List *commands = NIL; + + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + Assert(stmt->relationType == OBJECT_TYPE); + + address = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(address)) + { + return NIL; + } + + /* + * we should not get to a point where an alter happens on a distributed type during an + * extension statement, but better safe then sorry. + */ + if (creating_extension) + { + /* + * extensions should be created separately on the workers, types cascading from an + * extension should therefor not be propagated here. + */ + return NIL; + } + + QualifyTreeNode((Node *) stmt); + + sql = DeparseTreeNode((Node *) stmt); + + EnsureSequentialModeForTypeDDL(); + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) sql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * PlanAlterTypeSchemaStmt is executed before the statement is applied to the local + * postgres instance. + * + * In this stage we can prepare the commands that need to be run on all workers. + */ +List * +PlanAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString) +{ + const char *sql = NULL; + const ObjectAddress *typeAddress = NULL; + List *commands = NIL; + + Assert(stmt->objectType == OBJECT_TYPE); + + if (creating_extension) + { + /* types from extensions are managed by extensions, skipping */ + return NIL; + } + + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(typeAddress)) + { + /* not distributed to the workers, nothing to do */ + return NIL; + } + + EnsureCoordinator(); + + QualifyTreeNode((Node *) stmt); + sql = DeparseTreeNode((Node *) stmt); + + EnsureSequentialModeForTypeDDL(); + + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) sql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * ProcessAlterTypeSchemaStmt is executed after the change has been applied locally, we + * can now use the new dependencies of the type to ensure all its dependencies exist on + * the workers before we apply the commands remotely. + */ +void +ProcessAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString) +{ + const ObjectAddress *typeAddress = NULL; + + Assert(stmt->objectType == OBJECT_TYPE); + + if (creating_extension) + { + /* types from extensions are managed by extensions, skipping */ + return; + } + + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(typeAddress)) + { + /* not distributed to the workers, nothing to do */ + return; + } + + /* dependencies have changed (schema) lets ensure they exist */ + EnsureDependenciesExistsOnAllNodes(typeAddress); +} + + +/* + * PlanAlterTypeOwnerStmt is called for change of owner ship of types before the owner + * ship is changed on the local instance. + * + * If the type for which the owner is changed is distributed we execute the change on all + * the workers to keep the type in sync across the cluster. + */ +List * +PlanAlterTypeOwnerStmt(AlterOwnerStmt *stmt, const char *queryString) +{ + const ObjectAddress *typeAddress = NULL; + const char *sql = NULL; + List *commands = NULL; + + Assert(stmt->objectType == OBJECT_TYPE); + + if (creating_extension) + { + /* types from extensions are managed by extensions, skipping */ + return NIL; + } + + typeAddress = GetObjectAddressFromParseTree((Node *) stmt, false); + if (!IsObjectDistributed(typeAddress)) + { + return NIL; + } + + EnsureCoordinator(); + + QualifyTreeNode((Node *) stmt); + sql = DeparseTreeNode((Node *) stmt); + + EnsureSequentialModeForTypeDDL(); + commands = list_make3(DISABLE_DDL_PROPAGATION, + (void *) sql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(ALL_WORKERS, commands); +} + + +/* + * CreateTypeStmtByObjectAddress returns a parsetree for the CREATE TYPE statement to + * recreate the type by its object address. + */ +Node * +CreateTypeStmtByObjectAddress(const ObjectAddress *address) +{ + Assert(address->classId == TypeRelationId); + + switch (get_typtype(address->objectId)) + { + case TYPTYPE_ENUM: + { + return (Node *) RecreateEnumStmt(address->objectId); + } + + case TYPTYPE_COMPOSITE: + { + return (Node *) RecreateCompositeTypeStmt(address->objectId); + } + + default: + { + ereport(ERROR, (errmsg("unsupported type to generate create statement for"), + errdetail("only enum and composite types can be recreated"))); + } + } +} + + +/* + * RecreateCompositeTypeStmt is called for composite types to create its parsetree for the + * CREATE TYPE statement that would recreate the composite type. + */ +static CompositeTypeStmt * +RecreateCompositeTypeStmt(Oid typeOid) +{ + CompositeTypeStmt *stmt = NULL; + List *names = NIL; + + Assert(get_typtype(typeOid) == TYPTYPE_COMPOSITE); + + stmt = makeNode(CompositeTypeStmt); + names = stringToQualifiedNameList(format_type_be_qualified(typeOid)); + stmt->typevar = makeRangeVarFromNameList(names); + stmt->coldeflist = CompositeTypeColumnDefList(typeOid); + + return stmt; +} + + +/* + * attributeFormToColumnDef returns a ColumnDef * describing the field and its property + * for a pg_attribute entry. + * + * Note: Current implementation is only covering the features supported by composite types + */ +static ColumnDef * +attributeFormToColumnDef(Form_pg_attribute attributeForm) +{ + return makeColumnDef(NameStr(attributeForm->attname), + attributeForm->atttypid, + -1, + attributeForm->attcollation); +} + + +/* + * CompositeTypeColumnDefList returns a list of ColumnDef *'s that make up all the fields + * of the composite type. + */ +static List * +CompositeTypeColumnDefList(Oid typeOid) +{ + Relation relation = NULL; + Oid relationId = InvalidOid; + TupleDesc tupleDescriptor = NULL; + int attributeIndex = 0; + List *columnDefs = NIL; + + relationId = typeidTypeRelid(typeOid); + relation = relation_open(relationId, AccessShareLock); + + tupleDescriptor = RelationGetDescr(relation); + for (attributeIndex = 0; attributeIndex < tupleDescriptor->natts; attributeIndex++) + { + Form_pg_attribute attributeForm = TupleDescAttr(tupleDescriptor, attributeIndex); + + if (attributeForm->attisdropped) + { + /* skip logically hidden attributes */ + continue; + } + + columnDefs = lappend(columnDefs, attributeFormToColumnDef(attributeForm)); + } + + relation_close(relation, AccessShareLock); + + return columnDefs; +} + + +/* + * RecreateEnumStmt returns a parsetree for a CREATE TYPE ... AS ENUM statement that would + * recreate the given enum type. + */ +static CreateEnumStmt * +RecreateEnumStmt(Oid typeOid) +{ + CreateEnumStmt *stmt = NULL; + + Assert(get_typtype(typeOid) == TYPTYPE_ENUM); + + stmt = makeNode(CreateEnumStmt); + stmt->typeName = stringToQualifiedNameList(format_type_be_qualified(typeOid)); + stmt->vals = EnumValsList(typeOid); + + return stmt; +} + + +/* + * EnumValsList returns a list of String values containing the enum values for the given + * enum type. + */ +static List * +EnumValsList(Oid typeOid) +{ + Relation enum_rel = NULL; + SysScanDesc enum_scan = NULL; + HeapTuple enum_tuple = NULL; + ScanKeyData skey = { 0 }; + + List *vals = NIL; + + /* Scan pg_enum for the members of the target enum type. */ + ScanKeyInit(&skey, + Anum_pg_enum_enumtypid, + BTEqualStrategyNumber, F_OIDEQ, + ObjectIdGetDatum(typeOid)); + + enum_rel = heap_open(EnumRelationId, AccessShareLock); + enum_scan = systable_beginscan(enum_rel, + EnumTypIdLabelIndexId, + true, NULL, + 1, &skey); + + /* collect all value names in CREATE TYPE ... AS ENUM stmt */ + while (HeapTupleIsValid(enum_tuple = systable_getnext(enum_scan))) + { + Form_pg_enum en = (Form_pg_enum) GETSTRUCT(enum_tuple); + vals = lappend(vals, makeString(pstrdup(NameStr(en->enumlabel)))); + } + + systable_endscan(enum_scan); + heap_close(enum_rel, AccessShareLock); + return vals; +} + + +/* + * CompositeTypeStmtObjectAddress finds the ObjectAddress for the composite type described + * by the CompositeTypeStmt. If missing_ok is false this function throws an error if the + * type does not exist. + * + * Never returns NULL, but the objid in the address could be invalid if missing_ok was set + * to true. + */ +const ObjectAddress * +CompositeTypeStmtObjectAddress(CompositeTypeStmt *stmt, bool missing_ok) +{ + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + ObjectAddress *address = NULL; + + typeName = MakeTypeNameFromRangeVar(stmt->typevar); + typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * CreateEnumStmtObjectAddress finds the ObjectAddress for the enum type described by the + * CreateEnumStmt. If missing_ok is false this function throws an error if the type does + * not exist. + * + * Never returns NULL, but the objid in the address could be invalid if missing_ok was set + * to true. + */ +const ObjectAddress * +CreateEnumStmtObjectAddress(CreateEnumStmt *stmt, bool missing_ok) +{ + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + ObjectAddress *address = NULL; + + typeName = makeTypeNameFromNameList(stmt->typeName); + typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * AlterTypeStmtObjectAddress finds the ObjectAddress for the type described by the ALTER + * TYPE statement. If missing_ok is false this function throws an error if the type does + * not exist. + * + * Never returns NULL, but the objid in the address could be invalid if missing_ok was set + * to true. + */ +const ObjectAddress * +AlterTypeStmtObjectAddress(AlterTableStmt *stmt, bool missing_ok) +{ + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + ObjectAddress *address = NULL; + + Assert(stmt->relkind == OBJECT_TYPE); + + typeName = MakeTypeNameFromRangeVar(stmt->relation); + typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * AlterEnumStmtObjectAddress return the ObjectAddress of the enum type that is the + * subject of the AlterEnumStmt. Errors is missing_ok is false. + */ +const ObjectAddress * +AlterEnumStmtObjectAddress(AlterEnumStmt *stmt, bool missing_ok) +{ + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + ObjectAddress *address = NULL; + + typeName = makeTypeNameFromNameList(stmt->typeName); + typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * RenameTypeStmtObjectAddress returns the ObjectAddress of the type that is the subject + * of the RenameStmt. Errors if missing_ok is false. + */ +const ObjectAddress * +RenameTypeStmtObjectAddress(RenameStmt *stmt, bool missing_ok) +{ + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + ObjectAddress *address = NULL; + + Assert(stmt->renameType == OBJECT_TYPE); + + typeName = makeTypeNameFromNameList((List *) stmt->object); + typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * AlterTypeSchemaStmtObjectAddress returns the ObjectAddress of the type that is the + * subject of the AlterObjectSchemaStmt. Errors if missing_ok is false. + * + * This could be called both before or after it has been applied locally. It will look in + * the old schema first, if the type cannot be found in that schema it will look in the + * new schema. Errors if missing_ok is false and the type cannot be found in either of the + * schemas. + */ +const ObjectAddress * +AlterTypeSchemaStmtObjectAddress(AlterObjectSchemaStmt *stmt, bool missing_ok) +{ + ObjectAddress *address = NULL; + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + List *names = NIL; + + Assert(stmt->objectType == OBJECT_TYPE); + + names = (List *) stmt->object; + + /* + * we hardcode missing_ok here during LookupTypeNameOid because if we can't find it it + * might have already been moved in this transaction. + */ + typeName = makeTypeNameFromNameList(names); + typeOid = LookupTypeNameOid(NULL, typeName, true); + + if (typeOid == InvalidOid) + { + /* + * couldn't find the type, might have already been moved to the new schema, we + * construct a new typename that uses the new schema to search in. + */ + + /* typename is the last in the list of names */ + Value *typeNameStr = lfirst(list_tail(names)); + + /* + * we don't error here either, as the error would be not a good user facing + * error if the type didn't exist in the first place. + */ + names = list_make2(makeString(stmt->newschema), typeNameStr); + typeName = makeTypeNameFromNameList(names); + typeOid = LookupTypeNameOid(NULL, typeName, true); + + /* + * if the type is still invalid we couldn't find the type, error with the same + * message postgres would error with it missing_ok is false (not ok to miss) + */ + if (!missing_ok && typeOid == InvalidOid) + { + names = (List *) stmt->object; + typeName = makeTypeNameFromNameList(names); + + ereport(ERROR, (errcode(ERRCODE_UNDEFINED_OBJECT), + errmsg("type \"%s\" does not exist", + TypeNameToString(typeName)))); + } + } + + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * RenameTypeAttributeStmtObjectAddress returns the ObjectAddress of the type that is the + * subject of the RenameStmt. Errors if missing_ok is false. + * + * The ObjectAddress is that of the type, not that of the attributed for which the name is + * changed as Attributes are not distributed on their own but as a side effect of the + * whole type distribution. + */ +const ObjectAddress * +RenameTypeAttributeStmtObjectAddress(RenameStmt *stmt, bool missing_ok) +{ + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + ObjectAddress *address = NULL; + + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + Assert(stmt->relationType == OBJECT_TYPE); + + typeName = MakeTypeNameFromRangeVar(stmt->relation); + typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * AlterTypeOwnerObjectAddress returns the ObjectAddress of the type that is the subject + * of the AlterOwnerStmt. Errors if missing_ok is false. + */ +const ObjectAddress * +AlterTypeOwnerObjectAddress(AlterOwnerStmt *stmt, bool missing_ok) +{ + TypeName *typeName = NULL; + Oid typeOid = InvalidOid; + ObjectAddress *address = NULL; + + Assert(stmt->objectType == OBJECT_TYPE); + + typeName = makeTypeNameFromNameList((List *) stmt->object); + typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + address = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*address, TypeRelationId, typeOid); + + return address; +} + + +/* + * CreateDropStmtBasedOnCompositeTypeStmt returns, given a CREATE TYPE statement, a + * corresponding statement to drop the type that is to be created. The type does not need + * to exists in this postgres for this function to succeed. + */ +DropStmt * +CreateDropStmtBasedOnCompositeTypeStmt(CompositeTypeStmt *stmt) +{ + List *names = MakeNameListFromRangeVar(stmt->typevar); + TypeName *typeName = makeTypeNameFromNameList(names); + + DropStmt *dropStmt = makeNode(DropStmt); + dropStmt->removeType = OBJECT_TYPE; + dropStmt->objects = list_make1(typeName); + return dropStmt; +} + + +/* + * CreateDropStmtBasedOnEnumStmt returns, given a CREATE TYPE ... AS ENUM statement, a + * corresponding statement to drop the type that is to be created. The type does not need + * to exists in this postgres for this function to succeed. + */ +DropStmt * +CreateDropStmtBasedOnEnumStmt(CreateEnumStmt *stmt) +{ + TypeName *typeName = makeTypeNameFromNameList(stmt->typeName); + + DropStmt *dropStmt = makeNode(DropStmt); + dropStmt->removeType = OBJECT_TYPE; + dropStmt->objects = list_make1(typeName); + return dropStmt; +} + + +/* + * CreateTypeDDLCommandsIdempotent returns a list of DDL statements (const char *) to be + * executed on a node to recreate the type addressed by the typeAddress. + */ +List * +CreateTypeDDLCommandsIdempotent(const ObjectAddress *typeAddress) +{ + List *ddlCommands = NIL; + const char *ddlCommand = NULL; + Node *stmt = NULL; + StringInfoData buf = { 0 }; + const char *username = NULL; + + Assert(typeAddress->classId == TypeRelationId); + + if (type_is_array(typeAddress->objectId)) + { + /* + * array types cannot be created on their own, but could be a direct dependency of + * a table. In that case they are on the dependency graph and tried to be created. + * + * By returning an empty list we will not send any commands to create this type. + */ + return NIL; + } + + stmt = CreateTypeStmtByObjectAddress(typeAddress); + + /* capture ddl command for recreation and wrap in create if not exists construct */ + ddlCommand = DeparseTreeNode(stmt); + ddlCommand = WrapCreateOrReplace(ddlCommand); + ddlCommands = lappend(ddlCommands, (void *) ddlCommand); + + /* add owner ship change so the creation command can be run as a different user */ + username = GetUserNameFromId(GetTypeOwner(typeAddress->objectId), false); + initStringInfo(&buf); + appendStringInfo(&buf, ALTER_TYPE_OWNER_COMMAND, getObjectIdentity(typeAddress), + quote_identifier(username)); + ddlCommands = lappend(ddlCommands, buf.data); + + return ddlCommands; +} + + +/* + * WrapCreateOrReplace takes a sql CREATE command and wraps it in a call to citus' udf to + * create or replace the existing object based on its create command. + */ +const char * +WrapCreateOrReplace(const char *sql) +{ + StringInfoData buf = { 0 }; + initStringInfo(&buf); + appendStringInfo(&buf, CREATE_OR_REPLACE_COMMAND, quote_literal_cstr(sql)); + return buf.data; +} + + +/* + * FilterNameListForDistributedTypes takes a list of objects to delete, for Types this + * will be a list of TypeName. This list is filtered against the types that are + * distributed. + * + * The original list will not be touched, a new list will be created with only the objects + * in there. + */ +static List * +FilterNameListForDistributedTypes(List *objects, bool missing_ok) +{ + ListCell *objectCell = NULL; + List *result = NIL; + foreach(objectCell, objects) + { + TypeName *typeName = castNode(TypeName, lfirst(objectCell)); + Oid typeOid = LookupTypeNameOid(NULL, typeName, missing_ok); + ObjectAddress typeAddress = { 0 }; + + if (!OidIsValid(typeOid)) + { + continue; + } + + ObjectAddressSet(typeAddress, TypeRelationId, typeOid); + if (IsObjectDistributed(&typeAddress)) + { + result = lappend(result, typeName); + } + } + return result; +} + + +/* + * TypeNameListToObjectAddresses transforms a List * of TypeName *'s into a List * of + * ObjectAddress *'s. For this to succeed all Types identiefied by the TypeName *'s should + * exist on this postgres, an error will be thrown otherwise. + */ +static List * +TypeNameListToObjectAddresses(List *objects) +{ + ListCell *objectCell = NULL; + List *result = NIL; + foreach(objectCell, objects) + { + TypeName *typeName = castNode(TypeName, lfirst(objectCell)); + Oid typeOid = LookupTypeNameOid(NULL, typeName, false); + ObjectAddress *typeAddress = palloc0(sizeof(ObjectAddress)); + ObjectAddressSet(*typeAddress, TypeRelationId, typeOid); + result = lappend(result, typeAddress); + } + return result; +} + + +/* + * GetTypeOwner + * + * Given the type OID, find its owner + */ +static Oid +GetTypeOwner(Oid typeOid) +{ + Oid result = InvalidOid; + HeapTuple tp = NULL; + + tp = SearchSysCache1(TYPEOID, ObjectIdGetDatum(typeOid)); + if (HeapTupleIsValid(tp)) + { + Form_pg_type typtup = (Form_pg_type) GETSTRUCT(tp); + + result = typtup->typowner; + ReleaseSysCache(tp); + } + + return result; +} + + +/* + * MakeTypeNameFromRangeVar creates a TypeName based on a RangeVar. + */ +static TypeName * +MakeTypeNameFromRangeVar(const RangeVar *relation) +{ + List *names = NIL; + if (relation->schemaname) + { + names = lappend(names, makeString(relation->schemaname)); + } + names = lappend(names, makeString(relation->relname)); + + return makeTypeNameFromNameList(names); +} + + +/* + * EnsureSequentialModeForTypeDDL makes sure that the current transaction is already in + * sequential mode, or can still safely be put in sequential mode, it errors if that is + * not possible. The error contains information for the user to retry the transaction with + * sequential mode set from the beginnig. + * + * As types are node scoped objects there exists only 1 instance of the type used by + * potentially multiple shards. To make sure all shards in the transaction can interact + * with the type the type needs to be visible on all connections used by the transaction, + * meaning we can only use 1 connection per node. + */ +static void +EnsureSequentialModeForTypeDDL(void) +{ + if (!IsTransactionBlock()) + { + /* we do not need to switch to sequential mode if we are not in a transaction */ + return; + } + + if (ParallelQueryExecutedInTransaction()) + { + ereport(ERROR, (errmsg("cannot create or modify type because there was a " + "parallel operation on a distributed table in the " + "transaction"), + errdetail("When creating or altering a type, Citus needs to " + "perform all operations over a single connection per " + "node to ensure consistency."), + errhint("Try re-running the transaction with " + "\"SET LOCAL citus.multi_shard_modify_mode TO " + "\'sequential\';\""))); + } + + ereport(DEBUG1, (errmsg("switching to sequential query execution mode"), + errdetail("Type is created or altered. To make sure subsequent " + "commands see the type correctly we need to make sure to " + "use only one connection for all future commands"))); + SetLocalMultiShardModifyModeToSequential(); +} + + +/* + * ShouldPropagateTypeCreate returns if we should propagate the creation of a type. + * + * There are two moments we decide to not directly propagate the creation of a type. + * - During the creation of an Extension; we assume the type will be created by creating + * the extension on the worker + * - During a transaction block; if types are used in a distributed table in the same + * block we can only provide parallelism on the table if we do not change to sequential + * mode. Types will be propagated outside of this transaction to the workers so that + * the transaction can use 1 connection per shard and fully utilize citus' parallelism + */ +static bool +ShouldPropagateTypeCreate() +{ + if (creating_extension) + { + /* + * extensions should be created separately on the workers, types cascading from an + * extension should therefor not be propagated here. + */ + return false; + } + + /* + * by not propagating in a transaction block we allow for parallelism to be used when + * this type will be used as a column in a table that will be created and distributed + * in this same transaction. + */ + if (IsMultiStatementTransaction()) + { + return false; + } + + return true; +} diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index de7758032..e9244b762 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -41,6 +41,7 @@ #include "distributed/commands.h" #include "distributed/commands/multi_copy.h" #include "distributed/commands/utility_hook.h" /* IWYU pragma: keep */ +#include "distributed/listutils.h" #include "distributed/local_executor.h" #include "distributed/maintenanced.h" #include "distributed/master_protocol.h" @@ -52,6 +53,7 @@ #include "distributed/version_compat.h" #include "distributed/worker_transaction.h" #include "lib/stringinfo.h" +#include "nodes/pg_list.h" #include "tcop/utility.h" #include "utils/builtins.h" #include "utils/lsyscache.h" @@ -69,6 +71,10 @@ static void ExecuteDistributedDDLJob(DDLJob *ddlJob); static char * SetSearchPathToCurrentSearchPathCommand(void); static char * CurrentSearchPath(void); static void PostProcessUtility(Node *parsetree); +static List * PlanRenameAttributeStmt(RenameStmt *stmt, const char *queryString); +static List * PlanAlterOwnerStmt(AlterOwnerStmt *stmt, const char *queryString); + +static void ExecuteNodeBaseDDLCommands(List *taskList); /* @@ -337,27 +343,50 @@ multi_ProcessUtility(PlannedStmt *pstmt, if (IsA(parsetree, DropStmt)) { DropStmt *dropStatement = (DropStmt *) parsetree; - if (dropStatement->removeType == OBJECT_INDEX) + switch (dropStatement->removeType) { - ddlJobs = PlanDropIndexStmt(dropStatement, queryString); - } + case OBJECT_INDEX: + { + ddlJobs = PlanDropIndexStmt(dropStatement, queryString); + break; + } - if (dropStatement->removeType == OBJECT_TABLE) - { - ProcessDropTableStmt(dropStatement); - } + case OBJECT_TABLE: + { + ProcessDropTableStmt(dropStatement); + break; + } - if (dropStatement->removeType == OBJECT_SCHEMA) - { - ProcessDropSchemaStmt(dropStatement); - } + case OBJECT_SCHEMA: + { + ProcessDropSchemaStmt(dropStatement); + break; + } - if (dropStatement->removeType == OBJECT_POLICY) - { - ddlJobs = PlanDropPolicyStmt(dropStatement, queryString); + case OBJECT_POLICY: + { + ddlJobs = PlanDropPolicyStmt(dropStatement, queryString); + break; + } + + case OBJECT_TYPE: + { + ddlJobs = PlanDropTypeStmt(dropStatement, queryString); + break; + } + + default: + { + /* unsupported type, skipping*/ + } } } + if (IsA(parsetree, AlterEnumStmt)) + { + ddlJobs = PlanAlterEnumStmt(castNode(AlterEnumStmt, parsetree), queryString); + } + if (IsA(parsetree, AlterTableStmt)) { AlterTableStmt *alterTableStmt = (AlterTableStmt *) parsetree; @@ -367,6 +396,11 @@ multi_ProcessUtility(PlannedStmt *pstmt, { ddlJobs = PlanAlterTableStmt(alterTableStmt, queryString); } + + if (alterTableStmt->relkind == OBJECT_TYPE) + { + ddlJobs = PlanAlterTypeStmt(alterTableStmt, queryString); + } } /* @@ -375,7 +409,28 @@ multi_ProcessUtility(PlannedStmt *pstmt, */ if (IsA(parsetree, RenameStmt)) { - ddlJobs = PlanRenameStmt((RenameStmt *) parsetree, queryString); + RenameStmt *renameStmt = (RenameStmt *) parsetree; + + switch (renameStmt->renameType) + { + case OBJECT_TYPE: + { + ddlJobs = PlanRenameTypeStmt(renameStmt, queryString); + break; + } + + case OBJECT_ATTRIBUTE: + { + ddlJobs = PlanRenameAttributeStmt(renameStmt, queryString); + break; + } + + default: + { + ddlJobs = PlanRenameStmt(renameStmt, queryString); + break; + } + } } /* handle distributed CLUSTER statements */ @@ -390,8 +445,8 @@ multi_ProcessUtility(PlannedStmt *pstmt, */ if (IsA(parsetree, AlterObjectSchemaStmt)) { - AlterObjectSchemaStmt *setSchemaStmt = (AlterObjectSchemaStmt *) parsetree; - ddlJobs = PlanAlterObjectSchemaStmt(setSchemaStmt, queryString); + ddlJobs = PlanAlterObjectSchemaStmt( + castNode(AlterObjectSchemaStmt, parsetree), queryString); } if (IsA(parsetree, GrantStmt)) @@ -399,6 +454,12 @@ multi_ProcessUtility(PlannedStmt *pstmt, ddlJobs = PlanGrantStmt((GrantStmt *) parsetree); } + if (IsA(parsetree, AlterOwnerStmt)) + { + ddlJobs = PlanAlterOwnerStmt(castNode(AlterOwnerStmt, parsetree), + queryString); + } + if (IsA(parsetree, CreatePolicyStmt)) { ddlJobs = PlanCreatePolicyStmt((CreatePolicyStmt *) parsetree); @@ -409,6 +470,18 @@ multi_ProcessUtility(PlannedStmt *pstmt, ddlJobs = PlanAlterPolicyStmt((AlterPolicyStmt *) parsetree); } + if (IsA(parsetree, CompositeTypeStmt)) + { + ddlJobs = PlanCompositeTypeStmt(castNode(CompositeTypeStmt, parsetree), + queryString); + } + + if (IsA(parsetree, CreateEnumStmt)) + { + ddlJobs = PlanCreateEnumStmt(castNode(CreateEnumStmt, parsetree), + queryString); + } + /* * ALTER TABLE ALL IN TABLESPACE statements have their node type as * AlterTableMoveAllStmt. At the moment we do not support this functionality in @@ -504,6 +577,22 @@ multi_ProcessUtility(PlannedStmt *pstmt, standard_ProcessUtility(pstmt, queryString, context, params, queryEnv, dest, completionTag); + /* + * Postgres added the following CommandCounterIncrement as a patch in: + * - 10.7 -> 10.8 + * - 11.2 -> 11.3 + * The patch was a response to bug #15631. + * + * CommandCounterIncrement is used to make changes to the catalog visible for post + * processing of create commands (eg. create type). It is safe to call + * CommandCounterIncrement twice, as the call is a no-op if the command id is not + * used yet. + * + * Once versions older then above are not deemed important anymore this patch can + * be remove from citus. + */ + CommandCounterIncrement(); + if (IsA(parsetree, AlterTableStmt)) { activeAlterTables--; @@ -520,6 +609,33 @@ multi_ProcessUtility(PlannedStmt *pstmt, } PG_END_TRY(); + /* + * Post process for ddl statements + */ + if (EnableDDLPropagation) + { + if (IsA(parsetree, CompositeTypeStmt)) + { + ProcessCompositeTypeStmt(castNode(CompositeTypeStmt, parsetree), queryString); + } + + if (IsA(parsetree, CreateEnumStmt)) + { + ProcessCreateEnumStmt(castNode(CreateEnumStmt, parsetree), queryString); + } + + if (IsA(parsetree, AlterObjectSchemaStmt)) + { + ProcessAlterObjectSchemaStmt(castNode(AlterObjectSchemaStmt, parsetree), + queryString); + } + + if (IsA(parsetree, AlterEnumStmt)) + { + ProcessAlterEnumStmt(castNode(AlterEnumStmt, parsetree), queryString); + } + } + /* * We only process CREATE TABLE ... PARTITION OF commands in the function below * to handle the case when user creates a table as a partition of distributed table. @@ -592,6 +708,57 @@ multi_ProcessUtility(PlannedStmt *pstmt, } +/* + * PlanRenameAttributeStmt called for RenameStmt's that are targetting an attribute eg. + * type attributes. Based on the relation type the attribute gets renamed it dispatches to + * a specialized implementation if present, otherwise return an empty list for its DDLJobs + */ +static List * +PlanRenameAttributeStmt(RenameStmt *stmt, const char *queryString) +{ + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + + switch (stmt->relationType) + { + case OBJECT_TYPE: + { + return PlanRenameTypeAttributeStmt(stmt, queryString); + } + + default: + { + /* unsupported relation for attribute rename, do nothing */ + return NIL; + } + } +} + + +/* + * PlanAlterOwnerStmt gets called for statements that change the ownership of an object. + * Based on the type of object the ownership gets changed for it dispatches to a + * specialized implementation or returns an empty list of DDLJobs for objects that do not + * have an implementation provided. + */ +static List * +PlanAlterOwnerStmt(AlterOwnerStmt *stmt, const char *queryString) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + return PlanAlterTypeOwnerStmt(stmt, queryString); + } + + default: + { + /* do nothing for unsupported alter owner statements */ + return NIL; + } + } +} + + /* * ExecuteDistributedDDLJob simply executes a provided DDLJob in a distributed trans- * action, including metadata sync if needed. If the multi shard commit protocol is @@ -607,12 +774,33 @@ multi_ProcessUtility(PlannedStmt *pstmt, static void ExecuteDistributedDDLJob(DDLJob *ddlJob) { - bool shouldSyncMetadata = ShouldSyncTableMetadata(ddlJob->targetRelationId); + bool shouldSyncMetadata = false; EnsureCoordinator(); - EnsurePartitionTableNotReplicated(ddlJob->targetRelationId); - if (!ddlJob->concurrentIndexCmd) + if (ddlJob->targetRelationId != InvalidOid) + { + /* + * Only for ddlJobs that are targetting a relation (table) we want to sync its + * metadata and verify some properties around the table. + */ + shouldSyncMetadata = ShouldSyncTableMetadata(ddlJob->targetRelationId); + EnsurePartitionTableNotReplicated(ddlJob->targetRelationId); + } + + if (TaskExecutorType != MULTI_EXECUTOR_ADAPTIVE && + ddlJob->targetRelationId == InvalidOid) + { + /* + * Some ddl jobs can only be run by the adaptive executor and not our legacy ones. + * + * These are tasks that are not pinned to any relation nor shards. We can execute + * these very naively with a simple for loop that sends them to the target worker. + */ + + ExecuteNodeBaseDDLCommands(ddlJob->taskList); + } + else if (!ddlJob->concurrentIndexCmd) { if (shouldSyncMetadata) { @@ -680,6 +868,34 @@ ExecuteDistributedDDLJob(DDLJob *ddlJob) } +/* + * ExecuteNodeBaseDDLCommands executes ddl commands naively only when we are not using the + * adaptive executor. It gets connections to the target placements and executes the + * commands. + */ +static void +ExecuteNodeBaseDDLCommands(List *taskList) +{ + ListCell *taskCell = NULL; + + foreach(taskCell, taskList) + { + Task *task = (Task *) lfirst(taskCell); + ListCell *taskPlacementCell = NULL; + + /* these tasks should not be pinned to any shard */ + Assert(task->anchorShardId == INVALID_SHARD_ID); + + foreach(taskPlacementCell, task->taskPlacementList) + { + ShardPlacement *placement = (ShardPlacement *) lfirst(taskPlacementCell); + SendCommandToWorkerAsUser(placement->nodeName, placement->nodePort, NULL, + task->queryString); + } + } +} + + /* * SetSearchPathToCurrentSearchPathCommand generates a command which can * set the search path to the exact same search path that the issueing node @@ -843,6 +1059,55 @@ DDLTaskList(Oid relationId, const char *commandString) } +/* + * NodeDDLTaskList builds a list of tasks to execute a DDL command on a + * given target set of nodes. + */ +List * +NodeDDLTaskList(TargetWorkerSet targets, List *commands) +{ + List *workerNodes = TargetWorkerSetNodeList(targets); + char *concatenatedCommands = StringJoin(commands, ';'); + DDLJob *ddlJob = NULL; + ListCell *workerNodeCell = NULL; + Task *task = NULL; + + if (list_length(workerNodes) <= 0) + { + /* + * if there are no nodes we don't have to plan any ddl tasks. Planning them would + * cause a hang in the executor. + */ + return NIL; + } + + task = CitusMakeNode(Task); + task->taskType = DDL_TASK; + task->queryString = concatenatedCommands; + + foreach(workerNodeCell, workerNodes) + { + WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); + ShardPlacement *targetPlacement = NULL; + + targetPlacement = CitusMakeNode(ShardPlacement); + targetPlacement->nodeName = workerNode->workerName; + targetPlacement->nodePort = workerNode->workerPort; + targetPlacement->groupId = workerNode->groupId; + + task->taskPlacementList = lappend(task->taskPlacementList, targetPlacement); + } + + ddlJob = palloc0(sizeof(DDLJob)); + ddlJob->targetRelationId = InvalidOid; + ddlJob->concurrentIndexCmd = false; + ddlJob->commandString = NULL; + ddlJob->taskList = list_make1(task); + + return list_make1(ddlJob); +} + + /* * AlterTableInProgress returns true if we're processing an ALTER TABLE command * right now. diff --git a/src/backend/distributed/connection/remote_commands.c b/src/backend/distributed/connection/remote_commands.c index 54df5f8a5..30ee869cc 100644 --- a/src/backend/distributed/connection/remote_commands.c +++ b/src/backend/distributed/connection/remote_commands.c @@ -356,7 +356,8 @@ LogRemoteCommand(MultiConnection *connection, const char *command) } ereport(LOG, (errmsg("issuing %s", ApplyLogRedaction(command)), - errdetail("on server %s:%d", connection->hostname, connection->port))); + errdetail("on server %s@%s:%d", connection->user, connection->hostname, + connection->port))); } @@ -440,8 +441,20 @@ ExecuteOptionalRemoteCommand(MultiConnection *connection, const char *command, return RESPONSE_NOT_OKAY; } - *result = localResult; - return 0; + /* + * store result if result has been set, when the user is not interested in the result + * a NULL pointer could be passed and the result will be cleared + */ + if (result) + { + *result = localResult; + } + else + { + PQclear(localResult); + ForgetResults(connection); + } + return RESPONSE_OKAY; } diff --git a/src/backend/distributed/deparser/deparse.c b/src/backend/distributed/deparser/deparse.c new file mode 100644 index 000000000..79e85dc22 --- /dev/null +++ b/src/backend/distributed/deparser/deparse.c @@ -0,0 +1,204 @@ +/*------------------------------------------------------------------------- + * + * deparse.c + * Entrypoint for deparsing parsetrees. + * + * The goal of deparsing parsetrees is to reconstruct sql statements + * from any parsed sql statement by ParseTreeNode. Deparsed statements + * can be used to reapply them on remote postgres nodes like the citus + * workers. + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/deparser.h" + +static const char * DeparseDropStmt(DropStmt *stmt); +static const char * DeparseAlterTableStmt(AlterTableStmt *stmt); +static const char * DeparseRenameStmt(RenameStmt *stmt); +static const char * DeparseRenameAttributeStmt(RenameStmt *stmt); +static const char * DeparseAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt); +static const char * DeparseAlterOwnerStmt(AlterOwnerStmt *stmt); + + +/* + * DeparseTreeNode aims to be the inverse of postgres' ParseTreeNode. Currently with + * limited support. Check support before using, and add support for new statements as + * required. + * + * Currently supported: + * - CREATE TYPE + * - ALTER TYPE + * - DROP TYPE + */ +const char * +DeparseTreeNode(Node *stmt) +{ + switch (nodeTag(stmt)) + { + case T_DropStmt: + { + return DeparseDropStmt(castNode(DropStmt, stmt)); + } + + case T_CompositeTypeStmt: + { + return DeparseCompositeTypeStmt(castNode(CompositeTypeStmt, stmt)); + } + + case T_CreateEnumStmt: + { + return DeparseCreateEnumStmt(castNode(CreateEnumStmt, stmt)); + } + + case T_AlterTableStmt: + { + return DeparseAlterTableStmt(castNode(AlterTableStmt, stmt)); + } + + case T_AlterEnumStmt: + { + return DeparseAlterEnumStmt(castNode(AlterEnumStmt, stmt)); + } + + case T_RenameStmt: + { + return DeparseRenameStmt(castNode(RenameStmt, stmt)); + } + + case T_AlterObjectSchemaStmt: + { + return DeparseAlterObjectSchemaStmt(castNode(AlterObjectSchemaStmt, stmt)); + } + + case T_AlterOwnerStmt: + { + return DeparseAlterOwnerStmt(castNode(AlterOwnerStmt, stmt)); + } + + default: + { + ereport(ERROR, (errmsg("unsupported statement for deparsing"))); + } + } +} + + +static const char * +DeparseDropStmt(DropStmt *stmt) +{ + switch (stmt->removeType) + { + case OBJECT_TYPE: + { + return DeparseDropTypeStmt(stmt); + } + + default: + { + ereport(ERROR, (errmsg("unsupported drop statement for deparsing"))); + } + } +} + + +static const char * +DeparseAlterTableStmt(AlterTableStmt *stmt) +{ + switch (stmt->relkind) + { + case OBJECT_TYPE: + { + return DeparseAlterTypeStmt(stmt); + } + + default: + { + ereport(ERROR, (errmsg("unsupported alter statement for deparsing"))); + } + } +} + + +static const char * +DeparseRenameStmt(RenameStmt *stmt) +{ + switch (stmt->renameType) + { + case OBJECT_TYPE: + { + return DeparseRenameTypeStmt(stmt); + } + + case OBJECT_ATTRIBUTE: + { + return DeparseRenameAttributeStmt(stmt); + } + + default: + { + ereport(ERROR, (errmsg("unsupported rename statement for deparsing"))); + } + } +} + + +static const char * +DeparseRenameAttributeStmt(RenameStmt *stmt) +{ + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + + switch (stmt->relationType) + { + case OBJECT_TYPE: + { + return DeparseRenameTypeAttributeStmt(stmt); + } + + default: + { + ereport(ERROR, (errmsg("unsupported rename attribute statement for" + " deparsing"))); + } + } +} + + +static const char * +DeparseAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + return DeparseAlterTypeSchemaStmt(stmt); + } + + default: + { + ereport(ERROR, (errmsg("unsupported rename statement for deparsing"))); + } + } +} + + +static const char * +DeparseAlterOwnerStmt(AlterOwnerStmt *stmt) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + return DeparseAlterTypeOwnerStmt(stmt); + } + + default: + { + ereport(ERROR, (errmsg("unsupported alter owner statement for deparsing"))); + } + } +} diff --git a/src/backend/distributed/deparser/deparse_type_stmts.c b/src/backend/distributed/deparser/deparse_type_stmts.c new file mode 100644 index 000000000..b1ba56254 --- /dev/null +++ b/src/backend/distributed/deparser/deparse_type_stmts.c @@ -0,0 +1,509 @@ +/*------------------------------------------------------------------------- + * + * deparse_type_stmts.c + * All routines to deparse type statements. + * This file contains all entry points specific for type statement deparsing as well as + * functions that are currently only used for deparsing of the type statements. + * + * Functions that could move later are AppendColumnDef, AppendColumnDefList, etc. These + * should be reused across multiple statements and should live in their own deparse + * file. + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "catalog/namespace.h" +#include "lib/stringinfo.h" +#include "nodes/makefuncs.h" +#include "nodes/parsenodes.h" +#include "parser/parse_type.h" +#include "utils/builtins.h" + +#include "distributed/citus_ruleutils.h" +#include "distributed/commands.h" +#include "distributed/deparser.h" +#include "distributed/metadata/namespace.h" + +#define AlterEnumIsRename(stmt) (stmt->oldVal != NULL) +#define AlterEnumIsAddValue(stmt) (stmt->oldVal == NULL) + +/* forward declaration for deparse functions */ +static void AppendCompositeTypeStmt(StringInfo str, CompositeTypeStmt *stmt); +static void AppendColumnDef(StringInfo str, ColumnDef *columnDef); +static void AppendColumnDefList(StringInfo str, List *columnDefs); + +static void AppendCreateEnumStmt(StringInfo str, CreateEnumStmt *stmt); +static void AppendStringList(StringInfo str, List *strings); + +static void AppendDropTypeStmt(StringInfo buf, DropStmt *stmt); +static void AppendTypeNameList(StringInfo buf, List *objects); + +static void AppendAlterEnumStmt(StringInfo buf, AlterEnumStmt *stmt); + +static void AppendAlterTypeStmt(StringInfo buf, AlterTableStmt *stmt); +static void AppendAlterTypeCmd(StringInfo buf, AlterTableCmd *alterTableCmd); +static void AppendAlterTypeCmdAddColumn(StringInfo buf, AlterTableCmd *alterTableCmd); +static void AppendAlterTypeCmdDropColumn(StringInfo buf, AlterTableCmd *alterTableCmd); +static void AppendAlterTypeCmdAlterColumnType(StringInfo buf, + AlterTableCmd *alterTableCmd); + +static void AppendRenameTypeStmt(StringInfo buf, RenameStmt *stmt); +static void AppendRenameTypeAttributeStmt(StringInfo buf, RenameStmt *stmt); +static void AppendAlterTypeSchemaStmt(StringInfo buf, AlterObjectSchemaStmt *stmt); +static void AppendAlterTypeOwnerStmt(StringInfo buf, AlterOwnerStmt *stmt); + + +/* + * DeparseCompositeTypeStmt builds and returns a string representing the + * CompositeTypeStmt for application on a remote server. + */ +const char * +DeparseCompositeTypeStmt(CompositeTypeStmt *stmt) +{ + StringInfoData sql = { 0 }; + initStringInfo(&sql); + + AppendCompositeTypeStmt(&sql, stmt); + + return sql.data; +} + + +const char * +DeparseCreateEnumStmt(CreateEnumStmt *stmt) +{ + StringInfoData sql = { 0 }; + initStringInfo(&sql); + + AppendCreateEnumStmt(&sql, stmt); + + return sql.data; +} + + +const char * +DeparseAlterEnumStmt(AlterEnumStmt *stmt) +{ + StringInfoData sql = { 0 }; + initStringInfo(&sql); + + AppendAlterEnumStmt(&sql, stmt); + + return sql.data; +} + + +const char * +DeparseDropTypeStmt(DropStmt *stmt) +{ + StringInfoData str = { 0 }; + initStringInfo(&str); + + Assert(stmt->removeType == OBJECT_TYPE); + + AppendDropTypeStmt(&str, stmt); + + return str.data; +} + + +const char * +DeparseAlterTypeStmt(AlterTableStmt *stmt) +{ + StringInfoData str = { 0 }; + initStringInfo(&str); + + Assert(stmt->relkind == OBJECT_TYPE); + + AppendAlterTypeStmt(&str, stmt); + + return str.data; +} + + +static void +AppendAlterTypeStmt(StringInfo buf, AlterTableStmt *stmt) +{ + const char *identifier = quote_qualified_identifier(stmt->relation->schemaname, + stmt->relation->relname); + ListCell *cmdCell = NULL; + + Assert(stmt->relkind = OBJECT_TYPE); + + appendStringInfo(buf, "ALTER TYPE %s", identifier); + foreach(cmdCell, stmt->cmds) + { + AlterTableCmd *alterTableCmd = NULL; + + if (cmdCell != list_head(stmt->cmds)) + { + appendStringInfoString(buf, ", "); + } + + alterTableCmd = castNode(AlterTableCmd, lfirst(cmdCell)); + AppendAlterTypeCmd(buf, alterTableCmd); + } + + appendStringInfoString(buf, ";"); +} + + +static void +AppendAlterTypeCmd(StringInfo buf, AlterTableCmd *alterTableCmd) +{ + switch (alterTableCmd->subtype) + { + case AT_AddColumn: + { + AppendAlterTypeCmdAddColumn(buf, alterTableCmd); + break; + } + + case AT_DropColumn: + { + AppendAlterTypeCmdDropColumn(buf, alterTableCmd); + break; + } + + case AT_AlterColumnType: + { + AppendAlterTypeCmdAlterColumnType(buf, alterTableCmd); + break; + } + + default: + { + ereport(ERROR, (errmsg("unsupported subtype for alter table command"), + errdetail("sub command type: %d", alterTableCmd->subtype))); + } + } +} + + +static void +AppendAlterTypeCmdAddColumn(StringInfo buf, AlterTableCmd *alterTableCmd) +{ + Assert(alterTableCmd->subtype == AT_AddColumn); + + appendStringInfoString(buf, " ADD ATTRIBUTE "); + AppendColumnDef(buf, castNode(ColumnDef, alterTableCmd->def)); +} + + +static void +AppendAlterTypeCmdDropColumn(StringInfo buf, AlterTableCmd *alterTableCmd) +{ + Assert(alterTableCmd->subtype == AT_DropColumn); + appendStringInfo(buf, " DROP ATTRIBUTE %s", quote_identifier(alterTableCmd->name)); + + if (alterTableCmd->behavior == DROP_CASCADE) + { + appendStringInfoString(buf, " CASCADE"); + } +} + + +static void +AppendAlterTypeCmdAlterColumnType(StringInfo buf, AlterTableCmd *alterTableCmd) +{ + Assert(alterTableCmd->subtype == AT_AlterColumnType); + appendStringInfo(buf, " ALTER ATTRIBUTE %s SET DATA TYPE ", quote_identifier( + alterTableCmd->name)); + AppendColumnDef(buf, castNode(ColumnDef, alterTableCmd->def)); + + if (alterTableCmd->behavior == DROP_CASCADE) + { + appendStringInfoString(buf, " CASCADE"); + } +} + + +static void +AppendAlterEnumStmt(StringInfo buf, AlterEnumStmt *stmt) +{ + appendStringInfo(buf, "ALTER TYPE %s", NameListToQuotedString(stmt->typeName)); + + if (AlterEnumIsRename(stmt)) + { + /* Rename an existing label */ + appendStringInfo(buf, " RENAME VALUE %s TO %s;", + quote_literal_cstr(stmt->oldVal), + quote_literal_cstr(stmt->newVal)); + } + else if (AlterEnumIsAddValue(stmt)) + { + /* Add a new label */ + appendStringInfoString(buf, " ADD VALUE "); + if (stmt->skipIfNewValExists) + { + appendStringInfoString(buf, "IF NOT EXISTS "); + } + appendStringInfoString(buf, quote_literal_cstr(stmt->newVal)); + + if (stmt->newValNeighbor) + { + appendStringInfo(buf, " %s %s", + stmt->newValIsAfter ? "AFTER" : "BEFORE", + quote_literal_cstr(stmt->newValNeighbor)); + } + + appendStringInfoString(buf, ";"); + } +} + + +static void +AppendDropTypeStmt(StringInfo buf, DropStmt *stmt) +{ + /* + * already tested at call site, but for future it might be collapsed in a + * DeparseDropStmt so be safe and check again + */ + Assert(stmt->removeType == OBJECT_TYPE); + + appendStringInfo(buf, "DROP TYPE "); + if (stmt->missing_ok) + { + appendStringInfoString(buf, "IF EXISTS "); + } + AppendTypeNameList(buf, stmt->objects); + if (stmt->behavior == DROP_CASCADE) + { + appendStringInfoString(buf, " CASCADE"); + } + appendStringInfoString(buf, ";"); +} + + +static void +AppendTypeNameList(StringInfo buf, List *objects) +{ + ListCell *objectCell = NULL; + foreach(objectCell, objects) + { + TypeName *typeName = castNode(TypeName, lfirst(objectCell)); + Oid typeOid = LookupTypeNameOid(NULL, typeName, false); + const char *identifier = format_type_be_qualified(typeOid); + + if (objectCell != list_head(objects)) + { + appendStringInfo(buf, ", "); + } + + appendStringInfoString(buf, identifier); + } +} + + +/* + * AppendCompositeTypeStmt appends the sql string to recreate a CompositeTypeStmt to the + * provided buffer, ending in a ; for concatination of multiple statements. + */ +static void +AppendCompositeTypeStmt(StringInfo str, CompositeTypeStmt *stmt) +{ + const char *identifier = quote_qualified_identifier(stmt->typevar->schemaname, + stmt->typevar->relname); + appendStringInfo(str, "CREATE TYPE %s AS (", identifier); + AppendColumnDefList(str, stmt->coldeflist); + appendStringInfo(str, ");"); +} + + +static void +AppendCreateEnumStmt(StringInfo str, CreateEnumStmt *stmt) +{ + RangeVar *typevar = NULL; + const char *identifier = NULL; + + typevar = makeRangeVarFromNameList(stmt->typeName); + + /* create the identifier from the fully qualified rangevar */ + identifier = quote_qualified_identifier(typevar->schemaname, typevar->relname); + + appendStringInfo(str, "CREATE TYPE %s AS ENUM (", identifier); + AppendStringList(str, stmt->vals); + appendStringInfo(str, ");"); +} + + +static void +AppendStringList(StringInfo str, List *strings) +{ + ListCell *stringCell = NULL; + foreach(stringCell, strings) + { + const char *string = strVal(lfirst(stringCell)); + if (stringCell != list_head(strings)) + { + appendStringInfoString(str, ", "); + } + + string = quote_literal_cstr(string); + appendStringInfoString(str, string); + } +} + + +/* + * AppendColumnDefList appends the definition of a list of ColumnDef items to the provided + * buffer, adding separators as necessary. + */ +static void +AppendColumnDefList(StringInfo str, List *columnDefs) +{ + ListCell *columnDefCell = NULL; + foreach(columnDefCell, columnDefs) + { + if (columnDefCell != list_head(columnDefs)) + { + appendStringInfoString(str, ", "); + } + AppendColumnDef(str, castNode(ColumnDef, lfirst(columnDefCell))); + } +} + + +/* + * AppendColumnDef appends the definition of one ColumnDef completely qualified to the + * provided buffer. + * + * If the colname is not set that part is ommitted. This is the case in alter column type + * statements. + */ +static void +AppendColumnDef(StringInfo str, ColumnDef *columnDef) +{ + Oid typeOid = LookupTypeNameOid(NULL, columnDef->typeName, false); + Oid collationOid = GetColumnDefCollation(NULL, columnDef, typeOid); + + Assert(!columnDef->is_not_null); /* not null is not supported on composite types */ + + if (columnDef->colname) + { + appendStringInfo(str, "%s ", quote_identifier(columnDef->colname)); + } + + appendStringInfo(str, "%s", format_type_be_qualified(typeOid)); + + if (OidIsValid(collationOid)) + { + const char *identifier = FormatCollateBEQualified(collationOid); + appendStringInfo(str, " COLLATE %s", identifier); + } +} + + +const char * +DeparseRenameTypeStmt(RenameStmt *stmt) +{ + StringInfoData str = { 0 }; + initStringInfo(&str); + + Assert(stmt->renameType == OBJECT_TYPE); + + AppendRenameTypeStmt(&str, stmt); + + return str.data; +} + + +static void +AppendRenameTypeStmt(StringInfo buf, RenameStmt *stmt) +{ + List *names = (List *) stmt->object; + + appendStringInfo(buf, "ALTER TYPE %s RENAME TO %s;", NameListToQuotedString(names), + quote_identifier(stmt->newname)); +} + + +const char * +DeparseRenameTypeAttributeStmt(RenameStmt *stmt) +{ + StringInfoData str = { 0 }; + initStringInfo(&str); + + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + Assert(stmt->relationType == OBJECT_TYPE); + + AppendRenameTypeAttributeStmt(&str, stmt); + + return str.data; +} + + +static void +AppendRenameTypeAttributeStmt(StringInfo buf, RenameStmt *stmt) +{ + appendStringInfo(buf, "ALTER TYPE %s RENAME ATTRIBUTE %s TO %s", + quote_qualified_identifier(stmt->relation->schemaname, + stmt->relation->relname), + quote_identifier(stmt->subname), + quote_identifier(stmt->newname)); + + if (stmt->behavior == DROP_CASCADE) + { + appendStringInfoString(buf, " CASCADE"); + } + + appendStringInfoString(buf, ";"); +} + + +const char * +DeparseAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt) +{ + StringInfoData str = { 0 }; + initStringInfo(&str); + + Assert(stmt->objectType == OBJECT_TYPE); + + AppendAlterTypeSchemaStmt(&str, stmt); + + return str.data; +} + + +static void +AppendAlterTypeSchemaStmt(StringInfo buf, AlterObjectSchemaStmt *stmt) +{ + List *names = NIL; + + Assert(stmt->objectType == OBJECT_TYPE); + + names = (List *) stmt->object; + appendStringInfo(buf, "ALTER TYPE %s SET SCHEMA %s;", NameListToQuotedString(names), + quote_identifier(stmt->newschema)); +} + + +const char * +DeparseAlterTypeOwnerStmt(AlterOwnerStmt *stmt) +{ + StringInfoData str = { 0 }; + initStringInfo(&str); + + Assert(stmt->objectType == OBJECT_TYPE); + + AppendAlterTypeOwnerStmt(&str, stmt); + + return str.data; +} + + +static void +AppendAlterTypeOwnerStmt(StringInfo buf, AlterOwnerStmt *stmt) +{ + List *names = NIL; + + Assert(stmt->objectType == OBJECT_TYPE); + + names = (List *) stmt->object; + appendStringInfo(buf, "ALTER TYPE %s OWNER TO %s;", NameListToQuotedString(names), + RoleSpecString(stmt->newowner)); +} diff --git a/src/backend/distributed/deparser/format_collate.c b/src/backend/distributed/deparser/format_collate.c new file mode 100644 index 000000000..0c67d1f50 --- /dev/null +++ b/src/backend/distributed/deparser/format_collate.c @@ -0,0 +1,104 @@ +/*------------------------------------------------------------------------- + * + * format_collate.c + * Display collate names "nicely". + * + * This file is modeled after postgres' utils/adt/format_*.c files + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "access/htup_details.h" +#include "catalog/namespace.h" +#include "catalog/pg_collation.h" +#include "utils/builtins.h" +#include "utils/lsyscache.h" +#include "utils/syscache.h" + +#include "distributed/deparser.h" + +/* + * This version is for use within the backend in error messages, etc. + * One difference is that it will fail for an invalid collate. + * + * The result is always a palloc'd string. + */ +char * +FormatCollateBE(Oid collate_oid) +{ + return FormatCollateExtended(collate_oid, 0); +} + + +/* + * This version returns a name that is always qualified. + */ +char * +FormatCollateBEQualified(Oid collate_oid) +{ + return FormatCollateExtended(collate_oid, FORMAT_COLLATE_FORCE_QUALIFY); +} + + +/* + * FormatCollateExtended - inspired by format_type_extended + * Generate a possibly-qualified collate name. + * + * The default behavior is to only qualify if the type is not in the search + * path, and to raise an error if a non-existent collate_oid is given. + * + * The following bits in 'flags' modify the behavior: + * - FORMAT_COLLATE_FORCE_QUALIFY + * always schema-qualify collate names, regardless of search_path + * + * Returns a palloc'd string. + */ +char * +FormatCollateExtended(Oid collid, bits16 flags) +{ + HeapTuple tuple = NULL; + Form_pg_collation collform = NULL; + char *buf = NULL; + char *nspname = NULL; + char *typname = NULL; + + if (collid == InvalidOid && (flags & FORMAT_COLLATE_ALLOW_INVALID) != 0) + { + return pstrdup("-"); + } + + tuple = SearchSysCache1(COLLOID, ObjectIdGetDatum(collid)); + if (!HeapTupleIsValid(tuple)) + { + if ((flags & FORMAT_COLLATE_ALLOW_INVALID) != 0) + { + return pstrdup("???"); + } + else + { + elog(ERROR, "cache lookup failed for collate %u", collid); + } + } + collform = (Form_pg_collation) GETSTRUCT(tuple); + + if ((flags & FORMAT_COLLATE_FORCE_QUALIFY) == 0 && CollationIsVisible(collid)) + { + nspname = NULL; + } + else + { + nspname = get_namespace_name_or_temp(collform->collnamespace); + } + + typname = NameStr(collform->collname); + + buf = quote_qualified_identifier(nspname, typname); + + ReleaseSysCache(tuple); + + return buf; +} diff --git a/src/backend/distributed/deparser/objectaddress.c b/src/backend/distributed/deparser/objectaddress.c new file mode 100644 index 000000000..40719fc81 --- /dev/null +++ b/src/backend/distributed/deparser/objectaddress.c @@ -0,0 +1,191 @@ +/*------------------------------------------------------------------------- + * + * objectaddress.c + * Parstrees almost always target a object that postgres can address by + * an ObjectAddress. Here we have a walker for parsetrees to find the + * address of the object targeted. + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/commands.h" +#include "distributed/deparser.h" + +static const ObjectAddress * AlterTableStmtObjectAddress(AlterTableStmt *stmt, + bool missing_ok); +static const ObjectAddress * RenameStmtObjectAddress(RenameStmt *stmt, bool missing_ok); +static const ObjectAddress * AlterObjectSchemaStmtObjectAddress( + AlterObjectSchemaStmt *stmt, bool missing_ok); +static const ObjectAddress * RenameAttributeStmtObjectAddress(RenameStmt *stmt, + bool missing_ok); +static const ObjectAddress * AlterOwnerStmtObjectAddress(AlterOwnerStmt *stmt, + bool missing_ok); + + +/* + * GetObjectAddressFromParseTree returns the ObjectAdderss of the main target of the parse + * tree. + */ +const ObjectAddress * +GetObjectAddressFromParseTree(Node *parseTree, bool missing_ok) +{ + switch (parseTree->type) + { + case T_CompositeTypeStmt: + { + return CompositeTypeStmtObjectAddress(castNode(CompositeTypeStmt, parseTree), + missing_ok); + } + + case T_AlterTableStmt: + { + return AlterTableStmtObjectAddress(castNode(AlterTableStmt, parseTree), + missing_ok); + } + + case T_CreateEnumStmt: + { + return CreateEnumStmtObjectAddress(castNode(CreateEnumStmt, parseTree), + missing_ok); + } + + case T_AlterEnumStmt: + { + return AlterEnumStmtObjectAddress(castNode(AlterEnumStmt, parseTree), + missing_ok); + } + + case T_RenameStmt: + { + return RenameStmtObjectAddress(castNode(RenameStmt, parseTree), missing_ok); + } + + case T_AlterObjectSchemaStmt: + { + return AlterObjectSchemaStmtObjectAddress(castNode(AlterObjectSchemaStmt, + parseTree), missing_ok); + } + + case T_AlterOwnerStmt: + { + return AlterOwnerStmtObjectAddress(castNode(AlterOwnerStmt, parseTree), + missing_ok); + } + + default: + { + /* + * should not be reached, indicates the coordinator is sending unsupported + * statements + */ + ereport(ERROR, (errmsg("unsupported statement to get object address for"))); + return NULL; + } + } +} + + +static const ObjectAddress * +AlterTableStmtObjectAddress(AlterTableStmt *stmt, bool missing_ok) +{ + switch (stmt->relkind) + { + case OBJECT_TYPE: + { + return AlterTypeStmtObjectAddress(stmt, missing_ok); + } + + default: + { + ereport(ERROR, (errmsg("unsupported alter statement to get object address for" + ))); + } + } +} + + +static const ObjectAddress * +RenameStmtObjectAddress(RenameStmt *stmt, bool missing_ok) +{ + switch (stmt->renameType) + { + case OBJECT_TYPE: + { + return RenameTypeStmtObjectAddress(stmt, missing_ok); + } + + case OBJECT_ATTRIBUTE: + { + return RenameAttributeStmtObjectAddress(stmt, missing_ok); + } + + default: + { + ereport(ERROR, (errmsg("unsupported rename statement to get object address " + "for"))); + } + } +} + + +static const ObjectAddress * +AlterObjectSchemaStmtObjectAddress(AlterObjectSchemaStmt *stmt, bool missing_ok) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + return AlterTypeSchemaStmtObjectAddress(stmt, missing_ok); + } + + default: + { + ereport(ERROR, (errmsg("unsupported alter schema statement to get object " + "address for"))); + } + } +} + + +static const ObjectAddress * +RenameAttributeStmtObjectAddress(RenameStmt *stmt, bool missing_ok) +{ + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + + switch (stmt->relationType) + { + case OBJECT_TYPE: + { + return RenameTypeAttributeStmtObjectAddress(stmt, missing_ok); + } + + default: + { + ereport(ERROR, (errmsg("unsupported alter rename attribute statement to get " + "object address for"))); + } + } +} + + +static const ObjectAddress * +AlterOwnerStmtObjectAddress(AlterOwnerStmt *stmt, bool missing_ok) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + return AlterTypeOwnerObjectAddress(stmt, missing_ok); + } + + default: + { + ereport(ERROR, (errmsg("unsupported alter owner statement to get object " + "address for"))); + } + } +} diff --git a/src/backend/distributed/deparser/qualify.c b/src/backend/distributed/deparser/qualify.c new file mode 100644 index 000000000..e8ea1333a --- /dev/null +++ b/src/backend/distributed/deparser/qualify.c @@ -0,0 +1,198 @@ +/*------------------------------------------------------------------------- + * + * qualify.c + * The deparser relies on fully qualified names on all statements to + * work transparently on a remote worker node. Here we have helpers to + * fully qualify parsetrees. + * + * Fully qualified parsetrees contain names for all identifiers that + * are search_path agnostic. Meaning we need to include the schema name + * for each and every identifier in the parsetree. + * + * This file contains mostly the distpatching functions to specialized + * functions for each class of objects. eg qualify_type_stmt.c contains + * all functions related to fully qualifying parsetrees that interact + * with types. + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/deparser.h" + + +static void QualifyRenameStmt(RenameStmt *stmt); +static void QualifyRenameAttributeStmt(RenameStmt *stmt); +static void QualifyAlterTableStmt(AlterTableStmt *stmt); +static void QualifyAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt); +static void QualifyAlterOwnerStmt(AlterOwnerStmt *stmt); + + +/* + * QualifyTreeNode transforms the statement in place and makes all (supported) statements + * fully qualified. Fully qualified statements allow for application on a remote postgres + * server irregardless of their search_path. + */ +void +QualifyTreeNode(Node *stmt) +{ + switch (nodeTag(stmt)) + { + case T_RenameStmt: + { + QualifyRenameStmt(castNode(RenameStmt, stmt)); + return; + } + + case T_AlterEnumStmt: + { + QualifyAlterEnumStmt(castNode(AlterEnumStmt, stmt)); + return; + } + + case T_AlterTableStmt: + { + QualifyAlterTableStmt(castNode(AlterTableStmt, stmt)); + return; + } + + case T_CompositeTypeStmt: + { + QualifyCompositeTypeStmt(castNode(CompositeTypeStmt, stmt)); + return; + } + + case T_CreateEnumStmt: + { + QualifyCreateEnumStmt(castNode(CreateEnumStmt, stmt)); + return; + } + + case T_AlterObjectSchemaStmt: + { + QualifyAlterObjectSchemaStmt(castNode(AlterObjectSchemaStmt, stmt)); + return; + } + + case T_AlterOwnerStmt: + { + QualifyAlterOwnerStmt(castNode(AlterOwnerStmt, stmt)); + return; + } + + default: + { + /* skip unsupported statements */ + break; + } + } +} + + +static void +QualifyRenameStmt(RenameStmt *stmt) +{ + switch (stmt->renameType) + { + case OBJECT_TYPE: + { + QualifyRenameTypeStmt(stmt); + return; + } + + case OBJECT_ATTRIBUTE: + { + QualifyRenameAttributeStmt(stmt); + return; + } + + default: + { + /* skip unsupported statements */ + break; + } + } +} + + +static void +QualifyRenameAttributeStmt(RenameStmt *stmt) +{ + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + + switch (stmt->relationType) + { + case OBJECT_TYPE: + { + QualifyRenameTypeAttributeStmt(stmt); + return; + } + + default: + { + return; + } + } +} + + +static void +QualifyAlterTableStmt(AlterTableStmt *stmt) +{ + switch (stmt->relkind) + { + case OBJECT_TYPE: + { + QualifyAlterTypeStmt(stmt); + return; + } + + default: + { + /* skip unsupported statements */ + break; + } + } +} + + +static void +QualifyAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + QualifyAlterTypeSchemaStmt(stmt); + return; + } + + default: + { + /* skip unsupported statements */ + break; + } + } +} + + +static void +QualifyAlterOwnerStmt(AlterOwnerStmt *stmt) +{ + switch (stmt->objectType) + { + case OBJECT_TYPE: + { + QualifyAlterTypeOwnerStmt(stmt); + return; + } + + default: + { + return; + } + } +} diff --git a/src/backend/distributed/deparser/qualify_type_stmt.c b/src/backend/distributed/deparser/qualify_type_stmt.c new file mode 100644 index 000000000..6642f35bf --- /dev/null +++ b/src/backend/distributed/deparser/qualify_type_stmt.c @@ -0,0 +1,190 @@ +/*------------------------------------------------------------------------- + * + * qualify_type_stmt.c + * Functions specialized in fully qualifying all type statements. These + * functions are dispatched from qualify.c + * + * Fully qualifying type statements consists of adding the schema name + * to the subject of the types as well as any other branch of the + * parsetree. + * + * Goal would be that the deparser functions for these statements can + * serialize the statement without any external lookups. + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "access/heapam.h" +#include "access/htup_details.h" +#include "catalog/namespace.h" +#include "catalog/objectaddress.h" +#include "catalog/pg_type.h" +#include "distributed/deparser.h" +#include "distributed/metadata/namespace.h" +#include "nodes/makefuncs.h" +#include "parser/parse_type.h" +#include "utils/lsyscache.h" + +static char * GetTypeNamespaceNameByNameList(List *names); +static Oid TypeOidGetNamespaceOid(Oid typeOid); + +/* + * GetTypeNamespaceNameByNameList resolved the schema name of a type by its namelist. + */ +static char * +GetTypeNamespaceNameByNameList(List *names) +{ + TypeName *typeName = makeTypeNameFromNameList(names); + Oid typeOid = LookupTypeNameOid(NULL, typeName, false); + Oid namespaceOid = TypeOidGetNamespaceOid(typeOid); + char *nspname = get_namespace_name_or_temp(namespaceOid); + return nspname; +} + + +/* + * TypeOidGetNamespaceOid resolves the namespace oid for a type identified by its type oid + */ +static Oid +TypeOidGetNamespaceOid(Oid typeOid) +{ + Form_pg_type typeData = NULL; + Relation catalog = heap_open(TypeRelationId, AccessShareLock); +#if PG_VERSION_NUM >= 120000 + HeapTuple typeTuple = get_catalog_object_by_oid(catalog, Anum_pg_type_oid, typeOid); +#else + HeapTuple typeTuple = get_catalog_object_by_oid(catalog, typeOid); +#endif + heap_close(catalog, AccessShareLock); + + typeData = (Form_pg_type) GETSTRUCT(typeTuple); + + return typeData->typnamespace; +} + + +void +QualifyRenameTypeStmt(RenameStmt *stmt) +{ + List *names = (List *) stmt->object; + + Assert(stmt->renameType == OBJECT_TYPE); + + if (list_length(names) == 1) + { + /* not qualified, lookup name and add namespace name to names */ + char *nspname = GetTypeNamespaceNameByNameList(names); + names = list_make2(makeString(nspname), linitial(names)); + + stmt->object = (Node *) names; + } +} + + +void +QualifyRenameTypeAttributeStmt(RenameStmt *stmt) +{ + Assert(stmt->renameType == OBJECT_ATTRIBUTE); + Assert(stmt->relationType == OBJECT_TYPE); + + if (stmt->relation->schemaname == NULL) + { + List *names = list_make1(makeString(stmt->relation->relname)); + char *nspname = GetTypeNamespaceNameByNameList(names); + stmt->relation->schemaname = nspname; + } +} + + +void +QualifyAlterEnumStmt(AlterEnumStmt *stmt) +{ + List *names = stmt->typeName; + + if (list_length(names) == 1) + { + /* not qualified, lookup name and add namespace name to names */ + char *nspname = GetTypeNamespaceNameByNameList(names); + names = list_make2(makeString(nspname), linitial(names)); + + stmt->typeName = names; + } +} + + +void +QualifyAlterTypeStmt(AlterTableStmt *stmt) +{ + Assert(stmt->relkind == OBJECT_TYPE); + + if (stmt->relation->schemaname == NULL) + { + List *names = MakeNameListFromRangeVar(stmt->relation); + char *nspname = GetTypeNamespaceNameByNameList(names); + stmt->relation->schemaname = nspname; + } +} + + +void +QualifyCompositeTypeStmt(CompositeTypeStmt *stmt) +{ + if (stmt->typevar->schemaname == NULL) + { + Oid creationSchema = RangeVarGetCreationNamespace(stmt->typevar); + stmt->typevar->schemaname = get_namespace_name(creationSchema); + } +} + + +void +QualifyCreateEnumStmt(CreateEnumStmt *stmt) +{ + if (list_length(stmt->typeName) == 1) + { + char *objname = NULL; + Oid creationSchema = QualifiedNameGetCreationNamespace(stmt->typeName, &objname); + stmt->typeName = list_make2(makeString(get_namespace_name(creationSchema)), + linitial(stmt->typeName)); + } +} + + +void +QualifyAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt) +{ + List *names = NIL; + + Assert(stmt->objectType == OBJECT_TYPE); + + names = (List *) stmt->object; + if (list_length(names) == 1) + { + /* not qualified with schema, lookup type and its schema s*/ + char *nspname = GetTypeNamespaceNameByNameList(names); + names = list_make2(makeString(nspname), linitial(names)); + stmt->object = (Node *) names; + } +} + + +void +QualifyAlterTypeOwnerStmt(AlterOwnerStmt *stmt) +{ + List *names = NIL; + + Assert(stmt->objectType == OBJECT_TYPE); + + names = (List *) stmt->object; + if (list_length(names) == 1) + { + /* not qualified with schema, lookup type and its schema s*/ + char *nspname = GetTypeNamespaceNameByNameList(names); + names = list_make2(makeString(nspname), linitial(names)); + stmt->object = (Node *) names; + } +} diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index 5bddd4f57..460d399c1 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -373,6 +373,12 @@ AcquireExecutorMultiShardLocks(List *taskList) Task *task = (Task *) lfirst(taskCell); LOCKMODE lockMode = NoLock; + if (task->anchorShardId == INVALID_SHARD_ID) + { + /* no shard locks to take if the task is not anchored to a shard */ + continue; + } + if (AllModificationsCommutative || list_length(task->taskPlacementList) == 1) { /* @@ -1465,7 +1471,6 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn int64 totalAffectedTupleCount = 0; ListCell *taskCell = NULL; Task *firstTask = NULL; - ShardInterval *firstShardInterval = NULL; int connectionFlags = 0; List *affectedTupleCountList = NIL; HTAB *shardConnectionHash = NULL; @@ -1486,11 +1491,15 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn * ProcessUtility, so we only need to do this for DML commands. */ firstTask = (Task *) linitial(taskList); - firstShardInterval = LoadShardInterval(firstTask->anchorShardId); - if (PartitionedTable(firstShardInterval->relationId) && - firstTask->taskType == MODIFY_TASK) + if (firstTask->taskType == MODIFY_TASK) { - LockPartitionRelations(firstShardInterval->relationId, RowExclusiveLock); + ShardInterval *firstShardInterval = NULL; + + firstShardInterval = LoadShardInterval(firstTask->anchorShardId); + if (PartitionedTable(firstShardInterval->relationId)) + { + LockPartitionRelations(firstShardInterval->relationId, RowExclusiveLock); + } } /* diff --git a/src/backend/distributed/master/master_repair_shards.c b/src/backend/distributed/master/master_repair_shards.c index a74ad5f85..2195191a2 100644 --- a/src/backend/distributed/master/master_repair_shards.c +++ b/src/backend/distributed/master/master_repair_shards.c @@ -322,6 +322,7 @@ RepairShardPlacement(int64 shardId, char *sourceNodeName, int32 sourceNodePort, ddlCommandList = lappend(ddlCommandList, copyShardDataCommand->data); } + EnsureNoModificationsHaveBeenDone(); SendCommandListToWorkerInSingleTransaction(targetNodeName, targetNodePort, tableOwner, ddlCommandList); diff --git a/src/backend/distributed/metadata/dependency.c b/src/backend/distributed/metadata/dependency.c index fed532d32..8c427107e 100644 --- a/src/backend/distributed/metadata/dependency.c +++ b/src/backend/distributed/metadata/dependency.c @@ -16,7 +16,9 @@ #include "access/skey.h" #include "catalog/dependency.h" #include "catalog/indexing.h" +#include "catalog/pg_class.h" #include "catalog/pg_depend.h" +#include "catalog/pg_type.h" #include "distributed/metadata/dependency.h" #include "distributed/metadata/distobject.h" #include "utils/fmgroids.h" @@ -55,6 +57,7 @@ static void recurse_pg_depend(const ObjectAddress *target, static bool FollowAllSupportedDependencies(void *context, Form_pg_depend pg_depend); static bool FollowNewSupportedDependencies(void *context, Form_pg_depend pg_depend); static void ApplyAddToDependencyList(void *context, Form_pg_depend pg_depend); +static List * ExpandCitusSupportedTypes(void *context, const ObjectAddress *target); /* forward declaration of support functions to decide what to follow */ static bool SupportedDependencyByCitus(const ObjectAddress *address); @@ -74,7 +77,7 @@ GetDependenciesForObject(const ObjectAddress *target) InitObjectAddressCollector(&collector); recurse_pg_depend(target, - NULL, + &ExpandCitusSupportedTypes, &FollowNewSupportedDependencies, &ApplyAddToDependencyList, &collector); @@ -114,7 +117,7 @@ OrderObjectAddressListInDependencyOrder(List *objectAddressList) } recurse_pg_depend(objectAddress, - NULL, + &ExpandCitusSupportedTypes, &FollowAllSupportedDependencies, &ApplyAddToDependencyList, &collector); @@ -313,6 +316,52 @@ SupportedDependencyByCitus(const ObjectAddress *address) return true; } + case OCLASS_TYPE: + { + switch (get_typtype(address->objectId)) + { + case TYPTYPE_ENUM: + case TYPTYPE_COMPOSITE: + { + return true; + } + + case TYPTYPE_BASE: + { + /* + * array types should be followed but not created, as they get created + * by the original type. + */ + return type_is_array(address->objectId); + } + + default: + { + /* type not supported */ + return false; + } + } + + /* + * should be unreachable, break here is to make sure the function has a path + * without return, instead of falling through to the next block */ + break; + } + + case OCLASS_CLASS: + { + /* + * composite types have a reference to a relation of composite type, we need + * to follow those to get the dependencies of type fields. + */ + if (get_rel_relkind(address->objectId) == RELKIND_COMPOSITE_TYPE) + { + return true; + } + + return false; + } + default: { /* unsupported type */ @@ -487,3 +536,78 @@ ApplyAddToDependencyList(void *context, Form_pg_depend pg_depend) CollectObjectAddress(collector, &address); } + + +/* + * ExpandCitusSupportedTypes base on supported types by citus we might want to expand + * the list of objects to visit in pg_depend. + * + * An example where we want to expand is for types. Their dependencies are not captured + * with an entry in pg_depend from their object address, but by the object address of the + * relation describing the type. + */ +static List * +ExpandCitusSupportedTypes(void *context, const ObjectAddress *target) +{ + List *result = NIL; + + switch (target->classId) + { + case TypeRelationId: + { + /* + * types depending on other types are not captured in pg_depend, instead they + * are described with their dependencies by the relation that describes the + * composite type. + */ + if (get_typtype(target->objectId) == TYPTYPE_COMPOSITE) + { + Form_pg_depend dependency = palloc0(sizeof(FormData_pg_depend)); + dependency->classid = target->classId; + dependency->objid = target->objectId; + dependency->objsubid = target->objectSubId; + + /* add outward edge to the type's relation */ + dependency->refclassid = RelationRelationId; + dependency->refobjid = get_typ_typrelid(target->objectId); + dependency->refobjsubid = 0; + + dependency->deptype = DEPENDENCY_NORMAL; + + result = lappend(result, dependency); + } + + /* + * array types don't have a normal dependency on their element type, instead + * their dependency is an internal one. We can't follow interal dependencies + * as that would cause a cyclic dependency on others, instead we expand here + * to follow the dependency on the element type. + */ + if (type_is_array(target->objectId)) + { + Form_pg_depend dependency = palloc0(sizeof(FormData_pg_depend)); + dependency->classid = target->classId; + dependency->objid = target->objectId; + dependency->objsubid = target->objectSubId; + + /* add outward edge to the element type */ + dependency->refclassid = TypeRelationId; + dependency->refobjid = get_element_type(target->objectId); + dependency->refobjsubid = 0; + + dependency->deptype = DEPENDENCY_NORMAL; + + result = lappend(result, dependency); + } + + break; + } + + default: + { + /* no expansion for unsupported types */ + break; + } + } + return result; +} diff --git a/src/backend/distributed/metadata/distobject.c b/src/backend/distributed/metadata/distobject.c index f46a8c073..0e034bbb2 100644 --- a/src/backend/distributed/metadata/distobject.c +++ b/src/backend/distributed/metadata/distobject.c @@ -89,6 +89,11 @@ master_unmark_object_distributed(PG_FUNCTION_ARGS) bool ObjectExists(const ObjectAddress *address) { + if (address == NULL) + { + return false; + } + if (is_objectclass_supported(address->classId)) { HeapTuple objtup; diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index ad42bd7f9..b8fe709ba 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -145,6 +145,7 @@ start_metadata_sync_to_node(PG_FUNCTION_ARGS) * createMetadataSnapshotCommandList in the same transaction that we send * nodeDeleteCommand and nodeInsertCommand commands below. */ + EnsureNoModificationsHaveBeenDone(); SendCommandListToWorkerInSingleTransaction(nodeNameString, nodePort, extensionOwner, recreateMetadataSnapshotCommandList); diff --git a/src/backend/distributed/metadata/namespace.c b/src/backend/distributed/metadata/namespace.c new file mode 100644 index 000000000..a468311d4 --- /dev/null +++ b/src/backend/distributed/metadata/namespace.c @@ -0,0 +1,35 @@ + +#include "postgres.h" + +#include "nodes/primnodes.h" +#include "nodes/value.h" + +#include "distributed/metadata/namespace.h" + +/* + * MakeNameListFromRangeVar makes a namelist from a RangeVar. Its behaviour should be the + * exact opposite of postgres' makeRangeVarFromNameList. + */ +List * +MakeNameListFromRangeVar(const RangeVar *rel) +{ + if (rel->catalogname != NULL) + { + Assert(rel->schemaname != NULL); + Assert(rel->relname != NULL); + return list_make3(makeString(rel->catalogname), + makeString(rel->schemaname), + makeString(rel->relname)); + } + else if (rel->schemaname != NULL) + { + Assert(rel->relname != NULL); + return list_make2(makeString(rel->schemaname), + makeString(rel->relname)); + } + else + { + Assert(rel->relname != NULL); + return list_make1(makeString(rel->relname)); + } +} diff --git a/src/backend/distributed/transaction/relation_access_tracking.c b/src/backend/distributed/transaction/relation_access_tracking.c index 21f080aaa..11ac91d0a 100644 --- a/src/backend/distributed/transaction/relation_access_tracking.c +++ b/src/backend/distributed/transaction/relation_access_tracking.c @@ -439,7 +439,10 @@ RecordRelationParallelDDLAccessForTask(Task *task) lastRelationId = currentRelationId; } - RecordParallelDDLAccess(RelationIdForShard(task->anchorShardId)); + if (task->anchorShardId != INVALID_SHARD_ID) + { + RecordParallelDDLAccess(RelationIdForShard(task->anchorShardId)); + } } diff --git a/src/backend/distributed/transaction/worker_transaction.c b/src/backend/distributed/transaction/worker_transaction.c index 152499843..064ec82f4 100644 --- a/src/backend/distributed/transaction/worker_transaction.c +++ b/src/backend/distributed/transaction/worker_transaction.c @@ -38,11 +38,23 @@ * 2PC. */ void -SendCommandToWorker(char *nodeName, int32 nodePort, char *command) +SendCommandToWorker(char *nodeName, int32 nodePort, const char *command) +{ + const char *nodeUser = CitusExtensionOwnerName(); + SendCommandToWorkerAsUser(nodeName, nodePort, nodeUser, command); +} + + +/* + * SendCommandToWorkerAsUSer sends a command to a particular worker as a particular user + * as part of the 2PC. + */ +void +SendCommandToWorkerAsUser(char *nodeName, int32 nodePort, const char *nodeUser, + const char *command) { MultiConnection *transactionConnection = NULL; - char *nodeUser = CitusExtensionOwnerName(); - int connectionFlags = 0; + uint connectionFlags = 0; BeginOrContinueCoordinatedTransaction(); CoordinatedTransactionUse2PC(); @@ -87,9 +99,42 @@ SendCommandToFirstWorker(char *command) * owner to ensure write access to the Citus metadata tables. */ void -SendCommandToWorkers(TargetWorkerSet targetWorkerSet, char *command) +SendCommandToWorkers(TargetWorkerSet targetWorkerSet, const char *command) { - SendCommandToWorkersParams(targetWorkerSet, command, 0, NULL, NULL); + SendCommandToWorkersParams(targetWorkerSet, command, CitusExtensionOwnerName(), + 0, NULL, NULL); +} + + +/* + * TargetWorkerSetNodeList returns a list of WorkerNode's that satisfies the + * TargetWorkerSet. + */ +List * +TargetWorkerSetNodeList(TargetWorkerSet targetWorkerSet) +{ + List *workerNodeList = ActivePrimaryNodeList(); + ListCell *workerNodeCell = NULL; + List *result = NIL; + + foreach(workerNodeCell, workerNodeList) + { + WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); + if (targetWorkerSet == WORKERS_WITH_METADATA && + !workerNode->hasMetadata) + { + continue; + } + if (targetWorkerSet == OTHER_WORKERS && + workerNode->groupId == GetLocalGroupId()) + { + continue; + } + + result = lappend(result, workerNode); + } + + return result; } @@ -103,7 +148,7 @@ SendCommandToWorkers(TargetWorkerSet targetWorkerSet, char *command) void SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList) { - List *workerNodeList = ActivePrimaryNodeList(); + List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet); ListCell *workerNodeCell = NULL; char *nodeUser = CitusExtensionOwnerName(); ListCell *commandCell = NULL; @@ -117,18 +162,6 @@ SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList) int nodePort = workerNode->workerPort; int connectionFlags = FORCE_NEW_CONNECTION; - if (targetWorkerSet == WORKERS_WITH_METADATA && - !workerNode->hasMetadata) - { - continue; - } - - if (targetWorkerSet == OTHER_WORKERS && - workerNode->groupId == GetLocalGroupId()) - { - continue; - } - workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName, nodePort, nodeUser, NULL); @@ -145,6 +178,52 @@ SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList) } +/* + * SendBareOptionalCommandListToWorkersAsUser sends a list of commands to a set of target + * workers in serial. Commands are committed immediately: new connections are + * always used and no transaction block is used (hence "bare"). + */ +int +SendBareOptionalCommandListToWorkersAsUser(TargetWorkerSet targetWorkerSet, + List *commandList, const char *user) +{ + List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet); + ListCell *workerNodeCell = NULL; + ListCell *commandCell = NULL; + int maxError = RESPONSE_OKAY; + + /* run commands serially */ + foreach(workerNodeCell, workerNodeList) + { + MultiConnection *workerConnection = NULL; + WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); + char *nodeName = workerNode->workerName; + int nodePort = workerNode->workerPort; + int connectionFlags = FORCE_NEW_CONNECTION; + + workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName, + nodePort, user, NULL); + + /* iterate over the commands and execute them in the same connection */ + foreach(commandCell, commandList) + { + char *commandString = lfirst(commandCell); + int result = ExecuteOptionalRemoteCommand(workerConnection, commandString, + NULL); + if (result != RESPONSE_OKAY) + { + maxError = Max(maxError, result); + break; + } + } + + CloseConnection(workerConnection); + } + + return maxError; +} + + /* * SendCommandToWorkersParams sends a command to all workers in parallel. * Commands are committed on the workers when the local transaction commits. The @@ -154,15 +233,14 @@ SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList) * respectively. */ void -SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command, - int parameterCount, const Oid *parameterTypes, - const char *const *parameterValues) +SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, const char *command, + const char *user, int parameterCount, + const Oid *parameterTypes, const char *const *parameterValues) { List *connectionList = NIL; ListCell *connectionCell = NULL; - List *workerNodeList = ActivePrimaryNodeList(); + List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet); ListCell *workerNodeCell = NULL; - char *nodeUser = CitusExtensionOwnerName(); BeginOrContinueCoordinatedTransaction(); CoordinatedTransactionUse2PC(); @@ -174,22 +252,10 @@ SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command, char *nodeName = workerNode->workerName; int nodePort = workerNode->workerPort; MultiConnection *connection = NULL; - int connectionFlags = 0; - - if (targetWorkerSet == WORKERS_WITH_METADATA && - !workerNode->hasMetadata) - { - continue; - } - - if (targetWorkerSet == OTHER_WORKERS && - workerNode->groupId == GetLocalGroupId()) - { - continue; - } + int32 connectionFlags = 0; connection = StartNodeUserDatabaseConnection(connectionFlags, nodeName, nodePort, - nodeUser, NULL); + user, NULL); MarkRemoteTransactionCritical(connection); @@ -197,12 +263,7 @@ SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command, } /* finish opening connections */ - foreach(connectionCell, connectionList) - { - MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); - - FinishConnectionEstablishment(connection); - } + FinishConnectionListEstablishment(connectionList); RemoteTransactionsBeginIfNecessary(connectionList); @@ -238,25 +299,34 @@ SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command, /* - * SendCommandListToWorkerInSingleTransaction opens connection to the node with the given - * nodeName and nodePort. Then, the connection starts a transaction on the remote - * node and executes the commands in the transaction. The function raises error if - * any of the queries fails. + * EnsureNoModificationsHaveBeenDone reports an error if we have performed any + * modification in the current transaction to prevent opening a connection is such cases. */ void -SendCommandListToWorkerInSingleTransaction(char *nodeName, int32 nodePort, char *nodeUser, - List *commandList) +EnsureNoModificationsHaveBeenDone() { - MultiConnection *workerConnection = NULL; - ListCell *commandCell = NULL; - int connectionFlags = FORCE_NEW_CONNECTION; - if (XactModificationLevel > XACT_MODIFICATION_NONE) { ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION), errmsg("cannot open new connections after the first modification " "command within a transaction"))); } +} + + +/* + * SendCommandListToWorkerInSingleTransaction opens connection to the node with the given + * nodeName and nodePort. Then, the connection starts a transaction on the remote + * node and executes the commands in the transaction. The function raises error if + * any of the queries fails. + */ +void +SendCommandListToWorkerInSingleTransaction(const char *nodeName, int32 nodePort, + const char *nodeUser, List *commandList) +{ + MultiConnection *workerConnection = NULL; + ListCell *commandCell = NULL; + int connectionFlags = FORCE_NEW_CONNECTION; workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName, nodePort, nodeUser, NULL); diff --git a/src/backend/distributed/utils/node_metadata.c b/src/backend/distributed/utils/node_metadata.c index 4fb789c0d..b35b4e421 100644 --- a/src/backend/distributed/utils/node_metadata.c +++ b/src/backend/distributed/utils/node_metadata.c @@ -468,6 +468,7 @@ ActivateNode(char *nodeName, int nodePort) if (WorkerNodeIsPrimary(workerNode)) { + EnsureNoModificationsHaveBeenDone(); ReplicateAllDependenciesToNode(nodeName, nodePort); ReplicateAllReferenceTablesToNode(nodeName, nodePort); } diff --git a/src/backend/distributed/utils/reference_table_utils.c b/src/backend/distributed/utils/reference_table_utils.c index 09b33d408..b8a0af0a4 100644 --- a/src/backend/distributed/utils/reference_table_utils.c +++ b/src/backend/distributed/utils/reference_table_utils.c @@ -316,6 +316,7 @@ ReplicateShardToNode(ShardInterval *shardInterval, char *nodeName, int nodePort) get_rel_name(shardInterval->relationId), nodeName, nodePort))); + EnsureNoModificationsHaveBeenDone(); SendCommandListToWorkerInSingleTransaction(nodeName, nodePort, tableOwner, ddlCommandList); if (targetPlacement == NULL) diff --git a/src/backend/distributed/worker/worker_create_or_replace.c b/src/backend/distributed/worker/worker_create_or_replace.c new file mode 100644 index 000000000..324c4c688 --- /dev/null +++ b/src/backend/distributed/worker/worker_create_or_replace.c @@ -0,0 +1,180 @@ +/*------------------------------------------------------------------------- + * + * worker_create_if_not_exist.c + * TODO rename file and document, was named after old function + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "catalog/dependency.h" +#include "catalog/pg_type.h" +#include "fmgr.h" +#include "nodes/makefuncs.h" +#include "nodes/nodes.h" +#include "parser/parse_type.h" +#include "tcop/dest.h" +#include "tcop/utility.h" +#include "utils/builtins.h" +#include "utils/regproc.h" + +#include "distributed/commands.h" +#include "distributed/commands/utility_hook.h" +#include "distributed/deparser.h" +#include "distributed/metadata/distobject.h" +#include "distributed/worker_protocol.h" + +static Node * CreateStmtByObjectAddress(const ObjectAddress *address); +static DropStmt * CreateDropStmtBasedOnCreateStmt(Node *createStmt); + + +PG_FUNCTION_INFO_V1(worker_create_or_replace_object); + + +/* + * worker_create_or_replace_object(statement text) + * + * function is called, by the coordinator, with a CREATE statement for an object. This + * function implements the CREATE ... IF NOT EXISTS functionality for objects that do not + * have this functionality or where their implementation is not sufficient. + * + * Besides checking if an object of said name exists it tries to compare the object to be + * created with the one in the local catalog. If there is a difference the on in the local + * catalog will be renamed after which the statement can be executed on this worker to + * create the object. + * + * Renaming has two purposes + * - free the identifier for creation + * - non destructive if there is data store that would be destroyed if the object was + * used in a table on this node, eg. types. If the type would be dropped with a cascade + * it would drop any column holding user data for this type. + */ +Datum +worker_create_or_replace_object(PG_FUNCTION_ARGS) +{ + text *sqlStatementText = PG_GETARG_TEXT_P(0); + const char *sqlStatement = text_to_cstring(sqlStatementText); + const ObjectAddress *address = NULL; + Node *parseTree = ParseTreeNode(sqlStatement); + + /* + * since going to the drop statement might require some resolving we will do a check + * if the type actually exists instead of adding the IF EXISTS keyword to the + * statement. + */ + address = GetObjectAddressFromParseTree(parseTree, true); + if (ObjectExists(address)) + { + Node *localCreateStmt = NULL; + const char *localSqlStatement = NULL; + DropStmt *dropStmtParseTree = NULL; + + localCreateStmt = CreateStmtByObjectAddress(address); + localSqlStatement = DeparseTreeNode(localCreateStmt); + if (strcmp(sqlStatement, localSqlStatement) == 0) + { + /* + * TODO string compare is a poor mans comparison, but calling equal on the + * parsetree's returns false because there is extra information list character + * position of some sort + */ + + /* + * parseTree sent by the coordinator is the same as we would create for our + * object, therefore we can omit the create statement locally and not create + * the object as it already exists. + * + * We let the coordinator know we didn't create the object. + */ + PG_RETURN_BOOL(false); + } + + /* TODO don't drop, instead rename as described in documentation */ + + /* + * there might be dependencies left on the worker on this type, these are not + * managed by citus anyway so it should be ok to drop, thus we cascade to any such + * dependencies + */ + dropStmtParseTree = CreateDropStmtBasedOnCreateStmt(parseTree); + + if (dropStmtParseTree != NULL) + { + const char *sqlDropStmt = NULL; + + /* force the drop */ + dropStmtParseTree->behavior = DROP_CASCADE; + + sqlDropStmt = DeparseTreeNode((Node *) dropStmtParseTree); + CitusProcessUtility((Node *) dropStmtParseTree, sqlDropStmt, + PROCESS_UTILITY_TOPLEVEL, NULL, None_Receiver, NULL); + } + } + + /* apply create statement locally */ + CitusProcessUtility(parseTree, sqlStatement, PROCESS_UTILITY_TOPLEVEL, NULL, + None_Receiver, NULL); + + /* type has been created */ + PG_RETURN_BOOL(true); +} + + +/* + * CreateStmtByObjectAddress returns a parsetree that will recreate the object addressed + * by the ObjectAddress provided. + * + * Note: this tree does not contain position information that is normally in a parsetree, + * therefore you cannot equal this tree against parsed statement. Instead it can be + * deparsed to do a string comparison. + */ +static Node * +CreateStmtByObjectAddress(const ObjectAddress *address) +{ + switch (getObjectClass(address)) + { + case OCLASS_TYPE: + { + return CreateTypeStmtByObjectAddress(address); + } + + default: + { + ereport(ERROR, (errmsg("unsupported object to construct a create statment"))); + } + } +} + + +/* TODO will be removed as we will not drop but rename instead */ +static DropStmt * +CreateDropStmtBasedOnCreateStmt(Node *createStmt) +{ + switch (nodeTag(createStmt)) + { + case T_CompositeTypeStmt: + { + return CreateDropStmtBasedOnCompositeTypeStmt( + castNode(CompositeTypeStmt, createStmt)); + } + + case T_CreateEnumStmt: + { + return CreateDropStmtBasedOnEnumStmt(castNode(CreateEnumStmt, createStmt)); + } + + default: + { + /* + * should not be reached, indicates the coordinator is sending unsupported + * statements + */ + ereport(ERROR, (errmsg("unsupported statement to transform to a drop stmt"), + errhint("The coordinator send an unsupported command to the " + "worker"))); + } + } +} diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index 883b58439..4e95bf4a5 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -84,9 +84,14 @@ extern void ErrorIfUnsupportedRenameStmt(RenameStmt *renameStmt); /* schema.c - forward declarations */ extern void ProcessDropSchemaStmt(DropStmt *dropSchemaStatement); +extern List * PlanAlterTableSchemaStmt(AlterObjectSchemaStmt *stmt, + const char *queryString); extern List * PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, const char *alterObjectSchemaCommand); +extern void ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt, + const char *queryString); + /* sequence.c - forward declarations */ extern void ErrorIfUnsupportedSeqStmt(CreateSeqStmt *createSeqStmt); @@ -117,6 +122,42 @@ extern void ErrorIfUnsupportedConstraint(Relation relation, char distributionMet /* truncate.c - forward declarations */ extern void ProcessTruncateStatement(TruncateStmt *truncateStatement); +/* type.c - forward declarations */ +extern List * PlanCompositeTypeStmt(CompositeTypeStmt *stmt, const char *queryString); +extern void ProcessCompositeTypeStmt(CompositeTypeStmt *stmt, const char *queryString); +extern List * PlanAlterTypeStmt(AlterTableStmt *stmt, const char *queryString); +extern List * PlanCreateEnumStmt(CreateEnumStmt *createEnumStmt, const char *queryString); +extern void ProcessCreateEnumStmt(CreateEnumStmt *stmt, const char *queryString); +extern List * PlanAlterEnumStmt(AlterEnumStmt *stmt, const char *queryString); +extern void ProcessAlterEnumStmt(AlterEnumStmt *stmt, const char *queryString); +extern List * PlanDropTypeStmt(DropStmt *stmt, const char *queryString); +extern List * PlanRenameTypeStmt(RenameStmt *stmt, const char *queryString); +extern List * PlanRenameTypeAttributeStmt(RenameStmt *stmt, const char *queryString); +extern List * PlanAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt, + const char *queryString); +extern List * PlanAlterTypeOwnerStmt(AlterOwnerStmt *stmt, const char *queryString); +extern void ProcessAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt, + const char *queryString); +extern Node * CreateTypeStmtByObjectAddress(const ObjectAddress *address); +extern const ObjectAddress * CompositeTypeStmtObjectAddress(CompositeTypeStmt *stmt, + bool missing_ok); +extern const ObjectAddress * CreateEnumStmtObjectAddress(CreateEnumStmt *stmt, + bool missing_ok); +extern const ObjectAddress * AlterTypeStmtObjectAddress(AlterTableStmt *stmt, + bool missing_ok); +extern const ObjectAddress * AlterEnumStmtObjectAddress(AlterEnumStmt *stmt, + bool missing_ok); +extern const ObjectAddress * RenameTypeStmtObjectAddress(RenameStmt *stmt, + bool missing_ok); +extern const ObjectAddress * AlterTypeSchemaStmtObjectAddress(AlterObjectSchemaStmt *stmt, + bool missing_ok); +extern const ObjectAddress * RenameTypeAttributeStmtObjectAddress(RenameStmt *stmt, + bool missing_ok); +extern const ObjectAddress * AlterTypeOwnerObjectAddress(AlterOwnerStmt *stmt, + bool missing_ok); +extern DropStmt * CreateDropStmtBasedOnCompositeTypeStmt(CompositeTypeStmt *stmt); +extern DropStmt * CreateDropStmtBasedOnEnumStmt(CreateEnumStmt *stmt); +extern List * CreateTypeDDLCommandsIdempotent(const ObjectAddress *typeAddress); /* vacuum.c - froward declarations */ extern void ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand); diff --git a/src/include/distributed/commands/utility_hook.h b/src/include/distributed/commands/utility_hook.h index 113e82eb0..7a22a6623 100644 --- a/src/include/distributed/commands/utility_hook.h +++ b/src/include/distributed/commands/utility_hook.h @@ -12,10 +12,12 @@ #include "postgres.h" -#include "distributed/version_compat.h" #include "utils/relcache.h" #include "tcop/utility.h" +#include "distributed/version_compat.h" +#include "distributed/worker_transaction.h" + typedef enum { PROPSETCMD_INVALID = -1, @@ -52,6 +54,7 @@ extern void CitusProcessUtility(Node *node, const char *queryString, extern void MarkInvalidateForeignKeyGraph(void); extern void InvalidateForeignKeyGraphForDDL(void); extern List * DDLTaskList(Oid relationId, const char *commandString); +extern List * NodeDDLTaskList(TargetWorkerSet targets, List *commands); extern bool AlterTableInProgress(void); #endif /* MULTI_UTILITY_H */ diff --git a/src/include/distributed/deparser.h b/src/include/distributed/deparser.h new file mode 100644 index 000000000..e6417e5ee --- /dev/null +++ b/src/include/distributed/deparser.h @@ -0,0 +1,55 @@ +/*------------------------------------------------------------------------- + * + * deparser.h + * Used when deparsing any ddl parsetree into its sql from. + * + * Copyright (c) 2019, Citus Data, Inc. + * + * $Id$ + * + *------------------------------------------------------------------------- + */ + +#ifndef CITUS_DEPARSER_H +#define CITUS_DEPARSER_H + +#include "postgres.h" + +#include "nodes/nodes.h" +#include "nodes/parsenodes.h" +#include "catalog/objectaddress.h" + +/* forward declarations for format_collate.c */ +/* Control flags for FormatCollateExtended, compatible with format_type_extended */ +#define FORMAT_COLLATE_ALLOW_INVALID 0x02 /* allow invalid types */ +#define FORMAT_COLLATE_FORCE_QUALIFY 0x04 /* force qualification of collate */ +extern char * FormatCollateBE(Oid collate_oid); +extern char * FormatCollateBEQualified(Oid collate_oid); +extern char * FormatCollateExtended(Oid collid, bits16 flags); + +extern void QualifyTreeNode(Node *stmt); +extern const char * DeparseTreeNode(Node *stmt); + +extern const char * DeparseCompositeTypeStmt(CompositeTypeStmt *stmt); +extern const char * DeparseCreateEnumStmt(CreateEnumStmt *stmt); +extern const char * DeparseDropTypeStmt(DropStmt *stmt); +extern const char * DeparseAlterEnumStmt(AlterEnumStmt *stmt); +extern const char * DeparseAlterTypeStmt(AlterTableStmt *stmt); +extern const char * DeparseRenameTypeStmt(RenameStmt *stmt); +extern const char * DeparseRenameTypeAttributeStmt(RenameStmt *stmt); +extern const char * DeparseAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt); +extern const char * DeparseAlterTypeOwnerStmt(AlterOwnerStmt *stmt); + +extern void QualifyRenameTypeStmt(RenameStmt *stmt); +extern void QualifyRenameTypeAttributeStmt(RenameStmt *stmt); +extern void QualifyAlterEnumStmt(AlterEnumStmt *stmt); +extern void QualifyAlterTypeStmt(AlterTableStmt *stmt); +extern void QualifyCompositeTypeStmt(CompositeTypeStmt *stmt); +extern void QualifyCreateEnumStmt(CreateEnumStmt *stmt); +extern void QualifyAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt); +extern void QualifyAlterTypeOwnerStmt(AlterOwnerStmt *stmt); + +extern const ObjectAddress * GetObjectAddressFromParseTree(Node *parseTree, bool + missing_ok); + +#endif /* CITUS_DEPARSER_H */ diff --git a/src/include/distributed/metadata/namespace.h b/src/include/distributed/metadata/namespace.h new file mode 100644 index 000000000..c3db7ca90 --- /dev/null +++ b/src/include/distributed/metadata/namespace.h @@ -0,0 +1,20 @@ +/*------------------------------------------------------------------------- + * + * namespace.h + * Helper functions for citus to work with postgres namespaces/schemas + * + * Copyright (c) 2019, Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#ifndef CITUS_NAMESPACE_H +#define CITUS_NAMESPACE_H + +#include "postgres.h" + +#include "nodes/primnodes.h" + +extern List * MakeNameListFromRangeVar(const RangeVar *rel); + +#endif /*CITUS_NAMESPACE_H */ diff --git a/src/include/distributed/remote_commands.h b/src/include/distributed/remote_commands.h index 0321feb2d..ca8c9266e 100644 --- a/src/include/distributed/remote_commands.h +++ b/src/include/distributed/remote_commands.h @@ -14,6 +14,7 @@ #include "distributed/connection_management.h" /* errors which ExecuteRemoteCommand might return */ +#define RESPONSE_OKAY 0 #define QUERY_SEND_FAILED 1 #define RESPONSE_NOT_OKAY 2 diff --git a/src/include/distributed/worker_transaction.h b/src/include/distributed/worker_transaction.h index 73c1eb3a2..3ac470b5f 100644 --- a/src/include/distributed/worker_transaction.h +++ b/src/include/distributed/worker_transaction.h @@ -29,16 +29,26 @@ typedef enum TargetWorkerSet /* Functions declarations for worker transactions */ extern List * GetWorkerTransactions(void); -extern void SendCommandToWorker(char *nodeName, int32 nodePort, char *command); +extern List * TargetWorkerSetNodeList(TargetWorkerSet targetWorkerSet); +extern void SendCommandToWorker(char *nodeName, int32 nodePort, const char *command); +extern void SendCommandToWorkerAsUser(char *nodeName, int32 nodePort, + const char *nodeUser, const char *command); extern void SendCommandToFirstWorker(char *command); -extern void SendCommandToWorkers(TargetWorkerSet targetWorkerSet, char *command); +extern void SendCommandToWorkers(TargetWorkerSet targetWorkerSet, const char *command); extern void SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList); -extern void SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command, +extern int SendBareOptionalCommandListToWorkersAsUser(TargetWorkerSet targetWorkerSet, + List *commandList, + const char *user); +extern void SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, + const char *command, const char *user, int parameterCount, const Oid *parameterTypes, const char *const *parameterValues); -extern void SendCommandListToWorkerInSingleTransaction(char *nodeName, int32 nodePort, - char *nodeUser, List *commandList); +extern void EnsureNoModificationsHaveBeenDone(void); +extern void SendCommandListToWorkerInSingleTransaction(const char *nodeName, + int32 nodePort, + const char *nodeUser, + List *commandList); extern void RemoveWorkerTransaction(char *nodeName, int32 nodePort); /* helper functions for worker transactions */ diff --git a/src/test/regress/Makefile b/src/test/regress/Makefile index 5398ecaff..1364c1b6e 100644 --- a/src/test/regress/Makefile +++ b/src/test/regress/Makefile @@ -57,6 +57,11 @@ check-base: all $(pg_regress_multi_check) --load-extension=citus \ -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/base_schedule $(EXTRA_TESTS) +check-base-non-adaptive: all tempinstall-main + $(pg_regress_multi_check) --load-extension=citus \ + --server-option=citus.task_executor_type=real-time \ + -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/base_schedule $(EXTRA_TESTS) + # check-minimal only sets up the cluster check-minimal: all $(pg_regress_multi_check) --load-extension=citus \ diff --git a/src/test/regress/expected/distributed_types.out b/src/test/regress/expected/distributed_types.out new file mode 100644 index 000000000..01f2a1182 --- /dev/null +++ b/src/test/regress/expected/distributed_types.out @@ -0,0 +1,338 @@ +SET citus.next_shard_id TO 20010000; +CREATE USER typeuser; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +SELECT run_command_on_workers($$CREATE USER typeuser;$$); + run_command_on_workers +----------------------------------- + (localhost,57637,t,"CREATE ROLE") + (localhost,57638,t,"CREATE ROLE") +(2 rows) + +CREATE SCHEMA type_tests AUTHORIZATION typeuser; +CREATE SCHEMA type_tests2 AUTHORIZATION typeuser; -- to test creation in a specific schema and moving to schema +SET search_path TO type_tests; +SET citus.shard_count TO 4; +-- single statement transactions with a simple type used in a table +CREATE TYPE tc1 AS (a int, b int); +CREATE TABLE t1 (a int PRIMARY KEY, b tc1); +SELECT create_distributed_table('t1','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t1 VALUES (1, (2,3)::tc1); +SELECT * FROM t1; + a | b +---+------- + 1 | (2,3) +(1 row) + +ALTER TYPE tc1 RENAME TO tc1_newname; +INSERT INTO t1 VALUES (3, (4,5)::tc1_newname); -- insert with a cast would fail if the rename didn't propagate +ALTER TYPE tc1_newname SET SCHEMA type_tests2; +INSERT INTO t1 VALUES (6, (7,8)::type_tests2.tc1_newname); -- insert with a cast would fail if the rename didn't propagate +-- single statement transactions with a an enum used in a table +CREATE TYPE te1 AS ENUM ('one', 'two', 'three'); +CREATE TABLE t2 (a int PRIMARY KEY, b te1); +SELECT create_distributed_table('t2','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t2 VALUES (1, 'two'); +SELECT * FROM t2; + a | b +---+----- + 1 | two +(1 row) + +-- rename enum, subsequent operations on the type would fail if the rename was not propagated +ALTER TYPE te1 RENAME TO te1_newname; +-- add an extra value to the enum and use in table +ALTER TYPE te1_newname ADD VALUE 'four'; +UPDATE t2 SET b = 'four'; +SELECT * FROM t2; + a | b +---+------ + 1 | four +(1 row) + +-- change the schema of the type and use the new fully qualified name in an insert +ALTER TYPE te1_newname SET SCHEMA type_tests2; +INSERT INTO t2 VALUES (3, 'three'::type_tests2.te1_newname); +-- transaction block with simple type +BEGIN; +CREATE TYPE tc2 AS (a int, b int); +CREATE TABLE t3 (a int PRIMARY KEY, b tc2); +SELECT create_distributed_table('t3','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t3 VALUES (4, (5,6)::tc2); +SELECT * FROM t3; + a | b +---+------- + 4 | (5,6) +(1 row) + +COMMIT; +-- transaction block with simple type +BEGIN; +CREATE TYPE te2 AS ENUM ('yes', 'no'); +CREATE TABLE t4 (a int PRIMARY KEY, b te2); +SELECT create_distributed_table('t4','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t4 VALUES (1, 'yes'); +SELECT * FROM t4; + a | b +---+----- + 1 | yes +(1 row) + +-- ALTER TYPE ... ADD VALUE does not work in transactions +COMMIT; +-- test some combination of types without ddl propagation, this will prevent the workers +-- from having those types created. They are created just-in-time on table distribution +SET citus.enable_ddl_propagation TO off; +CREATE TYPE tc3 AS (a int, b int); +CREATE TYPE tc4 AS (a int, b tc3[]); +CREATE TYPE tc5 AS (a int, b tc4); +CREATE TYPE te3 AS ENUM ('a','b'); +RESET citus.enable_ddl_propagation; +CREATE TABLE t5 (a int PRIMARY KEY, b tc5[], c te3); +SELECT create_distributed_table('t5','a'); + create_distributed_table +-------------------------- + +(1 row) + +-- test adding an attribute to a type and a column to a table both for a non-distributed type +SET citus.enable_ddl_propagation TO off; +CREATE TYPE te4 AS ENUM ('c','d'); +CREATE TYPE tc6 AS (a int, b int); +CREATE TYPE tc6c AS (a int, b int); +RESET citus.enable_ddl_propagation; +-- types need to be fully qualified because of the search_path which is not supported by ALTER TYPE ... ADD COLUMN +ALTER TABLE t5 ADD COLUMN d type_tests.te4; +ALTER TABLE t5 ADD COLUMN e type_tests.tc6; +ALTER TYPE tc6 ADD ATTRIBUTE c tc6c; +-- last two values are only there if above commands succeeded +INSERT INTO t5 VALUES (1, NULL, 'a', 'd', (1,2,(4,5)::tc6c)::tc6); +-- test renaming an attribute of a distrbuted type and read it by its new name to verify propagation +ALTER TYPE tc6 RENAME ATTRIBUTE b TO d; +SELECT (e::tc6).d FROM t5 ORDER BY 1; + d +--- + 2 +(1 row) + +-- change owner of supported types and check ownership on remote server +ALTER TYPE te4 OWNER TO typeuser; +SELECT typname, usename FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid; + typname | usename +---------+---------- + te4 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(te4,typeuser)") + (localhost,57638,t,"(te4,typeuser)") +(2 rows) + +ALTER TYPE tc6 OWNER TO typeuser; +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid; + typname | usename +---------+---------- + tc6 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(tc6,typeuser)") + (localhost,57638,t,"(tc6,typeuser)") +(2 rows) + +-- create a type as a different user +SET ROLE typeuser; +-- create directly on the worker +CREATE TYPE tc7 AS (a int, b int); +CREATE TYPE te5 AS ENUM ('a','b','c'); +-- cascade to the worker when table gets created +SET citus.enable_ddl_propagation TO off; +CREATE TYPE tc8 AS (a int, b int); +CREATE TYPE te6 AS ENUM ('a','b','c'); +RESET citus.enable_ddl_propagation; +CREATE TABLE t6 (a int, b tc8, c te6); +SELECT create_distributed_table('t6', 'a'); + create_distributed_table +-------------------------- + +(1 row) + +RESET ROLE; +-- test ownership of all types +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid; + typname | usename +---------+---------- + tc7 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(tc7,typeuser)") + (localhost,57638,t,"(tc7,typeuser)") +(2 rows) + +SELECT typname, usename FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid; + typname | usename +---------+---------- + te5 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(te5,typeuser)") + (localhost,57638,t,"(te5,typeuser)") +(2 rows) + +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid; + typname | usename +---------+---------- + tc8 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(tc8,typeuser)") + (localhost,57638,t,"(tc8,typeuser)") +(2 rows) + +SELECT typname, usename FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid; + typname | usename +---------+---------- + te6 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(te6,typeuser)") + (localhost,57638,t,"(te6,typeuser)") +(2 rows) + +-- deleting the enum cascade will remove the type from the table and the workers +DROP TYPE te3 CASCADE; +NOTICE: drop cascades to column c of table t5 +-- DELETE multiple types at once +DROP TYPE tc3, tc4, tc5 CASCADE; +NOTICE: drop cascades to column b of table t5 +-- test if the types are deleted +SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname; + typname +--------- +(0 rows) + +SELECT run_command_on_workers($$SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;$$); + run_command_on_workers +------------------------ + (localhost,57637,t,"") + (localhost,57638,t,"") +(2 rows) + +-- make sure attribute names are quoted correctly, no errors indicates types are propagated correctly +CREATE TYPE tc9 AS ("field-with-dashes" text COLLATE "en_US"); +ALTER TYPE tc9 ADD ATTRIBUTE "some-more" int, ADD ATTRIBUTE normal int; +ALTER TYPE tc9 RENAME ATTRIBUTE normal TO "not-so-normal"; +-- test alter statements for non-distributed types, if they would be propagated they would +-- error, preventing from changing them +SET citus.enable_ddl_propagation TO off; +CREATE TYPE non_distributed_composite_type AS (a int, b int); +CREATE TYPE non_distributed_enum_type AS ENUM ('a', 'c'); +SET citus.enable_ddl_propagation TO on; +ALTER TYPE non_distributed_composite_type ADD ATTRIBUTE c int; +ALTER TYPE non_distributed_composite_type RENAME ATTRIBUTE c TO d; +ALTER TYPE non_distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE; +ALTER TYPE non_distributed_composite_type DROP ATTRIBUTE d; +ALTER TYPE non_distributed_composite_type OWNER TO typeuser; +ALTER TYPE non_distributed_composite_type RENAME TO non_distributed_composite_type_renamed; +ALTER TYPE non_distributed_composite_type_renamed RENAME TO non_distributed_composite_type; +ALTER TYPE non_distributed_composite_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.non_distributed_composite_type SET SCHEMA type_tests; +ALTER TYPE non_distributed_enum_type OWNER TO typeuser; +ALTER TYPE non_distributed_enum_type RENAME TO non_distributed_enum_type_renamed; +ALTER TYPE non_distributed_enum_type_renamed RENAME TO non_distributed_enum_type; +ALTER TYPE non_distributed_enum_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.non_distributed_enum_type SET SCHEMA type_tests; +ALTER TYPE non_distributed_enum_type ADD VALUE 'b' BEFORE 'c'; +ALTER TYPE non_distributed_enum_type ADD VALUE 'd' AFTER 'c'; +ALTER TYPE non_distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff'; +-- test all forms of alter statements on distributed types +CREATE TYPE distributed_composite_type AS (a int, b int); +CREATE TYPE distributed_enum_type AS ENUM ('a', 'c'); +-- enforce distribution of types in every case +CREATE TABLE type_proc (a int, b distributed_composite_type, c distributed_enum_type); +SELECT create_distributed_table('type_proc','a'); + create_distributed_table +-------------------------- + +(1 row) + +DROP TABLE type_proc; +ALTER TYPE distributed_composite_type ADD ATTRIBUTE c int; +ALTER TYPE distributed_composite_type RENAME ATTRIBUTE c TO d; +ALTER TYPE distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE; +ALTER TYPE distributed_composite_type DROP ATTRIBUTE d; +ALTER TYPE distributed_composite_type OWNER TO typeuser; +ALTER TYPE distributed_composite_type RENAME TO distributed_composite_type_renamed; +ALTER TYPE distributed_composite_type_renamed RENAME TO distributed_composite_type; +ALTER TYPE distributed_composite_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.distributed_composite_type SET SCHEMA type_tests; +ALTER TYPE distributed_enum_type OWNER TO typeuser; +ALTER TYPE distributed_enum_type RENAME TO distributed_enum_type_renamed; +ALTER TYPE distributed_enum_type_renamed RENAME TO distributed_enum_type; +ALTER TYPE distributed_enum_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.distributed_enum_type SET SCHEMA type_tests; +ALTER TYPE distributed_enum_type ADD VALUE 'b' BEFORE 'c'; +ALTER TYPE distributed_enum_type ADD VALUE 'd' AFTER 'c'; +ALTER TYPE distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff'; +-- clear objects +SET client_min_messages TO fatal; -- suppress cascading objects dropping +DROP SCHEMA type_tests CASCADE; +SELECT run_command_on_workers($$DROP SCHEMA type_tests CASCADE;$$); + run_command_on_workers +----------------------------------- + (localhost,57637,t,"DROP SCHEMA") + (localhost,57638,t,"DROP SCHEMA") +(2 rows) + +DROP SCHEMA type_tests2 CASCADE; +SELECT run_command_on_workers($$DROP SCHEMA type_tests2 CASCADE;$$); + run_command_on_workers +----------------------------------- + (localhost,57637,t,"DROP SCHEMA") + (localhost,57638,t,"DROP SCHEMA") +(2 rows) + +DROP USER typeuser; +SELECT run_command_on_workers($$DROP USER typeuser;$$); + run_command_on_workers +--------------------------------- + (localhost,57637,t,"DROP ROLE") + (localhost,57638,t,"DROP ROLE") +(2 rows) + diff --git a/src/test/regress/expected/distributed_types_0.out b/src/test/regress/expected/distributed_types_0.out new file mode 100644 index 000000000..a62d32cb1 --- /dev/null +++ b/src/test/regress/expected/distributed_types_0.out @@ -0,0 +1,338 @@ +SET citus.next_shard_id TO 20010000; +CREATE USER typeuser; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +HINT: Connect to worker nodes directly to manually create all necessary users and roles. +SELECT run_command_on_workers($$CREATE USER typeuser;$$); + run_command_on_workers +----------------------------------- + (localhost,57637,t,"CREATE ROLE") + (localhost,57638,t,"CREATE ROLE") +(2 rows) + +CREATE SCHEMA type_tests AUTHORIZATION typeuser; +CREATE SCHEMA type_tests2 AUTHORIZATION typeuser; -- to test creation in a specific schema and moving to schema +SET search_path TO type_tests; +SET citus.shard_count TO 4; +-- single statement transactions with a simple type used in a table +CREATE TYPE tc1 AS (a int, b int); +CREATE TABLE t1 (a int PRIMARY KEY, b tc1); +SELECT create_distributed_table('t1','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t1 VALUES (1, (2,3)::tc1); +SELECT * FROM t1; + a | b +---+------- + 1 | (2,3) +(1 row) + +ALTER TYPE tc1 RENAME TO tc1_newname; +INSERT INTO t1 VALUES (3, (4,5)::tc1_newname); -- insert with a cast would fail if the rename didn't propagate +ALTER TYPE tc1_newname SET SCHEMA type_tests2; +INSERT INTO t1 VALUES (6, (7,8)::type_tests2.tc1_newname); -- insert with a cast would fail if the rename didn't propagate +-- single statement transactions with a an enum used in a table +CREATE TYPE te1 AS ENUM ('one', 'two', 'three'); +CREATE TABLE t2 (a int PRIMARY KEY, b te1); +SELECT create_distributed_table('t2','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t2 VALUES (1, 'two'); +SELECT * FROM t2; + a | b +---+----- + 1 | two +(1 row) + +-- rename enum, subsequent operations on the type would fail if the rename was not propagated +ALTER TYPE te1 RENAME TO te1_newname; +-- add an extra value to the enum and use in table +ALTER TYPE te1_newname ADD VALUE 'four'; +UPDATE t2 SET b = 'four'; +SELECT * FROM t2; + a | b +---+------ + 1 | four +(1 row) + +-- change the schema of the type and use the new fully qualified name in an insert +ALTER TYPE te1_newname SET SCHEMA type_tests2; +INSERT INTO t2 VALUES (3, 'three'::type_tests2.te1_newname); +-- transaction block with simple type +BEGIN; +CREATE TYPE tc2 AS (a int, b int); +CREATE TABLE t3 (a int PRIMARY KEY, b tc2); +SELECT create_distributed_table('t3','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t3 VALUES (4, (5,6)::tc2); +SELECT * FROM t3; + a | b +---+------- + 4 | (5,6) +(1 row) + +COMMIT; +-- transaction block with simple type +BEGIN; +CREATE TYPE te2 AS ENUM ('yes', 'no'); +CREATE TABLE t4 (a int PRIMARY KEY, b te2); +SELECT create_distributed_table('t4','a'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO t4 VALUES (1, 'yes'); +SELECT * FROM t4; + a | b +---+----- + 1 | yes +(1 row) + +-- ALTER TYPE ... ADD VALUE does not work in transactions +COMMIT; +-- test some combination of types without ddl propagation, this will prevent the workers +-- from having those types created. They are created just-in-time on table distribution +SET citus.enable_ddl_propagation TO off; +CREATE TYPE tc3 AS (a int, b int); +CREATE TYPE tc4 AS (a int, b tc3[]); +CREATE TYPE tc5 AS (a int, b tc4); +CREATE TYPE te3 AS ENUM ('a','b'); +RESET citus.enable_ddl_propagation; +CREATE TABLE t5 (a int PRIMARY KEY, b tc5[], c te3); +SELECT create_distributed_table('t5','a'); + create_distributed_table +-------------------------- + +(1 row) + +-- test adding an attribute to a type and a column to a table both for a non-distributed type +SET citus.enable_ddl_propagation TO off; +CREATE TYPE te4 AS ENUM ('c','d'); +CREATE TYPE tc6 AS (a int, b int); +CREATE TYPE tc6c AS (a int, b int); +RESET citus.enable_ddl_propagation; +-- types need to be fully qualified because of the search_path which is not supported by ALTER TYPE ... ADD COLUMN +ALTER TABLE t5 ADD COLUMN d type_tests.te4; +ALTER TABLE t5 ADD COLUMN e type_tests.tc6; +ALTER TYPE tc6 ADD ATTRIBUTE c tc6c; +-- last two values are only there if above commands succeeded +INSERT INTO t5 VALUES (1, NULL, 'a', 'd', (1,2,(4,5)::tc6c)::tc6); +-- test renaming an attribute of a distrbuted type and read it by its new name to verify propagation +ALTER TYPE tc6 RENAME ATTRIBUTE b TO d; +SELECT (e::tc6).d FROM t5 ORDER BY 1; + d +--- + 2 +(1 row) + +-- change owner of supported types and check ownership on remote server +ALTER TYPE te4 OWNER TO typeuser; +SELECT typname, usename FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid; + typname | usename +---------+---------- + te4 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(te4,typeuser)") + (localhost,57638,t,"(te4,typeuser)") +(2 rows) + +ALTER TYPE tc6 OWNER TO typeuser; +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid; + typname | usename +---------+---------- + tc6 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(tc6,typeuser)") + (localhost,57638,t,"(tc6,typeuser)") +(2 rows) + +-- create a type as a different user +SET ROLE typeuser; +-- create directly on the worker +CREATE TYPE tc7 AS (a int, b int); +CREATE TYPE te5 AS ENUM ('a','b','c'); +-- cascade to the worker when table gets created +SET citus.enable_ddl_propagation TO off; +CREATE TYPE tc8 AS (a int, b int); +CREATE TYPE te6 AS ENUM ('a','b','c'); +RESET citus.enable_ddl_propagation; +CREATE TABLE t6 (a int, b tc8, c te6); +SELECT create_distributed_table('t6', 'a'); + create_distributed_table +-------------------------- + +(1 row) + +RESET ROLE; +-- test ownership of all types +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid; + typname | usename +---------+---------- + tc7 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(tc7,typeuser)") + (localhost,57638,t,"(tc7,typeuser)") +(2 rows) + +SELECT typname, usename FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid; + typname | usename +---------+---------- + te5 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(te5,typeuser)") + (localhost,57638,t,"(te5,typeuser)") +(2 rows) + +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid; + typname | usename +---------+---------- + tc8 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(tc8,typeuser)") + (localhost,57638,t,"(tc8,typeuser)") +(2 rows) + +SELECT typname, usename FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid; + typname | usename +---------+---------- + te6 | typeuser +(1 row) + +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;$$); + run_command_on_workers +-------------------------------------- + (localhost,57637,t,"(te6,typeuser)") + (localhost,57638,t,"(te6,typeuser)") +(2 rows) + +-- deleting the enum cascade will remove the type from the table and the workers +DROP TYPE te3 CASCADE; +NOTICE: drop cascades to table t5 column c +-- DELETE multiple types at once +DROP TYPE tc3, tc4, tc5 CASCADE; +NOTICE: drop cascades to table t5 column b +-- test if the types are deleted +SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname; + typname +--------- +(0 rows) + +SELECT run_command_on_workers($$SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;$$); + run_command_on_workers +------------------------ + (localhost,57637,t,"") + (localhost,57638,t,"") +(2 rows) + +-- make sure attribute names are quoted correctly, no errors indicates types are propagated correctly +CREATE TYPE tc9 AS ("field-with-dashes" text COLLATE "en_US"); +ALTER TYPE tc9 ADD ATTRIBUTE "some-more" int, ADD ATTRIBUTE normal int; +ALTER TYPE tc9 RENAME ATTRIBUTE normal TO "not-so-normal"; +-- test alter statements for non-distributed types, if they would be propagated they would +-- error, preventing from changing them +SET citus.enable_ddl_propagation TO off; +CREATE TYPE non_distributed_composite_type AS (a int, b int); +CREATE TYPE non_distributed_enum_type AS ENUM ('a', 'c'); +SET citus.enable_ddl_propagation TO on; +ALTER TYPE non_distributed_composite_type ADD ATTRIBUTE c int; +ALTER TYPE non_distributed_composite_type RENAME ATTRIBUTE c TO d; +ALTER TYPE non_distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE; +ALTER TYPE non_distributed_composite_type DROP ATTRIBUTE d; +ALTER TYPE non_distributed_composite_type OWNER TO typeuser; +ALTER TYPE non_distributed_composite_type RENAME TO non_distributed_composite_type_renamed; +ALTER TYPE non_distributed_composite_type_renamed RENAME TO non_distributed_composite_type; +ALTER TYPE non_distributed_composite_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.non_distributed_composite_type SET SCHEMA type_tests; +ALTER TYPE non_distributed_enum_type OWNER TO typeuser; +ALTER TYPE non_distributed_enum_type RENAME TO non_distributed_enum_type_renamed; +ALTER TYPE non_distributed_enum_type_renamed RENAME TO non_distributed_enum_type; +ALTER TYPE non_distributed_enum_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.non_distributed_enum_type SET SCHEMA type_tests; +ALTER TYPE non_distributed_enum_type ADD VALUE 'b' BEFORE 'c'; +ALTER TYPE non_distributed_enum_type ADD VALUE 'd' AFTER 'c'; +ALTER TYPE non_distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff'; +-- test all forms of alter statements on distributed types +CREATE TYPE distributed_composite_type AS (a int, b int); +CREATE TYPE distributed_enum_type AS ENUM ('a', 'c'); +-- enforce distribution of types in every case +CREATE TABLE type_proc (a int, b distributed_composite_type, c distributed_enum_type); +SELECT create_distributed_table('type_proc','a'); + create_distributed_table +-------------------------- + +(1 row) + +DROP TABLE type_proc; +ALTER TYPE distributed_composite_type ADD ATTRIBUTE c int; +ALTER TYPE distributed_composite_type RENAME ATTRIBUTE c TO d; +ALTER TYPE distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE; +ALTER TYPE distributed_composite_type DROP ATTRIBUTE d; +ALTER TYPE distributed_composite_type OWNER TO typeuser; +ALTER TYPE distributed_composite_type RENAME TO distributed_composite_type_renamed; +ALTER TYPE distributed_composite_type_renamed RENAME TO distributed_composite_type; +ALTER TYPE distributed_composite_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.distributed_composite_type SET SCHEMA type_tests; +ALTER TYPE distributed_enum_type OWNER TO typeuser; +ALTER TYPE distributed_enum_type RENAME TO distributed_enum_type_renamed; +ALTER TYPE distributed_enum_type_renamed RENAME TO distributed_enum_type; +ALTER TYPE distributed_enum_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.distributed_enum_type SET SCHEMA type_tests; +ALTER TYPE distributed_enum_type ADD VALUE 'b' BEFORE 'c'; +ALTER TYPE distributed_enum_type ADD VALUE 'd' AFTER 'c'; +ALTER TYPE distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff'; +-- clear objects +SET client_min_messages TO fatal; -- suppress cascading objects dropping +DROP SCHEMA type_tests CASCADE; +SELECT run_command_on_workers($$DROP SCHEMA type_tests CASCADE;$$); + run_command_on_workers +----------------------------------- + (localhost,57637,t,"DROP SCHEMA") + (localhost,57638,t,"DROP SCHEMA") +(2 rows) + +DROP SCHEMA type_tests2 CASCADE; +SELECT run_command_on_workers($$DROP SCHEMA type_tests2 CASCADE;$$); + run_command_on_workers +----------------------------------- + (localhost,57637,t,"DROP SCHEMA") + (localhost,57638,t,"DROP SCHEMA") +(2 rows) + +DROP USER typeuser; +SELECT run_command_on_workers($$DROP USER typeuser;$$); + run_command_on_workers +--------------------------------- + (localhost,57637,t,"DROP ROLE") + (localhost,57638,t,"DROP ROLE") +(2 rows) + diff --git a/src/test/regress/expected/failure_create_distributed_table_non_empty.out b/src/test/regress/expected/failure_create_distributed_table_non_empty.out index 1d57554d0..d6ed907f0 100644 --- a/src/test/regress/expected/failure_create_distributed_table_non_empty.out +++ b/src/test/regress/expected/failure_create_distributed_table_non_empty.out @@ -97,7 +97,7 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); run_command_on_workers ------------------------ - (localhost,9060,t,1) + (localhost,9060,t,0) (localhost,57637,t,1) (2 rows) @@ -108,6 +108,15 @@ SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_n (localhost,57637,t,"DROP SCHEMA") (2 rows) +-- this triggers a schema creation which prevents further transactions around dependency propagation +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- kill as soon as the coordinator sends begin SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy @@ -639,6 +648,15 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W (localhost,57637,t,0) (2 rows) +-- this triggers a schema creation which prevents further transactions around dependency propagation +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- kill as soon as the coordinator sends begin SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy diff --git a/src/test/regress/expected/failure_create_distributed_table_non_empty_9.out b/src/test/regress/expected/failure_create_distributed_table_non_empty_9.out index 7783b4c16..8b5acb091 100644 --- a/src/test/regress/expected/failure_create_distributed_table_non_empty_9.out +++ b/src/test/regress/expected/failure_create_distributed_table_non_empty_9.out @@ -97,7 +97,7 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); run_command_on_workers ------------------------ - (localhost,9060,t,1) + (localhost,9060,t,0) (localhost,57637,t,1) (2 rows) @@ -108,6 +108,15 @@ SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_n (localhost,57637,t,"DROP SCHEMA") (2 rows) +-- this triggers a schema creation which prevents further transactions around dependency propagation +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- kill as soon as the coordinator sends begin SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy @@ -635,6 +644,15 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W (localhost,57637,t,0) (2 rows) +-- this triggers a schema creation which prevents further transactions around dependency propagation +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- kill as soon as the coordinator sends begin SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy diff --git a/src/test/regress/expected/failure_create_reference_table.out b/src/test/regress/expected/failure_create_reference_table.out index abeaa40a8..713fbec2a 100644 --- a/src/test/regress/expected/failure_create_reference_table.out +++ b/src/test/regress/expected/failure_create_reference_table.out @@ -10,6 +10,10 @@ SELECT citus.mitmproxy('conn.allow()'); (1 row) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; CREATE TABLE ref_table(id int); INSERT INTO ref_table VALUES(1),(2),(3); -- Kill on sending first query to worker node, should error @@ -21,10 +25,10 @@ SELECT citus.mitmproxy('conn.onQuery().kill()'); (1 row) SELECT create_reference_table('ref_table'); -ERROR: server closed the connection unexpectedly - This probably means the server terminated abnormally - before or while processing the request. +WARNING: connection not open CONTEXT: while executing command on localhost:9060 +ERROR: connection error: localhost:9060 +DETAIL: connection not open SELECT count(*) FROM pg_dist_shard_placement; count ------- @@ -106,7 +110,7 @@ SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY 3").kill()'); SELECT create_reference_table('ref_table'); NOTICE: Copying data from local table... -ERROR: failed to COPY to shard 10000004 on localhost:9060 +ERROR: failed to COPY to shard 10000005 on localhost:9060 SELECT count(*) FROM pg_dist_shard_placement; count ------- @@ -170,8 +174,8 @@ SELECT create_reference_table('ref_table'); SELECT shardid, nodeport, shardstate FROM pg_dist_shard_placement ORDER BY shardid, nodeport; shardid | nodeport | shardstate ----------+----------+------------ - 10000007 | 9060 | 1 - 10000007 | 57637 | 1 + 10000008 | 9060 | 1 + 10000008 | 57637 | 1 (2 rows) SET client_min_messages TO NOTICE; @@ -195,10 +199,11 @@ SELECT citus.mitmproxy('conn.onQuery().kill()'); BEGIN; SELECT create_reference_table('ref_table'); -ERROR: server closed the connection unexpectedly +WARNING: server closed the connection unexpectedly This probably means the server terminated abnormally before or while processing the request. CONTEXT: while executing command on localhost:9060 +ERROR: failure on connection marked as essential: localhost:9060 COMMIT; -- kill on ROLLBACK, should be rollbacked SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()'); diff --git a/src/test/regress/expected/failure_create_reference_table_9.out b/src/test/regress/expected/failure_create_reference_table_9.out index 1946c7e4f..9cd6b4a97 100644 --- a/src/test/regress/expected/failure_create_reference_table_9.out +++ b/src/test/regress/expected/failure_create_reference_table_9.out @@ -10,6 +10,10 @@ SELECT citus.mitmproxy('conn.allow()'); (1 row) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; CREATE TABLE ref_table(id int); INSERT INTO ref_table VALUES(1),(2),(3); -- Kill on sending first query to worker node, should error @@ -21,10 +25,10 @@ SELECT citus.mitmproxy('conn.onQuery().kill()'); (1 row) SELECT create_reference_table('ref_table'); -ERROR: server closed the connection unexpectedly +ERROR: connection error: localhost:9060 +DETAIL: server closed the connection unexpectedly This probably means the server terminated abnormally before or while processing the request. -CONTEXT: while executing command on localhost:9060 SELECT count(*) FROM pg_dist_shard_placement; count ------- @@ -106,7 +110,7 @@ SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY 3").kill()'); SELECT create_reference_table('ref_table'); NOTICE: Copying data from local table... -ERROR: failed to COPY to shard 10000004 on localhost:9060 +ERROR: failed to COPY to shard 10000005 on localhost:9060 SELECT count(*) FROM pg_dist_shard_placement; count ------- @@ -170,8 +174,8 @@ SELECT create_reference_table('ref_table'); SELECT shardid, nodeport, shardstate FROM pg_dist_shard_placement ORDER BY shardid, nodeport; shardid | nodeport | shardstate ----------+----------+------------ - 10000007 | 9060 | 1 - 10000007 | 57637 | 1 + 10000008 | 9060 | 1 + 10000008 | 57637 | 1 (2 rows) SET client_min_messages TO NOTICE; @@ -195,10 +199,11 @@ SELECT citus.mitmproxy('conn.onQuery().kill()'); BEGIN; SELECT create_reference_table('ref_table'); -ERROR: server closed the connection unexpectedly +WARNING: server closed the connection unexpectedly This probably means the server terminated abnormally before or while processing the request. CONTEXT: while executing command on localhost:9060 +ERROR: failure on connection marked as essential: localhost:9060 COMMIT; -- kill on ROLLBACK, should be rollbacked SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()'); diff --git a/src/test/regress/expected/failure_create_table.out b/src/test/regress/expected/failure_create_table.out index 5d376bdd9..8160981b2 100644 --- a/src/test/regress/expected/failure_create_table.out +++ b/src/test/regress/expected/failure_create_table.out @@ -77,6 +77,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata (localhost,57637,t,1) (2 rows) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- Now, kill the connection while opening transaction on workers. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy @@ -372,6 +376,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W (localhost,57637,t,0) (2 rows) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- Now, kill the connection while creating transaction on workers in transaction. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy @@ -522,6 +530,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W (localhost,57637,t,0) (2 rows) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy diff --git a/src/test/regress/expected/failure_create_table_9.out b/src/test/regress/expected/failure_create_table_9.out index cc4b90e94..9a74ae556 100644 --- a/src/test/regress/expected/failure_create_table_9.out +++ b/src/test/regress/expected/failure_create_table_9.out @@ -77,6 +77,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata (localhost,57637,t,1) (2 rows) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- Now, kill the connection while opening transaction on workers. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy @@ -367,6 +371,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W (localhost,57637,t,0) (2 rows) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- Now, kill the connection while creating transaction on workers in transaction. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy @@ -512,6 +520,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W (localhost,57637,t,0) (2 rows) +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; -- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); mitmproxy diff --git a/src/test/regress/expected/foreign_key_to_reference_table.out b/src/test/regress/expected/foreign_key_to_reference_table.out index 1eb149fa7..eee29f719 100644 --- a/src/test/regress/expected/foreign_key_to_reference_table.out +++ b/src/test/regress/expected/foreign_key_to_reference_table.out @@ -15,13 +15,6 @@ SET citus.shard_count TO 8; SET citus.next_shard_id TO 7000000; SET citus.next_placement_id TO 7000000; CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); -SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$); - run_command_on_workers ------------------------------------ - (localhost,57637,t,"CREATE TYPE") - (localhost,57638,t,"CREATE TYPE") -(2 rows) - CREATE VIEW table_fkeys_in_workers AS SELECT (json_populate_record(NULL::foreign_details, @@ -576,13 +569,6 @@ DROP TABLE referencing_table; DROP TABLE referenced_table; -- foreign key as composite key CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int); -SELECT run_command_on_workers($$CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int)$$) ORDER BY 1; - run_command_on_workers ------------------------------------ - (localhost,57637,t,"CREATE TYPE") - (localhost,57638,t,"CREATE TYPE") -(2 rows) - CREATE TABLE referenced_table(test_column composite, PRIMARY KEY(test_column)); CREATE TABLE referencing_table(id int, referencing_composite composite); SELECT create_reference_table('referenced_table'); diff --git a/src/test/regress/expected/intermediate_results.out b/src/test/regress/expected/intermediate_results.out index bb4a7671a..47223cc1f 100644 --- a/src/test/regress/expected/intermediate_results.out +++ b/src/test/regress/expected/intermediate_results.out @@ -126,13 +126,6 @@ ERROR: invalid input syntax for integer: "PGCOPY" END; -- try a composite type CREATE TYPE intermediate_results.square_type AS (x text, x2 int); -SELECT run_command_on_workers('CREATE TYPE intermediate_results.square_type AS (x text, x2 int)'); - run_command_on_workers ------------------------------------ - (localhost,57637,t,"CREATE TYPE") - (localhost,57638,t,"CREATE TYPE") -(2 rows) - CREATE TABLE stored_squares (user_id text, square intermediate_results.square_type, metadata jsonb); INSERT INTO stored_squares VALUES ('jon', '(2,4)'::intermediate_results.square_type, '{"value":2}'); INSERT INTO stored_squares VALUES ('jon', '(3,9)'::intermediate_results.square_type, '{"value":3}'); diff --git a/src/test/regress/expected/isolation_citus_dist_activity.out b/src/test/regress/expected/isolation_citus_dist_activity.out index 3120cc577..5f8d727ee 100644 --- a/src/test/regress/expected/isolation_citus_dist_activity.out +++ b/src/test/regress/expected/isolation_citus_dist_activity.out @@ -41,16 +41,16 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -SELECT worker_apply_shard_ddl_command (102141, 'public', ' +SELECT worker_apply_shard_ddl_command (102145, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT worker_apply_shard_ddl_command (102140, 'public', ' +SELECT worker_apply_shard_ddl_command (102144, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT worker_apply_shard_ddl_command (102139, 'public', ' +SELECT worker_apply_shard_ddl_command (102143, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT worker_apply_shard_ddl_command (102138, 'public', ' +SELECT worker_apply_shard_ddl_command (102142, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression step s2-rollback: @@ -104,7 +104,7 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -INSERT INTO public.test_table_102144 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression +INSERT INTO public.test_table_102148 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression step s2-rollback: ROLLBACK; @@ -159,10 +159,10 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -COPY (SELECT count(*) AS count FROM test_table_102149 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -COPY (SELECT count(*) AS count FROM test_table_102148 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression -COPY (SELECT count(*) AS count FROM test_table_102147 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -COPY (SELECT count(*) AS count FROM test_table_102146 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression +COPY (SELECT count(*) AS count FROM test_table_102153 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression +COPY (SELECT count(*) AS count FROM test_table_102152 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression +COPY (SELECT count(*) AS count FROM test_table_102151 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression +COPY (SELECT count(*) AS count FROM test_table_102150 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression step s2-rollback: ROLLBACK; @@ -217,7 +217,7 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -SELECT count(*) AS count FROM public.test_table_102151 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 0 idle Client ClientRead postgres regression +SELECT count(*) AS count FROM public.test_table_102155 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 0 idle Client ClientRead postgres regression step s2-rollback: ROLLBACK; diff --git a/src/test/regress/expected/isolation_citus_dist_activity_9.out b/src/test/regress/expected/isolation_citus_dist_activity_9.out index 686ad1fee..8f30ca0dd 100644 --- a/src/test/regress/expected/isolation_citus_dist_activity_9.out +++ b/src/test/regress/expected/isolation_citus_dist_activity_9.out @@ -41,16 +41,16 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -SELECT worker_apply_shard_ddl_command (102141, 'public', ' +SELECT worker_apply_shard_ddl_command (102145, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT worker_apply_shard_ddl_command (102140, 'public', ' +SELECT worker_apply_shard_ddl_command (102144, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT worker_apply_shard_ddl_command (102139, 'public', ' +SELECT worker_apply_shard_ddl_command (102143, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT worker_apply_shard_ddl_command (102138, 'public', ' +SELECT worker_apply_shard_ddl_command (102142, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression step s2-rollback: @@ -104,7 +104,7 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -INSERT INTO public.test_table_102144 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression +INSERT INTO public.test_table_102148 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression step s2-rollback: ROLLBACK; @@ -159,10 +159,10 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -SELECT count(*) AS count FROM test_table_102149 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT count(*) AS count FROM test_table_102148 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT count(*) AS count FROM test_table_102147 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression -SELECT count(*) AS count FROM test_table_102146 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression +SELECT count(*) AS count FROM test_table_102153 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression +SELECT count(*) AS count FROM test_table_102152 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression +SELECT count(*) AS count FROM test_table_102151 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression +SELECT count(*) AS count FROM test_table_102150 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression step s2-rollback: ROLLBACK; @@ -217,7 +217,7 @@ step s3-view-worker: query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname -SELECT count(*) AS count FROM public.test_table_102151 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression +SELECT count(*) AS count FROM public.test_table_102155 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression step s2-rollback: ROLLBACK; diff --git a/src/test/regress/expected/isolation_distributed_transaction_id.out b/src/test/regress/expected/isolation_distributed_transaction_id.out index 6b3cfefef..cd6e9f130 100644 --- a/src/test/regress/expected/isolation_distributed_transaction_id.out +++ b/src/test/regress/expected/isolation_distributed_transaction_id.out @@ -56,7 +56,7 @@ step s3-commit: COMMIT; -starting permutation: s1-create-table s1-begin s1-insert s1-get-current-transaction-id s2-get-first-worker-active-transactions s1-commit +starting permutation: s1-create-table s1-begin s1-insert s1-verify-current-xact-is-on-worker s1-commit step s1-create-table: -- some tests also use distributed table CREATE TABLE distributed_transaction_id_table(some_value int, other_value int); @@ -72,23 +72,22 @@ step s1-begin: step s1-insert: INSERT INTO distributed_transaction_id_table VALUES (1, 1); -step s1-get-current-transaction-id: - SELECT row(initiator_node_identifier, transaction_number) FROM get_current_transaction_id(); +step s1-verify-current-xact-is-on-worker: + SELECT + remote.nodeport, + remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists + FROM + get_current_transaction_id() as xact, + run_command_on_workers($$ + SELECT row(initiator_node_identifier, transaction_number) + FROM get_all_active_transactions(); + $$) as remote + ORDER BY remote.nodeport ASC; -row +nodeport xact_exists -(0,229) -step s2-get-first-worker-active-transactions: - SELECT * FROM run_command_on_workers('SELECT row(initiator_node_identifier, transaction_number) - FROM - get_all_active_transactions(); - ') - WHERE nodeport = 57637; -; - -nodename nodeport success result - -localhost 57637 t (0,229) +57637 t +57638 t step s1-commit: COMMIT; diff --git a/src/test/regress/expected/isolation_dump_global_wait_edges.out b/src/test/regress/expected/isolation_dump_global_wait_edges.out index 30fb66054..0a21b6b7b 100644 --- a/src/test/regress/expected/isolation_dump_global_wait_edges.out +++ b/src/test/regress/expected/isolation_dump_global_wait_edges.out @@ -29,11 +29,11 @@ step detector-dump-wait-edges: waiting_transaction_numblocking_transaction_numblocking_transaction_waiting -232 231 f +246 245 f transactionnumberwaitingtransactionnumbers -231 -232 231 +245 +246 245 step s1-abort: ABORT; @@ -77,14 +77,14 @@ step detector-dump-wait-edges: waiting_transaction_numblocking_transaction_numblocking_transaction_waiting -236 235 f -237 235 f -237 236 t +250 249 f +251 249 f +251 250 t transactionnumberwaitingtransactionnumbers -235 -236 235 -237 235,236 +249 +250 249 +251 249,250 step s1-abort: ABORT; diff --git a/src/test/regress/expected/isolation_dump_global_wait_edges_0.out b/src/test/regress/expected/isolation_dump_global_wait_edges_0.out new file mode 100644 index 000000000..e49912948 --- /dev/null +++ b/src/test/regress/expected/isolation_dump_global_wait_edges_0.out @@ -0,0 +1,98 @@ +Parsed test spec with 4 sessions + +starting permutation: s1-begin s2-begin s1-update s2-update detector-dump-wait-edges s1-abort s2-abort +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s1-update: + UPDATE distributed_table SET y = 1 WHERE x = 1; + +step s2-update: + UPDATE distributed_table SET y = 2 WHERE x = 1; + +step detector-dump-wait-edges: + SELECT + waiting_transaction_num, + blocking_transaction_num, + blocking_transaction_waiting + FROM + dump_global_wait_edges() + ORDER BY + waiting_transaction_num, + blocking_transaction_num, + blocking_transaction_waiting; + + SELECT * FROM get_adjacency_list_wait_graph() ORDER BY 1; + +waiting_transaction_numblocking_transaction_numblocking_transaction_waiting + +247 246 f +transactionnumberwaitingtransactionnumbers + +246 +247 246 +step s1-abort: + ABORT; + +step s2-update: <... completed> +step s2-abort: + ABORT; + + +starting permutation: s1-begin s2-begin s3-begin s1-update s2-update s3-update detector-dump-wait-edges s1-abort s2-abort s3-abort +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s3-begin: + BEGIN; + +step s1-update: + UPDATE distributed_table SET y = 1 WHERE x = 1; + +step s2-update: + UPDATE distributed_table SET y = 2 WHERE x = 1; + +step s3-update: + UPDATE distributed_table SET y = 3 WHERE x = 1; + +step detector-dump-wait-edges: + SELECT + waiting_transaction_num, + blocking_transaction_num, + blocking_transaction_waiting + FROM + dump_global_wait_edges() + ORDER BY + waiting_transaction_num, + blocking_transaction_num, + blocking_transaction_waiting; + + SELECT * FROM get_adjacency_list_wait_graph() ORDER BY 1; + +waiting_transaction_numblocking_transaction_numblocking_transaction_waiting + +251 250 f +252 250 f +252 251 t +transactionnumberwaitingtransactionnumbers + +250 +251 250 +252 250,251 +step s1-abort: + ABORT; + +step s2-update: <... completed> +step s2-abort: + ABORT; + +step s3-update: <... completed> +step s3-abort: + ABORT; + diff --git a/src/test/regress/expected/isolation_ensure_dependency_activate_node.out b/src/test/regress/expected/isolation_ensure_dependency_activate_node.out index 1b6e9a7e5..289ed798f 100644 --- a/src/test/regress/expected/isolation_ensure_dependency_activate_node.out +++ b/src/test/regress/expected/isolation_ensure_dependency_activate_node.out @@ -14,6 +14,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -26,6 +30,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -64,6 +75,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address count @@ -71,6 +86,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -92,6 +114,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -104,6 +130,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -148,6 +181,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address count @@ -155,6 +192,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -176,6 +220,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -188,6 +236,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -232,6 +287,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address count @@ -239,6 +298,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -260,6 +326,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -272,6 +342,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -311,6 +388,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address (schema,{myschema},{}) @@ -321,6 +402,13 @@ run_command_on_workers (localhost,57637,t,1) (localhost,57638,t,1) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) master_remove_node @@ -340,6 +428,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -352,6 +444,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -397,6 +496,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address (schema,{myschema},{}) @@ -407,6 +510,13 @@ run_command_on_workers (localhost,57637,t,1) (localhost,57638,t,1) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) master_remove_node @@ -426,6 +536,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -438,6 +552,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -483,6 +604,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address (schema,{myschema},{}) @@ -493,6 +618,13 @@ run_command_on_workers (localhost,57637,t,1) (localhost,57638,t,1) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) master_remove_node @@ -512,6 +644,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -524,6 +660,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -588,6 +731,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address (schema,{myschema},{}) @@ -598,6 +745,13 @@ run_command_on_workers (localhost,57637,t,1) (localhost,57638,t,1) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) master_remove_node @@ -617,6 +771,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -629,6 +787,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -712,6 +877,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address (schema,{myschema},{}) @@ -722,6 +891,13 @@ run_command_on_workers (localhost,57637,t,1) (localhost,57638,t,1) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) master_remove_node @@ -741,6 +917,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -753,6 +933,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -810,6 +997,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address (schema,{myschema},{}) @@ -820,6 +1011,13 @@ run_command_on_workers (localhost,57637,t,1) (localhost,57638,t,1) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) master_remove_node @@ -839,6 +1037,10 @@ step s1-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); ?column? @@ -851,6 +1053,13 @@ count 0 run_command_on_workers +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + (localhost,57637,t,0) (localhost,57638,t,0) master_remove_node @@ -916,6 +1125,10 @@ step s2-print-distributed-objects: SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + pg_identify_object_as_address (schema,{myschema},{}) @@ -925,6 +1138,314 @@ count 1 run_command_on_workers +(localhost,57637,t,1) +(localhost,57638,t,1) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +master_remove_node + + + + +starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-create-type s1-commit s2-print-distributed-objects +?column? + +1 +step s1-print-distributed-objects: + SELECT 1 FROM master_add_node('localhost', 57638); + + -- print an overview of all distributed objects + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + + -- print if the schema has been created + SELECT count(*) FROM pg_namespace where nspname = 'myschema'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + + SELECT master_remove_node('localhost', 57638); + +?column? + +1 +pg_identify_object_as_address + +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +master_remove_node + + +step s1-begin: + BEGIN; + +step s1-add-worker: + SELECT 1 FROM master_add_node('localhost', 57638); + +?column? + +1 +step s2-public-schema: + SET search_path TO public; + +step s2-create-type: + CREATE TYPE tt1 AS (a int, b int); + +step s1-commit: + COMMIT; + +step s2-create-type: <... completed> +step s2-print-distributed-objects: + -- print an overview of all distributed objects + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + + -- print if the schema has been created + SELECT count(*) FROM pg_namespace where nspname = 'myschema'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + +pg_identify_object_as_address + +(type,{public.tt1},{}) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +1 +run_command_on_workers + +(localhost,57637,t,1) +(localhost,57638,t,1) +master_remove_node + + + + +starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-create-type s1-add-worker s1-commit s2-print-distributed-objects +?column? + +1 +step s1-print-distributed-objects: + SELECT 1 FROM master_add_node('localhost', 57638); + + -- print an overview of all distributed objects + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + + -- print if the schema has been created + SELECT count(*) FROM pg_namespace where nspname = 'myschema'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + + SELECT master_remove_node('localhost', 57638); + +?column? + +1 +pg_identify_object_as_address + +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +master_remove_node + + +step s1-begin: + BEGIN; + +step s2-public-schema: + SET search_path TO public; + +step s2-create-type: + CREATE TYPE tt1 AS (a int, b int); + +step s1-add-worker: + SELECT 1 FROM master_add_node('localhost', 57638); + +?column? + +1 +step s1-commit: + COMMIT; + +step s2-print-distributed-objects: + -- print an overview of all distributed objects + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + + -- print if the schema has been created + SELECT count(*) FROM pg_namespace where nspname = 'myschema'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + +pg_identify_object_as_address + +(type,{public.tt1},{}) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +1 +run_command_on_workers + +(localhost,57637,t,1) +(localhost,57638,t,1) +master_remove_node + + + + +starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-schema s2-create-type s2-create-table-with-type s1-add-worker s2-commit s1-commit s2-print-distributed-objects +?column? + +1 +step s1-print-distributed-objects: + SELECT 1 FROM master_add_node('localhost', 57638); + + -- print an overview of all distributed objects + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + + -- print if the schema has been created + SELECT count(*) FROM pg_namespace where nspname = 'myschema'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + + SELECT master_remove_node('localhost', 57638); + +?column? + +1 +pg_identify_object_as_address + +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +count + +0 +run_command_on_workers + +(localhost,57637,t,0) +(localhost,57638,t,0) +master_remove_node + + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s2-create-schema: + CREATE SCHEMA myschema; + SET search_path TO myschema; + +step s2-create-type: + CREATE TYPE tt1 AS (a int, b int); + +step s2-create-table-with-type: + CREATE TABLE t1 (a int, b tt1); + -- session needs to have replication factor set to 1, can't do in setup + SET citus.shard_replication_factor TO 1; + SELECT create_distributed_table('t1', 'a'); + +create_distributed_table + + +step s1-add-worker: + SELECT 1 FROM master_add_node('localhost', 57638); + +step s2-commit: + COMMIT; + +step s1-add-worker: <... completed> +?column? + +1 +step s1-commit: + COMMIT; + +step s2-print-distributed-objects: + -- print an overview of all distributed objects + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + + -- print if the schema has been created + SELECT count(*) FROM pg_namespace where nspname = 'myschema'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + +pg_identify_object_as_address + +(schema,{myschema},{}) +(type,{myschema.tt1},{}) +count + +1 +run_command_on_workers + +(localhost,57637,t,1) +(localhost,57638,t,1) +count + +1 +run_command_on_workers + (localhost,57637,t,1) (localhost,57638,t,1) master_remove_node diff --git a/src/test/regress/expected/isolation_replace_wait_function.out b/src/test/regress/expected/isolation_replace_wait_function.out index 8cea30f25..155eaad07 100644 --- a/src/test/regress/expected/isolation_replace_wait_function.out +++ b/src/test/regress/expected/isolation_replace_wait_function.out @@ -16,7 +16,7 @@ step s1-commit: COMMIT; step s2-insert: <... completed> -error in steps s1-commit s2-insert: ERROR: duplicate key value violates unique constraint "test_locking_a_key_102413" +error in steps s1-commit s2-insert: ERROR: duplicate key value violates unique constraint "test_locking_a_key_102417" step s2-commit: COMMIT; diff --git a/src/test/regress/expected/multi_data_types.out b/src/test/regress/expected/multi_data_types.out index 2ff571146..4f814656f 100644 --- a/src/test/regress/expected/multi_data_types.out +++ b/src/test/regress/expected/multi_data_types.out @@ -9,23 +9,44 @@ CREATE TYPE test_composite_type AS ( i2 integer ); -- ... as well as a function to use as its comparator... -CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean -LANGUAGE 'internal' -AS 'record_eq' -IMMUTABLE -RETURNS NULL ON NULL INPUT; -CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int -LANGUAGE 'internal' -AS 'btrecordcmp' -IMMUTABLE -RETURNS NULL ON NULL INPUT; +SELECT run_command_on_coordinator_and_workers($cf$ + CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean + LANGUAGE 'internal' + AS 'record_eq' + IMMUTABLE + RETURNS NULL ON NULL INPUT; +$cf$); + run_command_on_coordinator_and_workers +---------------------------------------- + +(1 row) + +SELECT run_command_on_coordinator_and_workers($cf$ + CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int + LANGUAGE 'internal' + AS 'btrecordcmp' + IMMUTABLE + RETURNS NULL ON NULL INPUT; +$cf$); + run_command_on_coordinator_and_workers +---------------------------------------- + +(1 row) + -- ... use that function to create a custom equality operator... -CREATE OPERATOR = ( - LEFTARG = test_composite_type, - RIGHTARG = test_composite_type, - PROCEDURE = equal_test_composite_type_function, - HASHES -); +SELECT run_command_on_coordinator_and_workers($co$ + CREATE OPERATOR = ( + LEFTARG = test_composite_type, + RIGHTARG = test_composite_type, + PROCEDURE = equal_test_composite_type_function, + HASHES + ); +$co$); + run_command_on_coordinator_and_workers +---------------------------------------- + +(1 row) + -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY cats_op_fam USING hash; -- ... create a test HASH function. Though it is a poor hash function, diff --git a/src/test/regress/expected/multi_function_in_join.out b/src/test/regress/expected/multi_function_in_join.out index 021aab809..886fe70c5 100644 --- a/src/test/regress/expected/multi_function_in_join.out +++ b/src/test/regress/expected/multi_function_in_join.out @@ -158,10 +158,12 @@ DEBUG: Plan 13 query after replacing subqueries and CTEs: SELECT f.a, f.b, tabl (3 rows) -- Custom Type returning function used in a join +RESET client_min_messages; CREATE TYPE min_and_max AS ( minimum INT, maximum INT ); +SET client_min_messages TO DEBUG1; CREATE OR REPLACE FUNCTION max_and_min () RETURNS min_and_max AS $$ DECLARE diff --git a/src/test/regress/expected/multi_mx_create_table.out b/src/test/regress/expected/multi_mx_create_table.out index 3cdccb5f2..45a488c6d 100644 --- a/src/test/regress/expected/multi_mx_create_table.out +++ b/src/test/regress/expected/multi_mx_create_table.out @@ -60,7 +60,6 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); -- now create required stuff in the worker 1 \c - - - :worker_1_port -- create schema to test schema support -CREATE SCHEMA citus_mx_test_schema; CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_2; -- create UDFs in worker node @@ -98,13 +97,9 @@ CREATE OPERATOR citus_mx_test_schema.=== ( ); SET search_path TO public; CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); -SET search_path TO public; -CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text); -CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); -- now create required stuff in the worker 2 \c - - - :worker_2_port -- create schema to test schema support -CREATE SCHEMA citus_mx_test_schema; CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_2; -- create UDF @@ -142,9 +137,6 @@ CREATE OPERATOR citus_mx_test_schema.=== ( ); SET search_path TO public; CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); -SET search_path TO public; -CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text); -CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); -- connect back to the master, and do some more tests \c - - - :master_port SET citus.shard_replication_factor TO 1; diff --git a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out index 853a94812..4fc55596f 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -51,10 +51,7 @@ CREATE TABLE repartition_udt_other ( -- proceed with type creation as above; thus the OIDs will be different. -- so that the OID is off. \c - - - :worker_1_port -CREATE TYPE test_udt AS (i integer, i2 integer); -DROP TYPE test_udt CASCADE; -- START type creation -CREATE TYPE test_udt AS (i integer, i2 integer); -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' @@ -90,7 +87,6 @@ FUNCTION 1 test_udt_hash(test_udt); -- END type creation \c - - - :worker_2_port -- START type creation -CREATE TYPE test_udt AS (i integer, i2 integer); -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' diff --git a/src/test/regress/expected/multi_partition_pruning.out b/src/test/regress/expected/multi_partition_pruning.out index 4ddecc47a..210a9c4e6 100644 --- a/src/test/regress/expected/multi_partition_pruning.out +++ b/src/test/regress/expected/multi_partition_pruning.out @@ -110,13 +110,13 @@ INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, ORDER BY nodename, nodeport ASC LIMIT 1; -- Create composite type partitioned table +RESET client_min_messages; -- avoid debug messages CREATE TYPE composite_type AS ( text_column text, double_column decimal, varchar_column varchar(50) ); -RESET client_min_messages; -- avoid debug messages about toast index creation CREATE TABLE composite_partitioned_table ( composite_column composite_type diff --git a/src/test/regress/expected/multi_prepare_plsql.out b/src/test/regress/expected/multi_prepare_plsql.out index 1901ab389..0e168bbdc 100644 --- a/src/test/regress/expected/multi_prepare_plsql.out +++ b/src/test/regress/expected/multi_prepare_plsql.out @@ -1254,7 +1254,6 @@ SELECT schemaname, indexrelname FROM pg_stat_all_indexes WHERE indexrelname = 'p -- cleanup DROP TABLE prepare_ddl; -DROP SCHEMA otherschema; RESET search_path; -- test prepared COPY CREATE OR REPLACE FUNCTION copy_in_plpgsql() @@ -1297,10 +1296,49 @@ SELECT local_copy_in_plpgsql(); (1 row) +-- types statements should not crash nor leak schema specifications on to cached statements +CREATE TYPE prepare_ddl_type AS (x int, y int); +SET search_path TO 'otherschema', public; +CREATE OR REPLACE FUNCTION public.type_ddl_plpgsql() +RETURNS void +LANGUAGE plpgsql +AS $function$ +DECLARE +BEGIN + ALTER TYPE prepare_ddl_type RENAME TO prepare_ddl_type_backup; +END; +$function$; +SELECT type_ddl_plpgsql(); + type_ddl_plpgsql +------------------ + +(1 row) + +-- create same type in new schema, owner of this new type should change +CREATE TYPE prepare_ddl_type AS (x int, y int); +SELECT type_ddl_plpgsql(); + type_ddl_plpgsql +------------------ + +(1 row) + +-- find all renamed types to verify the schema name didn't leak, nor a crash happened +SELECT nspname, typname FROM pg_type JOIN pg_namespace ON pg_namespace.oid = pg_type.typnamespace WHERE typname = 'prepare_ddl_type_backup'; + nspname | typname +-------------+------------------------- + public | prepare_ddl_type_backup + otherschema | prepare_ddl_type_backup +(2 rows) + +DROP TYPE prepare_ddl_type_backup; +RESET search_path; +DROP TYPE prepare_ddl_type_backup; +DROP FUNCTION type_ddl_plpgsql(); DROP FUNCTION ddl_in_plpgsql(); DROP FUNCTION copy_in_plpgsql(); DROP TABLE prepare_ddl; DROP TABLE local_ddl; +DROP SCHEMA otherschema; -- clean-up functions DROP FUNCTION plpgsql_test_1(); DROP FUNCTION plpgsql_test_2(); diff --git a/src/test/regress/expected/multi_reference_table.out b/src/test/regress/expected/multi_reference_table.out index eb390afed..d8f6e9a3c 100644 --- a/src/test/regress/expected/multi_reference_table.out +++ b/src/test/regress/expected/multi_reference_table.out @@ -1522,11 +1522,6 @@ TRUNCATE reference_table_test; -- function etc. -- first create the type on all nodes CREATE TYPE reference_comp_key as (key text, value text); -\c - - - :worker_1_port -CREATE TYPE reference_comp_key as (key text, value text); -\c - - - :worker_2_port -CREATE TYPE reference_comp_key as (key text, value text); -\c - - - :master_port CREATE TABLE reference_table_composite (id int PRIMARY KEY, data reference_comp_key); SELECT create_reference_table('reference_table_composite'); create_reference_table diff --git a/src/test/regress/expected/multi_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index 2d98f3096..5e6842292 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -51,10 +51,7 @@ CREATE TABLE repartition_udt_other ( -- proceed with type creation as above; thus the OIDs will be different. -- so that the OID is off. \c - - - :worker_1_port -CREATE TYPE test_udt AS (i integer, i2 integer); -DROP TYPE test_udt CASCADE; -- START type creation -CREATE TYPE test_udt AS (i integer, i2 integer); -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' @@ -90,7 +87,6 @@ FUNCTION 1 test_udt_hash(test_udt); -- END type creation \c - - - :worker_2_port -- START type creation -CREATE TYPE test_udt AS (i integer, i2 integer); -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' diff --git a/src/test/regress/expected/multi_replicate_reference_table.out b/src/test/regress/expected/multi_replicate_reference_table.out index 46de2dddc..86baacd65 100644 --- a/src/test/regress/expected/multi_replicate_reference_table.out +++ b/src/test/regress/expected/multi_replicate_reference_table.out @@ -394,7 +394,7 @@ ORDER BY logicalrelid; logicalrelid | partmethod | colocationid | repmodel -----------------------------------------+------------+--------------+---------- replicate_reference_table_reference_one | n | 10004 | t - replicate_reference_table_hash | h | 1360004 | c + replicate_reference_table_hash | h | 1360005 | c (2 rows) BEGIN; @@ -481,7 +481,6 @@ SELECT create_reference_table('replicate_reference_table_insert'); BEGIN; INSERT INTO replicate_reference_table_insert VALUES(1); SELECT 1 FROM master_add_node('localhost', :worker_2_port); -NOTICE: Replicating reference table "replicate_reference_table_insert" to the node localhost:57638 ERROR: cannot open new connections after the first modification command within a transaction ROLLBACK; DROP TABLE replicate_reference_table_insert; @@ -496,7 +495,6 @@ SELECT create_reference_table('replicate_reference_table_copy'); BEGIN; COPY replicate_reference_table_copy FROM STDIN; SELECT 1 FROM master_add_node('localhost', :worker_2_port); -NOTICE: Replicating reference table "replicate_reference_table_copy" to the node localhost:57638 ERROR: cannot open new connections after the first modification command within a transaction ROLLBACK; DROP TABLE replicate_reference_table_copy; @@ -511,7 +509,6 @@ SELECT create_reference_table('replicate_reference_table_ddl'); BEGIN; ALTER TABLE replicate_reference_table_ddl ADD column2 int; SELECT 1 FROM master_add_node('localhost', :worker_2_port); -NOTICE: Replicating reference table "replicate_reference_table_ddl" to the node localhost:57638 ERROR: cannot open new connections after the first modification command within a transaction ROLLBACK; DROP TABLE replicate_reference_table_ddl; diff --git a/src/test/regress/expected/multi_schema_support.out b/src/test/regress/expected/multi_schema_support.out index a42cd2a17..4a62930e7 100644 --- a/src/test/regress/expected/multi_schema_support.out +++ b/src/test/regress/expected/multi_schema_support.out @@ -536,13 +536,6 @@ SELECT n_comment FROM nation_hash_collation_search_path ORDER BY n_comment COLLA --test composite types with schema SET search_path TO public; CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); --- create type in worker node 1 in schema -\c - - - :worker_1_port -CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); --- create type in worker node 2 in schema -\c - - - :worker_2_port -CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); -\c - - - :master_port CREATE TABLE test_schema_support.nation_hash_composite_types( n_nationkey integer not null, n_name char(25) not null, @@ -1002,13 +995,6 @@ SET search_path TO public; ALTER TABLE test_schema_support.nation_hash SET SCHEMA public; WARNING: not propagating ALTER ... SET SCHEMA commands to worker nodes HINT: Connect to worker nodes directly to manually change schemas of affected objects. --- we will use this function in next test -CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text) -RETURNS void LANGUAGE plpgsql AS $$ -BEGIN - EXECUTE p_sql; - PERFORM run_command_on_workers(p_sql); -END;$$; -- test schema propagation with user other than current user SELECT run_command_on_coordinator_and_workers('CREATE USER "test-user"'); NOTICE: not propagating CREATE ROLE/USER commands to worker nodes diff --git a/src/test/regress/expected/multi_subquery.out b/src/test/regress/expected/multi_subquery.out index f4c73fbd1..691c0ad32 100644 --- a/src/test/regress/expected/multi_subquery.out +++ b/src/test/regress/expected/multi_subquery.out @@ -1120,17 +1120,6 @@ LIMIT -- CASCADE NOTICE messagez SET client_min_messages TO WARNING; DROP TABLE users, events; -SELECT run_command_on_master_and_workers($f$ - - DROP TYPE user_composite_type CASCADE; - -$f$); - run_command_on_master_and_workers ------------------------------------ - -(1 row) - --- createed in multi_behavioral_analytics_create_table -DROP FUNCTION run_command_on_master_and_workers(p_sql text); +DROP TYPE user_composite_type CASCADE; SET client_min_messages TO DEFAULT; SET citus.subquery_pushdown to OFF; diff --git a/src/test/regress/expected/multi_test_helpers.out b/src/test/regress/expected/multi_test_helpers.out index 4bc18f8e8..240872882 100644 --- a/src/test/regress/expected/multi_test_helpers.out +++ b/src/test/regress/expected/multi_test_helpers.out @@ -133,3 +133,10 @@ BEGIN END LOOP; RETURN; END; $$ language plpgsql; +-- helper function to quickly run SQL on the whole cluster +CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text) +RETURNS void LANGUAGE plpgsql AS $$ +BEGIN + EXECUTE p_sql; + PERFORM run_command_on_workers(p_sql); +END;$$; diff --git a/src/test/regress/expected/multi_upgrade_reference_table.out b/src/test/regress/expected/multi_upgrade_reference_table.out index bb2e4ac3d..1e390acab 100644 --- a/src/test/regress/expected/multi_upgrade_reference_table.out +++ b/src/test/regress/expected/multi_upgrade_reference_table.out @@ -80,9 +80,6 @@ ERROR: could not find any healthy placement for shard 1360006 DROP TABLE upgrade_reference_table_unhealthy; -- test with table containing composite type CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text); -\c - - - :worker_1_port -CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text); -\c - - - :master_port SET citus.shard_count TO 1; SET citus.shard_replication_factor TO 1; CREATE TABLE upgrade_reference_table_composite(column1 int, column2 upgrade_test_composite_type); @@ -95,8 +92,11 @@ SELECT create_distributed_table('upgrade_reference_table_composite', 'column1'); UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid='upgrade_reference_table_composite'::regclass; SELECT upgrade_to_reference_table('upgrade_reference_table_composite'); NOTICE: Replicating reference table "upgrade_reference_table_composite" to the node localhost:57638 -ERROR: type "public.upgrade_test_composite_type" does not exist -CONTEXT: while executing command on localhost:57638 + upgrade_to_reference_table +---------------------------- + +(1 row) + DROP TABLE upgrade_reference_table_composite; -- test with reference table CREATE TABLE upgrade_reference_table_reference(column1 int); @@ -240,7 +240,7 @@ WHERE logicalrelid = 'upgrade_reference_table_one_worker'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360000 | c + h | f | 1360001 | c (1 row) SELECT @@ -262,7 +262,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_one_worker'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360000 | 1 | 1 | 23 + 1360001 | 1 | 1 | 23 (1 row) SELECT @@ -354,7 +354,7 @@ WHERE logicalrelid = 'upgrade_reference_table_one_unhealthy'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360001 | c + h | f | 1360002 | c (1 row) SELECT @@ -376,7 +376,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_one_unhealthy'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360001 | 1 | 2 | 23 + 1360002 | 1 | 2 | 23 (1 row) SELECT @@ -468,7 +468,7 @@ WHERE logicalrelid = 'upgrade_reference_table_both_healthy'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360002 | c + h | f | 1360003 | c (1 row) SELECT @@ -490,7 +490,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_both_healthy'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360002 | 1 | 2 | 23 + 1360003 | 1 | 2 | 23 (1 row) SELECT @@ -584,7 +584,7 @@ WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360003 | c + h | f | 1360004 | c (1 row) SELECT @@ -606,7 +606,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360003 | 1 | 1 | 23 + 1360004 | 1 | 1 | 23 (1 row) SELECT @@ -639,7 +639,7 @@ WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360003 | c + h | f | 1360004 | c (1 row) SELECT @@ -661,7 +661,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360003 | 1 | 1 | 23 + 1360004 | 1 | 1 | 23 (1 row) SELECT @@ -697,7 +697,7 @@ WHERE logicalrelid = 'upgrade_reference_table_transaction_commit'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360003 | c + h | f | 1360004 | c (1 row) SELECT @@ -719,7 +719,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_transaction_commit'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360003 | 1 | 1 | 23 + 1360004 | 1 | 1 | 23 (1 row) SELECT @@ -823,7 +823,7 @@ WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360004 | s + h | f | 1360005 | s (1 row) SELECT @@ -845,7 +845,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360004 | 1 | 1 | 23 + 1360005 | 1 | 1 | 23 (1 row) SELECT @@ -875,7 +875,7 @@ WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360004 | s + h | f | 1360005 | s (1 row) SELECT @@ -897,7 +897,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360004 | 1 | 1 | 23 + 1360005 | 1 | 1 | 23 (1 row) SELECT @@ -944,7 +944,7 @@ WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass; partmethod | partkeyisnull | colocationid | repmodel ------------+---------------+--------------+---------- - h | f | 1360005 | c + h | f | 1360006 | c (1 row) SELECT @@ -966,7 +966,7 @@ WHERE colocationid IN WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass); colocationid | shardcount | replicationfactor | distributioncolumntype --------------+------------+-------------------+------------------------ - 1360005 | 1 | 2 | 23 + 1360006 | 1 | 2 | 23 (1 row) SELECT diff --git a/src/test/regress/expected/mx_foreign_key_to_reference_table.out b/src/test/regress/expected/mx_foreign_key_to_reference_table.out index d0d3058f4..a1d8d51ae 100644 --- a/src/test/regress/expected/mx_foreign_key_to_reference_table.out +++ b/src/test/regress/expected/mx_foreign_key_to_reference_table.out @@ -7,13 +7,6 @@ SET citus.next_placement_id TO 7000000; SET citus.replication_model TO streaming; -- Setup the view so that we can check if the foreign keys are created properly CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); -SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$); - run_command_on_workers ------------------------------------ - (localhost,57637,t,"CREATE TYPE") - (localhost,57638,t,"CREATE TYPE") -(2 rows) - CREATE VIEW table_fkeys_in_workers AS SELECT (json_populate_record(NULL::foreign_details, diff --git a/src/test/regress/expected/subquery_prepared_statements.out b/src/test/regress/expected/subquery_prepared_statements.out index fa1e2b1e0..3e8ae81cb 100644 --- a/src/test/regress/expected/subquery_prepared_statements.out +++ b/src/test/regress/expected/subquery_prepared_statements.out @@ -2,22 +2,15 @@ -- test recursive planning functionality on prepared statements -- =================================================================== CREATE SCHEMA subquery_prepared_statements; -SET search_path TO subquery_prepared_statements, public; -CREATE TYPE xy AS (x int, y int); -SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements'); +SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements;'); run_command_on_workers ------------------------------------- (localhost,57637,t,"CREATE SCHEMA") (localhost,57638,t,"CREATE SCHEMA") (2 rows) -SELECT run_command_on_workers('CREATE TYPE subquery_prepared_statements.xy AS (x int, y int)'); - run_command_on_workers ------------------------------------ - (localhost,57637,t,"CREATE TYPE") - (localhost,57638,t,"CREATE TYPE") -(2 rows) - +SET search_path TO subquery_prepared_statements, public; +CREATE TYPE subquery_prepared_statements.xy AS (x int, y int); SET client_min_messages TO DEBUG1; PREPARE subquery_prepare_without_param AS SELECT diff --git a/src/test/regress/expected/validate_constraint.out b/src/test/regress/expected/validate_constraint.out index ad180b13b..a8e1e19dc 100644 --- a/src/test/regress/expected/validate_constraint.out +++ b/src/test/regress/expected/validate_constraint.out @@ -28,14 +28,6 @@ SET citus.shard_count TO 8; SET citus.next_shard_id TO 8000000; SET citus.next_placement_id TO 8000000; CREATE TYPE constraint_validity AS (name text, validated bool); -SELECT run_command_on_workers( - $$CREATE TYPE constraint_validity AS (name text, validated bool)$$); - run_command_on_workers ------------------------------------ - (localhost,57637,t,"CREATE TYPE") - (localhost,57638,t,"CREATE TYPE") -(2 rows) - CREATE VIEW constraint_validations_in_workers AS SELECT (json_populate_record(NULL :: constraint_validity, json_array_elements_text((run_command_on_workers($$ diff --git a/src/test/regress/expected/with_basics.out b/src/test/regress/expected/with_basics.out index 04e5a57cd..40b72ed29 100644 --- a/src/test/regress/expected/with_basics.out +++ b/src/test/regress/expected/with_basics.out @@ -1,13 +1,6 @@ -- Test the basic CTE functionality and expected error messages SET search_path TO 'with_basics'; -CREATE TYPE xy AS (x int, y int); -SELECT run_command_on_workers('CREATE TYPE with_basics.xy AS (x int, y int)'); - run_command_on_workers ------------------------------------ - (localhost,57637,t,"CREATE TYPE") - (localhost,57638,t,"CREATE TYPE") -(2 rows) - +CREATE TYPE with_basics.xy AS (x int, y int); -- CTEs in FROM should work WITH cte AS ( SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 1,2 LIMIT 5 diff --git a/src/test/regress/input/multi_behavioral_analytics_create_table.source b/src/test/regress/input/multi_behavioral_analytics_create_table.source index f37c554fe..045b50f6c 100644 --- a/src/test/regress/input/multi_behavioral_analytics_create_table.source +++ b/src/test/regress/input/multi_behavioral_analytics_create_table.source @@ -81,14 +81,11 @@ CREATE INDEX is_index5 ON users_table(value_2); CREATE INDEX is_index6 ON events_table(value_2); -- Create composite type to use in subquery pushdown -SELECT run_command_on_master_and_workers($f$ - - CREATE TYPE user_composite_type AS - ( - tenant_id BIGINT, - user_id BIGINT - ); -$f$); +CREATE TYPE user_composite_type AS +( + tenant_id BIGINT, + user_id BIGINT +); SELECT run_command_on_master_and_workers($f$ diff --git a/src/test/regress/input/multi_complex_count_distinct.source b/src/test/regress/input/multi_complex_count_distinct.source index 3a268c878..f9cad3f72 100644 --- a/src/test/regress/input/multi_complex_count_distinct.source +++ b/src/test/regress/input/multi_complex_count_distinct.source @@ -464,11 +464,6 @@ CREATE TYPE test_item AS id INTEGER, duration INTEGER ); -SELECT * FROM run_command_on_workers($$CREATE TYPE test_item AS -( - id INTEGER, - duration INTEGER -)$$) ORDER BY nodeport; CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]); SELECT create_distributed_table('test_count_distinct_array', 'key'); @@ -491,7 +486,6 @@ LIMIT 5; DROP TABLE test_count_distinct_array; DROP TYPE test_item; -SELECT * FROM run_command_on_workers($$DROP TYPE test_item$$) ORDER BY nodeport; -- other distinct aggregate are not supported SELECT * diff --git a/src/test/regress/input/multi_copy.source b/src/test/regress/input/multi_copy.source index f4727aa28..e418a9d58 100644 --- a/src/test/regress/input/multi_copy.source +++ b/src/test/regress/input/multi_copy.source @@ -443,37 +443,6 @@ CREATE TYPE super_number_pack AS ( packed_number2 number_pack ); --- Create same types in worker1 -\c - - - :worker_1_port - -CREATE TYPE number_pack AS ( - number1 integer, - number2 integer -); - -CREATE TYPE super_number_pack AS ( - packed_number1 number_pack, - packed_number2 number_pack -); - --- Create same types in worker2 -\c - - - :worker_2_port - -CREATE TYPE number_pack AS ( - number1 integer, - number2 integer -); - -CREATE TYPE super_number_pack AS ( - packed_number1 number_pack, - packed_number2 number_pack -); - - --- Connect back to master -\c - - - :master_port - - -- Test array of user-defined type with hash distribution CREATE TABLE packed_numbers_hash ( id integer, diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index c095f4e3f..8a93290eb 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -276,3 +276,8 @@ test: multi_task_string_size # connection encryption tests # --------- test: ssl_by_default + +# --------- +# object distribution tests +# --------- +test: distributed_types diff --git a/src/test/regress/output/multi_behavioral_analytics_create_table.source b/src/test/regress/output/multi_behavioral_analytics_create_table.source index ab075538e..15660b577 100644 --- a/src/test/regress/output/multi_behavioral_analytics_create_table.source +++ b/src/test/regress/output/multi_behavioral_analytics_create_table.source @@ -121,19 +121,11 @@ CREATE INDEX is_index4 ON events_table(event_type); CREATE INDEX is_index5 ON users_table(value_2); CREATE INDEX is_index6 ON events_table(value_2); -- Create composite type to use in subquery pushdown -SELECT run_command_on_master_and_workers($f$ - - CREATE TYPE user_composite_type AS - ( - tenant_id BIGINT, - user_id BIGINT - ); -$f$); - run_command_on_master_and_workers ------------------------------------ - -(1 row) - +CREATE TYPE user_composite_type AS +( + tenant_id BIGINT, + user_id BIGINT +); SELECT run_command_on_master_and_workers($f$ CREATE FUNCTION cmp_user_composite_type_function(user_composite_type, user_composite_type) RETURNS int diff --git a/src/test/regress/output/multi_complex_count_distinct.source b/src/test/regress/output/multi_complex_count_distinct.source index b63c92f66..a25d910fd 100644 --- a/src/test/regress/output/multi_complex_count_distinct.source +++ b/src/test/regress/output/multi_complex_count_distinct.source @@ -942,17 +942,6 @@ CREATE TYPE test_item AS id INTEGER, duration INTEGER ); -SELECT * FROM run_command_on_workers($$CREATE TYPE test_item AS -( - id INTEGER, - duration INTEGER -)$$) ORDER BY nodeport; - nodename | nodeport | success | result ------------+----------+---------+------------- - localhost | 57637 | t | CREATE TYPE - localhost | 57638 | t | CREATE TYPE -(2 rows) - CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]); SELECT create_distributed_table('test_count_distinct_array', 'key'); create_distributed_table @@ -984,13 +973,6 @@ LIMIT 5; DROP TABLE test_count_distinct_array; DROP TYPE test_item; -SELECT * FROM run_command_on_workers($$DROP TYPE test_item$$) ORDER BY nodeport; - nodename | nodeport | success | result ------------+----------+---------+----------- - localhost | 57637 | t | DROP TYPE - localhost | 57638 | t | DROP TYPE -(2 rows) - -- other distinct aggregate are not supported SELECT * FROM ( diff --git a/src/test/regress/output/multi_copy.source b/src/test/regress/output/multi_copy.source index 97ce664ae..ac3405042 100644 --- a/src/test/regress/output/multi_copy.source +++ b/src/test/regress/output/multi_copy.source @@ -575,28 +575,6 @@ CREATE TYPE super_number_pack AS ( packed_number1 number_pack, packed_number2 number_pack ); --- Create same types in worker1 -\c - - - :worker_1_port -CREATE TYPE number_pack AS ( - number1 integer, - number2 integer -); -CREATE TYPE super_number_pack AS ( - packed_number1 number_pack, - packed_number2 number_pack -); --- Create same types in worker2 -\c - - - :worker_2_port -CREATE TYPE number_pack AS ( - number1 integer, - number2 integer -); -CREATE TYPE super_number_pack AS ( - packed_number1 number_pack, - packed_number2 number_pack -); --- Connect back to master -\c - - - :master_port -- Test array of user-defined type with hash distribution CREATE TABLE packed_numbers_hash ( id integer, diff --git a/src/test/regress/pg_regress_multi.pl b/src/test/regress/pg_regress_multi.pl index b92954602..210852645 100755 --- a/src/test/regress/pg_regress_multi.pl +++ b/src/test/regress/pg_regress_multi.pl @@ -69,11 +69,9 @@ my $postgresSrcdir = ""; my $majorversion = ""; my @extensions = (); my @userPgOptions = (); -my %dataTypes = (); my %fdws = (); my %fdwServers = (); my %functions = (); -my %operators = (); my $valgrind = 0; my $valgrindPath = "valgrind"; my $valgrindLogFile = "valgrind_test_log.txt"; @@ -386,19 +384,8 @@ for my $option (@userPgOptions) push(@pgOptions, '-c', $option); } -#define data types as a name->definition -%dataTypes = ('dummy_type', '(i integer)', - 'order_side', ' ENUM (\'buy\', \'sell\')', - 'test_composite_type', '(i integer, i2 integer)', - 'bug_status', ' ENUM (\'new\', \'open\', \'closed\')'); - # define functions as signature->definition -%functions = ('fake_fdw_handler()', 'fdw_handler AS \'citus\' LANGUAGE C STRICT;', - 'equal_test_composite_type_function(test_composite_type, test_composite_type)', - 'boolean AS \'select $1.i = $2.i AND $1.i2 = $2.i2;\' LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT;'); - - -%operators = ('=', '(LEFTARG = test_composite_type, RIGHTARG = test_composite_type, PROCEDURE = equal_test_composite_type_function, HASHES)'); +%functions = ('fake_fdw_handler()', 'fdw_handler AS \'citus\' LANGUAGE C STRICT;'); #define fdws as name->handler name %fdws = ('fake_fdw', 'fake_fdw_handler'); @@ -750,14 +737,6 @@ for my $port (@workerPorts) or die "Could not create extension on worker"; } - foreach my $dataType (keys %dataTypes) - { - system(catfile($bindir, "psql"), - ('-X', '-h', $host, '-p', $port, '-U', $user, "-d", "regression", - '-c', "CREATE TYPE $dataType AS $dataTypes{$dataType};")) == 0 - or die "Could not create TYPE $dataType on worker"; - } - foreach my $function (keys %functions) { system(catfile($bindir, "psql"), @@ -766,14 +745,6 @@ for my $port (@workerPorts) or die "Could not create FUNCTION $function on worker"; } - foreach my $operator (keys %operators) - { - system(catfile($bindir, "psql"), - ('-X', '-h', $host, '-p', $port, '-U', $user, "-d", "regression", - '-c', "CREATE OPERATOR $operator $operators{$operator};")) == 0 - or die "Could not create OPERATOR $operator on worker"; - } - foreach my $fdw (keys %fdws) { system(catfile($bindir, "psql"), diff --git a/src/test/regress/specs/isolation_distributed_transaction_id.spec b/src/test/regress/specs/isolation_distributed_transaction_id.spec index 834bc512f..4809557be 100644 --- a/src/test/regress/specs/isolation_distributed_transaction_id.spec +++ b/src/test/regress/specs/isolation_distributed_transaction_id.spec @@ -45,9 +45,18 @@ step "s1-insert" INSERT INTO distributed_transaction_id_table VALUES (1, 1); } -step "s1-get-current-transaction-id" +step "s1-verify-current-xact-is-on-worker" { - SELECT row(initiator_node_identifier, transaction_number) FROM get_current_transaction_id(); + SELECT + remote.nodeport, + remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists + FROM + get_current_transaction_id() as xact, + run_command_on_workers($$ + SELECT row(initiator_node_identifier, transaction_number) + FROM get_all_active_transactions(); + $$) as remote + ORDER BY remote.nodeport ASC; } step "s1-get-all-transactions" @@ -121,7 +130,7 @@ permutation "s1-begin" "s1-assign-transaction-id" "s1-get-all-transactions" "s2- # now show that distributed transaction id on the coordinator # is the same with the one on the worker -permutation "s1-create-table" "s1-begin" "s1-insert" "s1-get-current-transaction-id" "s2-get-first-worker-active-transactions" "s1-commit" +permutation "s1-create-table" "s1-begin" "s1-insert" "s1-verify-current-xact-is-on-worker" "s1-commit" # we would initially forget the distributed transaction ID on pg_dist_partition invalidations permutation "s1-begin" "s1-assign-transaction-id" "s1-has-transaction-number" "s2-vacuum" "s1-has-transaction-number" "s1-commit" diff --git a/src/test/regress/specs/isolation_ensure_dependency_activate_node.spec b/src/test/regress/specs/isolation_ensure_dependency_activate_node.spec index 5949ae3d4..acdaf444c 100644 --- a/src/test/regress/specs/isolation_ensure_dependency_activate_node.spec +++ b/src/test/regress/specs/isolation_ensure_dependency_activate_node.spec @@ -19,6 +19,7 @@ teardown DROP TABLE IF EXISTS t1 CASCADE; DROP TABLE IF EXISTS t2 CASCADE; DROP TABLE IF EXISTS t3 CASCADE; + DROP TYPE IF EXISTS tt1 CASCADE; SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node; } @@ -53,6 +54,10 @@ step "s1-print-distributed-objects" SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); + SELECT master_remove_node('localhost', 57638); } @@ -77,6 +82,19 @@ step "s2-create-table" SELECT create_distributed_table('t1', 'a'); } +step "s2-create-type" +{ + CREATE TYPE tt1 AS (a int, b int); +} + +step "s2-create-table-with-type" +{ + CREATE TABLE t1 (a int, b tt1); + -- session needs to have replication factor set to 1, can't do in setup + SET citus.shard_replication_factor TO 1; + SELECT create_distributed_table('t1', 'a'); +} + step "s2-begin" { BEGIN; @@ -97,6 +115,10 @@ step "s2-print-distributed-objects" -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); + + -- print if the type has been created + SELECT count(*) FROM pg_type where typname = 'tt1'; + SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$); } session "s3" @@ -178,3 +200,8 @@ permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-beg permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-begin" "s3-begin" "s4-begin" "s1-add-worker" "s2-create-table" "s3-use-schema" "s3-create-table" "s4-use-schema" "s4-create-table" "s1-commit" "s2-commit" "s3-commit" "s4-commit" "s2-print-distributed-objects" permutation "s1-print-distributed-objects" "s1-add-worker" "s2-create-schema" "s2-begin" "s3-begin" "s3-use-schema" "s2-create-table" "s3-create-table" "s2-commit" "s3-commit" "s2-print-distributed-objects" permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s4-begin" "s1-add-worker" "s2-create-schema" "s4-create-schema2" "s2-create-table" "s4-create-table" "s1-commit" "s2-commit" "s4-commit" "s2-print-distributed-objects" + +# type and schema tests +permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-create-type" "s1-commit" "s2-print-distributed-objects" +permutation "s1-print-distributed-objects" "s1-begin" "s2-public-schema" "s2-create-type" "s1-add-worker" "s1-commit" "s2-print-distributed-objects" +permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-create-schema" "s2-create-type" "s2-create-table-with-type" "s1-add-worker" "s2-commit" "s1-commit" "s2-print-distributed-objects" diff --git a/src/test/regress/sql/distributed_types.sql b/src/test/regress/sql/distributed_types.sql new file mode 100644 index 000000000..36dcae491 --- /dev/null +++ b/src/test/regress/sql/distributed_types.sql @@ -0,0 +1,222 @@ +SET citus.next_shard_id TO 20010000; + +CREATE USER typeuser; +SELECT run_command_on_workers($$CREATE USER typeuser;$$); + +CREATE SCHEMA type_tests AUTHORIZATION typeuser; +CREATE SCHEMA type_tests2 AUTHORIZATION typeuser; -- to test creation in a specific schema and moving to schema +SET search_path TO type_tests; +SET citus.shard_count TO 4; + +-- single statement transactions with a simple type used in a table +CREATE TYPE tc1 AS (a int, b int); +CREATE TABLE t1 (a int PRIMARY KEY, b tc1); +SELECT create_distributed_table('t1','a'); +INSERT INTO t1 VALUES (1, (2,3)::tc1); +SELECT * FROM t1; +ALTER TYPE tc1 RENAME TO tc1_newname; +INSERT INTO t1 VALUES (3, (4,5)::tc1_newname); -- insert with a cast would fail if the rename didn't propagate +ALTER TYPE tc1_newname SET SCHEMA type_tests2; +INSERT INTO t1 VALUES (6, (7,8)::type_tests2.tc1_newname); -- insert with a cast would fail if the rename didn't propagate + +-- single statement transactions with a an enum used in a table +CREATE TYPE te1 AS ENUM ('one', 'two', 'three'); +CREATE TABLE t2 (a int PRIMARY KEY, b te1); +SELECT create_distributed_table('t2','a'); +INSERT INTO t2 VALUES (1, 'two'); +SELECT * FROM t2; + +-- rename enum, subsequent operations on the type would fail if the rename was not propagated +ALTER TYPE te1 RENAME TO te1_newname; + +-- add an extra value to the enum and use in table +ALTER TYPE te1_newname ADD VALUE 'four'; +UPDATE t2 SET b = 'four'; +SELECT * FROM t2; + +-- change the schema of the type and use the new fully qualified name in an insert +ALTER TYPE te1_newname SET SCHEMA type_tests2; +INSERT INTO t2 VALUES (3, 'three'::type_tests2.te1_newname); + +-- transaction block with simple type +BEGIN; +CREATE TYPE tc2 AS (a int, b int); +CREATE TABLE t3 (a int PRIMARY KEY, b tc2); +SELECT create_distributed_table('t3','a'); +INSERT INTO t3 VALUES (4, (5,6)::tc2); +SELECT * FROM t3; +COMMIT; + +-- transaction block with simple type +BEGIN; +CREATE TYPE te2 AS ENUM ('yes', 'no'); +CREATE TABLE t4 (a int PRIMARY KEY, b te2); +SELECT create_distributed_table('t4','a'); +INSERT INTO t4 VALUES (1, 'yes'); +SELECT * FROM t4; +-- ALTER TYPE ... ADD VALUE does not work in transactions +COMMIT; + +-- test some combination of types without ddl propagation, this will prevent the workers +-- from having those types created. They are created just-in-time on table distribution +SET citus.enable_ddl_propagation TO off; +CREATE TYPE tc3 AS (a int, b int); +CREATE TYPE tc4 AS (a int, b tc3[]); +CREATE TYPE tc5 AS (a int, b tc4); +CREATE TYPE te3 AS ENUM ('a','b'); +RESET citus.enable_ddl_propagation; + +CREATE TABLE t5 (a int PRIMARY KEY, b tc5[], c te3); +SELECT create_distributed_table('t5','a'); + +-- test adding an attribute to a type and a column to a table both for a non-distributed type +SET citus.enable_ddl_propagation TO off; +CREATE TYPE te4 AS ENUM ('c','d'); +CREATE TYPE tc6 AS (a int, b int); +CREATE TYPE tc6c AS (a int, b int); +RESET citus.enable_ddl_propagation; + +-- types need to be fully qualified because of the search_path which is not supported by ALTER TYPE ... ADD COLUMN +ALTER TABLE t5 ADD COLUMN d type_tests.te4; +ALTER TABLE t5 ADD COLUMN e type_tests.tc6; + +ALTER TYPE tc6 ADD ATTRIBUTE c tc6c; + +-- last two values are only there if above commands succeeded +INSERT INTO t5 VALUES (1, NULL, 'a', 'd', (1,2,(4,5)::tc6c)::tc6); + +-- test renaming an attribute of a distrbuted type and read it by its new name to verify propagation +ALTER TYPE tc6 RENAME ATTRIBUTE b TO d; +SELECT (e::tc6).d FROM t5 ORDER BY 1; + +-- change owner of supported types and check ownership on remote server +ALTER TYPE te4 OWNER TO typeuser; +SELECT typname, usename FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid; +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;$$); + +ALTER TYPE tc6 OWNER TO typeuser; +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid; +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;$$); + +-- create a type as a different user +SET ROLE typeuser; +-- create directly on the worker +CREATE TYPE tc7 AS (a int, b int); +CREATE TYPE te5 AS ENUM ('a','b','c'); + +-- cascade to the worker when table gets created +SET citus.enable_ddl_propagation TO off; +CREATE TYPE tc8 AS (a int, b int); +CREATE TYPE te6 AS ENUM ('a','b','c'); +RESET citus.enable_ddl_propagation; +CREATE TABLE t6 (a int, b tc8, c te6); +SELECT create_distributed_table('t6', 'a'); +RESET ROLE; + +-- test ownership of all types +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid; +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;$$); + +SELECT typname, usename FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid; +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;$$); + +SELECT typname, usename FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid; +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;$$); + +SELECT typname, usename FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid; +SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;$$); + + +-- deleting the enum cascade will remove the type from the table and the workers +DROP TYPE te3 CASCADE; + +-- DELETE multiple types at once +DROP TYPE tc3, tc4, tc5 CASCADE; + +-- test if the types are deleted +SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname; +SELECT run_command_on_workers($$SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;$$); + +-- make sure attribute names are quoted correctly, no errors indicates types are propagated correctly +CREATE TYPE tc9 AS ("field-with-dashes" text COLLATE "en_US"); +ALTER TYPE tc9 ADD ATTRIBUTE "some-more" int, ADD ATTRIBUTE normal int; +ALTER TYPE tc9 RENAME ATTRIBUTE normal TO "not-so-normal"; + +-- test alter statements for non-distributed types, if they would be propagated they would +-- error, preventing from changing them +SET citus.enable_ddl_propagation TO off; +CREATE TYPE non_distributed_composite_type AS (a int, b int); +CREATE TYPE non_distributed_enum_type AS ENUM ('a', 'c'); +SET citus.enable_ddl_propagation TO on; + +ALTER TYPE non_distributed_composite_type ADD ATTRIBUTE c int; +ALTER TYPE non_distributed_composite_type RENAME ATTRIBUTE c TO d; +ALTER TYPE non_distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE; +ALTER TYPE non_distributed_composite_type DROP ATTRIBUTE d; + +ALTER TYPE non_distributed_composite_type OWNER TO typeuser; + +ALTER TYPE non_distributed_composite_type RENAME TO non_distributed_composite_type_renamed; +ALTER TYPE non_distributed_composite_type_renamed RENAME TO non_distributed_composite_type; + +ALTER TYPE non_distributed_composite_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.non_distributed_composite_type SET SCHEMA type_tests; + + +ALTER TYPE non_distributed_enum_type OWNER TO typeuser; + +ALTER TYPE non_distributed_enum_type RENAME TO non_distributed_enum_type_renamed; +ALTER TYPE non_distributed_enum_type_renamed RENAME TO non_distributed_enum_type; + +ALTER TYPE non_distributed_enum_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.non_distributed_enum_type SET SCHEMA type_tests; + +ALTER TYPE non_distributed_enum_type ADD VALUE 'b' BEFORE 'c'; +ALTER TYPE non_distributed_enum_type ADD VALUE 'd' AFTER 'c'; + +ALTER TYPE non_distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff'; + + +-- test all forms of alter statements on distributed types +CREATE TYPE distributed_composite_type AS (a int, b int); +CREATE TYPE distributed_enum_type AS ENUM ('a', 'c'); +-- enforce distribution of types in every case +CREATE TABLE type_proc (a int, b distributed_composite_type, c distributed_enum_type); +SELECT create_distributed_table('type_proc','a'); +DROP TABLE type_proc; + +ALTER TYPE distributed_composite_type ADD ATTRIBUTE c int; +ALTER TYPE distributed_composite_type RENAME ATTRIBUTE c TO d; +ALTER TYPE distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE; +ALTER TYPE distributed_composite_type DROP ATTRIBUTE d; + +ALTER TYPE distributed_composite_type OWNER TO typeuser; + +ALTER TYPE distributed_composite_type RENAME TO distributed_composite_type_renamed; +ALTER TYPE distributed_composite_type_renamed RENAME TO distributed_composite_type; + +ALTER TYPE distributed_composite_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.distributed_composite_type SET SCHEMA type_tests; + + +ALTER TYPE distributed_enum_type OWNER TO typeuser; + +ALTER TYPE distributed_enum_type RENAME TO distributed_enum_type_renamed; +ALTER TYPE distributed_enum_type_renamed RENAME TO distributed_enum_type; + +ALTER TYPE distributed_enum_type SET SCHEMA type_tests2; +ALTER TYPE type_tests2.distributed_enum_type SET SCHEMA type_tests; + +ALTER TYPE distributed_enum_type ADD VALUE 'b' BEFORE 'c'; +ALTER TYPE distributed_enum_type ADD VALUE 'd' AFTER 'c'; + +ALTER TYPE distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff'; + +-- clear objects +SET client_min_messages TO fatal; -- suppress cascading objects dropping +DROP SCHEMA type_tests CASCADE; +SELECT run_command_on_workers($$DROP SCHEMA type_tests CASCADE;$$); +DROP SCHEMA type_tests2 CASCADE; +SELECT run_command_on_workers($$DROP SCHEMA type_tests2 CASCADE;$$); +DROP USER typeuser; +SELECT run_command_on_workers($$DROP USER typeuser;$$); \ No newline at end of file diff --git a/src/test/regress/sql/failure_create_distributed_table_non_empty.sql b/src/test/regress/sql/failure_create_distributed_table_non_empty.sql index 8303a880b..7776e14b5 100644 --- a/src/test/regress/sql/failure_create_distributed_table_non_empty.sql +++ b/src/test/regress/sql/failure_create_distributed_table_non_empty.sql @@ -46,6 +46,11 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_non_empty_failure$$); +-- this triggers a schema creation which prevents further transactions around dependency propagation +SELECT citus.mitmproxy('conn.allow()'); +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; + -- kill as soon as the coordinator sends begin SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT create_distributed_table('test_table', 'id'); @@ -225,6 +230,11 @@ SELECT citus.mitmproxy('conn.allow()'); SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass; SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$); +-- this triggers a schema creation which prevents further transactions around dependency propagation +SELECT citus.mitmproxy('conn.allow()'); +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; + -- kill as soon as the coordinator sends begin SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT create_distributed_table('test_table', 'id'); diff --git a/src/test/regress/sql/failure_create_reference_table.sql b/src/test/regress/sql/failure_create_reference_table.sql index 47a9592fb..36fa10a93 100644 --- a/src/test/regress/sql/failure_create_reference_table.sql +++ b/src/test/regress/sql/failure_create_reference_table.sql @@ -9,6 +9,11 @@ SET citus.next_shard_id TO 10000000; SELECT citus.mitmproxy('conn.allow()'); +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; + CREATE TABLE ref_table(id int); INSERT INTO ref_table VALUES(1),(2),(3); diff --git a/src/test/regress/sql/failure_create_table.sql b/src/test/regress/sql/failure_create_table.sql index fe9790fc9..de1a15288 100644 --- a/src/test/regress/sql/failure_create_table.sql +++ b/src/test/regress/sql/failure_create_table.sql @@ -30,6 +30,11 @@ SELECT citus.mitmproxy('conn.allow()'); SELECT count(*) FROM pg_dist_shard; SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'failure_create_table'$$); +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; + -- Now, kill the connection while opening transaction on workers. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT create_distributed_table('test_table','id'); @@ -123,6 +128,11 @@ SELECT citus.mitmproxy('conn.allow()'); SELECT count(*) FROM pg_dist_shard; SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$); +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; + -- Now, kill the connection while creating transaction on workers in transaction. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); @@ -179,6 +189,11 @@ SELECT citus.mitmproxy('conn.allow()'); SELECT count(*) FROM pg_dist_shard; SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$); +-- this is merely used to get the schema creation propagated. Without there are failures +-- not related to reference tables but schema creation due to dependency creation on workers +CREATE TYPE schema_proc AS (a int); +DROP TYPE schema_proc; + -- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN. SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); diff --git a/src/test/regress/sql/foreign_key_to_reference_table.sql b/src/test/regress/sql/foreign_key_to_reference_table.sql index b4acc19aa..79c35099d 100644 --- a/src/test/regress/sql/foreign_key_to_reference_table.sql +++ b/src/test/regress/sql/foreign_key_to_reference_table.sql @@ -12,7 +12,6 @@ SET citus.next_shard_id TO 7000000; SET citus.next_placement_id TO 7000000; CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); -SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$); CREATE VIEW table_fkeys_in_workers AS SELECT @@ -273,7 +272,6 @@ DROP TABLE referenced_table; -- foreign key as composite key CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int); -SELECT run_command_on_workers($$CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int)$$) ORDER BY 1; CREATE TABLE referenced_table(test_column composite, PRIMARY KEY(test_column)); CREATE TABLE referencing_table(id int, referencing_composite composite); diff --git a/src/test/regress/sql/intermediate_results.sql b/src/test/regress/sql/intermediate_results.sql index 3c2f27eef..c81d0f6ab 100644 --- a/src/test/regress/sql/intermediate_results.sql +++ b/src/test/regress/sql/intermediate_results.sql @@ -80,7 +80,6 @@ END; -- try a composite type CREATE TYPE intermediate_results.square_type AS (x text, x2 int); -SELECT run_command_on_workers('CREATE TYPE intermediate_results.square_type AS (x text, x2 int)'); CREATE TABLE stored_squares (user_id text, square intermediate_results.square_type, metadata jsonb); INSERT INTO stored_squares VALUES ('jon', '(2,4)'::intermediate_results.square_type, '{"value":2}'); diff --git a/src/test/regress/sql/multi_data_types.sql b/src/test/regress/sql/multi_data_types.sql index a9c772adb..7f05fc888 100644 --- a/src/test/regress/sql/multi_data_types.sql +++ b/src/test/regress/sql/multi_data_types.sql @@ -6,7 +6,6 @@ SET citus.next_shard_id TO 530000; - -- create a custom type... CREATE TYPE test_composite_type AS ( i integer, @@ -14,25 +13,31 @@ CREATE TYPE test_composite_type AS ( ); -- ... as well as a function to use as its comparator... -CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean -LANGUAGE 'internal' -AS 'record_eq' -IMMUTABLE -RETURNS NULL ON NULL INPUT; +SELECT run_command_on_coordinator_and_workers($cf$ + CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean + LANGUAGE 'internal' + AS 'record_eq' + IMMUTABLE + RETURNS NULL ON NULL INPUT; +$cf$); -CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int -LANGUAGE 'internal' -AS 'btrecordcmp' -IMMUTABLE -RETURNS NULL ON NULL INPUT; +SELECT run_command_on_coordinator_and_workers($cf$ + CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int + LANGUAGE 'internal' + AS 'btrecordcmp' + IMMUTABLE + RETURNS NULL ON NULL INPUT; +$cf$); -- ... use that function to create a custom equality operator... -CREATE OPERATOR = ( - LEFTARG = test_composite_type, - RIGHTARG = test_composite_type, - PROCEDURE = equal_test_composite_type_function, - HASHES -); +SELECT run_command_on_coordinator_and_workers($co$ + CREATE OPERATOR = ( + LEFTARG = test_composite_type, + RIGHTARG = test_composite_type, + PROCEDURE = equal_test_composite_type_function, + HASHES + ); +$co$); -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY cats_op_fam USING hash; diff --git a/src/test/regress/sql/multi_function_in_join.sql b/src/test/regress/sql/multi_function_in_join.sql index 964b66577..89a562fe1 100644 --- a/src/test/regress/sql/multi_function_in_join.sql +++ b/src/test/regress/sql/multi_function_in_join.sql @@ -93,10 +93,12 @@ SELECT * FROM ROWS FROM (next_k_integers(5), next_k_integers(10)) AS f(a, b), -- Custom Type returning function used in a join +RESET client_min_messages; CREATE TYPE min_and_max AS ( minimum INT, maximum INT ); +SET client_min_messages TO DEBUG1; CREATE OR REPLACE FUNCTION max_and_min () RETURNS min_and_max AS $$ diff --git a/src/test/regress/sql/multi_mx_create_table.sql b/src/test/regress/sql/multi_mx_create_table.sql index 33e085fbf..09b798981 100644 --- a/src/test/regress/sql/multi_mx_create_table.sql +++ b/src/test/regress/sql/multi_mx_create_table.sql @@ -61,7 +61,6 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); \c - - - :worker_1_port -- create schema to test schema support -CREATE SCHEMA citus_mx_test_schema; CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_2; @@ -105,15 +104,10 @@ CREATE OPERATOR citus_mx_test_schema.=== ( SET search_path TO public; CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); -SET search_path TO public; -CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text); -CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); - -- now create required stuff in the worker 2 \c - - - :worker_2_port -- create schema to test schema support -CREATE SCHEMA citus_mx_test_schema; CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_2; @@ -159,10 +153,6 @@ CREATE OPERATOR citus_mx_test_schema.=== ( SET search_path TO public; CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); -SET search_path TO public; -CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text); -CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); - -- connect back to the master, and do some more tests \c - - - :master_port diff --git a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql index 2e6631bb3..129b1ad75 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql @@ -67,13 +67,7 @@ CREATE TABLE repartition_udt_other ( \c - - - :worker_1_port -CREATE TYPE test_udt AS (i integer, i2 integer); -DROP TYPE test_udt CASCADE; - -- START type creation - -CREATE TYPE test_udt AS (i integer, i2 integer); - -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' @@ -118,9 +112,6 @@ FUNCTION 1 test_udt_hash(test_udt); \c - - - :worker_2_port -- START type creation - -CREATE TYPE test_udt AS (i integer, i2 integer); - -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' diff --git a/src/test/regress/sql/multi_partition_pruning.sql b/src/test/regress/sql/multi_partition_pruning.sql index 9b1ea1e62..0ab4d28e0 100644 --- a/src/test/regress/sql/multi_partition_pruning.sql +++ b/src/test/regress/sql/multi_partition_pruning.sql @@ -92,6 +92,7 @@ INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, LIMIT 1; -- Create composite type partitioned table +RESET client_min_messages; -- avoid debug messages CREATE TYPE composite_type AS ( @@ -100,7 +101,6 @@ CREATE TYPE composite_type AS varchar_column varchar(50) ); -RESET client_min_messages; -- avoid debug messages about toast index creation CREATE TABLE composite_partitioned_table ( composite_column composite_type diff --git a/src/test/regress/sql/multi_prepare_plsql.sql b/src/test/regress/sql/multi_prepare_plsql.sql index 09c23ed31..61ff17475 100644 --- a/src/test/regress/sql/multi_prepare_plsql.sql +++ b/src/test/regress/sql/multi_prepare_plsql.sql @@ -582,8 +582,6 @@ SELECT schemaname, indexrelname FROM pg_stat_all_indexes WHERE indexrelname = 'p -- cleanup DROP TABLE prepare_ddl; -DROP SCHEMA otherschema; - RESET search_path; -- test prepared COPY @@ -612,10 +610,39 @@ $BODY$ LANGUAGE plpgsql; SELECT local_copy_in_plpgsql(); SELECT local_copy_in_plpgsql(); +-- types statements should not crash nor leak schema specifications on to cached statements + +CREATE TYPE prepare_ddl_type AS (x int, y int); +SET search_path TO 'otherschema', public; + +CREATE OR REPLACE FUNCTION public.type_ddl_plpgsql() +RETURNS void +LANGUAGE plpgsql +AS $function$ +DECLARE +BEGIN + ALTER TYPE prepare_ddl_type RENAME TO prepare_ddl_type_backup; +END; +$function$; + +SELECT type_ddl_plpgsql(); +-- create same type in new schema, owner of this new type should change +CREATE TYPE prepare_ddl_type AS (x int, y int); +SELECT type_ddl_plpgsql(); + +-- find all renamed types to verify the schema name didn't leak, nor a crash happened +SELECT nspname, typname FROM pg_type JOIN pg_namespace ON pg_namespace.oid = pg_type.typnamespace WHERE typname = 'prepare_ddl_type_backup'; + +DROP TYPE prepare_ddl_type_backup; +RESET search_path; + +DROP TYPE prepare_ddl_type_backup; +DROP FUNCTION type_ddl_plpgsql(); DROP FUNCTION ddl_in_plpgsql(); DROP FUNCTION copy_in_plpgsql(); DROP TABLE prepare_ddl; DROP TABLE local_ddl; +DROP SCHEMA otherschema; -- clean-up functions DROP FUNCTION plpgsql_test_1(); diff --git a/src/test/regress/sql/multi_reference_table.sql b/src/test/regress/sql/multi_reference_table.sql index bdccc4fc0..5f49736a5 100644 --- a/src/test/regress/sql/multi_reference_table.sql +++ b/src/test/regress/sql/multi_reference_table.sql @@ -954,12 +954,7 @@ TRUNCATE reference_table_test; -- first create the type on all nodes CREATE TYPE reference_comp_key as (key text, value text); -\c - - - :worker_1_port -CREATE TYPE reference_comp_key as (key text, value text); -\c - - - :worker_2_port -CREATE TYPE reference_comp_key as (key text, value text); -\c - - - :master_port CREATE TABLE reference_table_composite (id int PRIMARY KEY, data reference_comp_key); SELECT create_reference_table('reference_table_composite'); diff --git a/src/test/regress/sql/multi_repartition_udt.sql b/src/test/regress/sql/multi_repartition_udt.sql index b9c3610f2..ffcf851dc 100644 --- a/src/test/regress/sql/multi_repartition_udt.sql +++ b/src/test/regress/sql/multi_repartition_udt.sql @@ -67,13 +67,7 @@ CREATE TABLE repartition_udt_other ( \c - - - :worker_1_port -CREATE TYPE test_udt AS (i integer, i2 integer); -DROP TYPE test_udt CASCADE; - -- START type creation - -CREATE TYPE test_udt AS (i integer, i2 integer); - -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' @@ -118,9 +112,6 @@ FUNCTION 1 test_udt_hash(test_udt); \c - - - :worker_2_port -- START type creation - -CREATE TYPE test_udt AS (i integer, i2 integer); - -- ... as well as a function to use as its comparator... CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' diff --git a/src/test/regress/sql/multi_schema_support.sql b/src/test/regress/sql/multi_schema_support.sql index fd1540111..1afa50c49 100644 --- a/src/test/regress/sql/multi_schema_support.sql +++ b/src/test/regress/sql/multi_schema_support.sql @@ -381,15 +381,6 @@ SELECT n_comment FROM nation_hash_collation_search_path ORDER BY n_comment COLLA SET search_path TO public; CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); --- create type in worker node 1 in schema -\c - - - :worker_1_port -CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); - --- create type in worker node 2 in schema -\c - - - :worker_2_port -CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); - -\c - - - :master_port CREATE TABLE test_schema_support.nation_hash_composite_types( n_nationkey integer not null, n_name char(25) not null, @@ -742,14 +733,6 @@ SET citus.task_executor_type TO "real-time"; SET search_path TO public; ALTER TABLE test_schema_support.nation_hash SET SCHEMA public; --- we will use this function in next test -CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text) -RETURNS void LANGUAGE plpgsql AS $$ -BEGIN - EXECUTE p_sql; - PERFORM run_command_on_workers(p_sql); -END;$$; - -- test schema propagation with user other than current user SELECT run_command_on_coordinator_and_workers('CREATE USER "test-user"'); SELECT run_command_on_coordinator_and_workers('GRANT ALL ON DATABASE postgres to "test-user"'); diff --git a/src/test/regress/sql/multi_subquery.sql b/src/test/regress/sql/multi_subquery.sql index d65453486..10a274a8c 100644 --- a/src/test/regress/sql/multi_subquery.sql +++ b/src/test/regress/sql/multi_subquery.sql @@ -840,14 +840,7 @@ LIMIT SET client_min_messages TO WARNING; DROP TABLE users, events; -SELECT run_command_on_master_and_workers($f$ - - DROP TYPE user_composite_type CASCADE; - -$f$); - --- createed in multi_behavioral_analytics_create_table -DROP FUNCTION run_command_on_master_and_workers(p_sql text); +DROP TYPE user_composite_type CASCADE; SET client_min_messages TO DEFAULT; diff --git a/src/test/regress/sql/multi_test_helpers.sql b/src/test/regress/sql/multi_test_helpers.sql index 6dd0f1c16..7582f1ee1 100644 --- a/src/test/regress/sql/multi_test_helpers.sql +++ b/src/test/regress/sql/multi_test_helpers.sql @@ -132,3 +132,11 @@ BEGIN END LOOP; RETURN; END; $$ language plpgsql; + +-- helper function to quickly run SQL on the whole cluster +CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text) +RETURNS void LANGUAGE plpgsql AS $$ +BEGIN + EXECUTE p_sql; + PERFORM run_command_on_workers(p_sql); +END;$$; diff --git a/src/test/regress/sql/multi_upgrade_reference_table.sql b/src/test/regress/sql/multi_upgrade_reference_table.sql index 4c6d6372d..1f94dd51b 100644 --- a/src/test/regress/sql/multi_upgrade_reference_table.sql +++ b/src/test/regress/sql/multi_upgrade_reference_table.sql @@ -55,10 +55,6 @@ DROP TABLE upgrade_reference_table_unhealthy; -- test with table containing composite type CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text); -\c - - - :worker_1_port -CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text); - -\c - - - :master_port SET citus.shard_count TO 1; SET citus.shard_replication_factor TO 1; CREATE TABLE upgrade_reference_table_composite(column1 int, column2 upgrade_test_composite_type); diff --git a/src/test/regress/sql/mx_foreign_key_to_reference_table.sql b/src/test/regress/sql/mx_foreign_key_to_reference_table.sql index 8b19a2efe..5fab2b1c8 100644 --- a/src/test/regress/sql/mx_foreign_key_to_reference_table.sql +++ b/src/test/regress/sql/mx_foreign_key_to_reference_table.sql @@ -8,7 +8,6 @@ SET citus.replication_model TO streaming; -- Setup the view so that we can check if the foreign keys are created properly CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); -SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$); CREATE VIEW table_fkeys_in_workers AS SELECT diff --git a/src/test/regress/sql/subquery_prepared_statements.sql b/src/test/regress/sql/subquery_prepared_statements.sql index ca09234d7..6ee8cd66b 100644 --- a/src/test/regress/sql/subquery_prepared_statements.sql +++ b/src/test/regress/sql/subquery_prepared_statements.sql @@ -2,12 +2,11 @@ -- test recursive planning functionality on prepared statements -- =================================================================== CREATE SCHEMA subquery_prepared_statements; +SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements;'); + SET search_path TO subquery_prepared_statements, public; -CREATE TYPE xy AS (x int, y int); - -SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements'); -SELECT run_command_on_workers('CREATE TYPE subquery_prepared_statements.xy AS (x int, y int)'); +CREATE TYPE subquery_prepared_statements.xy AS (x int, y int); SET client_min_messages TO DEBUG1; diff --git a/src/test/regress/sql/validate_constraint.sql b/src/test/regress/sql/validate_constraint.sql index f2447285b..294e9a8b2 100644 --- a/src/test/regress/sql/validate_constraint.sql +++ b/src/test/regress/sql/validate_constraint.sql @@ -34,8 +34,6 @@ SET citus.next_shard_id TO 8000000; SET citus.next_placement_id TO 8000000; CREATE TYPE constraint_validity AS (name text, validated bool); -SELECT run_command_on_workers( - $$CREATE TYPE constraint_validity AS (name text, validated bool)$$); CREATE VIEW constraint_validations_in_workers AS SELECT (json_populate_record(NULL :: constraint_validity, diff --git a/src/test/regress/sql/with_basics.sql b/src/test/regress/sql/with_basics.sql index 75c9b3f3b..ff029b8c1 100644 --- a/src/test/regress/sql/with_basics.sql +++ b/src/test/regress/sql/with_basics.sql @@ -1,7 +1,6 @@ -- Test the basic CTE functionality and expected error messages SET search_path TO 'with_basics'; -CREATE TYPE xy AS (x int, y int); -SELECT run_command_on_workers('CREATE TYPE with_basics.xy AS (x int, y int)'); +CREATE TYPE with_basics.xy AS (x int, y int); -- CTEs in FROM should work WITH cte AS (