diff --git a/src/backend/distributed/commands/collation.c b/src/backend/distributed/commands/collation.c index d712c18ab..7f047ec1d 100644 --- a/src/backend/distributed/commands/collation.c +++ b/src/backend/distributed/commands/collation.c @@ -37,7 +37,7 @@ static char * CreateCollationDDLInternal(Oid collationId, Oid *collowner, char **quotedCollationName); static List * FilterNameListForDistributedCollations(List *objects, bool missing_ok, List **addresses); - +static bool ShouldPropagateDefineCollationStmt(void); /* * GetCreateCollationDDLInternal returns a CREATE COLLATE sql string for the @@ -519,6 +519,26 @@ DefineCollationStmtObjectAddress(Node *node, bool missing_ok) } +/* + * PreprocessDefineCollationStmt executed before the collation has been + * created locally to ensure that if the collation create statement will + * be propagated, the node is a coordinator node + */ +List * +PreprocessDefineCollationStmt(Node *node, const char *queryString, + ProcessUtilityContext processUtilityContext) +{ + Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION); + + if (ShouldPropagateDefineCollationStmt()) + { + EnsureCoordinator(); + } + + return NIL; +} + + /* * PostprocessDefineCollationStmt executed after the collation has been * created locally and before we create it on the worker nodes. @@ -531,16 +551,7 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString) { Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION); - if (!ShouldPropagate()) - { - return NIL; - } - - /* - * If the create collation command is a part of a multi-statement transaction, - * do not propagate it - */ - if (IsMultiStatementTransaction()) + if (!ShouldPropagateDefineCollationStmt()) { return NIL; } @@ -548,13 +559,38 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString) ObjectAddress collationAddress = DefineCollationStmtObjectAddress(node, false); - if (IsObjectDistributed(&collationAddress)) - { - EnsureCoordinator(); - } - EnsureDependenciesExistOnAllNodes(&collationAddress); - return NodeDDLTaskList(NON_COORDINATOR_NODES, CreateCollationDDLsIdempotent( + /* to prevent recursion with mx we disable ddl propagation */ + List *commands = list_make1(DISABLE_DDL_PROPAGATION); + commands = list_concat(commands, CreateCollationDDLsIdempotent( collationAddress.objectId)); + commands = lappend(commands, ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(NON_COORDINATOR_NODES, commands); +} + + +/* + * ShouldPropagateDefineCollationStmt checks if collation define + * statement should be propagated. Don't propagate if: + * - metadata syncing if off + * - statement is part of a multi stmt transaction and the multi shard connection + * type is not sequential + */ +static bool +ShouldPropagateDefineCollationStmt() +{ + if (!ShouldPropagate()) + { + return false; + } + + if (IsMultiStatementTransaction() && + MultiShardConnectionType != SEQUENTIAL_CONNECTION) + { + return false; + } + + return true; } diff --git a/src/backend/distributed/commands/distribute_object_ops.c b/src/backend/distributed/commands/distribute_object_ops.c index 9d680a467..37758aec3 100644 --- a/src/backend/distributed/commands/distribute_object_ops.c +++ b/src/backend/distributed/commands/distribute_object_ops.c @@ -175,7 +175,7 @@ static DistributeObjectOps Any_CreateFunction = { .preprocess = PreprocessCreateFunctionStmt, .postprocess = PostprocessCreateFunctionStmt, .address = CreateFunctionStmtObjectAddress, - .markDistributed = false, + .markDistributed = true, }; static DistributeObjectOps Any_CreatePolicy = { .deparse = NULL, @@ -276,7 +276,7 @@ static DistributeObjectOps Collation_AlterOwner = { static DistributeObjectOps Collation_Define = { .deparse = NULL, .qualify = NULL, - .preprocess = NULL, + .preprocess = PreprocessDefineCollationStmt, .postprocess = PostprocessDefineCollationStmt, .address = DefineCollationStmtObjectAddress, .markDistributed = true, diff --git a/src/backend/distributed/commands/function.c b/src/backend/distributed/commands/function.c index 95b11fd0c..80b870b62 100644 --- a/src/backend/distributed/commands/function.c +++ b/src/backend/distributed/commands/function.c @@ -25,6 +25,7 @@ #include "access/htup_details.h" #include "access/xact.h" #include "catalog/pg_aggregate.h" +#include "catalog/dependency.h" #include "catalog/namespace.h" #include "catalog/pg_proc.h" #include "catalog/pg_type.h" @@ -38,6 +39,7 @@ #include "distributed/listutils.h" #include "distributed/maintenanced.h" #include "distributed/metadata_utility.h" +#include "distributed/metadata/dependency.h" #include "distributed/coordinator_protocol.h" #include "distributed/metadata/distobject.h" #include "distributed/metadata/pg_dist_object.h" @@ -80,6 +82,7 @@ static void EnsureFunctionCanBeColocatedWithTable(Oid functionOid, Oid static bool ShouldPropagateCreateFunction(CreateFunctionStmt *stmt); static bool ShouldPropagateAlterFunction(const ObjectAddress *address); static bool ShouldAddFunctionSignature(FunctionParameterMode mode); +static ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress); static ObjectAddress FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs, bool missing_ok); @@ -759,7 +762,7 @@ UpdateFunctionDistributionInfo(const ObjectAddress *distAddress, /* * GetFunctionDDLCommand returns the complete "CREATE OR REPLACE FUNCTION ..." statement for - * the specified function followed by "ALTER FUNCTION .. SET OWNER ..". + * the specified function. * * useCreateOrReplace is ignored for non-aggregate functions. */ @@ -1170,46 +1173,23 @@ GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace) /* * ShouldPropagateCreateFunction tests if we need to propagate a CREATE FUNCTION - * statement. We only propagate replace's of distributed functions to keep the function on - * the workers in sync with the one on the coordinator. + * statement. */ static bool ShouldPropagateCreateFunction(CreateFunctionStmt *stmt) { - if (creating_extension) + if (!ShouldPropagate()) { - /* - * extensions should be created separately on the workers, functions cascading - * from an extension should therefore not be propagated. - */ - return false; - } - - if (!EnableMetadataSync) - { - /* - * we are configured to disable object propagation, should not propagate anything - */ - return false; - } - - if (!stmt->replace) - { - /* - * Since we only care for a replace of distributed functions if the statement is - * not a replace we are going to ignore. - */ return false; } /* - * Even though its a replace we should accept an non-existing function, it will just - * not be distributed + * If the create command is a part of a multi-statement transaction that is not in + * sequential mode, don't propagate. */ - ObjectAddress address = GetObjectAddressFromParseTree((Node *) stmt, true); - if (!IsObjectDistributed(&address)) + if (IsMultiStatementTransaction() && + MultiShardConnectionType != SEQUENTIAL_CONNECTION) { - /* do not propagate alter function for non-distributed functions */ return false; } @@ -1253,12 +1233,10 @@ ShouldPropagateAlterFunction(const ObjectAddress *address) /* * PreprocessCreateFunctionStmt is called during the planning phase for CREATE [OR REPLACE] - * FUNCTION. We primarily care for the replace variant of this statement to keep - * distributed functions in sync. We bail via a check on ShouldPropagateCreateFunction - * which checks for the OR REPLACE modifier. + * FUNCTION before it is created on the local node internally. * * Since we use pg_get_functiondef to get the ddl command we actually do not do any - * planning here, instead we defer the plan creation to the processing step. + * planning here, instead we defer the plan creation to the postprocessing step. * * Instead we do our basic housekeeping where we make sure we are on the coordinator and * can propagate the function in sequential mode. @@ -1279,7 +1257,7 @@ PreprocessCreateFunctionStmt(Node *node, const char *queryString, EnsureSequentialMode(OBJECT_FUNCTION); /* - * ddl jobs will be generated during the Processing phase as we need the function to + * ddl jobs will be generated during the postprocessing phase as we need the function to * be updated in the catalog to get its sql representation */ return NIL; @@ -1290,6 +1268,11 @@ PreprocessCreateFunctionStmt(Node *node, const char *queryString, * PostprocessCreateFunctionStmt actually creates the plan we need to execute for function * propagation. This is the downside of using pg_get_functiondef to get the sql statement. * + * If function depends on any non-distributed relation (except sequence and composite type), + * Citus can not distribute it. In order to not to prevent users from creating local + * functions on the coordinator WARNING message will be sent to the customer about the case + * instead of erroring out. + * * Besides creating the plan we also make sure all (new) dependencies of the function are * created on all nodes. */ @@ -1303,18 +1286,113 @@ PostprocessCreateFunctionStmt(Node *node, const char *queryString) return NIL; } - ObjectAddress address = GetObjectAddressFromParseTree((Node *) stmt, false); - EnsureDependenciesExistOnAllNodes(&address); + ObjectAddress functionAddress = GetObjectAddressFromParseTree((Node *) stmt, false); - List *commands = list_make4(DISABLE_DDL_PROPAGATION, - GetFunctionDDLCommand(address.objectId, true), - GetFunctionAlterOwnerCommand(address.objectId), - ENABLE_DDL_PROPAGATION); + if (IsObjectAddressOwnedByExtension(&functionAddress, NULL)) + { + return NIL; + } + + /* + * This check should have been valid for all objects not only for functions. Though, + * we do this limited check for now as functions are more likely to be used with + * such dependencies, and we want to scope it for now. + */ + ObjectAddress *undistributableDependency = GetUndistributableDependency( + &functionAddress); + if (undistributableDependency != NULL) + { + if (SupportedDependencyByCitus(undistributableDependency)) + { + /* + * Citus can't distribute some relations as dependency, although those + * types as supported by Citus. So we can use get_rel_name directly + */ + RangeVar *functionRangeVar = makeRangeVarFromNameList(stmt->funcname); + char *functionName = functionRangeVar->relname; + char *dependentRelationName = + get_rel_name(undistributableDependency->objectId); + + ereport(WARNING, (errmsg("Citus can't distribute function \"%s\" having " + "dependency on non-distributed relation \"%s\"", + functionName, dependentRelationName), + errdetail("Function will be created only locally"), + errhint("To distribute function, distribute dependent " + "relations first. Then, re-create the function"))); + } + else + { + char *objectType = NULL; + #if PG_VERSION_NUM >= PG_VERSION_14 + objectType = getObjectTypeDescription(undistributableDependency, false); + #else + objectType = getObjectTypeDescription(undistributableDependency); + #endif + ereport(WARNING, (errmsg("Citus can't distribute functions having " + "dependency on unsupported object of type \"%s\"", + objectType), + errdetail("Function will be created only locally"))); + } + + return NIL; + } + + EnsureDependenciesExistOnAllNodes(&functionAddress); + + List *commands = list_make1(DISABLE_DDL_PROPAGATION); + commands = list_concat(commands, CreateFunctionDDLCommandsIdempotent( + &functionAddress)); + commands = list_concat(commands, list_make1(ENABLE_DDL_PROPAGATION)); return NodeDDLTaskList(NON_COORDINATOR_NODES, commands); } +/* + * GetUndistributableDependency checks whether object has any non-distributable + * dependency. If any one found, it will be returned. + */ +static ObjectAddress * +GetUndistributableDependency(ObjectAddress *objectAddress) +{ + List *dependencies = GetAllDependenciesForObject(objectAddress); + ObjectAddress *dependency = NULL; + foreach_ptr(dependency, dependencies) + { + if (IsObjectDistributed(dependency)) + { + continue; + } + + if (!SupportedDependencyByCitus(dependency)) + { + /* + * Since roles should be handled manually with Citus community, skip them. + */ + if (getObjectClass(dependency) != OCLASS_ROLE) + { + return dependency; + } + } + + if (getObjectClass(dependency) == OCLASS_CLASS) + { + /* + * Citus can only distribute dependent non-distributed sequence + * and composite types. + */ + char relKind = get_rel_relkind(dependency->objectId); + if (relKind != RELKIND_SEQUENCE && relKind != RELKIND_COMPOSITE_TYPE) + { + return dependency; + } + } + } + + return NULL; +} + + /* * CreateFunctionStmtObjectAddress returns the ObjectAddress for the subject of the * CREATE [OR REPLACE] FUNCTION statement. If missing_ok is false it will error with the diff --git a/src/backend/distributed/commands/type.c b/src/backend/distributed/commands/type.c index 524124285..eb59e8522 100644 --- a/src/backend/distributed/commands/type.c +++ b/src/backend/distributed/commands/type.c @@ -957,6 +957,20 @@ CreateTypeDDLCommandsIdempotent(const ObjectAddress *typeAddress) return NIL; } + HeapTuple tup = SearchSysCacheCopy1(TYPEOID, ObjectIdGetDatum(typeAddress->objectId)); + if (!HeapTupleIsValid(tup)) + { + elog(ERROR, "cache lookup failed for type %u", typeAddress->objectId); + } + + /* Don't send any command if the type is a table's row type */ + Form_pg_type typTup = (Form_pg_type) GETSTRUCT(tup); + if (typTup->typtype == TYPTYPE_COMPOSITE && + get_rel_relkind(typTup->typrelid) != RELKIND_COMPOSITE_TYPE) + { + return NIL; + } + Node *stmt = CreateTypeStmtByObjectAddress(typeAddress); /* capture ddl command for recreation and wrap in create if not exists construct */ diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 93f7baf7a..28498e0f2 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -237,16 +237,17 @@ CitusExecutorRun(QueryDesc *queryDesc, * transactions. */ CitusTableCacheFlushInvalidatedEntries(); - - /* - * Within a 2PC, when a function is delegated to a remote node, we pin - * the distribution argument as the shard key for all the SQL in the - * function's block. The restriction is imposed to not to access other - * nodes from the current node and violate the transactional integrity - * of the 2PC. Now that the query is ending, reset the shard key to NULL. - */ - ResetAllowedShardKeyValue(); + InTopLevelDelegatedFunctionCall = false; } + + /* + * Within a 2PC, when a function is delegated to a remote node, we pin + * the distribution argument as the shard key for all the SQL in the + * function's block. The restriction is imposed to not to access other + * nodes from the current node, and violate the transactional integrity + * of the 2PC. Now that the query is ending, reset the shard key to NULL. + */ + CheckAndResetAllowedShardKeyValueIfNeeded(); } PG_CATCH(); { @@ -260,13 +261,15 @@ CitusExecutorRun(QueryDesc *queryDesc, if (ExecutorLevel == 0 && PlannerLevel == 0) { - /* - * In case of an exception, reset the pinned shard-key, for more - * details see the function header. - */ - ResetAllowedShardKeyValue(); + InTopLevelDelegatedFunctionCall = false; } + /* + * In case of an exception, reset the pinned shard-key, for more + * details see the function header. + */ + CheckAndResetAllowedShardKeyValueIfNeeded(); + PG_RE_THROW(); } PG_END_TRY(); diff --git a/src/backend/distributed/metadata/dependency.c b/src/backend/distributed/metadata/dependency.c index 9d58c87cd..2d1f67ad3 100644 --- a/src/backend/distributed/metadata/dependency.c +++ b/src/backend/distributed/metadata/dependency.c @@ -156,6 +156,8 @@ static bool FollowAllSupportedDependencies(ObjectAddressCollector *collector, DependencyDefinition *definition); static bool FollowNewSupportedDependencies(ObjectAddressCollector *collector, DependencyDefinition *definition); +static bool FollowAllDependencies(ObjectAddressCollector *collector, + DependencyDefinition *definition); static void ApplyAddToDependencyList(ObjectAddressCollector *collector, DependencyDefinition *definition); static List * ExpandCitusSupportedTypes(ObjectAddressCollector *collector, @@ -212,15 +214,42 @@ GetDependenciesForObject(const ObjectAddress *target) /* - * GetAllDependenciesForObject returns a list of all the ObjectAddresses to be - * created in order before the target object could safely be created on a - * worker. As a caller, you probably need GetDependenciesForObject() which - * eliminates already distributed objects from the returned list. + * GetAllSupportedDependenciesForObject returns a list of all the ObjectAddresses to be + * created in order before the target object could safely be created on a worker, if all + * dependent objects are distributable. As a caller, you probably need to use + * GetDependenciesForObject() which eliminates already distributed objects from the returned + * list. * * Some of the object might already be created on a worker. It should be created * in an idempotent way. */ List * +GetAllSupportedDependenciesForObject(const ObjectAddress *target) +{ + ObjectAddressCollector collector = { 0 }; + InitObjectAddressCollector(&collector); + + RecurseObjectDependencies(*target, + &ExpandCitusSupportedTypes, + &FollowAllSupportedDependencies, + &ApplyAddToDependencyList, + &collector); + + return collector.dependencyList; +} + + +/* + * GetAllDependenciesForObject returns a list of all the dependent objects of the given + * object irrespective of whether the dependent object is supported by Citus or not, if + * the object can be found as dependency with RecurseObjectDependencies and + * ExpandCitusSupportedTypes. + * + * This function will be used to provide meaningful error messages if any dependent + * object for a given object is not supported. If you want to create dependencies for + * an object, you probably need to use GetDependenciesForObject(). + */ +List * GetAllDependenciesForObject(const ObjectAddress *target) { ObjectAddressCollector collector = { 0 }; @@ -228,7 +257,7 @@ GetAllDependenciesForObject(const ObjectAddress *target) RecurseObjectDependencies(*target, &ExpandCitusSupportedTypes, - &FollowAllSupportedDependencies, + &FollowAllDependencies, &ApplyAddToDependencyList, &collector); @@ -903,10 +932,61 @@ FollowAllSupportedDependencies(ObjectAddressCollector *collector, /* - * ApplyAddToDependencyList is an apply function for RecurseObjectDependencies that will collect - * all the ObjectAddresses for pg_depend entries to the context. The context here is - * assumed to be a (ObjectAddressCollector *) to the location where all ObjectAddresses - * will be collected. + * FollowAllDependencies applies filters on pg_depend entries to follow the dependency + * tree of objects in depth first order. We will visit all objects irrespective of it is + * supported by Citus or not. + */ +static bool +FollowAllDependencies(ObjectAddressCollector *collector, + DependencyDefinition *definition) +{ + if (definition->mode == DependencyPgDepend) + { + /* + * For dependencies found in pg_depend: + * + * Follow only normal and extension dependencies. The latter is used to reach the + * extensions, the objects that directly depend on the extension are eliminated + * during the "apply" phase. + * + * Other dependencies are internal dependencies and managed by postgres. + */ + if (definition->data.pg_depend.deptype != DEPENDENCY_NORMAL && + definition->data.pg_depend.deptype != DEPENDENCY_EXTENSION) + { + return false; + } + } + + /* rest of the tests are to see if we want to follow the actual dependency */ + ObjectAddress address = DependencyDefinitionObjectAddress(definition); + + /* + * If the object is already in our dependency list we do not have to follow any + * further + */ + if (IsObjectAddressCollected(address, collector)) + { + return false; + } + + if (CitusExtensionObject(&address)) + { + /* following citus extension could complicate role management */ + return false; + } + + return true; +} + + +/* + * ApplyAddToDependencyList is an apply function for RecurseObjectDependencies that will + * collect all the ObjectAddresses for pg_depend entries to the context, except it is + * extension owned one. + * + * The context here is assumed to be a (ObjectAddressCollector *) to the location where + * all ObjectAddresses will be collected. */ static void ApplyAddToDependencyList(ObjectAddressCollector *collector, diff --git a/src/backend/distributed/planner/function_call_delegation.c b/src/backend/distributed/planner/function_call_delegation.c index 9ea275035..3ca22f3b1 100644 --- a/src/backend/distributed/planner/function_call_delegation.c +++ b/src/backend/distributed/planner/function_call_delegation.c @@ -723,6 +723,16 @@ FunctionInFromClause(List *fromlist, Query *query) static void EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId) { + /* + * If the distribution key is already set, the key is fixed until + * the force-delegation function returns. All nested force-delegation + * functions must use the same key. + */ + if (AllowedDistributionColumnValue.isActive) + { + return; + } + /* * The saved distribution argument need to persist through the life * of the query, both during the planning (where we save) and execution @@ -734,6 +744,7 @@ EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId) colocationId)); AllowedDistributionColumnValue.distributionColumnValue = copyObject(distArgument); AllowedDistributionColumnValue.colocationId = colocationId; + AllowedDistributionColumnValue.executorLevel = ExecutorLevel; AllowedDistributionColumnValue.isActive = true; MemoryContextSwitchTo(oldcontext); } @@ -747,15 +758,22 @@ EnableInForceDelegatedFuncExecution(Const *distArgument, uint32 colocationId) * the 2PC. Reset the distribution argument value once the function ends. */ void -ResetAllowedShardKeyValue(void) +CheckAndResetAllowedShardKeyValueIfNeeded(void) { - if (AllowedDistributionColumnValue.isActive) + /* + * If no distribution argument is pinned or the pinned argument was + * set by a nested-executor from upper level, nothing to reset. + */ + if (!AllowedDistributionColumnValue.isActive || + ExecutorLevel > AllowedDistributionColumnValue.executorLevel) { - pfree(AllowedDistributionColumnValue.distributionColumnValue); - AllowedDistributionColumnValue.isActive = false; + return; } - InTopLevelDelegatedFunctionCall = false; + Assert(ExecutorLevel == AllowedDistributionColumnValue.executorLevel); + pfree(AllowedDistributionColumnValue.distributionColumnValue); + AllowedDistributionColumnValue.isActive = false; + AllowedDistributionColumnValue.executorLevel = 0; } @@ -767,6 +785,7 @@ bool IsShardKeyValueAllowed(Const *shardKey, uint32 colocationId) { Assert(AllowedDistributionColumnValue.isActive); + Assert(ExecutorLevel > AllowedDistributionColumnValue.executorLevel); ereport(DEBUG4, errmsg("Comparing saved:%s with Shard key: %s colocationid:%d:%d", pretty_format_node_dump( diff --git a/src/backend/distributed/test/dependency.c b/src/backend/distributed/test/dependency.c index 79ad1c139..82e818b8c 100644 --- a/src/backend/distributed/test/dependency.c +++ b/src/backend/distributed/test/dependency.c @@ -47,7 +47,7 @@ citus_get_all_dependencies_for_object(PG_FUNCTION_ARGS) ObjectAddress address = { 0 }; ObjectAddressSubSet(address, classid, objid, objsubid); - List *dependencies = GetAllDependenciesForObject(&address); + List *dependencies = GetAllSupportedDependenciesForObject(&address); ObjectAddress *dependency = NULL; foreach_ptr(dependency, dependencies) { diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c index 4fe97e421..78e14367a 100644 --- a/src/backend/distributed/transaction/transaction_management.c +++ b/src/backend/distributed/transaction/transaction_management.c @@ -557,7 +557,8 @@ ResetGlobalVariables() MetadataSyncOnCommit = false; InTopLevelDelegatedFunctionCall = false; ResetWorkerErrorIndication(); - AllowedDistributionColumnValue.isActive = false; + memset(&AllowedDistributionColumnValue, 0, + sizeof(AllowedDistributionColumn)); } diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index 137ed2e01..1b2b92590 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -151,6 +151,8 @@ extern ObjectAddress AlterCollationSchemaStmtObjectAddress(Node *stmt, extern List * PostprocessAlterCollationSchemaStmt(Node *stmt, const char *queryString); extern char * GenerateBackupNameForCollationCollision(const ObjectAddress *address); extern ObjectAddress DefineCollationStmtObjectAddress(Node *stmt, bool missing_ok); +extern List * PreprocessDefineCollationStmt(Node *stmt, const char *queryString, + ProcessUtilityContext processUtilityContext); extern List * PostprocessDefineCollationStmt(Node *stmt, const char *queryString); /* database.c - forward declarations */ diff --git a/src/include/distributed/function_call_delegation.h b/src/include/distributed/function_call_delegation.h index 5471a3f1d..bf3dbe170 100644 --- a/src/include/distributed/function_call_delegation.h +++ b/src/include/distributed/function_call_delegation.h @@ -23,7 +23,7 @@ extern bool InTopLevelDelegatedFunctionCall; extern bool InDelegatedProcedureCall; PlannedStmt * TryToDelegateFunctionCall(DistributedPlanningContext *planContext); -extern void ResetAllowedShardKeyValue(void); +extern void CheckAndResetAllowedShardKeyValueIfNeeded(void); extern bool IsShardKeyValueAllowed(Const *shardKey, uint32 colocationId); #endif /* FUNCTION_CALL_DELEGATION_H */ diff --git a/src/include/distributed/metadata/dependency.h b/src/include/distributed/metadata/dependency.h index d20103011..141b2a628 100644 --- a/src/include/distributed/metadata/dependency.h +++ b/src/include/distributed/metadata/dependency.h @@ -19,6 +19,7 @@ extern List * GetUniqueDependenciesList(List *objectAddressesList); extern List * GetDependenciesForObject(const ObjectAddress *target); +extern List * GetAllSupportedDependenciesForObject(const ObjectAddress *target); extern List * GetAllDependenciesForObject(const ObjectAddress *target); extern List * OrderObjectAddressListInDependencyOrder(List *objectAddressList); extern bool SupportedDependencyByCitus(const ObjectAddress *address); diff --git a/src/include/distributed/transaction_management.h b/src/include/distributed/transaction_management.h index 48bce27a5..5ff34e21a 100644 --- a/src/include/distributed/transaction_management.h +++ b/src/include/distributed/transaction_management.h @@ -70,6 +70,9 @@ typedef struct AllowedDistributionColumn Const *distributionColumnValue; uint32 colocationId; bool isActive; + + /* In nested executor, track the level at which value is set */ + int executorLevel; } AllowedDistributionColumn; /* diff --git a/src/test/regress/expected/aggregate_support.out b/src/test/regress/expected/aggregate_support.out index 89ce53c70..bf35d7f4b 100644 --- a/src/test/regress/expected/aggregate_support.out +++ b/src/test/regress/expected/aggregate_support.out @@ -875,6 +875,7 @@ BEGIN RETURN $1 * $1; END; $function$; +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION square_func(int) RETURNS int LANGUAGE plpgsql @@ -883,6 +884,7 @@ BEGIN RETURN $1 * $1; END; $function$; +RESET citus.enable_metadata_sync; SELECT const_function(1), string_agg(a::character, ',') FROM t1; NOTICE: stable_fn called CONTEXT: PL/pgSQL function const_function(integer) line XX at RAISE diff --git a/src/test/regress/expected/citus_local_tables_queries_mx.out b/src/test/regress/expected/citus_local_tables_queries_mx.out index 684644476..52b73e7df 100644 --- a/src/test/regress/expected/citus_local_tables_queries_mx.out +++ b/src/test/regress/expected/citus_local_tables_queries_mx.out @@ -58,6 +58,7 @@ CREATE TABLE postgres_local_table(a int, b int); -- We shouldn't use LIMIT in INSERT SELECT queries to make the test faster as -- LIMIT would force planner to wrap SELECT query in an intermediate result and -- this might reduce the coverage of the test cases. +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$ BEGIN SET client_min_messages to ERROR; @@ -74,6 +75,7 @@ CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$ RESET client_min_messages; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; --------------------------------------------------------------------- ---- SELECT ---- --------------------------------------------------------------------- diff --git a/src/test/regress/expected/coordinator_evaluation.out b/src/test/regress/expected/coordinator_evaluation.out index bd309bdb2..f675c316f 100644 --- a/src/test/regress/expected/coordinator_evaluation.out +++ b/src/test/regress/expected/coordinator_evaluation.out @@ -523,6 +523,7 @@ BEGIN RETURN trunc(random() * (end_int-start_int) + start_int); END; $$ LANGUAGE 'plpgsql' STRICT; +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int) LANGUAGE plpgsql AS $procedure$ @@ -532,6 +533,7 @@ BEGIN PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey; END; $procedure$; +RESET citus.enable_metadata_sync; -- we couldn't find a meaningful query to write for this -- however this query fails before https://github.com/citusdata/citus/pull/3454 SET client_min_messages TO DEBUG2; @@ -564,6 +566,7 @@ BEGIN INSERT INTO coordinator_evaluation_table_2 VALUES (filterKey, filterKey); END; $procedure$; +DEBUG: switching to sequential query execution mode RESET citus.log_remote_commands ; RESET client_min_messages; -- these calls would INSERT key = 101, so test if insert succeeded diff --git a/src/test/regress/expected/coordinator_evaluation_modify.out b/src/test/regress/expected/coordinator_evaluation_modify.out index 906dbffef..e75ac60b1 100644 --- a/src/test/regress/expected/coordinator_evaluation_modify.out +++ b/src/test/regress/expected/coordinator_evaluation_modify.out @@ -833,11 +833,13 @@ EXECUTE router_with_only_function; SET citus.log_local_commands TO ON; SET search_path TO coordinator_evaluation_combinations_modify; -- returns 2 on the worker +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION get_constant_stable() RETURNS INT AS $$ BEGIN RETURN 2; END; $$ language plpgsql STABLE; +RESET citus.enable_metadata_sync; -- all local values INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test3'', 3)'), (4, '(''test4'', 4)'), (7, '(''test7'', 7)'), diff --git a/src/test/regress/expected/coordinator_shouldhaveshards.out b/src/test/regress/expected/coordinator_shouldhaveshards.out index 2cd9f2f20..5d5fa982c 100644 --- a/src/test/regress/expected/coordinator_shouldhaveshards.out +++ b/src/test/regress/expected/coordinator_shouldhaveshards.out @@ -898,9 +898,11 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT in -- a helper function which return true if the coordinated -- trannsaction uses 2PC +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION coordinated_transaction_should_use_2PC() RETURNS BOOL LANGUAGE C STRICT VOLATILE AS 'citus', $$coordinated_transaction_should_use_2PC$$; +RESET citus.enable_metadata_sync; -- a local SELECT followed by remote SELECTs -- does not trigger 2PC BEGIN; diff --git a/src/test/regress/expected/distributed_collations.out b/src/test/regress/expected/distributed_collations.out index f2413d0e5..bc6a5a859 100644 --- a/src/test/regress/expected/distributed_collations.out +++ b/src/test/regress/expected/distributed_collations.out @@ -163,3 +163,19 @@ SELECT run_command_on_workers($$DROP USER collationuser;$$); (localhost,57638,t,"DROP ROLE") (2 rows) +\c - - - :worker_1_port +-- test creating a collation on a worker +CREATE COLLATION another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk'); +ERROR: operation is not allowed on this node +HINT: Connect to the coordinator and run it again. +-- test if creating a collation on a worker on a local +-- schema raises the right error +SET citus.enable_ddl_propagation TO off; +CREATE SCHEMA collation_creation_on_worker; +SET citus.enable_ddl_propagation TO on; +CREATE COLLATION collation_creation_on_worker.another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk'); +ERROR: operation is not allowed on this node +HINT: Connect to the coordinator and run it again. +SET citus.enable_ddl_propagation TO off; +DROP SCHEMA collation_creation_on_worker; +SET citus.enable_ddl_propagation TO on; diff --git a/src/test/regress/expected/distributed_functions_conflict.out b/src/test/regress/expected/distributed_functions_conflict.out index b961fb955..354aea9c0 100644 --- a/src/test/regress/expected/distributed_functions_conflict.out +++ b/src/test/regress/expected/distributed_functions_conflict.out @@ -3,6 +3,7 @@ CREATE SCHEMA proc_conflict; \c - - - :worker_1_port SET search_path TO proc_conflict; +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$ BEGIN RETURN state * 2 + i; @@ -12,6 +13,7 @@ CREATE AGGREGATE existing_agg(int) ( SFUNC = existing_func, STYPE = int ); +RESET citus.enable_metadata_sync; \c - - - :master_port SET search_path TO proc_conflict; CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$ @@ -62,6 +64,7 @@ DROP AGGREGATE existing_agg(int) CASCADE; DROP FUNCTION existing_func(int, int) CASCADE; \c - - - :worker_1_port SET search_path TO proc_conflict; +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$ BEGIN RETURN state * 3 + i; @@ -71,6 +74,7 @@ CREATE AGGREGATE existing_agg(int) ( SFUNC = existing_func, STYPE = int ); +RESET citus.enable_metadata_sync; \c - - - :master_port SET search_path TO proc_conflict; CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$ diff --git a/src/test/regress/expected/fast_path_router_modify.out b/src/test/regress/expected/fast_path_router_modify.out index f99aac5d4..bf7e07362 100644 --- a/src/test/regress/expected/fast_path_router_modify.out +++ b/src/test/regress/expected/fast_path_router_modify.out @@ -228,7 +228,10 @@ BEGIN DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT modify_fast_path_plpsql(1,1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2" PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement @@ -241,6 +244,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem (1 row) SELECT modify_fast_path_plpsql(2,2); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2" PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement @@ -253,6 +257,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem (1 row) SELECT modify_fast_path_plpsql(3,3); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2" PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement @@ -265,6 +270,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem (1 row) SELECT modify_fast_path_plpsql(4,4); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2" PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement @@ -277,6 +283,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem (1 row) SELECT modify_fast_path_plpsql(5,5); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2" PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement @@ -289,6 +296,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem (1 row) SELECT modify_fast_path_plpsql(6,6); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2" PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statement @@ -301,6 +309,7 @@ PL/pgSQL function modify_fast_path_plpsql(integer,integer) line XX at SQL statem (1 row) SELECT modify_fast_path_plpsql(6,6); +DEBUG: function does not have co-located tables modify_fast_path_plpsql --------------------------------------------------------------------- diff --git a/src/test/regress/expected/forcedelegation_functions.out b/src/test/regress/expected/forcedelegation_functions.out index c4e07d5f9..c26f7b75b 100644 --- a/src/test/regress/expected/forcedelegation_functions.out +++ b/src/test/regress/expected/forcedelegation_functions.out @@ -293,6 +293,8 @@ BEGIN RETURN ret_val; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands CREATE OR REPLACE FUNCTION func_calls_forcepush_func() RETURNS NUMERIC AS $$ DECLARE incremented_val NUMERIC; @@ -302,6 +304,8 @@ BEGIN RETURN incremented_val; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('func_calls_forcepush_func()'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -354,12 +358,16 @@ PL/pgSQL function func_calls_forcepush_func() line XX at SQL statement 101 (1 row) +-- Block distributing that function as distributing it causes +-- different test output on PG 14. +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION get_val() RETURNS INT AS $$ BEGIN RETURN 100::INT; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- -- UDF calling another UDF in a FROM clause -- fn() @@ -377,7 +385,10 @@ BEGIN RETURN incremented_val; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT func_calls_forcepush_func_infrom(); +DEBUG: function does not have co-located tables DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT inner_force_delegation_function FROM inner_force_delegation_function(add_val + 100)" PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement @@ -395,6 +406,7 @@ PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement BEGIN; SELECT func_calls_forcepush_func_infrom(); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT inner_force_delegation_function FROM inner_force_delegation_function(add_val + 100)" PL/pgSQL function func_calls_forcepush_func_infrom() line XX at SQL statement @@ -428,7 +440,10 @@ BEGIN RETURN incremented_val; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT func_calls_forcepush_func_intarget(); +DEBUG: function does not have co-located tables DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT inner_force_delegation_function(100 + 100) OFFSET 0" PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement @@ -446,6 +461,7 @@ PL/pgSQL function func_calls_forcepush_func_intarget() line XX at SQL statement BEGIN; SELECT func_calls_forcepush_func_intarget(); +DEBUG: not pushing down function calls in a multi-statement transaction NOTICE: inner_force_delegation_function():201 DETAIL: from localhost:xxxxx CONTEXT: SQL statement "SELECT inner_force_delegation_function(100 + 100) OFFSET 0" @@ -473,6 +489,8 @@ BEGIN END if; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('test_recursive(int)', '$1', colocate_with := 'test_nested', force_delegation := true); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -544,6 +562,8 @@ BEGIN RETURN x + y; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'test_non_constant(int,bigint)', '$1', @@ -610,6 +630,8 @@ BEGIN INSERT INTO emp VALUES (empname, 33); END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands CREATE OR REPLACE FUNCTION outer_emp() RETURNS void AS $$ @@ -618,6 +640,8 @@ BEGIN PERFORM inner_emp('hello'); END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('inner_emp(text)','empname', force_delegation := true); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -627,6 +651,7 @@ DETAIL: A command for a distributed function is run. To make sure subsequent co (1 row) SELECT outer_emp(); +DEBUG: function does not have co-located tables DEBUG: Skipping pushdown of function from a PL/PgSQL simple expression CONTEXT: SQL statement "SELECT inner_emp('hello')" PL/pgSQL function outer_emp() line XX at PERFORM @@ -650,6 +675,8 @@ BEGIN INSERT INTO forcepushdown_schema.test_forcepushdown SELECT(a+1); END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_select_data(int)', 'a', colocate_with := 'test_forcepushdown', @@ -725,6 +752,8 @@ BEGIN SELECT intcol FROM forcepushdown_schema.test_forcepushdown_noncolocate; END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_select_data_nonlocal(int)', 'a', colocate_with := 'test_forcepushdown', @@ -803,6 +832,8 @@ BEGIN INSERT INTO forcepushdown_schema.test_forcepushdown_char VALUES (a); END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_data_char(char)', 'a', colocate_with := 'test_forcepushdown_char', @@ -821,6 +852,8 @@ BEGIN INSERT INTO forcepushdown_schema.test_forcepushdown_varchar VALUES (a); END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_data_varchar(varchar)', 'a', colocate_with := 'test_forcepushdown_varchar', @@ -839,6 +872,8 @@ BEGIN INSERT INTO forcepushdown_schema.test_forcepushdown_text VALUES (a); END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_data_text(text)', 'a', colocate_with := 'test_forcepushdown_text', @@ -947,6 +982,8 @@ BEGIN RAISE NOTICE 'Result: %', var; END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'select_data(int)', 'a', colocate_with := 'test_subquery', @@ -969,6 +1006,8 @@ BEGIN RAISE NOTICE 'Result: %', var; END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'select_data_noncolocate(int)', 'a', colocate_with := 'test_subquery', @@ -990,6 +1029,8 @@ BEGIN RAISE NOTICE 'Result: %', var; END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_select_data_cte1(int)', 'a', colocate_with := 'test_subquery', @@ -1011,6 +1052,8 @@ BEGIN RAISE NOTICE 'Result: %', var; END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_select_data_cte2(int)', 'a', colocate_with := 'test_subquery', @@ -1033,6 +1076,8 @@ BEGIN RAISE NOTICE 'Result: %', var; END; $fn$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function( 'insert_data_cte_nondist(int)', 'a', colocate_with := 'test_subquery', @@ -1203,6 +1248,8 @@ BEGIN RETURN x + y; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('test_prepare(int,int)','x',force_delegation :=true, colocate_with := 'table_test_prepare'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -1223,9 +1270,12 @@ BEGIN PERFORM 1, 1 + a FROM test_prepare(x + 1, y + 1) a; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands -- First 5 get delegated and succeeds BEGIN; SELECT outer_test_prepare(1,1); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1244,6 +1294,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM (1 row) SELECT outer_test_prepare(1,1); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1262,6 +1313,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM (1 row) SELECT outer_test_prepare(1,1); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1280,6 +1332,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM (1 row) SELECT outer_test_prepare(1,1); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1298,6 +1351,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM (1 row) SELECT outer_test_prepare(1,1); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1324,6 +1378,7 @@ SELECT COUNT(*) FROM table_test_prepare; -- 6th execution will be generic plan and should get delegated SELECT outer_test_prepare(1,1); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1342,6 +1397,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM (1 row) SELECT outer_test_prepare(1,1); +DEBUG: not pushing down function calls in a multi-statement transaction DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1362,6 +1418,7 @@ PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM END; -- Fails as expected SELECT outer_test_prepare(1,2); +DEBUG: function does not have co-located tables DEBUG: pushing down function call in a multi-statement transaction CONTEXT: SQL statement "SELECT FROM test_prepare(x, y)" PL/pgSQL function outer_test_prepare(integer,integer) line XX at PERFORM @@ -1397,6 +1454,8 @@ BEGIN RETURN x; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('test(int)', 'x', colocate_with := 'test_perform', force_delegation := true); DEBUG: switching to sequential query execution mode @@ -1418,10 +1477,471 @@ NOTICE: INPUT 3 CONTEXT: PL/pgSQL function test(integer) line XX at RAISE SQL statement "SELECT test(3)" PL/pgSQL function inline_code_block line XX at PERFORM +CREATE TABLE testnested_table (x int, y int); +SELECT create_distributed_table('testnested_table','x'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION inner_fn(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +-- Non-force function calling force-delegation function +CREATE OR REPLACE FUNCTION outer_local_fn() +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM 1 FROM inner_fn(1); + INSERT INTO forcepushdown_schema.testnested_table VALUES (2,3); + PERFORM 1 FROM inner_fn(4); + INSERT INTO forcepushdown_schema.testnested_table VALUES (5,6); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +SELECT create_distributed_function('inner_fn(int)','x', + colocate_with:='testnested_table', force_delegation := true); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT outer_local_fn(); +DEBUG: function does not have co-located tables +DEBUG: pushing down function call in a multi-statement transaction +CONTEXT: SQL statement "SELECT 1 FROM inner_fn(1)" +PL/pgSQL function outer_local_fn() line XX at PERFORM +DEBUG: pushing down the function call +CONTEXT: SQL statement "SELECT 1 FROM inner_fn(1)" +PL/pgSQL function outer_local_fn() line XX at PERFORM +DEBUG: pushing down function call in a multi-statement transaction +CONTEXT: SQL statement "SELECT 1 FROM inner_fn(4)" +PL/pgSQL function outer_local_fn() line XX at PERFORM +DEBUG: pushing down the function call +CONTEXT: SQL statement "SELECT 1 FROM inner_fn(4)" +PL/pgSQL function outer_local_fn() line XX at PERFORM + outer_local_fn +--------------------------------------------------------------------- + +(1 row) + +-- Rows from 1-6 should appear +SELECT * FROM testnested_table ORDER BY 1; + x | y +--------------------------------------------------------------------- + 1 | 1 + 2 | 3 + 4 | 4 + 5 | 6 +(4 rows) + +BEGIN; +SELECT outer_local_fn(); +DEBUG: not pushing down function calls in a multi-statement transaction + outer_local_fn +--------------------------------------------------------------------- + +(1 row) + +END; +SELECT * FROM testnested_table ORDER BY 1; + x | y +--------------------------------------------------------------------- + 1 | 1 + 1 | 1 + 2 | 3 + 2 | 3 + 4 | 4 + 4 | 4 + 5 | 6 + 5 | 6 +(8 rows) + +DROP FUNCTION inner_fn(int); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +DROP FUNCTION outer_local_fn(); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +TRUNCATE TABLE testnested_table; +CREATE OR REPLACE FUNCTION inner_fn(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +-- Force-delegation function calling non-force function +CREATE OR REPLACE FUNCTION outer_fn(y int, z int) +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM 1 FROM forcepushdown_schema.inner_fn(y); + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); + PERFORM 1 FROM forcepushdown_schema.inner_fn(z); + INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +SELECT create_distributed_function('inner_fn(int)','x', + colocate_with:='testnested_table', force_delegation := false); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function('outer_fn(int, int)','y', + colocate_with:='testnested_table', force_delegation := true); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT outer_fn(1, 2); +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x)" +PL/pgSQL function forcepushdown_schema.inner_fn(integer) line XX at SQL statement +SQL statement "SELECT 1 FROM forcepushdown_schema.inner_fn(z)" +PL/pgSQL function forcepushdown_schema.outer_fn(integer,integer) line XX at PERFORM +while executing command on localhost:xxxxx +BEGIN; +SELECT outer_fn(1, 2); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x)" +PL/pgSQL function forcepushdown_schema.inner_fn(integer) line XX at SQL statement +SQL statement "SELECT 1 FROM forcepushdown_schema.inner_fn(z)" +PL/pgSQL function forcepushdown_schema.outer_fn(integer,integer) line XX at PERFORM +while executing command on localhost:xxxxx +END; +-- No rows +SELECT * FROM testnested_table ORDER BY 1; + x | y +--------------------------------------------------------------------- +(0 rows) + +-- Force-delegation function calling force-delegation function +CREATE OR REPLACE FUNCTION force_push_inner(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +CREATE OR REPLACE FUNCTION force_push_outer(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); + PERFORM forcepushdown_schema.force_push_inner(x+1) LIMIT 1; +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +SELECT create_distributed_function( + 'force_push_outer(int)', 'x', + colocate_with := 'testnested_table', + force_delegation := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function( + 'force_push_inner(int)', 'y', + colocate_with := 'testnested_table', + force_delegation := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- Keys 7,8,9,14 fall on one node and 15 on a different node +-- Function gets delegated to node with shard-key = 7 and inner function +-- will not be delegated but inserts shard-key = 8 locally +SELECT force_push_outer(7); +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)" +PL/pgSQL function forcepushdown_schema.force_push_inner(integer) line XX at SQL statement +SQL statement "SELECT forcepushdown_schema.force_push_inner(x+1) LIMIT 1" +PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at PERFORM +while executing command on localhost:xxxxx +BEGIN; +-- Function gets delegated to node with shard-key = 8 and inner function +-- will not be delegated but inserts shard-key = 9 locally +SELECT force_push_outer(8); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)" +PL/pgSQL function forcepushdown_schema.force_push_inner(integer) line XX at SQL statement +SQL statement "SELECT forcepushdown_schema.force_push_inner(x+1) LIMIT 1" +PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at PERFORM +while executing command on localhost:xxxxx +END; +BEGIN; +-- Function gets delegated to node with shard-key = 14 and inner function +-- will not be delegated but fails to insert shard-key = 15 remotely +SELECT force_push_outer(14); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)" +PL/pgSQL function forcepushdown_schema.force_push_inner(integer) line XX at SQL statement +SQL statement "SELECT forcepushdown_schema.force_push_inner(x+1) LIMIT 1" +PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at PERFORM +while executing command on localhost:xxxxx +END; +SELECT * FROM testnested_table ORDER BY 1; + x | y +--------------------------------------------------------------------- +(0 rows) + +-- +-- Function-1() --> function-2() --> function-3() +-- +CREATE OR REPLACE FUNCTION force_push_1(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); + PERFORM forcepushdown_schema.force_push_2(x+1) LIMIT 1; +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +CREATE OR REPLACE FUNCTION force_push_2(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); + PERFORM forcepushdown_schema.force_push_3(y+1) LIMIT 1; +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +CREATE OR REPLACE FUNCTION force_push_3(z int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +SELECT create_distributed_function( + 'force_push_1(int)', 'x', + colocate_with := 'testnested_table', + force_delegation := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function( + 'force_push_2(int)', 'y', + colocate_with := 'testnested_table', + force_delegation := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function( + 'force_push_3(int)', 'z', + colocate_with := 'testnested_table', + force_delegation := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE TABLE testnested_table; +BEGIN; +-- All local inserts +SELECT force_push_1(7); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)" +PL/pgSQL function forcepushdown_schema.force_push_2(integer) line XX at SQL statement +SQL statement "SELECT forcepushdown_schema.force_push_2(x+1) LIMIT 1" +PL/pgSQL function forcepushdown_schema.force_push_1(integer) line XX at PERFORM +while executing command on localhost:xxxxx +END; +BEGIN; +-- Local(shard-keys 13, 15) + remote insert (shard-key 14) +SELECT force_push_1(13); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y)" +PL/pgSQL function forcepushdown_schema.force_push_2(integer) line XX at SQL statement +SQL statement "SELECT forcepushdown_schema.force_push_2(x+1) LIMIT 1" +PL/pgSQL function forcepushdown_schema.force_push_1(integer) line XX at PERFORM +while executing command on localhost:xxxxx +END; +SELECT * FROM testnested_table ORDER BY 1; + x | y +--------------------------------------------------------------------- +(0 rows) + +TRUNCATE TABLE testnested_table; +CREATE OR REPLACE FUNCTION force_push_inner(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +CREATE OR REPLACE FUNCTION force_push_outer(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM FROM forcepushdown_schema.force_push_inner(x); + INSERT INTO forcepushdown_schema.testnested_table VALUES (x+1,x+1); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +SELECT create_distributed_function( + 'force_push_inner(int)', 'y', + colocate_with := 'testnested_table', + force_delegation := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_function( + 'force_push_outer(int)', 'x', + colocate_with := 'testnested_table', + force_delegation := true +); +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +BEGIN; +SELECT force_push_outer(7); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +ERROR: queries must filter by the distribution argument in the same colocation group when using the forced function pushdown +HINT: consider disabling forced delegation through create_distributed_table(..., force_delegation := false) +CONTEXT: SQL statement "INSERT INTO forcepushdown_schema.testnested_table VALUES (x+1,x+1)" +PL/pgSQL function forcepushdown_schema.force_push_outer(integer) line XX at SQL statement +while executing command on localhost:xxxxx +END; +TABLE testnested_table ORDER BY 1; + x | y +--------------------------------------------------------------------- +(0 rows) + +CREATE OR REPLACE FUNCTION force_push_inner(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + RAISE NOTICE '%', y; +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +CREATE OR REPLACE FUNCTION force_push_outer(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM FROM forcepushdown_schema.force_push_inner(x+1); + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); +END; +$$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +BEGIN; +SELECT force_push_outer(9); +DEBUG: pushing down function call in a multi-statement transaction +DEBUG: pushing down the function call +NOTICE: 10 +DETAIL: from localhost:xxxxx + force_push_outer +--------------------------------------------------------------------- + +(1 row) + +END; +TABLE testnested_table ORDER BY 1; + x | y +--------------------------------------------------------------------- + 9 | 9 +(1 row) + RESET client_min_messages; SET citus.log_remote_commands TO off; DROP SCHEMA forcepushdown_schema CASCADE; -NOTICE: drop cascades to 38 other objects +NOTICE: drop cascades to 46 other objects DETAIL: drop cascades to table test_forcepushdown drop cascades to table test_forcepushdown_noncolocate drop cascades to function insert_data(integer) @@ -1460,3 +1980,11 @@ drop cascades to function test_prepare(integer,integer) drop cascades to function outer_test_prepare(integer,integer) drop cascades to table test_perform drop cascades to function test(integer) +drop cascades to table testnested_table +drop cascades to function inner_fn(integer) +drop cascades to function outer_fn(integer,integer) +drop cascades to function force_push_inner(integer) +drop cascades to function force_push_outer(integer) +drop cascades to function force_push_1(integer) +drop cascades to function force_push_2(integer) +drop cascades to function force_push_3(integer) diff --git a/src/test/regress/expected/function_propagation.out b/src/test/regress/expected/function_propagation.out new file mode 100644 index 000000000..a40f1f9a2 --- /dev/null +++ b/src/test/regress/expected/function_propagation.out @@ -0,0 +1,321 @@ +CREATE SCHEMA function_propagation_schema; +SET search_path TO 'function_propagation_schema'; +-- Check whether supported dependencies can be distributed while propagating functions +-- Check types +SET citus.enable_metadata_sync TO OFF; + CREATE TYPE function_prop_type AS (a int, b int); +RESET citus.enable_metadata_sync; +CREATE OR REPLACE FUNCTION func_1(param_1 function_prop_type) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Check all dependent objects and function depends on all nodes +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (schema,{function_propagation_schema},{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (type,{function_propagation_schema.function_prop_type},{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type}) +(1 row) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (schema,{function_propagation_schema},{}) + localhost | 57638 | t | (schema,{function_propagation_schema},{}) +(2 rows) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type},{}) + localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type},{}) +(2 rows) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type}) +(2 rows) + +SET citus.enable_metadata_sync TO OFF; + CREATE TYPE function_prop_type_2 AS (a int, b int); +RESET citus.enable_metadata_sync; +CREATE OR REPLACE FUNCTION func_2(param_1 int) +RETURNS function_prop_type_2 +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (type,{function_propagation_schema.function_prop_type_2},{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_2}",{integer}) +(1 row) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type_2},{}) + localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type_2},{}) +(2 rows) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_2}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_2}",{integer}) +(2 rows) + +-- Have a separate check for type created in transaction +BEGIN; + CREATE TYPE function_prop_type_3 AS (a int, b int); +COMMIT; +-- Objects in the body part is not found as dependency +CREATE OR REPLACE FUNCTION func_3(param_1 int) +RETURNS int +LANGUAGE plpgsql AS +$$ +DECLARE + internal_param1 function_prop_type_3; +BEGIN + return 1; +END; +$$; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_3}",{integer}) +(1 row) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_3}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_3}",{integer}) +(2 rows) + +-- Check table +CREATE TABLE function_prop_table(a int, b int); +-- Non-distributed table is not distributed as dependency +CREATE OR REPLACE FUNCTION func_4(param_1 function_prop_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +WARNING: Citus can't distribute function "func_4" having dependency on non-distributed relation "function_prop_table" +DETAIL: Function will be created only locally +HINT: To distribute function, distribute dependent relations first. Then, re-create the function +CREATE OR REPLACE FUNCTION func_5(param_1 int) +RETURNS function_prop_table +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +WARNING: Citus can't distribute function "func_5" having dependency on non-distributed relation "function_prop_table" +DETAIL: Function will be created only locally +HINT: To distribute function, distribute dependent relations first. Then, re-create the function +-- Functions can be created with distributed table dependency +SELECT create_distributed_table('function_prop_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION func_6(param_1 function_prop_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table}) +(1 row) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table}) +(2 rows) + +-- Views are not supported +CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table; +CREATE OR REPLACE FUNCTION func_7(param_1 function_prop_view) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +WARNING: Citus can't distribute functions having dependency on unsupported object of type "view" +DETAIL: Function will be created only locally +CREATE OR REPLACE FUNCTION func_8(param_1 int) +RETURNS function_prop_view +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +WARNING: Citus can't distribute functions having dependency on unsupported object of type "view" +DETAIL: Function will be created only locally +-- Check within transaction +BEGIN; + CREATE TYPE type_in_transaction AS (a int, b int); + CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction) + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + -- Within transaction functions are not distributed + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +COMMIT; +-- Show that recreating it outside transaction distributes the function and dependencies +CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (type,{function_propagation_schema.type_in_transaction},{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction}) +(1 row) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (type,{function_propagation_schema.type_in_transaction},{}) + localhost | 57638 | t | (type,{function_propagation_schema.type_in_transaction},{}) +(2 rows) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction}) +(2 rows) + +-- Test for SQL function with unsupported object in function body +CREATE TABLE table_in_sql_body(id int); +CREATE FUNCTION max_of_table() +RETURNS int +LANGUAGE SQL AS +$$ + SELECT max(id) FROM table_in_sql_body +$$; +-- Show that only function has propagated, since the table is not resolved as dependency +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,max_of_table}",{}) +(1 row) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,max_of_table}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,max_of_table}",{}) +(2 rows) + +-- Check extension owned table +CREATE TABLE extension_owned_table(a int); +SELECT run_command_on_workers($$ +CREATE TABLE function_propagation_schema.extension_owned_table(a int); +$$ +); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"CREATE TABLE") + (localhost,57638,t,"CREATE TABLE") +(2 rows) + +CREATE EXTENSION seg; +ALTER EXTENSION seg ADD TABLE extension_owned_table; +NOTICE: Citus does not propagate adding/dropping member objects +HINT: You can add/drop the member objects on the workers as well. +SELECT run_command_on_workers($$ +ALTER EXTENSION seg ADD TABLE function_propagation_schema.extension_owned_table; +$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"ALTER EXTENSION") + (localhost,57638,t,"ALTER EXTENSION") +(2 rows) + +CREATE OR REPLACE FUNCTION func_for_ext_check(param_1 extension_owned_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +RESET search_path; +SET client_min_messages TO WARNING; +DROP SCHEMA function_propagation_schema CASCADE; diff --git a/src/test/regress/expected/local_shard_copy.out b/src/test/regress/expected/local_shard_copy.out index ec9828a04..33ef9f7df 100644 --- a/src/test/regress/expected/local_shard_copy.out +++ b/src/test/regress/expected/local_shard_copy.out @@ -62,6 +62,7 @@ SET citus.log_local_commands TO ON; -- returns true of the distribution key filter -- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard -- placement which is local to this not +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$ DECLARE shard_is_local BOOLEAN := FALSE; @@ -84,6 +85,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R RETURN shard_is_local; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- pick some example values that reside on the shards locally and remote -- distribution key values of 1,6, 500 and 701 are LOCAL to shards, -- we'll use these values in the tests diff --git a/src/test/regress/expected/local_shard_execution.out b/src/test/regress/expected/local_shard_execution.out index 51daa4046..781a9c86c 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -94,6 +94,7 @@ SET search_path TO local_shard_execution; -- returns true of the distribution key filter -- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard -- placement which is local to this not +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$ DECLARE shard_is_local BOOLEAN := FALSE; @@ -116,6 +117,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R RETURN shard_is_local; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- test case for issue #3556 SET citus.log_intermediate_results TO TRUE; SET client_min_messages TO DEBUG1; @@ -801,6 +803,7 @@ BEGIN; ERROR: VACUUM cannot run inside a transaction block ROLLBACK; -- make sure that functions can use local execution +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE PROCEDURE only_local_execution() AS $$ DECLARE cnt INT; BEGIN @@ -896,6 +899,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$ SELECT count(*) INTO cnt FROM distributed_table; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CALL local_execution_followed_by_dist(); NOTICE: executing the command locally: INSERT INTO local_shard_execution.distributed_table_1470001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text CONTEXT: SQL statement "INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'" diff --git a/src/test/regress/expected/local_shard_execution_replicated.out b/src/test/regress/expected/local_shard_execution_replicated.out index 9d880f44b..731c825c3 100644 --- a/src/test/regress/expected/local_shard_execution_replicated.out +++ b/src/test/regress/expected/local_shard_execution_replicated.out @@ -770,6 +770,7 @@ NOTICE: executing the command locally: INSERT INTO local_shard_execution_replic ERROR: VACUUM cannot run inside a transaction block ROLLBACK; -- make sure that functions can use local execution +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE PROCEDURE only_local_execution() AS $$ DECLARE cnt INT; BEGIN @@ -865,6 +866,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$ SELECT count(*) INTO cnt FROM distributed_table; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CALL local_execution_followed_by_dist(); NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text CONTEXT: SQL statement "INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'" diff --git a/src/test/regress/expected/metadata_sync_helpers.out b/src/test/regress/expected/metadata_sync_helpers.out index 24aa7302f..9651fcc63 100644 --- a/src/test/regress/expected/metadata_sync_helpers.out +++ b/src/test/regress/expected/metadata_sync_helpers.out @@ -987,6 +987,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; ERROR: Node with group id 123123123 for shard placement xxxxx does not exist ROLLBACK; -- create a volatile function that returns the local node id +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION get_node_id() RETURNS INT AS $$ DECLARE localGroupId int; @@ -999,6 +1000,7 @@ BEGIN nodeport = 57637 AND nodename = 'localhost' AND isactive AND nodecluster = 'default'; RETURN localGroupId; END; $$ language plpgsql; +RESET citus.enable_metadata_sync; -- fails because we ingest more placements for the same shards to the same worker node BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); diff --git a/src/test/regress/expected/multi_create_table.out b/src/test/regress/expected/multi_create_table.out index a09f1fbcc..4ec8a27da 100644 --- a/src/test/regress/expected/multi_create_table.out +++ b/src/test/regress/expected/multi_create_table.out @@ -7,6 +7,7 @@ -- reference and hash-distributed version of orders, customer and part tables. SET citus.next_shard_id TO 360000; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -23,6 +24,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; CREATE TABLE lineitem ( l_orderkey bigint not null, l_partkey integer not null, diff --git a/src/test/regress/expected/multi_deparse_function.out b/src/test/regress/expected/multi_deparse_function.out index b85078258..656c89c40 100644 --- a/src/test/regress/expected/multi_deparse_function.out +++ b/src/test/regress/expected/multi_deparse_function.out @@ -689,17 +689,7 @@ SELECT create_distributed_function('func_custom_param(intpair)'); (1 row) RESET citus.enable_metadata_sync; -SELECT deparse_and_run_on_workers($cmd$ ALTER FUNCTION func_custom_param RENAME TO func_with_custom_param; -$cmd$); -INFO: Propagating deparsed query: ALTER FUNCTION function_tests.func_custom_param(function_tests.intpair) RENAME TO func_with_custom_param; -CONTEXT: PL/pgSQL function deparse_and_run_on_workers(text) line XX at RAISE - deparse_and_run_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"ALTER FUNCTION") - (localhost,57638,t,"ALTER FUNCTION") -(2 rows) - -- a function that returns TABLE CREATE FUNCTION func_returns_table(IN count INT) RETURNS TABLE (x INT, y INT) @@ -713,17 +703,7 @@ SELECT create_distributed_function('func_returns_table(INT)'); (1 row) RESET citus.enable_metadata_sync; -SELECT deparse_and_run_on_workers($cmd$ ALTER FUNCTION func_returns_table ROWS 100; -$cmd$); -INFO: Propagating deparsed query: ALTER FUNCTION function_tests.func_returns_table(integer) ROWS 100.000000; -CONTEXT: PL/pgSQL function deparse_and_run_on_workers(text) line XX at RAISE - deparse_and_run_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"ALTER FUNCTION") - (localhost,57638,t,"ALTER FUNCTION") -(2 rows) - -- clear objects SET client_min_messages TO WARNING; -- suppress cascading objects dropping DROP SCHEMA "CiTuS.TeeN" CASCADE; diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 3a25f71b5..12d6998d5 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -77,6 +77,8 @@ END $func$ LANGUAGE plpgsql; CREATE SCHEMA test; :create_function_test_maintenance_worker +WARNING: Citus can't distribute functions having dependency on unsupported object of type "view" +DETAIL: Function will be created only locally -- check maintenance daemon is started SELECT datname, current_database(), usename, (SELECT extowner::regrole::text FROM pg_extension WHERE extname = 'citus') @@ -1198,6 +1200,8 @@ HINT: You can manually create a database and its extensions on workers. CREATE EXTENSION citus; CREATE SCHEMA test; :create_function_test_maintenance_worker +WARNING: Citus can't distribute functions having dependency on unsupported object of type "view" +DETAIL: Function will be created only locally -- see that the daemon started SELECT datname, current_database(), usename, (SELECT extowner::regrole::text FROM pg_extension WHERE extname = 'citus') diff --git a/src/test/regress/expected/multi_function_in_join.out b/src/test/regress/expected/multi_function_in_join.out index c80b26c57..7d62e286b 100644 --- a/src/test/regress/expected/multi_function_in_join.out +++ b/src/test/regress/expected/multi_function_in_join.out @@ -39,14 +39,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta CREATE FUNCTION add(integer, integer) RETURNS integer AS 'SELECT $1 + $2;' LANGUAGE SQL; -SELECT create_distributed_function('add(integer,integer)'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands - create_distributed_function ---------------------------------------------------------------------- - -(1 row) - SELECT * FROM table1 JOIN add(3,5) sum ON (id = sum) ORDER BY id ASC; id | data | sum --------------------------------------------------------------------- @@ -60,7 +54,10 @@ BEGIN RETURN i + 1; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN increment(2) val ON (id = val) ORDER BY id ASC; +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT val FROM functions_in_joins.increment(2) val(val) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, val.val FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.val FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(val integer)) val ON ((table1.id OPERATOR(pg_catalog.=) val.val))) ORDER BY table1.id id | data | val @@ -69,6 +66,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta (1 row) -- a function that returns a set of integers +-- Block distributing function as we have tests below to test it locally +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION next_k_integers(IN first_value INTEGER, IN k INTEGER DEFAULT 3, OUT result INTEGER) @@ -77,6 +76,7 @@ BEGIN RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x); END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SELECT * FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result) ORDER BY id ASC; @@ -93,7 +93,10 @@ CREATE FUNCTION get_set_of_records() RETURNS SETOF RECORD AS $cmd$ SELECT x, x+1 FROM generate_series(0,4) f(x) $cmd$ LANGUAGE SQL; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN get_set_of_records() AS t2(x int, y int) ON (id = x) ORDER BY id ASC; +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM functions_in_joins.get_set_of_records() t2(x integer, y integer) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, t2.x, t2.y FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) t2 ON ((table1.id OPERATOR(pg_catalog.=) t2.x))) ORDER BY table1.id id | data | x | y @@ -108,7 +111,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta CREATE FUNCTION dup(int) RETURNS TABLE(f1 int, f2 text) AS $$ SELECT $1, CAST($1 AS text) || ' is text' $$ LANGUAGE SQL; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT f.* FROM table1 t JOIN dup(32) f ON (f1 = id); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT f1, f2 FROM functions_in_joins.dup(32) f(f1, f2) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FROM (functions_in_joins.table1 t JOIN (SELECT intermediate_result.f1, intermediate_result.f2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(f1 integer, f2 text)) f ON ((f.f1 OPERATOR(pg_catalog.=) t.id))) f1 | f2 @@ -119,7 +125,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FR -- a stable function CREATE OR REPLACE FUNCTION the_minimum_id() RETURNS INTEGER STABLE AS 'SELECT min(id) FROM table1' LANGUAGE SQL; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN the_minimum_id() min_id ON (id = min_id); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT min_id FROM functions_in_joins.the_minimum_id() min_id(min_id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, min_id.min_id FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.min_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(min_id integer)) min_id ON ((table1.id OPERATOR(pg_catalog.=) min_id.min_id))) id | data | min_id @@ -180,7 +189,10 @@ begin return result; end; $$ language plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN max_and_min() m ON (m.maximum = data OR m.minimum = data) ORDER BY 1,2,3,4; +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT minimum, maximum FROM functions_in_joins.max_and_min() m(minimum, maximum) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, m.minimum, m.maximum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.minimum, intermediate_result.maximum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(minimum integer, maximum integer)) m ON (((m.maximum OPERATOR(pg_catalog.=) table1.data) OR (m.minimum OPERATOR(pg_catalog.=) table1.data)))) ORDER BY table1.id, table1.data, m.minimum, m.maximum id | data | minimum | maximum @@ -198,6 +210,7 @@ SET client_min_messages TO ERROR; -- function joins in CTE results can create lateral joins that are not supported -- we execute the query within a function to consolidate the error messages -- between different executors +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION raise_failed_execution_func_join(query text) RETURNS void AS $$ BEGIN EXECUTE query; @@ -209,6 +222,7 @@ BEGIN END IF; END; $$LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SELECT raise_failed_execution_func_join($$ WITH one_row AS ( SELECT * FROM table1 WHERE id=52 @@ -219,8 +233,10 @@ SELECT raise_failed_execution_func_join($$ $$); ERROR: Task failed to execute -- a user-defined immutable function +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION the_answer_to_life() RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL; +RESET citus.enable_metadata_sync; SELECT raise_failed_execution_func_join($$ SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer); $$); diff --git a/src/test/regress/expected/multi_function_in_join_0.out b/src/test/regress/expected/multi_function_in_join_0.out index 5b818855b..5f2bd70c7 100644 --- a/src/test/regress/expected/multi_function_in_join_0.out +++ b/src/test/regress/expected/multi_function_in_join_0.out @@ -39,14 +39,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta CREATE FUNCTION add(integer, integer) RETURNS integer AS 'SELECT $1 + $2;' LANGUAGE SQL; -SELECT create_distributed_function('add(integer,integer)'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands - create_distributed_function ---------------------------------------------------------------------- - -(1 row) - SELECT * FROM table1 JOIN add(3,5) sum ON (id = sum) ORDER BY id ASC; DEBUG: generating subplan XXX_1 for subquery SELECT sum FROM functions_in_joins.add(3, 5) sum(sum) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, sum.sum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(sum integer)) sum ON ((table1.id OPERATOR(pg_catalog.=) sum.sum))) ORDER BY table1.id @@ -62,7 +56,10 @@ BEGIN RETURN i + 1; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN increment(2) val ON (id = val) ORDER BY id ASC; +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT val FROM functions_in_joins.increment(2) val(val) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, val.val FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.val FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(val integer)) val ON ((table1.id OPERATOR(pg_catalog.=) val.val))) ORDER BY table1.id id | data | val @@ -71,6 +68,8 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta (1 row) -- a function that returns a set of integers +-- Block distributing function as we have tests below to test it locally +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION next_k_integers(IN first_value INTEGER, IN k INTEGER DEFAULT 3, OUT result INTEGER) @@ -79,6 +78,7 @@ BEGIN RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x); END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SELECT * FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result) ORDER BY id ASC; @@ -95,7 +95,10 @@ CREATE FUNCTION get_set_of_records() RETURNS SETOF RECORD AS $cmd$ SELECT x, x+1 FROM generate_series(0,4) f(x) $cmd$ LANGUAGE SQL; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN get_set_of_records() AS t2(x int, y int) ON (id = x) ORDER BY id ASC; +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT x, y FROM functions_in_joins.get_set_of_records() t2(x integer, y integer) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, t2.x, t2.y FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) t2 ON ((table1.id OPERATOR(pg_catalog.=) t2.x))) ORDER BY table1.id id | data | x | y @@ -110,7 +113,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta CREATE FUNCTION dup(int) RETURNS TABLE(f1 int, f2 text) AS $$ SELECT $1, CAST($1 AS text) || ' is text' $$ LANGUAGE SQL; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT f.* FROM table1 t JOIN dup(32) f ON (f1 = id); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT f1, f2 FROM functions_in_joins.dup(32) f(f1, f2) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FROM (functions_in_joins.table1 t JOIN (SELECT intermediate_result.f1, intermediate_result.f2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(f1 integer, f2 text)) f ON ((f.f1 OPERATOR(pg_catalog.=) t.id))) f1 | f2 @@ -121,7 +127,10 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FR -- a stable function CREATE OR REPLACE FUNCTION the_minimum_id() RETURNS INTEGER STABLE AS 'SELECT min(id) FROM table1' LANGUAGE SQL; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN the_minimum_id() min_id ON (id = min_id); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT min_id FROM functions_in_joins.the_minimum_id() min_id(min_id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, min_id.min_id FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.min_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(min_id integer)) min_id ON ((table1.id OPERATOR(pg_catalog.=) min_id.min_id))) id | data | min_id @@ -182,7 +191,10 @@ begin return result; end; $$ language plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM table1 JOIN max_and_min() m ON (m.maximum = data OR m.minimum = data) ORDER BY 1,2,3,4; +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT minimum, maximum FROM functions_in_joins.max_and_min() m(minimum, maximum) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, m.minimum, m.maximum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.minimum, intermediate_result.maximum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(minimum integer, maximum integer)) m ON (((m.maximum OPERATOR(pg_catalog.=) table1.data) OR (m.minimum OPERATOR(pg_catalog.=) table1.data)))) ORDER BY table1.id, table1.data, m.minimum, m.maximum id | data | minimum | maximum @@ -200,6 +212,7 @@ SET client_min_messages TO ERROR; -- function joins in CTE results can create lateral joins that are not supported -- we execute the query within a function to consolidate the error messages -- between different executors +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION raise_failed_execution_func_join(query text) RETURNS void AS $$ BEGIN EXECUTE query; @@ -211,6 +224,7 @@ BEGIN END IF; END; $$LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SELECT raise_failed_execution_func_join($$ WITH one_row AS ( SELECT * FROM table1 WHERE id=52 @@ -221,8 +235,10 @@ SELECT raise_failed_execution_func_join($$ $$); ERROR: Task failed to execute -- a user-defined immutable function +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION the_answer_to_life() RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL; +RESET citus.enable_metadata_sync; SELECT raise_failed_execution_func_join($$ SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer); $$); diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index 99c5e8e56..c7679d02e 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -127,6 +127,7 @@ WHERE user_id < 0; NOTICE: evaluating on master -- make sure we don't evaluate stable functions with column arguments +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION evaluate_on_master(x int) RETURNS int LANGUAGE plpgsql STABLE AS $function$ @@ -135,6 +136,7 @@ BEGIN RETURN x; END; $function$; +RESET citus.enable_metadata_sync; INSERT INTO raw_events_second (user_id, value_1) SELECT user_id, evaluate_on_master(value_1) diff --git a/src/test/regress/expected/multi_metadata_sync.out b/src/test/regress/expected/multi_metadata_sync.out index 5680b73c3..be923eee3 100644 --- a/src/test/regress/expected/multi_metadata_sync.out +++ b/src/test/regress/expected/multi_metadata_sync.out @@ -26,10 +26,12 @@ ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000; SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset -- Create the necessary test utility function +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION activate_node_snapshot() RETURNS text[] LANGUAGE C STRICT AS 'citus'; +RESET citus.enable_metadata_sync; COMMENT ON FUNCTION activate_node_snapshot() IS 'commands to activate node snapshot'; -- Show that none of the existing tables are qualified to be MX tables @@ -89,12 +91,14 @@ SELECT unnest(activate_node_snapshot()) order by 1; (27 rows) -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) RETURNS void LANGUAGE C STRICT AS 'citus', $$master_create_distributed_table$$; +RESET citus.enable_metadata_sync; COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) diff --git a/src/test/regress/expected/multi_modifications.out b/src/test/regress/expected/multi_modifications.out index c6323c793..5b5764593 100644 --- a/src/test/regress/expected/multi_modifications.out +++ b/src/test/regress/expected/multi_modifications.out @@ -467,13 +467,6 @@ UPDATE limit_orders SET placed_at = LEAST(placed_at, now()::timestamp) WHERE id UPDATE limit_orders SET array_of_values = 1 || array_of_values WHERE id = 246; CREATE FUNCTION immutable_append(old_values int[], new_value int) RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; -\c - - - :worker_1_port -CREATE FUNCTION immutable_append(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; -\c - - - :worker_2_port -CREATE FUNCTION immutable_append(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; -\c - - - :master_port -- immutable function calls with vars are also allowed UPDATE limit_orders SET array_of_values = immutable_append(array_of_values, 2) WHERE id = 246; diff --git a/src/test/regress/expected/multi_modifying_xacts.out b/src/test/regress/expected/multi_modifying_xacts.out index 226529040..47b113378 100644 --- a/src/test/regress/expected/multi_modifying_xacts.out +++ b/src/test/regress/expected/multi_modifying_xacts.out @@ -329,22 +329,15 @@ SELECT count(*) FROM pg_dist_transaction; (1 row) -- create a check function -SELECT * from run_command_on_workers('CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$ +CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$ BEGIN IF (NEW.id > 30) THEN - RAISE ''illegal value''; + RAISE 'illegal value'; END IF; RETURN NEW; END; -$rli$ LANGUAGE plpgsql;') -ORDER BY nodeport; - nodename | nodeport | success | result ---------------------------------------------------------------------- - localhost | 57637 | t | CREATE FUNCTION - localhost | 57638 | t | CREATE FUNCTION -(2 rows) - +$rli$ LANGUAGE plpgsql; -- register after insert trigger SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()') ORDER BY nodeport, shardid; @@ -418,14 +411,7 @@ ORDER BY nodeport, shardid; localhost | 57638 | 1200001 | t | DROP TRIGGER (4 rows) -SELECT * FROM run_command_on_workers('drop function reject_large_id()') -ORDER BY nodeport; - nodename | nodeport | success | result ---------------------------------------------------------------------- - localhost | 57637 | t | DROP FUNCTION - localhost | 57638 | t | DROP FUNCTION -(2 rows) - +DROP FUNCTION reject_large_id(); -- ALTER and copy are compatible BEGIN; ALTER TABLE labs ADD COLUMN motto text; @@ -522,6 +508,7 @@ AND s.logicalrelid = 'objects'::regclass; -- create trigger on one worker to reject certain values \c - - - :worker_2_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ BEGIN IF (NEW.name = 'BAD') THEN @@ -531,6 +518,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad AFTER INSERT ON objects_1200003 DEFERRABLE INITIALLY IMMEDIATE @@ -573,6 +561,7 @@ DELETE FROM objects; -- there cannot be errors on different shards at different times -- because the first failure will fail the whole transaction \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ BEGIN IF (NEW.name = 'BAD') THEN @@ -582,6 +571,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad AFTER INSERT ON labs_1200002 DEFERRABLE INITIALLY IMMEDIATE @@ -850,6 +840,7 @@ SELECT * FROM reference_modifying_xacts; -- lets fail on of the workers at before the commit time \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$ BEGIN IF (NEW.key = 999) THEN @@ -859,6 +850,7 @@ CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad_reference AFTER INSERT ON reference_modifying_xacts_1200006 DEFERRABLE INITIALLY IMMEDIATE @@ -938,6 +930,7 @@ INSERT INTO hash_modifying_xacts VALUES (2, 2); ABORT; -- lets fail one of the workers before COMMIT time for the hash table \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$ BEGIN IF (NEW.key = 997) THEN @@ -947,6 +940,7 @@ CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad_hash AFTER INSERT ON hash_modifying_xacts_1200007 DEFERRABLE INITIALLY IMMEDIATE diff --git a/src/test/regress/expected/multi_multiuser.out b/src/test/regress/expected/multi_multiuser.out index 2c7be4c90..0e25a38d2 100644 --- a/src/test/regress/expected/multi_multiuser.out +++ b/src/test/regress/expected/multi_multiuser.out @@ -622,10 +622,12 @@ ERROR: permission denied for function worker_cleanup_job_schema_cache RESET ROLE; -- to test access to files created during repartition we will create some on worker 1 \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) RETURNS void AS 'citus' LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; SET ROLE full_access; SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]); worker_hash_partition_table @@ -636,10 +638,12 @@ SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS RESET ROLE; -- all attempts for transfer are initiated from other workers \c - - - :worker_2_port +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) RETURNS void AS 'citus' LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; -- super user should not be able to copy files created by a user SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port); WARNING: could not open file "base/pgsql_job_cache/job_0042/task_000001/p_00001.xxxx": No such file or directory diff --git a/src/test/regress/expected/multi_mx_add_coordinator.out b/src/test/regress/expected/multi_mx_add_coordinator.out index 045563d6f..ff6debaf9 100644 --- a/src/test/regress/expected/multi_mx_add_coordinator.out +++ b/src/test/regress/expected/multi_mx_add_coordinator.out @@ -139,6 +139,8 @@ BEGIN INSERT INTO mx_add_coordinator.ref(groupid) VALUES (gid); END; $$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('my_group_id()', colocate_with := 'ref'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands diff --git a/src/test/regress/expected/multi_mx_call.out b/src/test/regress/expected/multi_mx_call.out index 37c989885..5d440e67b 100644 --- a/src/test/regress/expected/multi_mx_call.out +++ b/src/test/regress/expected/multi_mx_call.out @@ -402,6 +402,7 @@ select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass -- Test that we handle transactional constructs correctly inside a procedure -- that is routed to the workers. +SET citus.enable_metadata_sync TO OFF; CREATE PROCEDURE mx_call_proc_tx(x int) LANGUAGE plpgsql AS $$ BEGIN INSERT INTO multi_mx_call.mx_call_dist_table_1 VALUES (x, -1), (x+1, 4); @@ -411,6 +412,7 @@ BEGIN -- Now do the final update! UPDATE multi_mx_call.mx_call_dist_table_1 SET val = val-1 WHERE id >= x; END;$$; +RESET citus.enable_metadata_sync; -- before distribution ... CALL multi_mx_call.mx_call_proc_tx(10); -- after distribution ... @@ -486,6 +488,8 @@ BEGIN RAISE WARNING 'warning'; RAISE EXCEPTION 'error'; END;$$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands select create_distributed_function('mx_call_proc_raise(int)', '$1', 'mx_call_dist_table_1'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -552,6 +556,8 @@ SET client_min_messages TO DEBUG1; -- CREATE FUNCTION mx_call_add(int, int) RETURNS int AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('mx_call_add(int,int)'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands diff --git a/src/test/regress/expected/multi_mx_call_0.out b/src/test/regress/expected/multi_mx_call_0.out index 6f33d4dae..474d3a637 100644 --- a/src/test/regress/expected/multi_mx_call_0.out +++ b/src/test/regress/expected/multi_mx_call_0.out @@ -402,6 +402,7 @@ select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass -- Test that we handle transactional constructs correctly inside a procedure -- that is routed to the workers. +SET citus.enable_metadata_sync TO OFF; CREATE PROCEDURE mx_call_proc_tx(x int) LANGUAGE plpgsql AS $$ BEGIN INSERT INTO multi_mx_call.mx_call_dist_table_1 VALUES (x, -1), (x+1, 4); @@ -411,6 +412,7 @@ BEGIN -- Now do the final update! UPDATE multi_mx_call.mx_call_dist_table_1 SET val = val-1 WHERE id >= x; END;$$; +RESET citus.enable_metadata_sync; -- before distribution ... CALL multi_mx_call.mx_call_proc_tx(10); -- after distribution ... @@ -486,6 +488,8 @@ BEGIN RAISE WARNING 'warning'; RAISE EXCEPTION 'error'; END;$$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands select create_distributed_function('mx_call_proc_raise(int)', '$1', 'mx_call_dist_table_1'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -552,6 +556,8 @@ SET client_min_messages TO DEBUG1; -- CREATE FUNCTION mx_call_add(int, int) RETURNS int AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('mx_call_add(int,int)'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands diff --git a/src/test/regress/expected/multi_mx_create_table.out b/src/test/regress/expected/multi_mx_create_table.out index ef67a626e..6036bd325 100644 --- a/src/test/regress/expected/multi_mx_create_table.out +++ b/src/test/regress/expected/multi_mx_create_table.out @@ -63,30 +63,7 @@ CREATE TABLE citus_local_table(a int); SELECT citus_add_local_table_to_metadata('citus_local_table'); ERROR: operation is not allowed on this node HINT: Connect to the coordinator and run it again. --- create UDFs in worker node -CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; SET search_path TO citus_mx_test_schema; -CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; -CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; -- create operator CREATE OPERATOR citus_mx_test_schema.=== ( LEFTARG = int, @@ -98,30 +75,7 @@ CREATE OPERATOR citus_mx_test_schema.=== ( ); -- now create required stuff in the worker 2 \c - - - :worker_2_port --- create UDF -CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; SET search_path TO citus_mx_test_schema; -CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; -CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; -- create operator CREATE OPERATOR citus_mx_test_schema.=== ( LEFTARG = int, diff --git a/src/test/regress/expected/multi_mx_explain.out b/src/test/regress/expected/multi_mx_explain.out index 2c58dd003..1c585a027 100644 --- a/src/test/regress/expected/multi_mx_explain.out +++ b/src/test/regress/expected/multi_mx_explain.out @@ -13,6 +13,7 @@ VACUUM ANALYZE customer_mx; VACUUM ANALYZE supplier_mx; \c - - - :worker_1_port -- Function that parses explain output as JSON +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION explain_json(query text) RETURNS jsonb AS $BODY$ @@ -36,6 +37,7 @@ END; $BODY$ LANGUAGE plpgsql; \c - - - :worker_2_port -- Function that parses explain output as JSON +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION explain_json(query text) RETURNS jsonb AS $BODY$ @@ -57,6 +59,7 @@ BEGIN RETURN result; END; $BODY$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- Test Text format EXPLAIN (COSTS FALSE, FORMAT TEXT) SELECT l_quantity, count(*) count_quantity FROM lineitem_mx diff --git a/src/test/regress/expected/multi_mx_function_call_delegation.out b/src/test/regress/expected/multi_mx_function_call_delegation.out index 954c23420..552af7bf5 100644 --- a/src/test/regress/expected/multi_mx_function_call_delegation.out +++ b/src/test/regress/expected/multi_mx_function_call_delegation.out @@ -409,8 +409,11 @@ BEGIN WHERE id >= x ORDER BY 1, 2; END;$$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands -- before distribution ... select mx_call_func_tbl(10); +DEBUG: function does not have co-located tables mx_call_func_tbl --------------------------------------------------------------------- (10,-1) @@ -441,6 +444,8 @@ BEGIN RAISE WARNING 'warning'; RAISE EXCEPTION 'error'; END;$$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands select create_distributed_function('mx_call_func_raise(int)', '$1', 'mx_call_dist_table_1'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -593,6 +598,8 @@ SET citus.shard_replication_factor = 1; -- CREATE FUNCTION mx_call_add(int, int) RETURNS int AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('mx_call_add(int,int)', '$1'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands diff --git a/src/test/regress/expected/multi_mx_function_call_delegation_0.out b/src/test/regress/expected/multi_mx_function_call_delegation_0.out index 66c0f89d2..cab2f6394 100644 --- a/src/test/regress/expected/multi_mx_function_call_delegation_0.out +++ b/src/test/regress/expected/multi_mx_function_call_delegation_0.out @@ -409,8 +409,11 @@ BEGIN WHERE id >= x ORDER BY 1, 2; END;$$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands -- before distribution ... select mx_call_func_tbl(10); +DEBUG: function does not have co-located tables mx_call_func_tbl --------------------------------------------------------------------- (10,-1) @@ -441,6 +444,8 @@ BEGIN RAISE WARNING 'warning'; RAISE EXCEPTION 'error'; END;$$; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands select create_distributed_function('mx_call_func_raise(int)', '$1', 'mx_call_dist_table_1'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands @@ -593,6 +598,8 @@ SET citus.shard_replication_factor = 1; -- CREATE FUNCTION mx_call_add(int, int) RETURNS int AS 'select $1 + $2;' LANGUAGE SQL IMMUTABLE; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('mx_call_add(int,int)', '$1'); DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands diff --git a/src/test/regress/expected/multi_mx_metadata.out b/src/test/regress/expected/multi_mx_metadata.out index dc2365dda..a35b8edb1 100644 --- a/src/test/regress/expected/multi_mx_metadata.out +++ b/src/test/regress/expected/multi_mx_metadata.out @@ -344,16 +344,6 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='distributed_mx (3 rows) \c - no_access_mx - :worker_1_port --- see the comment in the top of the file -CREATE OR REPLACE FUNCTION raise_failed_aclcheck(query text) RETURNS void AS $$ -BEGIN - EXECUTE query; - EXCEPTION WHEN OTHERS THEN - IF SQLERRM LIKE 'must be owner of%' THEN - RAISE 'must be owner of the object'; - END IF; -END; -$$LANGUAGE plpgsql; SELECT raise_failed_aclcheck($$ DROP TABLE distributed_mx_table; $$); diff --git a/src/test/regress/expected/multi_mx_modifications.out b/src/test/regress/expected/multi_mx_modifications.out index c290c4e5a..276766c30 100644 --- a/src/test/regress/expected/multi_mx_modifications.out +++ b/src/test/regress/expected/multi_mx_modifications.out @@ -266,6 +266,7 @@ UPDATE limit_orders_mx SET array_of_values = 1 || array_of_values WHERE id = 246 -- immutable function calls with vars are also allowed UPDATE limit_orders_mx SET array_of_values = immutable_append_mx(array_of_values, 2) WHERE id = 246; +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION stable_append_mx(old_values int[], new_value int) RETURNS int[] AS $$ BEGIN RETURN old_values || new_value; END; $$ LANGUAGE plpgsql STABLE; @@ -282,6 +283,7 @@ SELECT array_of_values FROM limit_orders_mx WHERE id = 246; -- STRICT functions work as expected CREATE FUNCTION temp_strict_func(integer,integer) RETURNS integer AS 'SELECT COALESCE($1, 2) + COALESCE($1, 3);' LANGUAGE SQL STABLE STRICT; +RESET citus.enable_metadata_sync; UPDATE limit_orders_mx SET bidder_id = temp_strict_func(1, null) WHERE id = 246; ERROR: null value in column "bidder_id" violates not-null constraint SELECT array_of_values FROM limit_orders_mx WHERE id = 246; diff --git a/src/test/regress/expected/multi_mx_modifying_xacts.out b/src/test/regress/expected/multi_mx_modifying_xacts.out index 8c08673b2..e486b8b1b 100644 --- a/src/test/regress/expected/multi_mx_modifying_xacts.out +++ b/src/test/regress/expected/multi_mx_modifying_xacts.out @@ -227,6 +227,7 @@ SELECT * FROM objects_mx WHERE id = 1; -- create trigger on one worker to reject certain values \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad_mx() RETURNS trigger AS $rb$ BEGIN IF (NEW.name = 'BAD') THEN @@ -236,6 +237,7 @@ CREATE FUNCTION reject_bad_mx() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad_mx AFTER INSERT ON objects_mx_1220103 DEFERRABLE INITIALLY IMMEDIATE diff --git a/src/test/regress/expected/multi_mx_node_metadata.out b/src/test/regress/expected/multi_mx_node_metadata.out index d46f49caf..a4f49f320 100644 --- a/src/test/regress/expected/multi_mx_node_metadata.out +++ b/src/test/regress/expected/multi_mx_node_metadata.out @@ -829,10 +829,12 @@ SELECT master_update_node(nodeid, 'localhost', 12345) FROM pg_dist_node; (1 row) +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION trigger_metadata_sync() RETURNS void LANGUAGE C STRICT AS 'citus'; +RESET citus.enable_metadata_sync; SELECT trigger_metadata_sync(); trigger_metadata_sync --------------------------------------------------------------------- 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 114e55a0f..c9357e954 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -52,13 +52,7 @@ CREATE TABLE repartition_udt_other ( -- so that the OID is off. \c - - - :worker_1_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -68,13 +62,6 @@ CREATE OPERATOR = ( ); -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 @@ -87,13 +74,7 @@ FUNCTION 1 test_udt_hash(test_udt); -- END type creation \c - - - :worker_2_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -103,13 +84,6 @@ CREATE OPERATOR = ( ); -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 diff --git a/src/test/regress/expected/multi_mx_router_planner.out b/src/test/regress/expected/multi_mx_router_planner.out index 4dcbac57e..d006b4bb8 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -1363,6 +1363,7 @@ DEBUG: query has a single distribution column value: 1 (5 rows) -- queries inside plpgsql functions could be router plannable +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION author_articles_max_id() RETURNS int AS $$ DECLARE max_id integer; @@ -1418,6 +1419,7 @@ PL/pgSQL function author_articles_id_word_count() line XX at RETURN QUERY 41 | 11814 (5 rows) +RESET citus.enable_metadata_sync; -- materialized views can be created for router plannable queries CREATE MATERIALIZED VIEW mv_articles_hash_mx AS SELECT * FROM articles_hash_mx WHERE author_id = 1; diff --git a/src/test/regress/expected/multi_name_lengths.out b/src/test/regress/expected/multi_name_lengths.out index e100ba7d8..e3cbca911 100644 --- a/src/test/regress/expected/multi_name_lengths.out +++ b/src/test/regress/expected/multi_name_lengths.out @@ -4,6 +4,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 225000; SET citus.shard_count TO 2; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -20,6 +21,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; -- Verify that a table name > 56 characters gets hashed properly. CREATE TABLE too_long_12345678901234567890123456789012345678901234567890 ( col1 integer not null, diff --git a/src/test/regress/expected/multi_prepare_sql.out b/src/test/regress/expected/multi_prepare_sql.out index 4f30bc716..6681f4112 100644 --- a/src/test/regress/expected/multi_prepare_sql.out +++ b/src/test/regress/expected/multi_prepare_sql.out @@ -1132,10 +1132,6 @@ DROP TABLE http_request; -- (re-)planning, but not when executing. -- first create helper function CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$; -\c - - - :worker_1_port -CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$; -\c - - - :worker_2_port -CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$; \c - - - :master_port -- test table CREATE TABLE test_table (test_id integer NOT NULL, data text); diff --git a/src/test/regress/expected/multi_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index c37bbba02..ee6ac8800 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -53,13 +53,7 @@ CREATE TABLE repartition_udt_other ( -- so that the OID is off. \c - - :public_worker_1_host :worker_1_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -69,13 +63,6 @@ CREATE OPERATOR = ( ); -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 @@ -88,13 +75,7 @@ FUNCTION 1 test_udt_hash(test_udt); -- END type creation \c - - :public_worker_2_host :worker_2_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -104,13 +85,6 @@ CREATE OPERATOR = ( ); -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 diff --git a/src/test/regress/expected/multi_repartitioned_subquery_udf.out b/src/test/regress/expected/multi_repartitioned_subquery_udf.out index 0048bb7eb..f6017d2ea 100644 --- a/src/test/regress/expected/multi_repartitioned_subquery_udf.out +++ b/src/test/regress/expected/multi_repartitioned_subquery_udf.out @@ -13,28 +13,6 @@ LANGUAGE sql IMMUTABLE AS $_$ ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2) OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub; $_$; -\c - - :public_worker_1_host :worker_1_port -DROP FUNCTION IF EXISTS median(double precision[]); -NOTICE: function median(pg_catalog.float8[]) does not exist, skipping -CREATE FUNCTION median(double precision[]) RETURNS double precision -LANGUAGE sql IMMUTABLE AS $_$ - SELECT AVG(val) FROM - (SELECT val FROM unnest($1) val - ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2) - OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub; -$_$; -\c - - :public_worker_2_host :worker_2_port -DROP FUNCTION IF EXISTS median(double precision[]); -NOTICE: function median(pg_catalog.float8[]) does not exist, skipping -CREATE FUNCTION median(double precision[]) RETURNS double precision -LANGUAGE sql IMMUTABLE AS $_$ - SELECT AVG(val) FROM - (SELECT val FROM unnest($1) val - ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2) - OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub; -$_$; --- Run query on master -\c - - :master_host :master_port SELECT * FROM (SELECT median(ARRAY[1,2,sum(l_suppkey)]) as median, count(*) FROM lineitem GROUP BY l_partkey) AS a WHERE median > 2; diff --git a/src/test/regress/expected/multi_router_planner.out b/src/test/regress/expected/multi_router_planner.out index 405659544..cc503f766 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -1592,10 +1592,13 @@ BEGIN RETURN QUERY SELECT 1; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT 1 FROM authors_reference r JOIN ( SELECT s.datid FROM number1() s LEFT JOIN pg_database d ON s.datid = d.oid ) num_db ON (r.id = num_db.datid) LIMIT 1; DEBUG: found no worker with all shard placements +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT datid FROM public.number1() s(datid) DEBUG: Creating router plan DEBUG: generating subplan XXX_2 for subquery SELECT s.datid FROM ((SELECT intermediate_result.datid FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(datid integer)) s LEFT JOIN pg_database d ON (((s.datid)::oid OPERATOR(pg_catalog.=) d.oid))) @@ -1610,6 +1613,7 @@ CREATE VIEW num_db AS SELECT s.datid FROM number1() s LEFT JOIN pg_database d ON s.datid = d.oid; SELECT 1 FROM authors_reference r JOIN num_db ON (r.id = num_db.datid) LIMIT 1; DEBUG: found no worker with all shard placements +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT datid FROM public.number1() s(datid) DEBUG: Creating router plan DEBUG: generating subplan XXX_2 for subquery SELECT s.datid FROM ((SELECT intermediate_result.datid FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(datid integer)) s LEFT JOIN pg_database d ON (((s.datid)::oid OPERATOR(pg_catalog.=) d.oid))) @@ -2014,6 +2018,8 @@ BEGIN END IF; END; $$LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SET client_min_messages TO ERROR; \set VERBOSITY terse SELECT raise_failed_execution_router($$ @@ -2307,7 +2313,10 @@ BEGIN return max_id; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT author_articles_max_id(); +DEBUG: function does not have co-located tables DEBUG: Creating router plan CONTEXT: SQL statement "SELECT MAX(id) FROM articles_hash ah WHERE author_id = 1" @@ -2332,7 +2341,10 @@ BEGIN END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT * FROM author_articles_id_word_count() ORDER BY 1; +DEBUG: function does not have co-located tables DEBUG: Creating router plan CONTEXT: SQL statement "SELECT ah.id, ah.word_count FROM articles_hash ah diff --git a/src/test/regress/expected/multi_router_planner_fast_path.out b/src/test/regress/expected/multi_router_planner_fast_path.out index eb4a91f7e..8fb8917ab 100644 --- a/src/test/regress/expected/multi_router_planner_fast_path.out +++ b/src/test/regress/expected/multi_router_planner_fast_path.out @@ -1225,6 +1225,7 @@ DEBUG: query has a single distribution column value: 1 (0 rows) -- create a dummy function to be used in filtering +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION someDummyFunction(regclass) RETURNS text AS $$ @@ -1247,6 +1248,7 @@ BEGIN END IF; END; $$LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SET client_min_messages TO ERROR; \set VERBOSITY terse -- fast path router plannable, but errors @@ -1611,11 +1613,14 @@ BEGIN return max_id; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands -- we don't want too many details. though we're omitting -- "DETAIL: distribution column value:", we see it acceptable -- since the query results verifies the correctness \set VERBOSITY terse SELECT author_articles_max_id(); +DEBUG: function does not have co-located tables DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan DEBUG: query has a single distribution column value: 1 @@ -1625,30 +1630,35 @@ DEBUG: query has a single distribution column value: 1 (1 row) SELECT author_articles_max_id(); +DEBUG: function does not have co-located tables author_articles_max_id --------------------------------------------------------------------- 41 (1 row) SELECT author_articles_max_id(); +DEBUG: function does not have co-located tables author_articles_max_id --------------------------------------------------------------------- 41 (1 row) SELECT author_articles_max_id(); +DEBUG: function does not have co-located tables author_articles_max_id --------------------------------------------------------------------- 41 (1 row) SELECT author_articles_max_id(); +DEBUG: function does not have co-located tables author_articles_max_id --------------------------------------------------------------------- 41 (1 row) SELECT author_articles_max_id(); +DEBUG: function does not have co-located tables author_articles_max_id --------------------------------------------------------------------- 41 @@ -1665,7 +1675,9 @@ BEGIN return max_id; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode SELECT author_articles_max_id(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan author_articles_max_id @@ -1674,6 +1686,7 @@ DEBUG: Creating router plan (1 row) SELECT author_articles_max_id(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan author_articles_max_id @@ -1682,6 +1695,7 @@ DEBUG: Creating router plan (1 row) SELECT author_articles_max_id(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan author_articles_max_id @@ -1690,6 +1704,7 @@ DEBUG: Creating router plan (1 row) SELECT author_articles_max_id(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan author_articles_max_id @@ -1698,6 +1713,7 @@ DEBUG: Creating router plan (1 row) SELECT author_articles_max_id(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan author_articles_max_id @@ -1706,6 +1722,7 @@ DEBUG: Creating router plan (1 row) SELECT author_articles_max_id(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan author_articles_max_id @@ -1724,7 +1741,9 @@ BEGIN END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode SELECT * FROM author_articles_id_word_count(); +DEBUG: function does not have co-located tables DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan DEBUG: query has a single distribution column value: 1 @@ -1738,6 +1757,7 @@ DEBUG: query has a single distribution column value: 1 (5 rows) SELECT * FROM author_articles_id_word_count(); +DEBUG: function does not have co-located tables id | word_count --------------------------------------------------------------------- 1 | 9572 @@ -1748,6 +1768,7 @@ SELECT * FROM author_articles_id_word_count(); (5 rows) SELECT * FROM author_articles_id_word_count(); +DEBUG: function does not have co-located tables id | word_count --------------------------------------------------------------------- 1 | 9572 @@ -1758,6 +1779,7 @@ SELECT * FROM author_articles_id_word_count(); (5 rows) SELECT * FROM author_articles_id_word_count(); +DEBUG: function does not have co-located tables id | word_count --------------------------------------------------------------------- 1 | 9572 @@ -1768,6 +1790,7 @@ SELECT * FROM author_articles_id_word_count(); (5 rows) SELECT * FROM author_articles_id_word_count(); +DEBUG: function does not have co-located tables id | word_count --------------------------------------------------------------------- 1 | 9572 @@ -1778,6 +1801,7 @@ SELECT * FROM author_articles_id_word_count(); (5 rows) SELECT * FROM author_articles_id_word_count(); +DEBUG: function does not have co-located tables id | word_count --------------------------------------------------------------------- 1 | 9572 @@ -1798,7 +1822,9 @@ BEGIN END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode SELECT * FROM author_articles_id_word_count(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan id | word_count @@ -1811,6 +1837,7 @@ DEBUG: Creating router plan (5 rows) SELECT * FROM author_articles_id_word_count(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan id | word_count @@ -1823,6 +1850,7 @@ DEBUG: Creating router plan (5 rows) SELECT * FROM author_articles_id_word_count(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan id | word_count @@ -1835,6 +1863,7 @@ DEBUG: Creating router plan (5 rows) SELECT * FROM author_articles_id_word_count(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan id | word_count @@ -1847,6 +1876,7 @@ DEBUG: Creating router plan (5 rows) SELECT * FROM author_articles_id_word_count(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan id | word_count @@ -1859,6 +1889,7 @@ DEBUG: Creating router plan (5 rows) SELECT * FROM author_articles_id_word_count(1); +DEBUG: function does not have co-located tables DEBUG: Deferred pruning for a fast-path router query DEBUG: Creating router plan id | word_count diff --git a/src/test/regress/expected/multi_schema_support.out b/src/test/regress/expected/multi_schema_support.out index 6e3ac0dc8..eb1c47401 100644 --- a/src/test/regress/expected/multi_schema_support.out +++ b/src/test/regress/expected/multi_schema_support.out @@ -194,31 +194,6 @@ BEGIN END; $$ LANGUAGE 'plpgsql' IMMUTABLE; --- create UDF in worker node 1 -\c - - - :worker_1_port -CREATE OR REPLACE FUNCTION dummyFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; --- create UDF in worker node 2 -\c - - - :worker_2_port -CREATE OR REPLACE FUNCTION dummyFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; -\c - - - :master_port -- UDF in public, table in a schema other than public, search_path is not set SELECT dummyFunction(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1; dummyfunction @@ -260,33 +235,6 @@ BEGIN END; $$ LANGUAGE 'plpgsql' IMMUTABLE; --- create UDF in worker node 1 in schema -\c - - - :worker_1_port -SET search_path TO test_schema_support; -CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; --- create UDF in worker node 2 in schema -\c - - - :worker_2_port -SET search_path TO test_schema_support; -CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; -\c - - - :master_port -- UDF in schema, table in a schema other than public, search_path is not set SET search_path TO public; SELECT test_schema_support.dummyFunction2(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1; diff --git a/src/test/regress/expected/multi_subquery_behavioral_analytics.out b/src/test/regress/expected/multi_subquery_behavioral_analytics.out index e357132b4..9284b7f92 100644 --- a/src/test/regress/expected/multi_subquery_behavioral_analytics.out +++ b/src/test/regress/expected/multi_subquery_behavioral_analytics.out @@ -1563,22 +1563,8 @@ LIMIT 10; (10 rows) -- this is one complex join query derived from a user's production query --- first declare the function on workers on master +-- declare the function on workers and master -- With array_index: -SELECT * FROM run_command_on_workers('CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) - RETURNS INT AS $$ - SELECT i - FROM (SELECT generate_series(array_lower($1, 1), array_upper($1, 1))) g(i) - WHERE $1 [i] = $2 - LIMIT 1; - $$ LANGUAGE sql') -ORDER BY 1,2; - nodename | nodeport | success | result ---------------------------------------------------------------------- - localhost | 57637 | t | CREATE FUNCTION - localhost | 57638 | t | CREATE FUNCTION -(2 rows) - CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) RETURNS INT AS $$ SELECT i @@ -1745,14 +1731,6 @@ LIMIT 10; (10 rows) -- drop created functions -SELECT * FROM run_command_on_workers('DROP FUNCTION array_index(ANYARRAY, ANYELEMENT)') -ORDER BY 1,2; - nodename | nodeport | success | result ---------------------------------------------------------------------- - localhost | 57637 | t | DROP FUNCTION - localhost | 57638 | t | DROP FUNCTION -(2 rows) - DROP FUNCTION array_index(ANYARRAY, ANYELEMENT); -- a query with a constant subquery SELECT count(*) as subquery_count @@ -1883,6 +1861,8 @@ CREATE OR REPLACE FUNCTION volatile_func_test() RETURNS INT AS $$ SELECT 1; $$ LANGUAGE sql VOLATILE; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands -- Citus should be able to evalute functions/row comparisons on the LIMIT/OFFSET SELECT user_id, array_length(events_table, 1) FROM ( @@ -1987,21 +1967,6 @@ CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; -SELECT run_command_on_workers($f$ - -CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool - AS 'select $1 > $2;' - LANGUAGE SQL - IMMUTABLE - RETURNS NULL ON NULL INPUT; - -$f$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"CREATE FUNCTION") - (localhost,57638,t,"CREATE FUNCTION") -(2 rows) - -- we don't support joins via functions SELECT user_id, array_length(events_table, 1) FROM ( @@ -2285,16 +2250,5 @@ LIMIT 1; (1 row) DROP FUNCTION test_join_function_2(integer, integer); -SELECT run_command_on_workers($f$ - - DROP FUNCTION test_join_function_2(integer, integer); - -$f$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"DROP FUNCTION") - (localhost,57638,t,"DROP FUNCTION") -(2 rows) - SET citus.enable_router_execution TO TRUE; SET citus.subquery_pushdown to OFF; diff --git a/src/test/regress/expected/multi_test_helpers_superuser.out b/src/test/regress/expected/multi_test_helpers_superuser.out index 238ed9f2d..4bd8b941d 100644 --- a/src/test/regress/expected/multi_test_helpers_superuser.out +++ b/src/test/regress/expected/multi_test_helpers_superuser.out @@ -38,6 +38,7 @@ SELECT dist_node_check.matches AND dist_placement_check.matches FROM dist_node_check CROSS JOIN dist_placement_check $$; -- partition_task_list_results tests the internal PartitionTasklistResults function +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.partition_task_list_results(resultIdPrefix text, query text, target_table regclass, @@ -63,3 +64,4 @@ BEGIN PERFORM public.wait_until_metadata_sync(30000); END; $function$; +RESET citus.enable_metadata_sync; diff --git a/src/test/regress/expected/multi_transactional_drop_shards.out b/src/test/regress/expected/multi_transactional_drop_shards.out index 5d59069a3..fd06e5d14 100644 --- a/src/test/regress/expected/multi_transactional_drop_shards.out +++ b/src/test/regress/expected/multi_transactional_drop_shards.out @@ -292,11 +292,13 @@ SET citus.override_table_visibility TO false; (1 row) -- test DROP table with failing worker +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION fail_drop_table() RETURNS event_trigger AS $fdt$ BEGIN RAISE 'illegal value'; END; $fdt$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE EVENT TRIGGER fail_drop_table ON sql_drop EXECUTE PROCEDURE fail_drop_table(); \c - - - :master_port \set VERBOSITY terse diff --git a/src/test/regress/expected/multi_unsupported_worker_operations.out b/src/test/regress/expected/multi_unsupported_worker_operations.out index 2e412a4a8..d54e5e84b 100644 --- a/src/test/regress/expected/multi_unsupported_worker_operations.out +++ b/src/test/regress/expected/multi_unsupported_worker_operations.out @@ -62,6 +62,7 @@ SELECT * FROM mx_table ORDER BY col_1; -- Try commands from metadata worker \c - - - :worker_1_port -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -78,6 +79,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; CREATE TABLE mx_table_worker(col_1 text); -- master_create_distributed_table SELECT master_create_distributed_table('mx_table_worker', 'col_1', 'hash'); diff --git a/src/test/regress/expected/non_colocated_leaf_subquery_joins.out b/src/test/regress/expected/non_colocated_leaf_subquery_joins.out index 1a4f20529..976f3d438 100644 --- a/src/test/regress/expected/non_colocated_leaf_subquery_joins.out +++ b/src/test/regress/expected/non_colocated_leaf_subquery_joins.out @@ -22,6 +22,7 @@ BEGIN RETURN result; END; $BODY$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode SHOW log_error_verbosity; log_error_verbosity --------------------------------------------------------------------- @@ -36,6 +37,7 @@ FROM (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar WHERE foo.user_id = bar.user_id;$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id) valid @@ -51,6 +53,7 @@ FROM (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (5,6,7,8)) as bar WHERE foo.user_id = bar.user_id;$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id) @@ -73,6 +76,7 @@ WHERE users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (5,6));$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer))) valid @@ -89,6 +93,7 @@ SELECT count(*) FROM q1, (SELECT users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) as bar WHERE bar.user_id = q1.user_id ;$$); +DEBUG: function does not have co-located tables DEBUG: CTE q1 is going to be inlined via distributed planning DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.user_id FROM public.users_table) q1, (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) bar WHERE (bar.user_id OPERATOR(pg_catalog.=) q1.user_id) @@ -101,6 +106,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c SELECT true AS valid FROM explain_json($$ (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) UNION (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8));$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) @@ -137,6 +143,7 @@ FROM ( ) q ORDER BY 2 DESC, 1; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: push down of limit count: 5 DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 OPERATOR(pg_catalog.>=) 5) AND (EXISTS (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))) LIMIT 5 @@ -158,6 +165,7 @@ FROM (SELECT users_table.user_id, value_1 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar WHERE foo.user_id = bar.value_1;$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.value_1) valid diff --git a/src/test/regress/expected/non_colocated_subquery_joins.out b/src/test/regress/expected/non_colocated_subquery_joins.out index 0776f47cf..c14502569 100644 --- a/src/test/regress/expected/non_colocated_subquery_joins.out +++ b/src/test/regress/expected/non_colocated_subquery_joins.out @@ -32,6 +32,7 @@ BEGIN RETURN result; END; $BODY$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode -- leaf queries contain colocated joins -- but not the subquery SELECT true AS valid FROM explain_json_2($$ @@ -43,6 +44,7 @@ SELECT true AS valid FROM explain_json_2($$ WHERE foo.value_2 = bar.value_2; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.value_2 FROM (SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) bar WHERE (foo.value_2 OPERATOR(pg_catalog.=) bar.value_2) valid @@ -63,6 +65,7 @@ SELECT true AS valid FROM explain_json_2($$ (SELECT event_type FROM events_table WHERE user_id < 100); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 100) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.events_table WHERE (event_type OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.event_type FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer))) valid @@ -82,6 +85,7 @@ SELECT true AS valid FROM explain_json_2($$ NOT IN (SELECT user_id FROM events_table WHERE event_type = 2); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.events_table WHERE (event_type OPERATOR(pg_catalog.=) 2) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.events_table WHERE (NOT (user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))) valid @@ -103,6 +107,7 @@ SELECT true AS valid FROM explain_json_2($$ foo.event_type IN (SELECT event_type FROM events_table WHERE user_id < 3); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 3) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (foo.event_type OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.event_type FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer)))) valid @@ -123,6 +128,7 @@ SELECT true AS valid FROM explain_json_2($$ foo.user_id = bar.user_id AND foo.user_id IN (SELECT user_id FROM events_table WHERE user_id < 10); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT (users_table.user_id OPERATOR(pg_catalog./) 2) AS user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (foo.user_id OPERATOR(pg_catalog.=) ANY (SELECT events_table.user_id FROM public.events_table WHERE (events_table.user_id OPERATOR(pg_catalog.<) 10)))) valid @@ -143,6 +149,7 @@ SELECT true AS valid FROM explain_json_2($$ foo.user_id = bar.user_id AND foo.user_id NOT IN (SELECT user_id FROM events_table WHERE user_id < 10); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT (users_table.user_id OPERATOR(pg_catalog./) 2) AS user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 10) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (NOT (foo.user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer))))) @@ -164,6 +171,7 @@ SELECT true AS valid FROM explain_json_2($$ foo.user_id = bar.user_id AND foo.event_type IN (SELECT event_type FROM events_table WHERE user_id < 4); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: generating subplan XXX_2 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.<) 4) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id FROM (SELECT intermediate_result.user_id, intermediate_result.event_type FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event_type integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE ((foo.user_id OPERATOR(pg_catalog.=) bar.user_id) AND (foo.event_type OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.event_type FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer)))) @@ -188,6 +196,7 @@ SELECT true AS valid FROM explain_json_2($$ ) as foo_top, events_table WHERE events_table.user_id = foo_top.user_id; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, events_table.event_type FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.event_type) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: generating subplan XXX_3 for subquery SELECT event_type FROM public.events_table WHERE (user_id OPERATOR(pg_catalog.=) 5) @@ -222,6 +231,7 @@ SELECT true AS valid FROM explain_json_2($$ ) as foo_top; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo5.value_1))) foo_top valid @@ -253,6 +263,7 @@ SELECT true AS valid FROM explain_json_2($$ foo1.user_id = foo5.user_id ) as foo_top; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo5.user_id))) foo_top valid @@ -282,6 +293,7 @@ SELECT true AS valid FROM explain_json_2($$ foo1.user_id = foo5.value_1 ) as foo_top; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo5.value_1))) foo_top @@ -313,6 +325,7 @@ SELECT true AS valid FROM explain_json_2($$ foo2.user_id = foo5.value_1 ) as foo_top; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[17, 18, 19, 20]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, random FROM (SELECT foo1.user_id, random() AS random FROM (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo1, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo2, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo3, (SELECT users_table.user_id, users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) foo4, (SELECT intermediate_result.user_id, intermediate_result.value_1 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer)) foo5 WHERE ((foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo2.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo3.user_id) AND (foo1.user_id OPERATOR(pg_catalog.=) foo4.user_id) AND (foo2.user_id OPERATOR(pg_catalog.=) foo5.value_1))) foo_top @@ -346,6 +359,7 @@ SELECT true AS valid FROM explain_json_2($$ foo.user_id = bar.user_id) as bar_top ON (foo_top.user_id = bar_top.user_id); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) foo_top JOIN (SELECT foo.user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) bar_top ON ((foo_top.user_id OPERATOR(pg_catalog.=) bar_top.user_id))) @@ -380,6 +394,7 @@ SELECT true AS valid FROM explain_json_2($$ ON (foo_top.value_2 = bar_top.user_id); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT foo.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo, (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.user_id, foo.value_2 FROM (SELECT DISTINCT users_table.user_id, users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) foo_top JOIN (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar_top ON ((foo_top.value_2 OPERATOR(pg_catalog.=) bar_top.user_id))) valid @@ -411,6 +426,7 @@ SELECT true AS valid FROM explain_json_2($$ foo.user_id = bar.user_id) as bar_top ON (foo_top.value_2 = bar_top.user_id); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[13, 14, 15, 16]))) DEBUG: generating subplan XXX_2 for subquery SELECT foo.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12])))) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.user_id, foo.value_2 FROM (SELECT DISTINCT users_table.user_id, users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8])))) bar WHERE (foo.user_id OPERATOR(pg_catalog.=) bar.user_id)) foo_top JOIN (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar_top ON ((foo_top.value_2 OPERATOR(pg_catalog.=) bar_top.user_id))) @@ -432,6 +448,7 @@ SELECT true AS valid FROM explain_json_2($$ WHERE foo.my_users = users_table.user_id) as mid_level_query ) as bar; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT events_table.user_id AS my_users FROM public.events_table, public.users_table WHERE (events_table.event_type OPERATOR(pg_catalog.=) users_table.user_id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT intermediate_result.my_users FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(my_users integer)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar valid @@ -451,6 +468,7 @@ SELECT true AS valid FROM explain_json_2($$ WHERE foo.my_users = users_table.user_id) as mid_level_query ) as bar; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT events_table.event_type AS my_users, random() AS random FROM public.events_table, public.users_table WHERE (events_table.user_id OPERATOR(pg_catalog.=) users_table.user_id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT intermediate_result.my_users, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(my_users integer, random double precision)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar valid @@ -474,6 +492,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c WHERE foo.my_users = users_table.user_id) as mid_level_query ) as bar; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT events_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT events_table.user_id AS my_users FROM public.events_table, (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) selected_users WHERE (events_table.event_type OPERATOR(pg_catalog.=) selected_users.user_id)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar valid @@ -507,6 +526,7 @@ SELECT true AS valid FROM explain_json_2($$ ) as bar; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT value_2 FROM public.events_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT mid_level_query.user_id FROM (SELECT DISTINCT users_table.user_id FROM public.users_table, (SELECT events_table.user_id AS my_users FROM public.events_table, (SELECT events_table_1.user_id FROM public.users_table users_table_1, public.events_table events_table_1 WHERE ((users_table_1.user_id OPERATOR(pg_catalog.=) events_table_1.user_id) AND (users_table_1.user_id OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer))))) selected_users WHERE (events_table.user_id OPERATOR(pg_catalog.=) selected_users.user_id)) foo WHERE (foo.my_users OPERATOR(pg_catalog.=) users_table.user_id)) mid_level_query) bar valid @@ -528,6 +548,7 @@ WHERE users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (5,6));$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer))) valid @@ -544,6 +565,7 @@ SELECT count(*) FROM q1, (SELECT users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) as bar WHERE bar.user_id = q1.user_id ;$$); +DEBUG: function does not have co-located tables DEBUG: CTE q1 is going to be inlined via distributed planning DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.user_id FROM public.users_table) q1, (SELECT intermediate_result.user_id, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, random double precision)) bar WHERE (bar.user_id OPERATOR(pg_catalog.=) q1.user_id) @@ -561,6 +583,7 @@ SELECT true AS valid FROM explain_json_2($$ users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (1,2,3,4)) as bar WHERE bar.user_id = q1.user_id ;$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for CTE q1: SELECT user_id FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) q1, (SELECT users_table.user_id, random() AS random FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) bar WHERE (bar.user_id OPERATOR(pg_catalog.=) q1.user_id) valid @@ -572,6 +595,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c SELECT true AS valid FROM explain_json_2($$ (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND event_type IN (1,2,3,4)) UNION (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8));$$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) @@ -608,6 +632,7 @@ FROM ( ) q ORDER BY 2 DESC, 1; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.value_2) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4]))) DEBUG: push down of limit count: 5 DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 OPERATOR(pg_catalog.>=) 5) AND (EXISTS (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)))) LIMIT 5 @@ -629,6 +654,7 @@ SELECT true AS valid FROM explain_json_2($$ FROM (users_table u1 JOIN users_table u2 using(value_1)) a JOIN (SELECT value_1, random() FROM users_table) as u3 USING (value_1); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT value_1, random() AS random FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((public.users_table u1 JOIN public.users_table u2 USING (value_1)) a(value_1, user_id, "time", value_2, value_3, value_4, user_id_1, time_1, value_2_1, value_3_1, value_4_1) JOIN (SELECT intermediate_result.value_1, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer, random double precision)) u3 USING (value_1)) ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns @@ -641,6 +667,7 @@ SELECT true AS valid FROM explain_json_2($$ FROM (SELECT * FROM users_table u1 JOIN users_table u2 using(value_1)) a JOIN (SELECT value_1, random() FROM users_table) as u3 USING (value_1); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT u1.value_1, u1.user_id, u1."time", u1.value_2, u1.value_3, u1.value_4, u2.user_id, u2."time", u2.value_2, u2.value_3, u2.value_4 FROM (public.users_table u1 JOIN public.users_table u2 USING (value_1)) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.value_1, intermediate_result.user_id, intermediate_result."time", intermediate_result.value_2, intermediate_result.value_3, intermediate_result.value_4, intermediate_result.user_id_1 AS user_id, intermediate_result.time_1 AS "time", intermediate_result.value_2_1 AS value_2, intermediate_result.value_3_1 AS value_3, intermediate_result.value_4_1 AS value_4 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer, user_id integer, "time" timestamp without time zone, value_2 integer, value_3 double precision, value_4 bigint, user_id_1 integer, time_1 timestamp without time zone, value_2_1 integer, value_3_1 double precision, value_4_1 bigint)) a(value_1, user_id, "time", value_2, value_3, value_4, user_id_1, time_1, value_2_1, value_3_1, value_4_1) JOIN (SELECT users_table.value_1, random() AS random FROM public.users_table) u3 USING (value_1)) valid @@ -660,6 +687,7 @@ SELECT true AS valid FROM explain_json_2($$ events_table using (value_2); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT value_2, random() AS random FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.value_2, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer, random double precision)) u1 JOIN public.events_table USING (value_2)) valid @@ -678,6 +706,7 @@ SELECT true AS valid FROM explain_json_2($$ (SELECT value_2, random() FROM users_table) as u2 USING(value_2); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT value_2, random() AS random FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT users_table.value_2, random() AS random FROM public.users_table) u1 LEFT JOIN (SELECT intermediate_result.value_2, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer, random double precision)) u2 USING (value_2)) valid @@ -698,6 +727,7 @@ SELECT true AS valid FROM explain_json_2($$ (SELECT value_2, random() FROM users_table) as u2 USING(value_2); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT value_2, random() AS random FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT users_table.value_2, random() AS random FROM public.users_table) u1 RIGHT JOIN (SELECT intermediate_result.value_2, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer, random double precision)) u2 USING (value_2)) ERROR: cannot pushdown the subquery @@ -717,6 +747,7 @@ SELECT true AS valid FROM explain_json_2($$ (SELECT value_1 FROM users_table) as foo ON (a.user_id = foo.value_1) ); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.users_table DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) @@ -741,6 +772,7 @@ SELECT true AS valid FROM explain_json_2($$ users_table as foo ON (a.user_id = foo.value_1) ); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.users_table DEBUG: generating subplan XXX_2 for subquery SELECT user_id FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) UNION SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) @@ -774,6 +806,7 @@ SELECT true AS valid FROM explain_json_2($$ ON(foo.user_id = bar.value_1) ); $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT value_1 FROM public.users_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.user_id, a.user_id, bar.value_1 FROM (((SELECT users_table.user_id FROM public.users_table) foo JOIN (SELECT users_table.user_id FROM public.users_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])) UNION SELECT users_table.user_id FROM public.users_table WHERE (users_table.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) a ON ((a.user_id OPERATOR(pg_catalog.=) foo.user_id))) JOIN (SELECT intermediate_result.value_1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_1 integer)) bar ON ((foo.user_id OPERATOR(pg_catalog.=) bar.value_1))) valid @@ -813,6 +846,7 @@ SELECT true AS valid FROM explain_json_2($$ WHERE non_colocated_subquery.value_2 != non_colocated_subquery_2.cnt $$); +DEBUG: function does not have co-located tables DEBUG: CTE non_colocated_subquery is going to be inlined via distributed planning DEBUG: CTE non_colocated_subquery_2 is going to be inlined via distributed planning DEBUG: generating subplan XXX_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) @@ -837,6 +871,7 @@ SELECT true AS valid FROM explain_json_2($$ AND foo.value_2 = baz.value_2 $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT users_table_local.value_2 FROM non_colocated_subquery.users_table_local, non_colocated_subquery.events_table_local WHERE ((users_table_local.user_id OPERATOR(pg_catalog.=) events_table_local.user_id) AND (events_table_local.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[5, 6, 7, 8]))) DEBUG: generating subplan XXX_2 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[9, 10, 11, 12]))) DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (events_table.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[1, 2, 3, 4])))) foo, (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) bar, (SELECT intermediate_result.value_2 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) baz WHERE ((foo.value_2 OPERATOR(pg_catalog.=) bar.value_2) AND (foo.value_2 OPERATOR(pg_catalog.=) baz.value_2)) @@ -872,6 +907,7 @@ SELECT true AS valid FROM explain_json_2($$ AND foo.user_id IN (SELECT users_table.user_id FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (1,2)) $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT value_1, value_2 FROM public.users_table DEBUG: generating subplan XXX_2 for subquery SELECT value_1 FROM public.users_table WHERE (value_2 OPERATOR(pg_catalog.<) 1) DEBUG: generating subplan XXX_3 for subquery SELECT value_2 FROM public.users_table WHERE (value_1 OPERATOR(pg_catalog.<) 2) @@ -894,6 +930,7 @@ SELECT true AS valid FROM explain_json_2($$ users_table_ref.user_id = foo.user_id AND foo.user_id = bar.value_2; $$); +DEBUG: function does not have co-located tables DEBUG: generating subplan XXX_1 for subquery SELECT user_id, value_2 FROM public.events_table DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM public.users_reference_table users_table_ref, (SELECT users_table.user_id FROM public.users_table) foo, (SELECT intermediate_result.user_id, intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_2 integer)) bar WHERE ((users_table_ref.user_id OPERATOR(pg_catalog.=) foo.user_id) AND (foo.user_id OPERATOR(pg_catalog.=) bar.value_2)) valid @@ -942,6 +979,7 @@ JOIN LATERAL WHERE user_id = users_table.user_id) AS bar LEFT JOIN users_table u2 ON u2.user_id = bar.value_2) AS foo ON TRUE $$); +DEBUG: function does not have co-located tables DEBUG: Router planner cannot handle multi-shard select queries DEBUG: skipping recursive planning for the subquery since it contains references to outer queries DEBUG: skipping recursive planning for the subquery since it contains references to outer queries diff --git a/src/test/regress/expected/pg13.out b/src/test/regress/expected/pg13.out index 7e91e3d2a..2ba782aaf 100644 --- a/src/test/regress/expected/pg13.out +++ b/src/test/regress/expected/pg13.out @@ -145,9 +145,13 @@ CREATE TYPE myvarchar; CREATE FUNCTION myvarcharin(cstring, oid, integer) RETURNS myvarchar LANGUAGE internal IMMUTABLE PARALLEL SAFE STRICT AS 'varcharin'; NOTICE: return type myvarchar is only a shell +WARNING: Citus can't distribute functions having dependency on unsupported object of type "type" +DETAIL: Function will be created only locally CREATE FUNCTION myvarcharout(myvarchar) RETURNS cstring LANGUAGE internal IMMUTABLE PARALLEL SAFE STRICT AS 'varcharout'; NOTICE: argument type myvarchar is only a shell +WARNING: Citus can't distribute functions having dependency on unsupported object of type "type" +DETAIL: Function will be created only locally CREATE TYPE myvarchar ( input = myvarcharin, output = myvarcharout, diff --git a/src/test/regress/expected/propagate_extension_commands.out b/src/test/regress/expected/propagate_extension_commands.out index 71966ed39..1c2fd19b4 100644 --- a/src/test/regress/expected/propagate_extension_commands.out +++ b/src/test/regress/expected/propagate_extension_commands.out @@ -186,22 +186,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init ); CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template); COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers'; -SELECT run_command_on_workers($$ -CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT; -$$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"CREATE FUNCTION") -(1 row) - -SELECT run_command_on_workers($$ -CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT; -$$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"CREATE FUNCTION") -(1 row) - SELECT run_command_on_workers($$ CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init ); $$); diff --git a/src/test/regress/expected/propagate_extension_commands_1.out b/src/test/regress/expected/propagate_extension_commands_1.out index 482ee5580..e6c85bee4 100644 --- a/src/test/regress/expected/propagate_extension_commands_1.out +++ b/src/test/regress/expected/propagate_extension_commands_1.out @@ -186,22 +186,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init ); CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template); COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers'; -SELECT run_command_on_workers($$ -CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT; -$$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"CREATE FUNCTION") -(1 row) - -SELECT run_command_on_workers($$ -CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT; -$$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"CREATE FUNCTION") -(1 row) - SELECT run_command_on_workers($$ CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init ); $$); diff --git a/src/test/regress/expected/replicate_reference_tables_to_coordinator.out b/src/test/regress/expected/replicate_reference_tables_to_coordinator.out index 987c52bdc..948adb050 100644 --- a/src/test/regress/expected/replicate_reference_tables_to_coordinator.out +++ b/src/test/regress/expected/replicate_reference_tables_to_coordinator.out @@ -43,11 +43,13 @@ SELECT create_reference_table('numbers'); INSERT INTO numbers VALUES (20), (21); NOTICE: executing the command locally: INSERT INTO replicate_ref_to_coordinator.numbers_8000001 AS citus_table_alias (a) VALUES (20), (21) +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION my_volatile_fn() RETURNS INT AS $$ BEGIN RETURN 1; END; $$ language plpgsql VOLATILE; +RESET citus.enable_metadata_sync; -- INSERT ... SELECT between reference tables BEGIN; EXPLAIN (COSTS OFF) INSERT INTO squares SELECT a, a*a FROM numbers; diff --git a/src/test/regress/expected/shard_move_deferred_delete.out b/src/test/regress/expected/shard_move_deferred_delete.out index ed0d46500..d878c18a4 100644 --- a/src/test/regress/expected/shard_move_deferred_delete.out +++ b/src/test/regress/expected/shard_move_deferred_delete.out @@ -142,22 +142,35 @@ SELECT master_move_shard_placement(20000000, 'localhost', :worker_2_port, 'local (1 row) -SELECT run_command_on_workers($cmd$ - -- override the function for testing purpose - create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) - as $BODY$ - begin - select 20 into available_disk_size; - select 8500 into total_disk_size; - end - $BODY$ language plpgsql; -$cmd$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"CREATE FUNCTION") - (localhost,57638,t,"CREATE FUNCTION") -(2 rows) - +-- override the function for testing purpose +-- since it is extension owned function, propagate it to workers manually +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 20 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; +\c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 20 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; +\c - - - :worker_2_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 20 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; +\c - - - :master_port +SET search_path TO shard_move_deferred_delete; SELECT citus_shard_cost_by_disk_size(20000001); citus_shard_cost_by_disk_size --------------------------------------------------------------------- @@ -187,22 +200,35 @@ $cmd$); (localhost,57638,t,1) (2 rows) -SELECT run_command_on_workers($cmd$ - -- override the function for testing purpose - create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) - as $BODY$ - begin - select 8300 into available_disk_size; - select 8500 into total_disk_size; - end - $BODY$ language plpgsql; -$cmd$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"CREATE FUNCTION") - (localhost,57638,t,"CREATE FUNCTION") -(2 rows) - +-- override the function for testing purpose +-- since it is extension owned function, propagate it to workers manually +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 8300 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; +\c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 8300 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; +\c - - - :worker_2_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 8300 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; +\c - - - :master_port +SET search_path TO shard_move_deferred_delete; -- When there would not be enough free space left after the move, the move should fail SELECT master_move_shard_placement(20000001, 'localhost', :worker_2_port, 'localhost', :worker_1_port); ERROR: not enough empty space on node if the shard is moved, actual available space after move will be 108 bytes, desired available space after move is 850 bytes,estimated size increase on node after move is 8192 bytes. diff --git a/src/test/regress/expected/shard_rebalancer.out b/src/test/regress/expected/shard_rebalancer.out index bbf537000..c63b50842 100644 --- a/src/test/regress/expected/shard_rebalancer.out +++ b/src/test/regress/expected/shard_rebalancer.out @@ -241,6 +241,7 @@ RETURNS boolean AS 'citus' LANGUAGE C STRICT VOLATILE; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -257,6 +258,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; SET citus.next_shard_id TO 123000; SELECT worker_node_responsive(node_name, node_port::int) FROM master_get_active_worker_nodes() diff --git a/src/test/regress/expected/subquery_and_cte.out b/src/test/regress/expected/subquery_and_cte.out index 4360bb69e..f3f12b975 100644 --- a/src/test/regress/expected/subquery_and_cte.out +++ b/src/test/regress/expected/subquery_and_cte.out @@ -126,6 +126,7 @@ WITH cte1 AS MATERIALIZED (SELECT id, value FROM func()) UPDATE dist_table dt SET value = cte1.value FROM cte1 WHERE dt.id = 1; DEBUG: generating subplan XXX_1 for CTE cte1: SELECT id, value FROM subquery_and_ctes.func() func(id, value) +DEBUG: function does not have co-located tables DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE subquery_and_ctes.dist_table dt SET value = cte1.value FROM (SELECT intermediate_result.id, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id integer, value integer)) cte1 WHERE (dt.id OPERATOR(pg_catalog.=) 1) -- CTEs are recursively planned, and subquery foo is also recursively planned -- final plan becomes a real-time plan since we also have events_table in the diff --git a/src/test/regress/expected/upgrade_basic_before.out b/src/test/regress/expected/upgrade_basic_before.out index 28a358428..2e63447df 100644 --- a/src/test/regress/expected/upgrade_basic_before.out +++ b/src/test/regress/expected/upgrade_basic_before.out @@ -53,6 +53,7 @@ SELECT create_distributed_table('tr', 'pk'); INSERT INTO tr SELECT c, c FROM generate_series(1, 5) as c; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -69,6 +70,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; CREATE TABLE t_range(id int, value_1 int); SELECT create_distributed_table('t_range', 'id', 'range'); create_distributed_table diff --git a/src/test/regress/expected/values.out b/src/test/regress/expected/values.out index 8509cd971..cc26f7432 100644 --- a/src/test/regress/expected/values.out +++ b/src/test/regress/expected/values.out @@ -587,6 +587,8 @@ CREATE OR REPLACE FUNCTION fixed_volatile_value() RETURNS integer VOLATILE AS $$ RETURN 1; END; $$ LANGUAGE plpgsql; +DEBUG: switching to sequential query execution mode +DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands EXECUTE test_values_pushdown(fixed_volatile_value(),2,3); NOTICE: evaluated on the coordinator CONTEXT: PL/pgSQL function fixed_volatile_value() line XX at RAISE diff --git a/src/test/regress/input/multi_alter_table_statements.source b/src/test/regress/input/multi_alter_table_statements.source index fb719a243..fa828bd7b 100644 --- a/src/test/regress/input/multi_alter_table_statements.source +++ b/src/test/regress/input/multi_alter_table_statements.source @@ -294,11 +294,13 @@ ALTER TABLE lineitem_alter_220000 DROP COLUMN first; CREATE TABLE ddl_commands (command text UNIQUE DEFERRABLE INITIALLY DEFERRED); -- Use an event trigger to log all DDL event tags in it +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION log_ddl_tag() RETURNS event_trigger AS $ldt$ BEGIN INSERT INTO ddl_commands VALUES (tg_tag); END; $ldt$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE EVENT TRIGGER log_ddl_tag ON ddl_command_end EXECUTE PROCEDURE log_ddl_tag(); @@ -499,13 +501,14 @@ CREATE TABLE trigger_table ( SELECT create_distributed_table('trigger_table', 'id'); -- first set a trigger on a shard \c - - - :worker_1_port - +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION update_value() RETURNS trigger AS $up$ BEGIN NEW.value := 'trigger enabled'; RETURN NEW; END; $up$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE TRIGGER update_value BEFORE INSERT ON trigger_table_220017 diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 35b6fdcc9..4d7f68e73 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -312,6 +312,8 @@ test: ssl_by_default # --------- # object distribution tests +# TODO: After deprecating parameterless create_distributed_function combine +# distributed_functions and function_propagation tests # --------- test: distributed_types distributed_types_conflict disable_object_propagation distributed_types_xact_add_enum_value text_search test: check_mx @@ -319,6 +321,7 @@ test: distributed_functions distributed_functions_conflict test: distributed_collations test: distributed_procedure test: distributed_collations_conflict +test: function_propagation test: check_mx # --------- diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index f1132e9d2..2e26ea425 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -642,11 +642,13 @@ ALTER TABLE lineitem_alter_220000 DROP COLUMN first; -- Create table to trigger at-xact-end (deferred) failure CREATE TABLE ddl_commands (command text UNIQUE DEFERRABLE INITIALLY DEFERRED); -- Use an event trigger to log all DDL event tags in it +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION log_ddl_tag() RETURNS event_trigger AS $ldt$ BEGIN INSERT INTO ddl_commands VALUES (tg_tag); END; $ldt$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE EVENT TRIGGER log_ddl_tag ON ddl_command_end EXECUTE PROCEDURE log_ddl_tag(); \c - - - :master_port -- The above trigger will cause failure at transaction end on one placement. @@ -985,12 +987,14 @@ SELECT create_distributed_table('trigger_table', 'id'); -- first set a trigger on a shard \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION update_value() RETURNS trigger AS $up$ BEGIN NEW.value := 'trigger enabled'; RETURN NEW; END; $up$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE TRIGGER update_value BEFORE INSERT ON trigger_table_220017 FOR EACH ROW EXECUTE PROCEDURE update_value(); diff --git a/src/test/regress/output/multi_copy.source b/src/test/regress/output/multi_copy.source index 2bd0c7b77..d6d72d963 100644 --- a/src/test/regress/output/multi_copy.source +++ b/src/test/regress/output/multi_copy.source @@ -635,6 +635,8 @@ INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::re INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0); INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0); SELECT 1 FROM master_activate_node('localhost', :worker_1_port); +NOTICE: Replicating postgres objects to node localhost:57637 +DETAIL: There are 114 objects to replicate, depending on your environment this might take a while ?column? --------------------------------------------------------------------- 1 diff --git a/src/test/regress/sql/aggregate_support.sql b/src/test/regress/sql/aggregate_support.sql index dc7215f79..7c82418a7 100644 --- a/src/test/regress/sql/aggregate_support.sql +++ b/src/test/regress/sql/aggregate_support.sql @@ -459,6 +459,7 @@ RETURN $1 * $1; END; $function$; +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION square_func(int) RETURNS int LANGUAGE plpgsql @@ -467,6 +468,7 @@ BEGIN RETURN $1 * $1; END; $function$; +RESET citus.enable_metadata_sync; SELECT const_function(1), string_agg(a::character, ',') FROM t1; SELECT const_function(1), count(b) FROM t1; diff --git a/src/test/regress/sql/citus_local_tables_queries_mx.sql b/src/test/regress/sql/citus_local_tables_queries_mx.sql index f4a53bffc..ddcc95d84 100644 --- a/src/test/regress/sql/citus_local_tables_queries_mx.sql +++ b/src/test/regress/sql/citus_local_tables_queries_mx.sql @@ -43,6 +43,7 @@ CREATE TABLE postgres_local_table(a int, b int); -- We shouldn't use LIMIT in INSERT SELECT queries to make the test faster as -- LIMIT would force planner to wrap SELECT query in an intermediate result and -- this might reduce the coverage of the test cases. +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$ BEGIN SET client_min_messages to ERROR; @@ -59,6 +60,7 @@ CREATE FUNCTION clear_and_init_test_tables() RETURNS void AS $$ RESET client_min_messages; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; ---------------- ---- SELECT ---- diff --git a/src/test/regress/sql/coordinator_evaluation.sql b/src/test/regress/sql/coordinator_evaluation.sql index e7479fd1a..1f313a76a 100644 --- a/src/test/regress/sql/coordinator_evaluation.sql +++ b/src/test/regress/sql/coordinator_evaluation.sql @@ -165,6 +165,7 @@ BEGIN END; $$ LANGUAGE 'plpgsql' STRICT; +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int) LANGUAGE plpgsql AS $procedure$ @@ -174,6 +175,7 @@ BEGIN PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey; END; $procedure$; +RESET citus.enable_metadata_sync; -- we couldn't find a meaningful query to write for this -- however this query fails before https://github.com/citusdata/citus/pull/3454 diff --git a/src/test/regress/sql/coordinator_evaluation_modify.sql b/src/test/regress/sql/coordinator_evaluation_modify.sql index e4f104ad9..a4c9f75b5 100644 --- a/src/test/regress/sql/coordinator_evaluation_modify.sql +++ b/src/test/regress/sql/coordinator_evaluation_modify.sql @@ -275,11 +275,13 @@ SET citus.log_local_commands TO ON; SET search_path TO coordinator_evaluation_combinations_modify; -- returns 2 on the worker +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION get_constant_stable() RETURNS INT AS $$ BEGIN RETURN 2; END; $$ language plpgsql STABLE; +RESET citus.enable_metadata_sync; -- all local values diff --git a/src/test/regress/sql/coordinator_shouldhaveshards.sql b/src/test/regress/sql/coordinator_shouldhaveshards.sql index 159a42d6c..bc79513c3 100644 --- a/src/test/regress/sql/coordinator_shouldhaveshards.sql +++ b/src/test/regress/sql/coordinator_shouldhaveshards.sql @@ -373,9 +373,11 @@ inserts AS ( -- a helper function which return true if the coordinated -- trannsaction uses 2PC +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION coordinated_transaction_should_use_2PC() RETURNS BOOL LANGUAGE C STRICT VOLATILE AS 'citus', $$coordinated_transaction_should_use_2PC$$; +RESET citus.enable_metadata_sync; -- a local SELECT followed by remote SELECTs -- does not trigger 2PC diff --git a/src/test/regress/sql/distributed_collations.sql b/src/test/regress/sql/distributed_collations.sql index 8b2bffc7e..669577a09 100644 --- a/src/test/regress/sql/distributed_collations.sql +++ b/src/test/regress/sql/distributed_collations.sql @@ -93,3 +93,19 @@ DROP SCHEMA collation_tests CASCADE; DROP SCHEMA collation_tests2 CASCADE; DROP USER collationuser; SELECT run_command_on_workers($$DROP USER collationuser;$$); + +\c - - - :worker_1_port +-- test creating a collation on a worker +CREATE COLLATION another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk'); + +-- test if creating a collation on a worker on a local +-- schema raises the right error +SET citus.enable_ddl_propagation TO off; +CREATE SCHEMA collation_creation_on_worker; +SET citus.enable_ddl_propagation TO on; + +CREATE COLLATION collation_creation_on_worker.another_german_phonebook (provider = icu, locale = 'de-u-co-phonebk'); + +SET citus.enable_ddl_propagation TO off; +DROP SCHEMA collation_creation_on_worker; +SET citus.enable_ddl_propagation TO on; diff --git a/src/test/regress/sql/distributed_functions_conflict.sql b/src/test/regress/sql/distributed_functions_conflict.sql index b4877880f..cee80548b 100644 --- a/src/test/regress/sql/distributed_functions_conflict.sql +++ b/src/test/regress/sql/distributed_functions_conflict.sql @@ -5,6 +5,7 @@ CREATE SCHEMA proc_conflict; \c - - - :worker_1_port SET search_path TO proc_conflict; +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$ BEGIN RETURN state * 2 + i; @@ -14,6 +15,7 @@ CREATE AGGREGATE existing_agg(int) ( SFUNC = existing_func, STYPE = int ); +RESET citus.enable_metadata_sync; \c - - - :master_port SET search_path TO proc_conflict; @@ -58,7 +60,7 @@ DROP FUNCTION existing_func(int, int) CASCADE; \c - - - :worker_1_port SET search_path TO proc_conflict; - +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$ BEGIN RETURN state * 3 + i; @@ -68,6 +70,7 @@ CREATE AGGREGATE existing_agg(int) ( SFUNC = existing_func, STYPE = int ); +RESET citus.enable_metadata_sync; \c - - - :master_port SET search_path TO proc_conflict; diff --git a/src/test/regress/sql/forcedelegation_functions.sql b/src/test/regress/sql/forcedelegation_functions.sql index 6f62c41d5..d95cfc75a 100644 --- a/src/test/regress/sql/forcedelegation_functions.sql +++ b/src/test/regress/sql/forcedelegation_functions.sql @@ -189,12 +189,16 @@ COMMIT; SELECT func_calls_forcepush_func(); +-- Block distributing that function as distributing it causes +-- different test output on PG 14. +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION get_val() RETURNS INT AS $$ BEGIN RETURN 100::INT; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- -- UDF calling another UDF in a FROM clause @@ -683,6 +687,250 @@ BEGIN END; $$ LANGUAGE plpgsql; +CREATE TABLE testnested_table (x int, y int); +SELECT create_distributed_table('testnested_table','x'); + +CREATE OR REPLACE FUNCTION inner_fn(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); +END; +$$ LANGUAGE plpgsql; + +-- Non-force function calling force-delegation function +CREATE OR REPLACE FUNCTION outer_local_fn() +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM 1 FROM inner_fn(1); + INSERT INTO forcepushdown_schema.testnested_table VALUES (2,3); + PERFORM 1 FROM inner_fn(4); + INSERT INTO forcepushdown_schema.testnested_table VALUES (5,6); +END; +$$ LANGUAGE plpgsql; + +SELECT create_distributed_function('inner_fn(int)','x', + colocate_with:='testnested_table', force_delegation := true); + +SELECT outer_local_fn(); +-- Rows from 1-6 should appear +SELECT * FROM testnested_table ORDER BY 1; + +BEGIN; +SELECT outer_local_fn(); +END; +SELECT * FROM testnested_table ORDER BY 1; + +DROP FUNCTION inner_fn(int); +DROP FUNCTION outer_local_fn(); +TRUNCATE TABLE testnested_table; + +CREATE OR REPLACE FUNCTION inner_fn(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); +END; +$$ LANGUAGE plpgsql; + +-- Force-delegation function calling non-force function +CREATE OR REPLACE FUNCTION outer_fn(y int, z int) +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM 1 FROM forcepushdown_schema.inner_fn(y); + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); + PERFORM 1 FROM forcepushdown_schema.inner_fn(z); + INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z); +END; +$$ LANGUAGE plpgsql; + +SELECT create_distributed_function('inner_fn(int)','x', + colocate_with:='testnested_table', force_delegation := false); +SELECT create_distributed_function('outer_fn(int, int)','y', + colocate_with:='testnested_table', force_delegation := true); + +SELECT outer_fn(1, 2); +BEGIN; +SELECT outer_fn(1, 2); +END; + +-- No rows +SELECT * FROM testnested_table ORDER BY 1; + +-- Force-delegation function calling force-delegation function +CREATE OR REPLACE FUNCTION force_push_inner(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION force_push_outer(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); + PERFORM forcepushdown_schema.force_push_inner(x+1) LIMIT 1; +END; +$$ LANGUAGE plpgsql; + +SELECT create_distributed_function( + 'force_push_outer(int)', 'x', + colocate_with := 'testnested_table', + force_delegation := true +); +SELECT create_distributed_function( + 'force_push_inner(int)', 'y', + colocate_with := 'testnested_table', + force_delegation := true +); + +-- Keys 7,8,9,14 fall on one node and 15 on a different node + +-- Function gets delegated to node with shard-key = 7 and inner function +-- will not be delegated but inserts shard-key = 8 locally +SELECT force_push_outer(7); + +BEGIN; +-- Function gets delegated to node with shard-key = 8 and inner function +-- will not be delegated but inserts shard-key = 9 locally +SELECT force_push_outer(8); +END; + +BEGIN; +-- Function gets delegated to node with shard-key = 14 and inner function +-- will not be delegated but fails to insert shard-key = 15 remotely +SELECT force_push_outer(14); +END; +SELECT * FROM testnested_table ORDER BY 1; + +-- +-- Function-1() --> function-2() --> function-3() +-- +CREATE OR REPLACE FUNCTION force_push_1(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); + PERFORM forcepushdown_schema.force_push_2(x+1) LIMIT 1; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION force_push_2(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); + PERFORM forcepushdown_schema.force_push_3(y+1) LIMIT 1; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION force_push_3(z int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (z,z); +END; +$$ LANGUAGE plpgsql; + +SELECT create_distributed_function( + 'force_push_1(int)', 'x', + colocate_with := 'testnested_table', + force_delegation := true +); +SELECT create_distributed_function( + 'force_push_2(int)', 'y', + colocate_with := 'testnested_table', + force_delegation := true +); +SELECT create_distributed_function( + 'force_push_3(int)', 'z', + colocate_with := 'testnested_table', + force_delegation := true +); + +TRUNCATE TABLE testnested_table; +BEGIN; +-- All local inserts +SELECT force_push_1(7); +END; + +BEGIN; +-- Local(shard-keys 13, 15) + remote insert (shard-key 14) +SELECT force_push_1(13); +END; + +SELECT * FROM testnested_table ORDER BY 1; + +TRUNCATE TABLE testnested_table; +CREATE OR REPLACE FUNCTION force_push_inner(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + INSERT INTO forcepushdown_schema.testnested_table VALUES (y,y); +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION force_push_outer(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM FROM forcepushdown_schema.force_push_inner(x); + INSERT INTO forcepushdown_schema.testnested_table VALUES (x+1,x+1); +END; +$$ LANGUAGE plpgsql; +SELECT create_distributed_function( + 'force_push_inner(int)', 'y', + colocate_with := 'testnested_table', + force_delegation := true +); +SELECT create_distributed_function( + 'force_push_outer(int)', 'x', + colocate_with := 'testnested_table', + force_delegation := true +); + +BEGIN; +SELECT force_push_outer(7); +END; +TABLE testnested_table ORDER BY 1; + +CREATE OR REPLACE FUNCTION force_push_inner(y int) +RETURNS void +AS $$ +DECLARE +BEGIN + RAISE NOTICE '%', y; +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION force_push_outer(x int) +RETURNS void +AS $$ +DECLARE +BEGIN + PERFORM FROM forcepushdown_schema.force_push_inner(x+1); + INSERT INTO forcepushdown_schema.testnested_table VALUES (x,x); +END; +$$ LANGUAGE plpgsql; + +BEGIN; +SELECT force_push_outer(9); +END; +TABLE testnested_table ORDER BY 1; + RESET client_min_messages; SET citus.log_remote_commands TO off; DROP SCHEMA forcepushdown_schema CASCADE; diff --git a/src/test/regress/sql/function_propagation.sql b/src/test/regress/sql/function_propagation.sql new file mode 100644 index 000000000..79168497b --- /dev/null +++ b/src/test/regress/sql/function_propagation.sql @@ -0,0 +1,193 @@ +CREATE SCHEMA function_propagation_schema; +SET search_path TO 'function_propagation_schema'; + +-- Check whether supported dependencies can be distributed while propagating functions + +-- Check types +SET citus.enable_metadata_sync TO OFF; + CREATE TYPE function_prop_type AS (a int, b int); +RESET citus.enable_metadata_sync; + +CREATE OR REPLACE FUNCTION func_1(param_1 function_prop_type) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +-- Check all dependent objects and function depends on all nodes +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2; + +SET citus.enable_metadata_sync TO OFF; + CREATE TYPE function_prop_type_2 AS (a int, b int); +RESET citus.enable_metadata_sync; + +CREATE OR REPLACE FUNCTION func_2(param_1 int) +RETURNS function_prop_type_2 +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2; + +-- Have a separate check for type created in transaction +BEGIN; + CREATE TYPE function_prop_type_3 AS (a int, b int); +COMMIT; + +-- Objects in the body part is not found as dependency +CREATE OR REPLACE FUNCTION func_3(param_1 int) +RETURNS int +LANGUAGE plpgsql AS +$$ +DECLARE + internal_param1 function_prop_type_3; +BEGIN + return 1; +END; +$$; + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2; + +-- Check table +CREATE TABLE function_prop_table(a int, b int); + +-- Non-distributed table is not distributed as dependency +CREATE OR REPLACE FUNCTION func_4(param_1 function_prop_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +CREATE OR REPLACE FUNCTION func_5(param_1 int) +RETURNS function_prop_table +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +-- Functions can be created with distributed table dependency +SELECT create_distributed_table('function_prop_table', 'a'); +CREATE OR REPLACE FUNCTION func_6(param_1 function_prop_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2; + +-- Views are not supported +CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table; +CREATE OR REPLACE FUNCTION func_7(param_1 function_prop_view) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +CREATE OR REPLACE FUNCTION func_8(param_1 int) +RETURNS function_prop_view +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +-- Check within transaction +BEGIN; + CREATE TYPE type_in_transaction AS (a int, b int); + CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction) + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Within transaction functions are not distributed + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; +COMMIT; + +-- Show that recreating it outside transaction distributes the function and dependencies +CREATE OR REPLACE FUNCTION func_in_transaction(param_1 type_in_transaction) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2; + +-- Test for SQL function with unsupported object in function body +CREATE TABLE table_in_sql_body(id int); + +CREATE FUNCTION max_of_table() +RETURNS int +LANGUAGE SQL AS +$$ + SELECT max(id) FROM table_in_sql_body +$$; + +-- Show that only function has propagated, since the table is not resolved as dependency +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2; + +-- Check extension owned table +CREATE TABLE extension_owned_table(a int); +SELECT run_command_on_workers($$ +CREATE TABLE function_propagation_schema.extension_owned_table(a int); +$$ +); +CREATE EXTENSION seg; +ALTER EXTENSION seg ADD TABLE extension_owned_table; +SELECT run_command_on_workers($$ +ALTER EXTENSION seg ADD TABLE function_propagation_schema.extension_owned_table; +$$); + +CREATE OR REPLACE FUNCTION func_for_ext_check(param_1 extension_owned_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +RESET search_path; +SET client_min_messages TO WARNING; +DROP SCHEMA function_propagation_schema CASCADE; diff --git a/src/test/regress/sql/local_shard_copy.sql b/src/test/regress/sql/local_shard_copy.sql index 688ba8b4b..094ba8fb4 100644 --- a/src/test/regress/sql/local_shard_copy.sql +++ b/src/test/regress/sql/local_shard_copy.sql @@ -48,6 +48,7 @@ SET citus.log_local_commands TO ON; -- returns true of the distribution key filter -- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard -- placement which is local to this not +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$ DECLARE shard_is_local BOOLEAN := FALSE; @@ -70,6 +71,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R RETURN shard_is_local; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- pick some example values that reside on the shards locally and remote diff --git a/src/test/regress/sql/local_shard_execution.sql b/src/test/regress/sql/local_shard_execution.sql index 6bda63d8d..c7dad6f00 100644 --- a/src/test/regress/sql/local_shard_execution.sql +++ b/src/test/regress/sql/local_shard_execution.sql @@ -71,6 +71,7 @@ SET search_path TO local_shard_execution; -- returns true of the distribution key filter -- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard -- placement which is local to this not +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) RETURNS bool AS $$ DECLARE shard_is_local BOOLEAN := FALSE; @@ -93,6 +94,7 @@ CREATE OR REPLACE FUNCTION shard_of_distribution_column_is_local(dist_key int) R RETURN shard_is_local; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- test case for issue #3556 SET citus.log_intermediate_results TO TRUE; @@ -418,6 +420,7 @@ BEGIN; ROLLBACK; -- make sure that functions can use local execution +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE PROCEDURE only_local_execution() AS $$ DECLARE cnt INT; BEGIN @@ -491,6 +494,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$ SELECT count(*) INTO cnt FROM distributed_table; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CALL local_execution_followed_by_dist(); diff --git a/src/test/regress/sql/local_shard_execution_replicated.sql b/src/test/regress/sql/local_shard_execution_replicated.sql index a341ac619..01af172a3 100644 --- a/src/test/regress/sql/local_shard_execution_replicated.sql +++ b/src/test/regress/sql/local_shard_execution_replicated.sql @@ -384,6 +384,7 @@ BEGIN; ROLLBACK; -- make sure that functions can use local execution +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE PROCEDURE only_local_execution() AS $$ DECLARE cnt INT; BEGIN @@ -457,6 +458,7 @@ CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$ SELECT count(*) INTO cnt FROM distributed_table; END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CALL local_execution_followed_by_dist(); diff --git a/src/test/regress/sql/metadata_sync_helpers.sql b/src/test/regress/sql/metadata_sync_helpers.sql index 1ab302ba2..cf123ff49 100644 --- a/src/test/regress/sql/metadata_sync_helpers.sql +++ b/src/test/regress/sql/metadata_sync_helpers.sql @@ -618,6 +618,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; ROLLBACK; -- create a volatile function that returns the local node id +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION get_node_id() RETURNS INT AS $$ DECLARE localGroupId int; @@ -630,6 +631,7 @@ BEGIN nodeport = 57637 AND nodename = 'localhost' AND isactive AND nodecluster = 'default'; RETURN localGroupId; END; $$ language plpgsql; +RESET citus.enable_metadata_sync; -- fails because we ingest more placements for the same shards to the same worker node BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; diff --git a/src/test/regress/sql/multi_create_table.sql b/src/test/regress/sql/multi_create_table.sql index 4c318e73a..4a1070514 100644 --- a/src/test/regress/sql/multi_create_table.sql +++ b/src/test/regress/sql/multi_create_table.sql @@ -10,6 +10,7 @@ SET citus.next_shard_id TO 360000; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -27,6 +28,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; CREATE TABLE lineitem ( l_orderkey bigint not null, diff --git a/src/test/regress/sql/multi_deparse_function.sql b/src/test/regress/sql/multi_deparse_function.sql index 1574ff22f..2b8450b4f 100644 --- a/src/test/regress/sql/multi_deparse_function.sql +++ b/src/test/regress/sql/multi_deparse_function.sql @@ -359,10 +359,7 @@ SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_function('func_custom_param(intpair)'); RESET citus.enable_metadata_sync; -SELECT deparse_and_run_on_workers($cmd$ ALTER FUNCTION func_custom_param RENAME TO func_with_custom_param; -$cmd$); - -- a function that returns TABLE CREATE FUNCTION func_returns_table(IN count INT) @@ -373,9 +370,7 @@ SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_function('func_returns_table(INT)'); RESET citus.enable_metadata_sync; -SELECT deparse_and_run_on_workers($cmd$ ALTER FUNCTION func_returns_table ROWS 100; -$cmd$); -- clear objects SET client_min_messages TO WARNING; -- suppress cascading objects dropping diff --git a/src/test/regress/sql/multi_function_in_join.sql b/src/test/regress/sql/multi_function_in_join.sql index 6abdafdad..1e2844f4e 100644 --- a/src/test/regress/sql/multi_function_in_join.sql +++ b/src/test/regress/sql/multi_function_in_join.sql @@ -33,7 +33,6 @@ SELECT * FROM table1 JOIN nextval('numbers') n ON (id = n) ORDER BY id ASC; CREATE FUNCTION add(integer, integer) RETURNS integer AS 'SELECT $1 + $2;' LANGUAGE SQL; -SELECT create_distributed_function('add(integer,integer)'); SELECT * FROM table1 JOIN add(3,5) sum ON (id = sum) ORDER BY id ASC; -- Check join of plpgsql functions @@ -46,6 +45,8 @@ $$ LANGUAGE plpgsql; SELECT * FROM table1 JOIN increment(2) val ON (id = val) ORDER BY id ASC; -- a function that returns a set of integers +-- Block distributing function as we have tests below to test it locally +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION next_k_integers(IN first_value INTEGER, IN k INTEGER DEFAULT 3, OUT result INTEGER) @@ -54,6 +55,7 @@ BEGIN RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x); END; $$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SELECT * FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result) ORDER BY id ASC; @@ -124,6 +126,7 @@ SET client_min_messages TO ERROR; -- function joins in CTE results can create lateral joins that are not supported -- we execute the query within a function to consolidate the error messages -- between different executors +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION raise_failed_execution_func_join(query text) RETURNS void AS $$ BEGIN EXECUTE query; @@ -135,6 +138,7 @@ BEGIN END IF; END; $$LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SELECT raise_failed_execution_func_join($$ WITH one_row AS ( @@ -146,8 +150,10 @@ SELECT raise_failed_execution_func_join($$ $$); -- a user-defined immutable function +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION the_answer_to_life() RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL; +RESET citus.enable_metadata_sync; SELECT raise_failed_execution_func_join($$ SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer); diff --git a/src/test/regress/sql/multi_insert_select.sql b/src/test/regress/sql/multi_insert_select.sql index 7de9f9763..92c802fe1 100644 --- a/src/test/regress/sql/multi_insert_select.sql +++ b/src/test/regress/sql/multi_insert_select.sql @@ -106,6 +106,7 @@ WHERE user_id < 0; -- make sure we don't evaluate stable functions with column arguments +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION evaluate_on_master(x int) RETURNS int LANGUAGE plpgsql STABLE AS $function$ @@ -114,6 +115,7 @@ BEGIN RETURN x; END; $function$; +RESET citus.enable_metadata_sync; INSERT INTO raw_events_second (user_id, value_1) SELECT diff --git a/src/test/regress/sql/multi_metadata_sync.sql b/src/test/regress/sql/multi_metadata_sync.sql index d85bede5f..13c984396 100644 --- a/src/test/regress/sql/multi_metadata_sync.sql +++ b/src/test/regress/sql/multi_metadata_sync.sql @@ -20,10 +20,12 @@ SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset -- Create the necessary test utility function +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION activate_node_snapshot() RETURNS text[] LANGUAGE C STRICT AS 'citus'; +RESET citus.enable_metadata_sync; COMMENT ON FUNCTION activate_node_snapshot() IS 'commands to activate node snapshot'; @@ -43,12 +45,14 @@ ALTER ROLE CURRENT_USER WITH PASSWORD 'dummypassword'; SELECT unnest(activate_node_snapshot()) order by 1; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) RETURNS void LANGUAGE C STRICT AS 'citus', $$master_create_distributed_table$$; +RESET citus.enable_metadata_sync; COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) diff --git a/src/test/regress/sql/multi_modifications.sql b/src/test/regress/sql/multi_modifications.sql index 55453e78b..7977325ea 100644 --- a/src/test/regress/sql/multi_modifications.sql +++ b/src/test/regress/sql/multi_modifications.sql @@ -363,16 +363,6 @@ UPDATE limit_orders SET array_of_values = 1 || array_of_values WHERE id = 246; CREATE FUNCTION immutable_append(old_values int[], new_value int) RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; -\c - - - :worker_1_port -CREATE FUNCTION immutable_append(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; - -\c - - - :worker_2_port -CREATE FUNCTION immutable_append(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; - -\c - - - :master_port - -- immutable function calls with vars are also allowed UPDATE limit_orders SET array_of_values = immutable_append(array_of_values, 2) WHERE id = 246; diff --git a/src/test/regress/sql/multi_modifying_xacts.sql b/src/test/regress/sql/multi_modifying_xacts.sql index a3b004546..a53153b9b 100644 --- a/src/test/regress/sql/multi_modifying_xacts.sql +++ b/src/test/regress/sql/multi_modifying_xacts.sql @@ -282,18 +282,16 @@ SELECT * FROM researchers WHERE lab_id = 6; -- verify 2pc SELECT count(*) FROM pg_dist_transaction; - -- create a check function -SELECT * from run_command_on_workers('CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$ +CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$ BEGIN IF (NEW.id > 30) THEN - RAISE ''illegal value''; + RAISE 'illegal value'; END IF; RETURN NEW; END; -$rli$ LANGUAGE plpgsql;') -ORDER BY nodeport; +$rli$ LANGUAGE plpgsql; -- register after insert trigger SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()') @@ -344,9 +342,7 @@ SELECT * FROM researchers WHERE lab_id = 6; SELECT * from run_command_on_placements('researchers', 'drop trigger reject_large_researcher_id on %s') ORDER BY nodeport, shardid; -SELECT * FROM run_command_on_workers('drop function reject_large_id()') -ORDER BY nodeport; - +DROP FUNCTION reject_large_id(); -- ALTER and copy are compatible BEGIN; ALTER TABLE labs ADD COLUMN motto text; @@ -416,6 +412,7 @@ AND s.logicalrelid = 'objects'::regclass; -- create trigger on one worker to reject certain values \c - - - :worker_2_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ BEGIN IF (NEW.name = 'BAD') THEN @@ -425,6 +422,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad AFTER INSERT ON objects_1200003 @@ -460,6 +458,7 @@ DELETE FROM objects; -- there cannot be errors on different shards at different times -- because the first failure will fail the whole transaction \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ BEGIN IF (NEW.name = 'BAD') THEN @@ -469,6 +468,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad AFTER INSERT ON labs_1200002 @@ -668,7 +668,7 @@ SELECT * FROM reference_modifying_xacts; -- lets fail on of the workers at before the commit time \c - - - :worker_1_port - +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$ BEGIN IF (NEW.key = 999) THEN @@ -678,6 +678,7 @@ CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad_reference AFTER INSERT ON reference_modifying_xacts_1200006 @@ -762,7 +763,7 @@ ABORT; -- lets fail one of the workers before COMMIT time for the hash table \c - - - :worker_1_port - +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$ BEGIN IF (NEW.key = 997) THEN @@ -772,6 +773,7 @@ CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad_hash AFTER INSERT ON hash_modifying_xacts_1200007 diff --git a/src/test/regress/sql/multi_multiuser.sql b/src/test/regress/sql/multi_multiuser.sql index d22fae1ec..f379c75f1 100644 --- a/src/test/regress/sql/multi_multiuser.sql +++ b/src/test/regress/sql/multi_multiuser.sql @@ -369,21 +369,24 @@ RESET ROLE; -- to test access to files created during repartition we will create some on worker 1 \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) RETURNS void AS 'citus' LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; SET ROLE full_access; SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]); RESET ROLE; -- all attempts for transfer are initiated from other workers \c - - - :worker_2_port - +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) RETURNS void AS 'citus' LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; -- super user should not be able to copy files created by a user SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port); diff --git a/src/test/regress/sql/multi_mx_call.sql b/src/test/regress/sql/multi_mx_call.sql index 4728b8948..ccdffd9aa 100644 --- a/src/test/regress/sql/multi_mx_call.sql +++ b/src/test/regress/sql/multi_mx_call.sql @@ -180,6 +180,7 @@ select colocate_proc_with_table('mx_call_proc', 'mx_call_dist_table_1'::regclass -- Test that we handle transactional constructs correctly inside a procedure -- that is routed to the workers. +SET citus.enable_metadata_sync TO OFF; CREATE PROCEDURE mx_call_proc_tx(x int) LANGUAGE plpgsql AS $$ BEGIN INSERT INTO multi_mx_call.mx_call_dist_table_1 VALUES (x, -1), (x+1, 4); @@ -189,6 +190,7 @@ BEGIN -- Now do the final update! UPDATE multi_mx_call.mx_call_dist_table_1 SET val = val-1 WHERE id >= x; END;$$; +RESET citus.enable_metadata_sync; -- before distribution ... CALL multi_mx_call.mx_call_proc_tx(10); diff --git a/src/test/regress/sql/multi_mx_create_table.sql b/src/test/regress/sql/multi_mx_create_table.sql index 4aa5a0164..0a685e5ce 100644 --- a/src/test/regress/sql/multi_mx_create_table.sql +++ b/src/test/regress/sql/multi_mx_create_table.sql @@ -63,33 +63,7 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell'); CREATE TABLE citus_local_table(a int); SELECT citus_add_local_table_to_metadata('citus_local_table'); --- create UDFs in worker node -CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - SET search_path TO citus_mx_test_schema; -CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - -CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; - -- create operator CREATE OPERATOR citus_mx_test_schema.=== ( LEFTARG = int, @@ -103,32 +77,7 @@ CREATE OPERATOR citus_mx_test_schema.=== ( -- now create required stuff in the worker 2 \c - - - :worker_2_port --- create UDF -CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - SET search_path TO citus_mx_test_schema; -CREATE OR REPLACE FUNCTION simpleTestFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - -CREATE FUNCTION public.immutable_append_mx(old_values int[], new_value int) -RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; -- create operator CREATE OPERATOR citus_mx_test_schema.=== ( diff --git a/src/test/regress/sql/multi_mx_explain.sql b/src/test/regress/sql/multi_mx_explain.sql index 9417b94f3..347d79d67 100644 --- a/src/test/regress/sql/multi_mx_explain.sql +++ b/src/test/regress/sql/multi_mx_explain.sql @@ -18,6 +18,7 @@ VACUUM ANALYZE supplier_mx; \c - - - :worker_1_port -- Function that parses explain output as JSON +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION explain_json(query text) RETURNS jsonb AS $BODY$ @@ -43,6 +44,7 @@ $BODY$ LANGUAGE plpgsql; \c - - - :worker_2_port -- Function that parses explain output as JSON +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION explain_json(query text) RETURNS jsonb AS $BODY$ @@ -65,6 +67,7 @@ BEGIN RETURN result; END; $BODY$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; -- Test Text format diff --git a/src/test/regress/sql/multi_mx_metadata.sql b/src/test/regress/sql/multi_mx_metadata.sql index 7bd09b900..169f3aa8b 100644 --- a/src/test/regress/sql/multi_mx_metadata.sql +++ b/src/test/regress/sql/multi_mx_metadata.sql @@ -218,17 +218,6 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='distributed_mx \c - no_access_mx - :worker_1_port --- see the comment in the top of the file -CREATE OR REPLACE FUNCTION raise_failed_aclcheck(query text) RETURNS void AS $$ -BEGIN - EXECUTE query; - EXCEPTION WHEN OTHERS THEN - IF SQLERRM LIKE 'must be owner of%' THEN - RAISE 'must be owner of the object'; - END IF; -END; -$$LANGUAGE plpgsql; - SELECT raise_failed_aclcheck($$ DROP TABLE distributed_mx_table; $$); diff --git a/src/test/regress/sql/multi_mx_modifications.sql b/src/test/regress/sql/multi_mx_modifications.sql index 72ad17b65..852bf3a42 100644 --- a/src/test/regress/sql/multi_mx_modifications.sql +++ b/src/test/regress/sql/multi_mx_modifications.sql @@ -202,6 +202,7 @@ UPDATE limit_orders_mx SET array_of_values = 1 || array_of_values WHERE id = 246 UPDATE limit_orders_mx SET array_of_values = immutable_append_mx(array_of_values, 2) WHERE id = 246; +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION stable_append_mx(old_values int[], new_value int) RETURNS int[] AS $$ BEGIN RETURN old_values || new_value; END; $$ LANGUAGE plpgsql STABLE; @@ -215,6 +216,8 @@ SELECT array_of_values FROM limit_orders_mx WHERE id = 246; -- STRICT functions work as expected CREATE FUNCTION temp_strict_func(integer,integer) RETURNS integer AS 'SELECT COALESCE($1, 2) + COALESCE($1, 3);' LANGUAGE SQL STABLE STRICT; +RESET citus.enable_metadata_sync; + UPDATE limit_orders_mx SET bidder_id = temp_strict_func(1, null) WHERE id = 246; SELECT array_of_values FROM limit_orders_mx WHERE id = 246; diff --git a/src/test/regress/sql/multi_mx_modifying_xacts.sql b/src/test/regress/sql/multi_mx_modifying_xacts.sql index 2c7d65b7c..cf60f023d 100644 --- a/src/test/regress/sql/multi_mx_modifying_xacts.sql +++ b/src/test/regress/sql/multi_mx_modifying_xacts.sql @@ -195,6 +195,7 @@ SELECT * FROM objects_mx WHERE id = 1; -- create trigger on one worker to reject certain values \c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION reject_bad_mx() RETURNS trigger AS $rb$ BEGIN IF (NEW.name = 'BAD') THEN @@ -204,6 +205,7 @@ CREATE FUNCTION reject_bad_mx() RETURNS trigger AS $rb$ RETURN NEW; END; $rb$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE CONSTRAINT TRIGGER reject_bad_mx AFTER INSERT ON objects_mx_1220103 diff --git a/src/test/regress/sql/multi_mx_node_metadata.sql b/src/test/regress/sql/multi_mx_node_metadata.sql index 09e8e49f1..a54917f87 100644 --- a/src/test/regress/sql/multi_mx_node_metadata.sql +++ b/src/test/regress/sql/multi_mx_node_metadata.sql @@ -367,11 +367,12 @@ UPDATE pg_dist_node SET hasmetadata = true; SELECT master_update_node(nodeid, 'localhost', 12345) FROM pg_dist_node; +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION trigger_metadata_sync() RETURNS void LANGUAGE C STRICT AS 'citus'; - +RESET citus.enable_metadata_sync; SELECT trigger_metadata_sync(); \c :datname - - :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 a2663c7a4..779f4c24e 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql @@ -68,14 +68,7 @@ CREATE TABLE repartition_udt_other ( \c - - - :worker_1_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -87,15 +80,6 @@ CREATE OPERATOR = ( -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - - -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 @@ -112,14 +96,7 @@ FUNCTION 1 test_udt_hash(test_udt); \c - - - :worker_2_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -131,15 +108,6 @@ CREATE OPERATOR = ( -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - - -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 diff --git a/src/test/regress/sql/multi_mx_router_planner.sql b/src/test/regress/sql/multi_mx_router_planner.sql index e8efbe244..fdfd81b07 100644 --- a/src/test/regress/sql/multi_mx_router_planner.sql +++ b/src/test/regress/sql/multi_mx_router_planner.sql @@ -609,6 +609,7 @@ PREPARE author_articles(int) as EXECUTE author_articles(1); -- queries inside plpgsql functions could be router plannable +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION author_articles_max_id() RETURNS int AS $$ DECLARE max_id integer; @@ -635,6 +636,7 @@ END; $$ LANGUAGE plpgsql; SELECT * FROM author_articles_id_word_count(); +RESET citus.enable_metadata_sync; -- materialized views can be created for router plannable queries CREATE MATERIALIZED VIEW mv_articles_hash_mx AS diff --git a/src/test/regress/sql/multi_name_lengths.sql b/src/test/regress/sql/multi_name_lengths.sql index df0a68f33..34a509fad 100644 --- a/src/test/regress/sql/multi_name_lengths.sql +++ b/src/test/regress/sql/multi_name_lengths.sql @@ -7,6 +7,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 225000; SET citus.shard_count TO 2; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -24,6 +25,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; -- Verify that a table name > 56 characters gets hashed properly. CREATE TABLE too_long_12345678901234567890123456789012345678901234567890 ( diff --git a/src/test/regress/sql/multi_prepare_sql.sql b/src/test/regress/sql/multi_prepare_sql.sql index 47fad0021..54893d025 100644 --- a/src/test/regress/sql/multi_prepare_sql.sql +++ b/src/test/regress/sql/multi_prepare_sql.sql @@ -610,10 +610,6 @@ DROP TABLE http_request; -- first create helper function CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$; -\c - - - :worker_1_port -CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$; -\c - - - :worker_2_port -CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IMMUTABLE AS $$BEGIN RAISE NOTICE '%', $1;RETURN 1;END$$; \c - - - :master_port -- test table diff --git a/src/test/regress/sql/multi_repartition_udt.sql b/src/test/regress/sql/multi_repartition_udt.sql index c244a2b61..2e5c1de24 100644 --- a/src/test/regress/sql/multi_repartition_udt.sql +++ b/src/test/regress/sql/multi_repartition_udt.sql @@ -69,14 +69,7 @@ CREATE TABLE repartition_udt_other ( \c - - :public_worker_1_host :worker_1_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -88,15 +81,6 @@ CREATE OPERATOR = ( -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - - -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 @@ -113,14 +97,7 @@ FUNCTION 1 test_udt_hash(test_udt); \c - - :public_worker_2_host :worker_2_port -- START type creation --- ... 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;' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - --- ... use that function to create a custom equality operator... +-- Use function to create a custom equality operator... CREATE OPERATOR = ( LEFTARG = test_udt, RIGHTARG = test_udt, @@ -132,15 +109,6 @@ CREATE OPERATOR = ( -- ... and create a custom operator family for hash indexes... CREATE OPERATOR FAMILY tudt_op_fam USING hash; --- ... create a test HASH function. Though it is a poor hash function, --- it is acceptable for our tests -CREATE FUNCTION test_udt_hash(test_udt) RETURNS int -AS 'SELECT hashtext( ($1.i + $1.i2)::text);' -LANGUAGE SQL -IMMUTABLE -RETURNS NULL ON NULL INPUT; - - -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 diff --git a/src/test/regress/sql/multi_repartitioned_subquery_udf.sql b/src/test/regress/sql/multi_repartitioned_subquery_udf.sql index ed432b797..6708c0ae1 100644 --- a/src/test/regress/sql/multi_repartitioned_subquery_udf.sql +++ b/src/test/regress/sql/multi_repartitioned_subquery_udf.sql @@ -18,32 +18,6 @@ LANGUAGE sql IMMUTABLE AS $_$ OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub; $_$; -\c - - :public_worker_1_host :worker_1_port -DROP FUNCTION IF EXISTS median(double precision[]); - -CREATE FUNCTION median(double precision[]) RETURNS double precision -LANGUAGE sql IMMUTABLE AS $_$ - SELECT AVG(val) FROM - (SELECT val FROM unnest($1) val - ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2) - OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub; -$_$; - -\c - - :public_worker_2_host :worker_2_port -DROP FUNCTION IF EXISTS median(double precision[]); - -CREATE FUNCTION median(double precision[]) RETURNS double precision -LANGUAGE sql IMMUTABLE AS $_$ - SELECT AVG(val) FROM - (SELECT val FROM unnest($1) val - ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2) - OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub; -$_$; - --- Run query on master -\c - - :master_host :master_port - - SELECT * FROM (SELECT median(ARRAY[1,2,sum(l_suppkey)]) as median, count(*) FROM lineitem GROUP BY l_partkey) AS a WHERE median > 2; diff --git a/src/test/regress/sql/multi_router_planner_fast_path.sql b/src/test/regress/sql/multi_router_planner_fast_path.sql index 1bf416826..01c9035d5 100644 --- a/src/test/regress/sql/multi_router_planner_fast_path.sql +++ b/src/test/regress/sql/multi_router_planner_fast_path.sql @@ -531,6 +531,7 @@ SELECT word_count, rank() OVER (PARTITION BY author_id ORDER BY word_count) WHERE author_id = 1 and 1=0; -- create a dummy function to be used in filtering +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION someDummyFunction(regclass) RETURNS text AS $$ @@ -554,6 +555,7 @@ BEGIN END IF; END; $$LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; SET client_min_messages TO ERROR; \set VERBOSITY terse diff --git a/src/test/regress/sql/multi_schema_support.sql b/src/test/regress/sql/multi_schema_support.sql index fdb922102..02ca41d01 100644 --- a/src/test/regress/sql/multi_schema_support.sql +++ b/src/test/regress/sql/multi_schema_support.sql @@ -203,34 +203,6 @@ END; $$ LANGUAGE 'plpgsql' IMMUTABLE; --- create UDF in worker node 1 -\c - - - :worker_1_port -CREATE OR REPLACE FUNCTION dummyFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - --- create UDF in worker node 2 -\c - - - :worker_2_port -CREATE OR REPLACE FUNCTION dummyFunction(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - -\c - - - :master_port - -- UDF in public, table in a schema other than public, search_path is not set SELECT dummyFunction(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1; @@ -251,36 +223,6 @@ END; $$ LANGUAGE 'plpgsql' IMMUTABLE; --- create UDF in worker node 1 in schema -\c - - - :worker_1_port -SET search_path TO test_schema_support; -CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - --- create UDF in worker node 2 in schema -\c - - - :worker_2_port -SET search_path TO test_schema_support; -CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer) - RETURNS text AS -$$ -DECLARE - strresult text; -BEGIN - RETURN theValue * 3 / 2 + 1; -END; -$$ -LANGUAGE 'plpgsql' IMMUTABLE; - -\c - - - :master_port - -- UDF in schema, table in a schema other than public, search_path is not set SET search_path TO public; SELECT test_schema_support.dummyFunction2(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1; diff --git a/src/test/regress/sql/multi_subquery_behavioral_analytics.sql b/src/test/regress/sql/multi_subquery_behavioral_analytics.sql index 3e8953ead..f7860ce58 100644 --- a/src/test/regress/sql/multi_subquery_behavioral_analytics.sql +++ b/src/test/regress/sql/multi_subquery_behavioral_analytics.sql @@ -1278,17 +1278,8 @@ ORDER BY LIMIT 10; -- this is one complex join query derived from a user's production query --- first declare the function on workers on master +-- declare the function on workers and master -- With array_index: -SELECT * FROM run_command_on_workers('CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) - RETURNS INT AS $$ - SELECT i - FROM (SELECT generate_series(array_lower($1, 1), array_upper($1, 1))) g(i) - WHERE $1 [i] = $2 - LIMIT 1; - $$ LANGUAGE sql') -ORDER BY 1,2; - CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) RETURNS INT AS $$ SELECT i @@ -1431,8 +1422,6 @@ LIMIT 10; -- drop created functions -SELECT * FROM run_command_on_workers('DROP FUNCTION array_index(ANYARRAY, ANYELEMENT)') -ORDER BY 1,2; DROP FUNCTION array_index(ANYARRAY, ANYELEMENT); -- a query with a constant subquery @@ -1629,16 +1618,6 @@ CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool IMMUTABLE RETURNS NULL ON NULL INPUT; -SELECT run_command_on_workers($f$ - -CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool - AS 'select $1 > $2;' - LANGUAGE SQL - IMMUTABLE - RETURNS NULL ON NULL INPUT; - -$f$); - -- we don't support joins via functions SELECT user_id, array_length(events_table, 1) FROM ( @@ -1869,11 +1848,5 @@ LIMIT 1; DROP FUNCTION test_join_function_2(integer, integer); -SELECT run_command_on_workers($f$ - - DROP FUNCTION test_join_function_2(integer, integer); - -$f$); - SET citus.enable_router_execution TO TRUE; SET citus.subquery_pushdown to OFF; diff --git a/src/test/regress/sql/multi_test_helpers_superuser.sql b/src/test/regress/sql/multi_test_helpers_superuser.sql index 2df7b1724..85900ee00 100644 --- a/src/test/regress/sql/multi_test_helpers_superuser.sql +++ b/src/test/regress/sql/multi_test_helpers_superuser.sql @@ -37,6 +37,7 @@ $$; -- partition_task_list_results tests the internal PartitionTasklistResults function +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.partition_task_list_results(resultIdPrefix text, query text, target_table regclass, @@ -49,7 +50,6 @@ CREATE OR REPLACE FUNCTION pg_catalog.partition_task_list_results(resultIdPrefix LANGUAGE C STRICT VOLATILE AS 'citus', $$partition_task_list_results$$; - -- get size of TopTransactionContext CREATE OR REPLACE FUNCTION top_transaction_context_size() RETURNS BIGINT LANGUAGE C STRICT VOLATILE @@ -65,3 +65,4 @@ BEGIN PERFORM public.wait_until_metadata_sync(30000); END; $function$; +RESET citus.enable_metadata_sync; diff --git a/src/test/regress/sql/multi_transactional_drop_shards.sql b/src/test/regress/sql/multi_transactional_drop_shards.sql index f88839d77..25c69031e 100644 --- a/src/test/regress/sql/multi_transactional_drop_shards.sql +++ b/src/test/regress/sql/multi_transactional_drop_shards.sql @@ -168,11 +168,13 @@ SET citus.override_table_visibility TO false; -- test DROP table with failing worker +SET citus.enable_metadata_sync TO OFF; CREATE FUNCTION fail_drop_table() RETURNS event_trigger AS $fdt$ BEGIN RAISE 'illegal value'; END; $fdt$ LANGUAGE plpgsql; +RESET citus.enable_metadata_sync; CREATE EVENT TRIGGER fail_drop_table ON sql_drop EXECUTE PROCEDURE fail_drop_table(); diff --git a/src/test/regress/sql/multi_unsupported_worker_operations.sql b/src/test/regress/sql/multi_unsupported_worker_operations.sql index 02e5397cc..5fac64e3d 100644 --- a/src/test/regress/sql/multi_unsupported_worker_operations.sql +++ b/src/test/regress/sql/multi_unsupported_worker_operations.sql @@ -51,6 +51,7 @@ SELECT * FROM mx_table ORDER BY col_1; \c - - - :worker_1_port -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -68,6 +69,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; CREATE TABLE mx_table_worker(col_1 text); diff --git a/src/test/regress/sql/propagate_extension_commands.sql b/src/test/regress/sql/propagate_extension_commands.sql index 6356d71e3..3640e2f17 100644 --- a/src/test/regress/sql/propagate_extension_commands.sql +++ b/src/test/regress/sql/propagate_extension_commands.sql @@ -116,13 +116,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init ); CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template); COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers'; -SELECT run_command_on_workers($$ -CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT; -$$); - -SELECT run_command_on_workers($$ -CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT; -$$); SELECT run_command_on_workers($$ CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init ); diff --git a/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql b/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql index 0587ebfaa..a3a5c94c8 100644 --- a/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql +++ b/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql @@ -32,12 +32,13 @@ CREATE TABLE numbers(a int); SELECT create_reference_table('numbers'); INSERT INTO numbers VALUES (20), (21); +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION my_volatile_fn() RETURNS INT AS $$ BEGIN RETURN 1; END; $$ language plpgsql VOLATILE; - +RESET citus.enable_metadata_sync; -- INSERT ... SELECT between reference tables BEGIN; diff --git a/src/test/regress/sql/shard_move_deferred_delete.sql b/src/test/regress/sql/shard_move_deferred_delete.sql index a052590d5..9dc4f82e3 100644 --- a/src/test/regress/sql/shard_move_deferred_delete.sql +++ b/src/test/regress/sql/shard_move_deferred_delete.sql @@ -80,17 +80,39 @@ $cmd$); SELECT master_move_shard_placement(20000000, 'localhost', :worker_2_port, 'localhost', :worker_1_port); -SELECT run_command_on_workers($cmd$ - -- override the function for testing purpose - create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) - as $BODY$ - begin - select 20 into available_disk_size; - select 8500 into total_disk_size; - end - $BODY$ language plpgsql; -$cmd$); +-- override the function for testing purpose +-- since it is extension owned function, propagate it to workers manually +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 20 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; +\c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 20 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; + +\c - - - :worker_2_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 20 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; + +\c - - - :master_port + +SET search_path TO shard_move_deferred_delete; SELECT citus_shard_cost_by_disk_size(20000001); @@ -108,16 +130,39 @@ SELECT run_command_on_workers($cmd$ SELECT count(*) FROM pg_class WHERE relname = 't1_20000000'; $cmd$); -SELECT run_command_on_workers($cmd$ - -- override the function for testing purpose - create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) - as $BODY$ - begin - select 8300 into available_disk_size; - select 8500 into total_disk_size; - end - $BODY$ language plpgsql; -$cmd$); +-- override the function for testing purpose +-- since it is extension owned function, propagate it to workers manually +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 8300 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; + +\c - - - :worker_1_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 8300 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; + +\c - - - :worker_2_port +SET citus.enable_metadata_sync TO OFF; +create or replace function pg_catalog.citus_local_disk_space_stats(OUT available_disk_size bigint, OUT total_disk_size bigint) +as $BODY$ +begin + select 8300 into available_disk_size; + select 8500 into total_disk_size; +end +$BODY$ language plpgsql; + +\c - - - :master_port + +SET search_path TO shard_move_deferred_delete; -- When there would not be enough free space left after the move, the move should fail SELECT master_move_shard_placement(20000001, 'localhost', :worker_2_port, 'localhost', :worker_1_port); diff --git a/src/test/regress/sql/shard_rebalancer.sql b/src/test/regress/sql/shard_rebalancer.sql index c9bad1983..8a9ae16d9 100644 --- a/src/test/regress/sql/shard_rebalancer.sql +++ b/src/test/regress/sql/shard_rebalancer.sql @@ -137,6 +137,7 @@ AS 'citus' LANGUAGE C STRICT VOLATILE; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -154,6 +155,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; SET citus.next_shard_id TO 123000; diff --git a/src/test/regress/sql/upgrade_basic_before.sql b/src/test/regress/sql/upgrade_basic_before.sql index 3b236cca0..7d518e510 100644 --- a/src/test/regress/sql/upgrade_basic_before.sql +++ b/src/test/regress/sql/upgrade_basic_before.sql @@ -34,6 +34,7 @@ CREATE TABLE tr(pk int, a int REFERENCES r(a) ON DELETE CASCADE ON UPDATE CASCAD SELECT create_distributed_table('tr', 'pk'); INSERT INTO tr SELECT c, c FROM generate_series(1, 5) as c; -- this function is dropped in Citus10, added here for tests +SET citus.enable_metadata_sync TO OFF; CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass, distribution_column text, distribution_method citus.distribution_type) @@ -50,6 +51,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex RETURNS void AS 'citus', $$master_create_worker_shards$$ LANGUAGE C STRICT; +RESET citus.enable_metadata_sync; CREATE TABLE t_range(id int, value_1 int); SELECT create_distributed_table('t_range', 'id', 'range'); SELECT master_create_empty_shard('t_range') as shardid1 \gset