From 28796894416680fe386711344462cca510f03aa0 Mon Sep 17 00:00:00 2001 From: Nils Dijk Date: Fri, 13 Sep 2019 17:46:07 +0200 Subject: [PATCH] Distribute Types to worker nodes (#2893) DESCRIPTION: Distribute Types to worker nodes When to propagate ============== There are two logical moments that types could be distributed to the worker nodes - When they get used ( just in time distribution ) - When they get created ( proactive distribution ) The just in time distribution follows the model used by how schema's get created right before we are going to create a table in that schema, for types this would be when the table uses a type as its column. The proactive distribution is suitable for situations where it is benificial to have the type on the worker nodes directly. They can later on be used in queries where an intermediate result gets created with a cast to this type. Just in time creation is always the last resort, you cannot create a distributed table before the type gets created. A good example use case is; you have an existing postgres server that needs to scale out. By adding the citus extension, add some nodes to the cluster, and distribute the table. The type got created before citus existed. There was no moment where citus could have propagated the creation of a type. Proactive is almost always a good option. Types are not resource intensive objects, there is no performance overhead of having 100's of types. If you want to use them in a query to represent an intermediate result (which happens in our test suite) they just work. There is however a moment when proactive type distribution is not beneficial; in transactions where the type is used in a distributed table. Lets assume the following transaction: ```sql BEGIN; CREATE TYPE tt1 AS (a int, b int); CREATE TABLE t1 AS (a int PRIMARY KEY, b tt1); SELECT create_distributed_table('t1', 'a'); \copy t1 FROM bigdata.csv ``` Types are node scoped objects; meaning the type exists once per worker. Shards however have best performance when they are created over their own connection. For the type to be visible on all connections it needs to be created and committed before we try to create the shards. Here the just in time situation is most beneficial and follows how we create schema's on the workers. Outside of a transaction block we will just use 1 connection to propagate the creation. How propagation works ================= Just in time ----------- Just in time propagation hooks into the infrastructure introduced in #2882. It adds types as a supported object in `SupportedDependencyByCitus`. This will make sure that any object being distributed by citus that depends on types will now cascade into types. When types are depending them self on other objects they will get created first. Creation later works by getting the ddl commands to create the object by its `ObjectAddress` in `GetDependencyCreateDDLCommands` which will dispatch types to `CreateTypeDDLCommandsIdempotent`. For the correct walking of the graph we follow array types, when later asked for the ddl commands for array types we return `NIL` (empty list) which makes that the object will not be recorded as distributed, (its an internal type, dependant on the user type). Proactive distribution --------------------- When the user creates a type (composite or enum) we will have a hook running in `multi_ProcessUtility` after the command has been applied locally. Running after running locally makes that we already have an `ObjectAddress` for the type. This is required to mark the type as being distributed. Keeping the type up to date ==================== For types that are recorded in `pg_dist_object` (eg. `IsObjectDistributed` returns true for the `ObjectAddress`) we will intercept the utility commands that alter the type. - `AlterTableStmt` with `relkind` set to `OBJECT_TYPE` encapsulate changes to the fields of a composite type. - `DropStmt` with removeType set to `OBJECT_TYPE` encapsulate `DROP TYPE`. - `AlterEnumStmt` encapsulates changes to enum values. Enum types can not be changed transactionally. When the execution on a worker fails a warning will be shown to the user the propagation was incomplete due to worker communication failure. An idempotent command is shown for the user to re-execute when the worker communication is fixed. Keeping types up to date is done via the executor. Before the statement is executed locally we create a plan on how to apply it on the workers. This plan is executed after we have applied the statement locally. All changes to types need to be done in the same transaction for types that have already been distributed and will fail with an error if parallel queries have already been executed in the same transaction. Much like foreign keys to reference tables. --- src/backend/distributed/Makefile | 2 +- .../distributed/citus--8.3-1--8.4-1.sql | 7 + .../distributed/commands/dependencies.c | 88 +- src/backend/distributed/commands/schema.c | 71 +- src/backend/distributed/commands/table.c | 22 +- src/backend/distributed/commands/type.c | 1415 +++++++++++++++++ .../distributed/commands/utility_hook.c | 305 +++- .../distributed/connection/remote_commands.c | 19 +- src/backend/distributed/deparser/deparse.c | 204 +++ .../distributed/deparser/deparse_type_stmts.c | 509 ++++++ .../distributed/deparser/format_collate.c | 104 ++ .../distributed/deparser/objectaddress.c | 191 +++ src/backend/distributed/deparser/qualify.c | 198 +++ .../distributed/deparser/qualify_type_stmt.c | 190 +++ .../executor/multi_router_executor.c | 19 +- .../distributed/master/master_repair_shards.c | 1 + src/backend/distributed/metadata/dependency.c | 128 +- src/backend/distributed/metadata/distobject.c | 5 + .../distributed/metadata/metadata_sync.c | 1 + src/backend/distributed/metadata/namespace.c | 35 + .../transaction/relation_access_tracking.c | 5 +- .../transaction/worker_transaction.c | 176 +- src/backend/distributed/utils/node_metadata.c | 1 + .../distributed/utils/reference_table_utils.c | 1 + .../worker/worker_create_or_replace.c | 180 +++ src/include/distributed/commands.h | 41 + .../distributed/commands/utility_hook.h | 5 +- src/include/distributed/deparser.h | 55 + src/include/distributed/metadata/namespace.h | 20 + src/include/distributed/remote_commands.h | 1 + src/include/distributed/worker_transaction.h | 20 +- src/test/regress/Makefile | 5 + .../regress/expected/distributed_types.out | 338 ++++ .../regress/expected/distributed_types_0.out | 338 ++++ ...ure_create_distributed_table_non_empty.out | 20 +- ...e_create_distributed_table_non_empty_9.out | 20 +- .../failure_create_reference_table.out | 19 +- .../failure_create_reference_table_9.out | 17 +- .../regress/expected/failure_create_table.out | 12 + .../expected/failure_create_table_9.out | 12 + .../foreign_key_to_reference_table.out | 14 - .../regress/expected/intermediate_results.out | 7 - .../isolation_citus_dist_activity.out | 20 +- .../isolation_citus_dist_activity_9.out | 20 +- .../isolation_distributed_transaction_id.out | 31 +- .../isolation_dump_global_wait_edges.out | 18 +- .../isolation_dump_global_wait_edges_0.out | 98 ++ ...lation_ensure_dependency_activate_node.out | 521 ++++++ .../isolation_replace_wait_function.out | 2 +- .../regress/expected/multi_data_types.out | 53 +- .../expected/multi_function_in_join.out | 2 + .../expected/multi_mx_create_table.out | 8 - .../multi_mx_repartition_udt_prepare.out | 4 - .../expected/multi_partition_pruning.out | 2 +- .../regress/expected/multi_prepare_plsql.out | 40 +- .../expected/multi_reference_table.out | 5 - .../expected/multi_repartition_udt.out | 4 - .../multi_replicate_reference_table.out | 5 +- .../regress/expected/multi_schema_support.out | 14 - src/test/regress/expected/multi_subquery.out | 13 +- .../regress/expected/multi_test_helpers.out | 7 + .../multi_upgrade_reference_table.out | 46 +- .../mx_foreign_key_to_reference_table.out | 7 - .../expected/subquery_prepared_statements.out | 13 +- .../regress/expected/validate_constraint.out | 8 - src/test/regress/expected/with_basics.out | 9 +- ...i_behavioral_analytics_create_table.source | 13 +- .../input/multi_complex_count_distinct.source | 6 - src/test/regress/input/multi_copy.source | 31 - src/test/regress/multi_schedule | 5 + ...i_behavioral_analytics_create_table.source | 18 +- .../multi_complex_count_distinct.source | 18 - src/test/regress/output/multi_copy.source | 22 - src/test/regress/pg_regress_multi.pl | 31 +- .../isolation_distributed_transaction_id.spec | 15 +- ...ation_ensure_dependency_activate_node.spec | 27 + src/test/regress/sql/distributed_types.sql | 222 +++ ...ure_create_distributed_table_non_empty.sql | 10 + .../sql/failure_create_reference_table.sql | 5 + src/test/regress/sql/failure_create_table.sql | 15 + .../sql/foreign_key_to_reference_table.sql | 2 - src/test/regress/sql/intermediate_results.sql | 1 - src/test/regress/sql/multi_data_types.sql | 39 +- .../regress/sql/multi_function_in_join.sql | 2 + .../regress/sql/multi_mx_create_table.sql | 10 - .../sql/multi_mx_repartition_udt_prepare.sql | 9 - .../regress/sql/multi_partition_pruning.sql | 2 +- src/test/regress/sql/multi_prepare_plsql.sql | 31 +- .../regress/sql/multi_reference_table.sql | 5 - .../regress/sql/multi_repartition_udt.sql | 9 - src/test/regress/sql/multi_schema_support.sql | 17 - src/test/regress/sql/multi_subquery.sql | 9 +- src/test/regress/sql/multi_test_helpers.sql | 8 + .../sql/multi_upgrade_reference_table.sql | 4 - .../sql/mx_foreign_key_to_reference_table.sql | 1 - .../sql/subquery_prepared_statements.sql | 7 +- src/test/regress/sql/validate_constraint.sql | 2 - src/test/regress/sql/with_basics.sql | 3 +- 98 files changed, 5790 insertions(+), 580 deletions(-) create mode 100644 src/backend/distributed/commands/type.c create mode 100644 src/backend/distributed/deparser/deparse.c create mode 100644 src/backend/distributed/deparser/deparse_type_stmts.c create mode 100644 src/backend/distributed/deparser/format_collate.c create mode 100644 src/backend/distributed/deparser/objectaddress.c create mode 100644 src/backend/distributed/deparser/qualify.c create mode 100644 src/backend/distributed/deparser/qualify_type_stmt.c create mode 100644 src/backend/distributed/metadata/namespace.c create mode 100644 src/backend/distributed/worker/worker_create_or_replace.c create mode 100644 src/include/distributed/deparser.h create mode 100644 src/include/distributed/metadata/namespace.h create mode 100644 src/test/regress/expected/distributed_types.out create mode 100644 src/test/regress/expected/distributed_types_0.out create mode 100644 src/test/regress/expected/isolation_dump_global_wait_edges_0.out create mode 100644 src/test/regress/sql/distributed_types.sql 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 (