diff --git a/src/backend/distributed/commands/aggregate.c b/src/backend/distributed/commands/aggregate.c new file mode 100644 index 000000000..4c25f41bf --- /dev/null +++ b/src/backend/distributed/commands/aggregate.c @@ -0,0 +1,107 @@ +/*------------------------------------------------------------------------- + * + * aggregate.c + * Commands for distributing AGGREGATE statements. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/commands.h" +#include "distributed/commands/utility_hook.h" +#include "distributed/deparser.h" +#include "distributed/listutils.h" +#include "distributed/metadata/dependency.h" +#include "distributed/metadata_sync.h" +#include "distributed/metadata/distobject.h" +#include "distributed/multi_executor.h" +#include "nodes/parsenodes.h" +#include "utils/lsyscache.h" + + +/* + * PostprocessDefineAggregateStmt actually creates the plan we need to execute for + * aggregate propagation. + * This is the downside of using the locally created aggregate to get the sql statement. + * + * If the aggregate depends on any non-distributed relation, Citus can not distribute it. + * In order to not to prevent users from creating local aggregates on the coordinator, + * a WARNING message will be sent to the user about the case instead of erroring out. + * + * Besides creating the plan we also make sure all (new) dependencies of the aggregate + * are created on all nodes. + */ +List * +PostprocessDefineAggregateStmt(Node *node, const char *queryString) +{ + QualifyTreeNode((Node *) node); + + DefineStmt *stmt = castNode(DefineStmt, node); + + if (!ShouldPropagate()) + { + return NIL; + } + + if (!ShouldPropagateCreateInCoordinatedTransction()) + { + return NIL; + } + + ObjectAddress address = GetObjectAddressFromParseTree((Node *) stmt, false); + + EnsureCoordinator(); + + EnsureSequentialMode(OBJECT_AGGREGATE); + + ObjectAddress *undistributableDependency = GetUndistributableDependency( + &address); + 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 *aggRangeVar = makeRangeVarFromNameList(stmt->defnames); + char *aggName = aggRangeVar->relname; + char *dependentRelationName = + get_rel_name(undistributableDependency->objectId); + + ereport(WARNING, (errmsg("Citus can't distribute aggregate \"%s\" having " + "dependency on non-distributed relation \"%s\"", + aggName, dependentRelationName), + errdetail("Aggregate will be created only locally"), + errhint("To distribute aggregate, distribute dependent " + "relations first. Then, re-create the aggregate"))); + } + 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("Aggregate will be created only locally"))); + } + + return NIL; + } + + EnsureDependenciesExistOnAllNodes(&address); + + List *commands = CreateFunctionDDLCommandsIdempotent(&address); + + commands = lcons(DISABLE_DDL_PROPAGATION, commands); + commands = lappend(commands, ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(NON_COORDINATOR_NODES, commands); +} diff --git a/src/backend/distributed/commands/citus_global_signal.c b/src/backend/distributed/commands/citus_global_signal.c index fc7618159..64bb67f0d 100644 --- a/src/backend/distributed/commands/citus_global_signal.c +++ b/src/backend/distributed/commands/citus_global_signal.c @@ -87,10 +87,11 @@ CitusSignalBackend(uint64 globalPID, uint64 timeout, int sig) } #endif - int nodeId = ExtractNodeIdFromGlobalPID(globalPID); + bool missingOk = false; + int nodeId = ExtractNodeIdFromGlobalPID(globalPID, missingOk); int processId = ExtractProcessIdFromGlobalPID(globalPID); - WorkerNode *workerNode = FindNodeWithNodeId(nodeId); + WorkerNode *workerNode = FindNodeWithNodeId(nodeId, missingOk); StringInfo cancelQuery = makeStringInfo(); diff --git a/src/backend/distributed/commands/collation.c b/src/backend/distributed/commands/collation.c index 7f047ec1d..12bf1404a 100644 --- a/src/backend/distributed/commands/collation.c +++ b/src/backend/distributed/commands/collation.c @@ -530,11 +530,14 @@ PreprocessDefineCollationStmt(Node *node, const char *queryString, { Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION); - if (ShouldPropagateDefineCollationStmt()) + if (!ShouldPropagateDefineCollationStmt()) { - EnsureCoordinator(); + return NIL; } + EnsureCoordinator(); + EnsureSequentialMode(OBJECT_COLLATION); + return NIL; } @@ -575,8 +578,7 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString) * 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 + * - create statement should be propagated according the the ddl propagation policy */ static bool ShouldPropagateDefineCollationStmt() @@ -586,8 +588,7 @@ ShouldPropagateDefineCollationStmt() return false; } - if (IsMultiStatementTransaction() && - MultiShardConnectionType != SEQUENTIAL_CONNECTION) + if (!ShouldPropagateCreateInCoordinatedTransction()) { return false; } diff --git a/src/backend/distributed/commands/dependencies.c b/src/backend/distributed/commands/dependencies.c index 84e875602..fe6e651fa 100644 --- a/src/backend/distributed/commands/dependencies.c +++ b/src/backend/distributed/commands/dependencies.c @@ -20,6 +20,8 @@ #include "distributed/metadata/dependency.h" #include "distributed/metadata/distobject.h" #include "distributed/metadata_sync.h" +#include "distributed/multi_executor.h" +#include "distributed/relation_access_tracking.h" #include "distributed/remote_commands.h" #include "distributed/worker_manager.h" #include "distributed/worker_transaction.h" @@ -247,7 +249,8 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency) * The commands will be added to both shards and metadata tables via the table * creation commands. */ - if (relKind == RELKIND_INDEX) + if (relKind == RELKIND_INDEX || + relKind == RELKIND_PARTITIONED_INDEX) { return NIL; } @@ -458,6 +461,88 @@ ShouldPropagate(void) } +/* + * ShouldPropagateCreateInCoordinatedTransction returns based the current state of the + * session and policies if Citus needs to propagate the creation of new objects. + * + * Creation of objects on other nodes could be postponed till the object is actually used + * in a sharded object (eg. distributed table or index on a distributed table). In certain + * use cases the opportunity for parallelism in a transaction block is preferred. When + * configured like that the creation of an object might be postponed and backfilled till + * the object is actually used. + */ +bool +ShouldPropagateCreateInCoordinatedTransction() +{ + if (!IsMultiStatementTransaction()) + { + /* + * If we are in a single statement transaction we will always propagate the + * creation of objects. There are no downsides in regard to performance or + * transactional limitations. These only arise with transaction blocks consisting + * of multiple statements. + */ + return true; + } + + if (MultiShardConnectionType == SEQUENTIAL_CONNECTION) + { + /* + * If we are in a transaction that is already switched to sequential, either by + * the user, or automatically by an other command, we will always propagate the + * creation of new objects to the workers. + * + * This guarantees no strange anomalies when the transaction aborts or on + * visibility of the newly created object. + */ + return true; + } + + switch (CreateObjectPropagationMode) + { + case CREATE_OBJECT_PROPAGATION_DEFERRED: + { + /* + * We prefer parallelism at this point. Since we did not already return while + * checking for sequential mode we are still in parallel mode. We don't want + * to switch that now, thus not propagating the creation. + */ + return false; + } + + case CREATE_OBJECT_PROPAGATION_AUTOMATIC: + { + /* + * When we run in optimistic mode we want to switch to sequential mode, only + * if this would _not_ give an error to the user. Meaning, we either are + * already in sequential mode (checked earlier), or there has been no parallel + * execution in the current transaction block. + * + * If switching to sequential would throw an error we would stay in parallel + * mode while creating new objects. We will rely on Citus' mechanism to ensure + * the existence if the object would be used in the same transaction. + */ + if (ParallelQueryExecutedInTransaction()) + { + return false; + } + + return true; + } + + case CREATE_OBJECT_PROPAGATION_IMMEDIATE: + { + return true; + } + + default: + { + elog(ERROR, "unsupported ddl propagation mode"); + } + } +} + + /* * ShouldPropagateObject determines if we should be propagating DDLs based * on their object address. diff --git a/src/backend/distributed/commands/distribute_object_ops.c b/src/backend/distributed/commands/distribute_object_ops.c index f1aa1fa1e..80a2b6628 100644 --- a/src/backend/distributed/commands/distribute_object_ops.c +++ b/src/backend/distributed/commands/distribute_object_ops.c @@ -43,11 +43,11 @@ static DistributeObjectOps Aggregate_AlterOwner = { }; static DistributeObjectOps Aggregate_Define = { .deparse = NULL, - .qualify = NULL, + .qualify = QualifyDefineAggregateStmt, .preprocess = NULL, - .postprocess = NULL, + .postprocess = PostprocessDefineAggregateStmt, .address = DefineAggregateStmtObjectAddress, - .markDistributed = false, + .markDistributed = true, }; static DistributeObjectOps Aggregate_Drop = { .deparse = DeparseDropFunctionStmt, diff --git a/src/backend/distributed/commands/extension.c b/src/backend/distributed/commands/extension.c index dc1363a65..3aa782c06 100644 --- a/src/backend/distributed/commands/extension.c +++ b/src/backend/distributed/commands/extension.c @@ -136,11 +136,8 @@ PostprocessCreateExtensionStmt(Node *node, const char *queryString) return NIL; } - /* - * If the extension command is a part of a multi-statement transaction, - * do not propagate it - */ - if (IsMultiStatementTransaction()) + /* check creation against multi-statement transaction policy */ + if (!ShouldPropagateCreateInCoordinatedTransction()) { return NIL; } @@ -148,16 +145,6 @@ PostprocessCreateExtensionStmt(Node *node, const char *queryString) /* extension management can only be done via coordinator node */ EnsureCoordinator(); - /* - * Make sure that no new nodes are added after this point until the end of the - * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the - * ExclusiveLock taken by citus_add_node. - * This guarantees that all active nodes will have the extension, because they will - * either get it now, or get it in citus_add_node after this transaction finishes and - * the pg_dist_object record becomes visible. - */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - /* * Make sure that the current transaction is already in sequential mode, * or can still safely be put in sequential mode @@ -259,16 +246,6 @@ PreprocessDropExtensionStmt(Node *node, const char *queryString, /* extension management can only be done via coordinator node */ EnsureCoordinator(); - /* - * Make sure that no new nodes are added after this point until the end of the - * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the - * ExclusiveLock taken by citus_add_node. - * This guarantees that all active nodes will drop the extension, because they will - * either get it now, or get it in citus_add_node after this transaction finishes and - * the pg_dist_object record becomes visible. - */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - /* * Make sure that the current transaction is already in sequential mode, * or can still safely be put in sequential mode @@ -395,15 +372,6 @@ PreprocessAlterExtensionSchemaStmt(Node *node, const char *queryString, /* extension management can only be done via coordinator node */ EnsureCoordinator(); - /* - * Make sure that no new nodes are added after this point until the end of the - * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the - * ExclusiveLock taken by citus_add_node. - * This guarantees that all active nodes will update the extension schema after - * this transaction finishes and the pg_dist_object record becomes visible. - */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - /* * Make sure that the current transaction is already in sequential mode, * or can still safely be put in sequential mode @@ -463,16 +431,6 @@ PreprocessAlterExtensionUpdateStmt(Node *node, const char *queryString, /* extension management can only be done via coordinator node */ EnsureCoordinator(); - /* - * Make sure that no new nodes are added after this point until the end of the - * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the - * ExclusiveLock taken by citus_add_node. - * This guarantees that all active nodes will update the extension version, because - * they will either get it now, or get it in citus_add_node after this transaction - * finishes and the pg_dist_object record becomes visible. - */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - /* * Make sure that the current transaction is already in sequential mode, * or can still safely be put in sequential mode @@ -555,6 +513,16 @@ MarkExistingObjectDependenciesDistributedIfSupported() ObjectAddress tableAddress = { 0 }; ObjectAddressSet(tableAddress, RelationRelationId, citusTableId); + if (ShouldSyncTableMetadata(citusTableId)) + { + /* we need to pass pointer allocated in the heap */ + ObjectAddress *addressPointer = palloc0(sizeof(ObjectAddress)); + *addressPointer = tableAddress; + + /* as of Citus 11, tables that should be synced are also considered object */ + resultingObjectAddresses = lappend(resultingObjectAddresses, addressPointer); + } + List *distributableDependencyObjectAddresses = GetDistributableDependenciesForObject(&tableAddress); @@ -578,11 +546,22 @@ MarkExistingObjectDependenciesDistributedIfSupported() /* remove duplicates from object addresses list for efficiency */ List *uniqueObjectAddresses = GetUniqueDependenciesList(resultingObjectAddresses); + /* + * We should sync the new dependencies during ALTER EXTENSION because + * we cannot know whether the nodes has already been upgraded or not. If + * the nodes are not upgraded at this point, we cannot sync the object. Also, + * when the workers upgraded, they'd get the same objects anyway. + */ + bool prevMetadataSyncValue = EnableMetadataSync; + SetLocalEnableMetadataSync(false); + ObjectAddress *objectAddress = NULL; foreach_ptr(objectAddress, uniqueObjectAddresses) { MarkObjectDistributed(objectAddress); } + + SetLocalEnableMetadataSync(prevMetadataSyncValue); } diff --git a/src/backend/distributed/commands/foreign_server.c b/src/backend/distributed/commands/foreign_server.c index ad1802ddb..0777814df 100644 --- a/src/backend/distributed/commands/foreign_server.c +++ b/src/backend/distributed/commands/foreign_server.c @@ -17,6 +17,7 @@ #include "distributed/listutils.h" #include "distributed/metadata/distobject.h" #include "distributed/metadata_sync.h" +#include "distributed/multi_executor.h" #include "distributed/worker_transaction.h" #include "foreign/foreign.h" #include "nodes/makefuncs.h" @@ -41,7 +42,14 @@ PreprocessCreateForeignServerStmt(Node *node, const char *queryString, return NIL; } + /* check creation against multi-statement transaction policy */ + if (!ShouldPropagateCreateInCoordinatedTransction()) + { + return NIL; + } + EnsureCoordinator(); + EnsureSequentialMode(OBJECT_FOREIGN_SERVER); char *sql = DeparseTreeNode(node); @@ -209,7 +217,18 @@ PreprocessDropForeignServerStmt(Node *node, const char *queryString, List * PostprocessCreateForeignServerStmt(Node *node, const char *queryString) { - bool missingOk = false; + if (!ShouldPropagate()) + { + return NIL; + } + + /* check creation against multi-statement transaction policy */ + if (!ShouldPropagateCreateInCoordinatedTransction()) + { + return NIL; + } + + const bool missingOk = false; ObjectAddress address = GetObjectAddressFromParseTree(node, missingOk); EnsureDependenciesExistOnAllNodes(&address); @@ -224,8 +243,14 @@ PostprocessCreateForeignServerStmt(Node *node, const char *queryString) List * PostprocessAlterForeignServerOwnerStmt(Node *node, const char *queryString) { - bool missingOk = false; + const bool missingOk = false; ObjectAddress address = GetObjectAddressFromParseTree(node, missingOk); + + if (!ShouldPropagateObject(&address)) + { + return NIL; + } + EnsureDependenciesExistOnAllNodes(&address); return NIL; diff --git a/src/backend/distributed/commands/function.c b/src/backend/distributed/commands/function.c index 80b870b62..46eb6a2a9 100644 --- a/src/backend/distributed/commands/function.c +++ b/src/backend/distributed/commands/function.c @@ -82,7 +82,6 @@ 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); @@ -1352,7 +1351,7 @@ PostprocessCreateFunctionStmt(Node *node, const char *queryString) * GetUndistributableDependency checks whether object has any non-distributable * dependency. If any one found, it will be returned. */ -static ObjectAddress * +ObjectAddress * GetUndistributableDependency(ObjectAddress *objectAddress) { List *dependencies = GetAllDependenciesForObject(objectAddress); @@ -1443,10 +1442,18 @@ DefineAggregateStmtObjectAddress(Node *node, bool missing_ok) ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs); objectWithArgs->objname = stmt->defnames; - FunctionParameter *funcParam = NULL; - foreach_ptr(funcParam, linitial(stmt->args)) + if (stmt->args != NIL) { - objectWithArgs->objargs = lappend(objectWithArgs->objargs, funcParam->argType); + FunctionParameter *funcParam = NULL; + foreach_ptr(funcParam, linitial(stmt->args)) + { + objectWithArgs->objargs = lappend(objectWithArgs->objargs, + funcParam->argType); + } + } + else + { + objectWithArgs->objargs = list_make1(makeTypeName("anyelement")); } return FunctionToObjectAddress(OBJECT_AGGREGATE, objectWithArgs, missing_ok); @@ -2019,10 +2026,10 @@ ShouldAddFunctionSignature(FunctionParameterMode mode) /* - * FunctionToObjectAddress returns the ObjectAddress of a Function or Procedure based on - * its type and ObjectWithArgs describing the Function/Procedure. If missing_ok is set to - * false an error will be raised by postgres explaining the Function/Procedure could not - * be found. + * FunctionToObjectAddress returns the ObjectAddress of a Function, Procedure or + * Aggregate based on its type and ObjectWithArgs describing the + * Function/Procedure/Aggregate. If missing_ok is set to false an error will be + * raised by postgres explaining the Function/Procedure could not be found. */ static ObjectAddress FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs, diff --git a/src/backend/distributed/commands/role.c b/src/backend/distributed/commands/role.c index c84899453..608dc0060 100644 --- a/src/backend/distributed/commands/role.c +++ b/src/backend/distributed/commands/role.c @@ -140,13 +140,6 @@ PostprocessAlterRoleStmt(Node *node, const char *queryString) AlterRoleStmt *stmt = castNode(AlterRoleStmt, node); - /* - * Make sure that no new nodes are added after this point until the end of the - * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the - * ExclusiveLock taken by citus_add_node. - */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - DefElem *option = NULL; foreach_ptr(option, stmt->options) { diff --git a/src/backend/distributed/commands/schema.c b/src/backend/distributed/commands/schema.c index 77f3fcc32..635e7adde 100644 --- a/src/backend/distributed/commands/schema.c +++ b/src/backend/distributed/commands/schema.c @@ -367,8 +367,8 @@ ShouldPropagateCreateSchemaStmt() return false; } - if (IsMultiStatementTransaction() && - MultiShardConnectionType != SEQUENTIAL_CONNECTION) + /* check creation against multi-statement transaction policy */ + if (!ShouldPropagateCreateInCoordinatedTransction()) { return false; } diff --git a/src/backend/distributed/commands/text_search.c b/src/backend/distributed/commands/text_search.c index be78057f7..53080c42b 100644 --- a/src/backend/distributed/commands/text_search.c +++ b/src/backend/distributed/commands/text_search.c @@ -73,16 +73,10 @@ PostprocessCreateTextSearchConfigurationStmt(Node *node, const char *queryString return NIL; } - /* - * If the create command is a part of a multi-statement transaction that is not in - * sequential mode, don't propagate. Instead we will rely on back filling. - */ - if (IsMultiStatementTransaction()) + /* check creation against multi-statement transaction policy */ + if (!ShouldPropagateCreateInCoordinatedTransction()) { - if (MultiShardConnectionType != SEQUENTIAL_CONNECTION) - { - return NIL; - } + return NIL; } EnsureCoordinator(); diff --git a/src/backend/distributed/commands/type.c b/src/backend/distributed/commands/type.c index eb59e8522..c124388d4 100644 --- a/src/backend/distributed/commands/type.c +++ b/src/backend/distributed/commands/type.c @@ -129,16 +129,6 @@ PreprocessCompositeTypeStmt(Node *node, const char *queryString, */ EnsureCoordinator(); - /* - * Make sure that no new nodes are added after this point until the end of the - * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the - * ExclusiveLock taken by citus_add_node. - * This guarantees that all active nodes will have the object, because they will - * either get it now, or get it in citus_add_node after this transaction finishes and - * the pg_dist_object record becomes visible. - */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - /* fully qualify before lookup and later deparsing */ QualifyTreeNode(node); @@ -1161,7 +1151,7 @@ ShouldPropagateTypeCreate() * this type will be used as a column in a table that will be created and distributed * in this same transaction. */ - if (IsMultiStatementTransaction()) + if (!ShouldPropagateCreateInCoordinatedTransction()) { return false; } diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index cf9012dd5..91e02a8ff 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -78,6 +78,7 @@ #include "utils/syscache.h" bool EnableDDLPropagation = true; /* ddl propagation is enabled */ +int CreateObjectPropagationMode = CREATE_OBJECT_PROPAGATION_DEFERRED; PropSetCmdBehavior PropagateSetCommands = PROPSETCMD_NONE; /* SET prop off */ static bool shouldInvalidateForeignKeyGraph = false; static int activeAlterTables = 0; @@ -258,7 +259,7 @@ multi_ProcessUtility(PlannedStmt *pstmt, else if (IsA(parsetree, DoStmt)) { /* - * All statements in a DO block are executed in a single transaciton, + * All statements in a DO block are executed in a single transaction, * so we need to keep track of whether we are inside a DO block. */ DoBlockLevel += 1; @@ -1563,7 +1564,8 @@ DDLTaskList(Oid relationId, const char *commandString) List * NodeDDLTaskList(TargetWorkerSet targets, List *commands) { - List *workerNodes = TargetWorkerSetNodeList(targets, NoLock); + /* don't allow concurrent node list changes that require an exclusive lock */ + List *workerNodes = TargetWorkerSetNodeList(targets, RowShareLock); if (list_length(workerNodes) <= 0) { diff --git a/src/backend/distributed/deparser/qualify_aggregate_stmts.c b/src/backend/distributed/deparser/qualify_aggregate_stmts.c new file mode 100644 index 000000000..9debc244a --- /dev/null +++ b/src/backend/distributed/deparser/qualify_aggregate_stmts.c @@ -0,0 +1,34 @@ +/*------------------------------------------------------------------------- + * + * qualify_aggregate_stmts.c + * Functions specialized in fully qualifying all aggregate statements. + * These functions are dispatched from qualify.c + * + * Fully qualifying aggregate statements consists of adding the schema name + * to the subject of the types as well as any other branch of the parsetree. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "catalog/namespace.h" +#include "distributed/deparser.h" +#include "nodes/makefuncs.h" +#include "utils/lsyscache.h" + +void +QualifyDefineAggregateStmt(Node *node) +{ + DefineStmt *stmt = castNode(DefineStmt, node); + + if (list_length(stmt->defnames) == 1) + { + char *objname = NULL; + Oid creationSchema = QualifiedNameGetCreationNamespace(stmt->defnames, &objname); + stmt->defnames = list_make2(makeString(get_namespace_name(creationSchema)), + linitial(stmt->defnames)); + } +} diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 9ff53cf37..1f04751bb 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -819,6 +819,9 @@ AdaptiveExecutor(CitusScanState *scanState) bool hasDependentJobs = HasDependentJobs(job); if (hasDependentJobs) { + /* jobs use intermediate results, which require a distributed transaction */ + UseCoordinatedTransaction(); + jobIdList = ExecuteDependentTasks(taskList, job); } @@ -828,9 +831,10 @@ AdaptiveExecutor(CitusScanState *scanState) targetPoolSize = 1; } + bool excludeFromXact = false; + TransactionProperties xactProperties = DecideTransactionPropertiesForTaskList( - distributedPlan->modLevel, taskList, - hasDependentJobs); + distributedPlan->modLevel, taskList, excludeFromXact); bool localExecutionSupported = true; DistributedExecution *execution = CreateDistributedExecution( @@ -873,11 +877,6 @@ AdaptiveExecutor(CitusScanState *scanState) FinishDistributedExecution(execution); - if (hasDependentJobs) - { - DoRepartitionCleanup(jobIdList); - } - if (SortReturning && distributedPlan->expectResults && commandType != CMD_SELECT) { SortTupleStore(scanState); @@ -963,6 +962,26 @@ ExecuteUtilityTaskListExtended(List *utilityTaskList, int poolSize, } +/* + * ExecuteTaskList is a proxy to ExecuteTaskListExtended + * with defaults for some of the arguments. + */ +uint64 +ExecuteTaskList(RowModifyLevel modLevel, List *taskList) +{ + bool localExecutionSupported = true; + ExecutionParams *executionParams = CreateBasicExecutionParams( + modLevel, taskList, MaxAdaptiveExecutorPoolSize, localExecutionSupported + ); + + bool excludeFromXact = false; + executionParams->xactProperties = DecideTransactionPropertiesForTaskList( + modLevel, taskList, excludeFromXact); + + return ExecuteTaskListExtended(executionParams); +} + + /* * ExecuteTaskListOutsideTransaction is a proxy to ExecuteTaskListExtended * with defaults for some of the arguments. @@ -2639,12 +2658,6 @@ RunDistributedExecution(DistributedExecution *execution) */ UnclaimAllSessionConnections(execution->sessionList); - /* do repartition cleanup if this is a repartition query*/ - if (list_length(execution->jobIdList) > 0) - { - DoRepartitionCleanup(execution->jobIdList); - } - if (execution->waitEventSet != NULL) { FreeWaitEventSet(execution->waitEventSet); diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 1c9800663..92bfbd715 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -150,8 +150,6 @@ RegisterCitusCustomScanMethods(void) static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags) { - MarkCitusInitiatedCoordinatorBackend(); - CitusScanState *scanState = (CitusScanState *) node; /* diff --git a/src/backend/distributed/executor/directed_acyclic_graph_execution.c b/src/backend/distributed/executor/directed_acyclic_graph_execution.c index 2245db420..73ae0b591 100644 --- a/src/backend/distributed/executor/directed_acyclic_graph_execution.c +++ b/src/backend/distributed/executor/directed_acyclic_graph_execution.c @@ -39,6 +39,7 @@ static HTAB * CreateTaskHashTable(void); static bool IsAllDependencyCompleted(Task *task, HTAB *completedTasks); static void AddCompletedTasks(List *curCompletedTasks, HTAB *completedTasks); static List * FindExecutableTasks(List *allTasks, HTAB *completedTasks); +static List * RemoveMergeTasks(List *taskList); static int TaskHashCompare(const void *key1, const void *key2, Size keysize); static uint32 TaskHash(const void *key, Size keysize); static bool IsTaskAlreadyCompleted(Task *task, HTAB *completedTasks); @@ -64,8 +65,13 @@ ExecuteTasksInDependencyOrder(List *allTasks, List *excludedTasks, List *jobIds) { break; } - ExecuteTaskListOutsideTransaction(ROW_MODIFY_NONE, curTasks, - MaxAdaptiveExecutorPoolSize, jobIds); + + /* merge tasks do not need to be executed */ + List *executableTasks = RemoveMergeTasks(curTasks); + if (list_length(executableTasks) > 0) + { + ExecuteTaskList(ROW_MODIFY_NONE, executableTasks); + } AddCompletedTasks(curTasks, completedTasks); curTasks = NIL; @@ -97,6 +103,29 @@ FindExecutableTasks(List *allTasks, HTAB *completedTasks) } +/* + * RemoveMergeTasks returns a copy of taskList that excludes all the + * merge tasks. We do this because merge tasks are currently only a + * logical concept that does not need to be executed. + */ +static List * +RemoveMergeTasks(List *taskList) +{ + List *prunedTaskList = NIL; + Task *task = NULL; + + foreach_ptr(task, taskList) + { + if (task->taskType != MERGE_TASK) + { + prunedTaskList = lappend(prunedTaskList, task); + } + } + + return prunedTaskList; +} + + /* * AddCompletedTasks adds the givens tasks to completedTasks HTAB. */ diff --git a/src/backend/distributed/executor/distributed_intermediate_results.c b/src/backend/distributed/executor/distributed_intermediate_results.c index 8a29e633d..f7d62e157 100644 --- a/src/backend/distributed/executor/distributed_intermediate_results.c +++ b/src/backend/distributed/executor/distributed_intermediate_results.c @@ -59,28 +59,6 @@ typedef struct PartitioningTupleDest } PartitioningTupleDest; -/* - * NodePair contains the source and destination node in a NodeToNodeFragmentsTransfer. - * It is a separate struct to use it as a key in a hash table. - */ -typedef struct NodePair -{ - uint32 sourceNodeId; - uint32 targetNodeId; -} NodePair; - - -/* - * NodeToNodeFragmentsTransfer contains all fragments that need to be fetched from - * the source node to the destination node in the NodePair. - */ -typedef struct NodeToNodeFragmentsTransfer -{ - NodePair nodes; - List *fragmentList; -} NodeToNodeFragmentsTransfer; - - /* forward declarations of local functions */ static List * WrapTasksForPartitioning(const char *resultIdPrefix, List *selectTaskList, @@ -98,9 +76,6 @@ static TupleDesc PartitioningTupleDestTupleDescForQuery(TupleDestination *self, queryNumber); static ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int datumCount, Oid typeId); -static void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount, - Oid intervalTypeId, ArrayType **minValueArray, - ArrayType **maxValueArray); static char * SourceShardPrefix(const char *resultPrefix, uint64 shardId); static DistributedResultFragment * TupleToDistributedResultFragment(HeapTuple heapTuple, TupleDesc tupleDesc, @@ -115,8 +90,6 @@ static List ** ColocateFragmentsWithRelation(List *fragmentList, static List * ColocationTransfers(List *fragmentList, CitusTableCacheEntry *targetRelation); static List * FragmentTransferTaskList(List *fragmentListTransfers); -static char * QueryStringForFragmentsTransfer( - NodeToNodeFragmentsTransfer *fragmentsTransfer); static void ExecuteFetchTaskList(List *fetchTaskList); @@ -360,7 +333,7 @@ SourceShardPrefix(const char *resultPrefix, uint64 shardId) * ShardMinMaxValueArrays returns min values and max values of given shard * intervals. Returned arrays are text arrays. */ -static void +void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount, Oid intervalTypeOutFunc, ArrayType **minValueArray, ArrayType **maxValueArray) @@ -632,7 +605,7 @@ FragmentTransferTaskList(List *fragmentListTransfers) * result fragments from source node to target node. See the structure of * NodeToNodeFragmentsTransfer for details of how these are decided. */ -static char * +char * QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer) { StringInfo queryString = makeStringInfo(); @@ -667,7 +640,7 @@ QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer) quote_literal_cstr(sourceNode->workerName), sourceNode->workerPort); - ereport(DEBUG3, (errmsg("fetch task on %s:%d: %s", sourceNode->workerName, + ereport(DEBUG4, (errmsg("fetch task on %s:%d: %s", sourceNode->workerName, sourceNode->workerPort, queryString->data))); return queryString->data; diff --git a/src/backend/distributed/executor/intermediate_results.c b/src/backend/distributed/executor/intermediate_results.c index 3442e23a3..20c95fe06 100644 --- a/src/backend/distributed/executor/intermediate_results.c +++ b/src/backend/distributed/executor/intermediate_results.c @@ -17,6 +17,7 @@ #include "pgstat.h" #include "catalog/pg_enum.h" +#include "catalog/pg_type.h" #include "commands/copy.h" #include "distributed/commands/multi_copy.h" #include "distributed/connection_management.h" @@ -916,6 +917,8 @@ fetch_intermediate_results(PG_FUNCTION_ARGS) StringInfo beginAndSetXactId = BeginAndSetDistributedTransactionIdCommand(); ExecuteCriticalRemoteCommand(connection, beginAndSetXactId->data); + CreateIntermediateResultsDirectory(); + for (resultIndex = 0; resultIndex < resultCount; resultIndex++) { char *resultId = TextDatumGetCString(resultIdArray[resultIndex]); @@ -938,6 +941,19 @@ fetch_intermediate_results(PG_FUNCTION_ARGS) static uint64 FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId) { + char *localPath = QueryResultFileName(resultId); + + struct stat fileStat; + int statOK = stat(localPath, &fileStat); + if (statOK == 0) + { + /* + * File exists, most likely because we are trying to fetch a + * a file from a node to itself. Skip doing work. + */ + return fileStat.st_size; + } + uint64 totalBytesWritten = 0; StringInfo copyCommand = makeStringInfo(); @@ -948,8 +964,6 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId) int socket = PQsocket(pgConn); bool raiseErrors = true; - CreateIntermediateResultsDirectory(); - appendStringInfo(copyCommand, "COPY \"%s\" TO STDOUT WITH (format result)", resultId); @@ -966,7 +980,6 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId) PQclear(result); - char *localPath = QueryResultFileName(resultId); File fileDesc = FileOpenForTransmit(localPath, fileFlags, fileMode); FileCompat fileCompat = FileCompatFromFileStart(fileDesc); diff --git a/src/backend/distributed/executor/local_executor.c b/src/backend/distributed/executor/local_executor.c index 26bf12ba0..7c0426c0a 100644 --- a/src/backend/distributed/executor/local_executor.c +++ b/src/backend/distributed/executor/local_executor.c @@ -915,6 +915,25 @@ TaskAccessesLocalNode(Task *task) } +/* + * EnsureCompatibleLocalExecutionState makes sure that the tasks won't have + * any visibility problems because of local execution. + */ +void +EnsureCompatibleLocalExecutionState(List *taskList) +{ + /* + * We have LOCAL_EXECUTION_REQUIRED check here to avoid unnecessarily + * iterating the task list in AnyTaskAccessesLocalNode. + */ + if (GetCurrentLocalExecutionStatus() == LOCAL_EXECUTION_REQUIRED && + AnyTaskAccessesLocalNode(taskList)) + { + ErrorIfTransactionAccessedPlacementsLocally(); + } +} + + /* * ErrorIfTransactionAccessedPlacementsLocally errors out if a local query * on any shard has already been executed in the same transaction. diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 28498e0f2..a47dc6a48 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -748,6 +748,11 @@ GetObjectTypeString(ObjectType objType) { switch (objType) { + case OBJECT_AGGREGATE: + { + return "aggregate"; + } + case OBJECT_COLLATION: { return "collation"; @@ -763,6 +768,11 @@ GetObjectTypeString(ObjectType objType) return "extension"; } + case OBJECT_FOREIGN_SERVER: + { + return "foreign server"; + } + case OBJECT_FUNCTION: { return "function"; diff --git a/src/backend/distributed/executor/partitioned_intermediate_results.c b/src/backend/distributed/executor/partitioned_intermediate_results.c index 8feb31a95..129a7d130 100644 --- a/src/backend/distributed/executor/partitioned_intermediate_results.c +++ b/src/backend/distributed/executor/partitioned_intermediate_results.c @@ -86,6 +86,9 @@ typedef struct PartitionedResultDestReceiver /* keeping track of which partitionDestReceivers have been started */ Bitmapset *startedDestReceivers; + + /* whether NULL partition column values are allowed */ + bool allowNullPartitionColumnValues; } PartitionedResultDestReceiver; static Portal StartPortalForQueryExecution(const char *queryString); @@ -99,7 +102,8 @@ static DestReceiver * CreatePartitionedResultDestReceiver(int partitionColumnInd shardSearchInfo, DestReceiver ** partitionedDestReceivers, - bool lazyStartup); + bool lazyStartup, + bool allowNullPartitionValues); static void PartitionedResultDestReceiverStartup(DestReceiver *dest, int operation, TupleDesc inputTupleDescriptor); static bool PartitionedResultDestReceiverReceive(TupleTableSlot *slot, @@ -148,6 +152,8 @@ worker_partition_query_result(PG_FUNCTION_ARGS) int32 maxValuesCount = ArrayObjectCount(maxValuesArray); bool binaryCopy = PG_GETARG_BOOL(6); + bool allowNullPartitionColumnValues = PG_GETARG_BOOL(7); + bool generateEmptyResults = PG_GETARG_BOOL(8); if (!IsMultiStatementTransaction()) { @@ -226,13 +232,21 @@ worker_partition_query_result(PG_FUNCTION_ARGS) dests[partitionIndex] = partitionDest; } - const bool lazyStartup = true; + /* + * If we are asked to generated empty results, use non-lazy startup. + * + * The rStartup of the FileDestReceiver will be called for all partitions + * and generate empty files, which may still have binary header/footer. + */ + const bool lazyStartup = !generateEmptyResults; + DestReceiver *dest = CreatePartitionedResultDestReceiver( partitionColumnIndex, partitionCount, shardSearchInfo, dests, - lazyStartup); + lazyStartup, + allowNullPartitionColumnValues); /* execute the query */ PortalRun(portal, FETCH_ALL, false, true, dest, dest, NULL); @@ -390,7 +404,8 @@ CreatePartitionedResultDestReceiver(int partitionColumnIndex, int partitionCount, CitusTableCacheEntry *shardSearchInfo, DestReceiver **partitionedDestReceivers, - bool lazyStartup) + bool lazyStartup, + bool allowNullPartitionColumnValues) { PartitionedResultDestReceiver *resultDest = palloc0(sizeof(PartitionedResultDestReceiver)); @@ -409,6 +424,7 @@ CreatePartitionedResultDestReceiver(int partitionColumnIndex, resultDest->partitionDestReceivers = partitionedDestReceivers; resultDest->startedDestReceivers = NULL; resultDest->lazyStartup = lazyStartup; + resultDest->allowNullPartitionColumnValues = allowNullPartitionColumnValues; return (DestReceiver *) resultDest; } @@ -458,23 +474,40 @@ PartitionedResultDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest) Datum *columnValues = slot->tts_values; bool *columnNulls = slot->tts_isnull; + int partitionIndex; + if (columnNulls[self->partitionColumnIndex]) { - ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), - errmsg("the partition column value cannot be NULL"))); + if (self->allowNullPartitionColumnValues) + { + /* + * NULL values go into the first partition for both hash- and range- + * partitioning, since that is the only way to guarantee that there is + * always a partition for NULL and that it is always the same partition. + */ + partitionIndex = 0; + } + else + { + ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), + errmsg("the partition column value cannot be NULL"))); + } } - - Datum partitionColumnValue = columnValues[self->partitionColumnIndex]; - ShardInterval *shardInterval = FindShardInterval(partitionColumnValue, - self->shardSearchInfo); - if (shardInterval == NULL) + else { - ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - errmsg("could not find shard for partition column " - "value"))); + Datum partitionColumnValue = columnValues[self->partitionColumnIndex]; + ShardInterval *shardInterval = FindShardInterval(partitionColumnValue, + self->shardSearchInfo); + if (shardInterval == NULL) + { + ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("could not find shard for partition column " + "value"))); + } + + partitionIndex = shardInterval->shardIndex; } - int partitionIndex = shardInterval->shardIndex; DestReceiver *partitionDest = self->partitionDestReceivers[partitionIndex]; /* check if this partitionDestReceiver has been started before, start if not */ diff --git a/src/backend/distributed/executor/repartition_join_execution.c b/src/backend/distributed/executor/repartition_join_execution.c index dfe8efd9e..29d994e59 100644 --- a/src/backend/distributed/executor/repartition_join_execution.c +++ b/src/backend/distributed/executor/repartition_join_execution.c @@ -44,12 +44,8 @@ #include "distributed/worker_transaction.h" -static List * CreateTemporarySchemasForMergeTasks(Job *topLevelJob); static List * ExtractJobsInJobTree(Job *job); static void TraverseJobTree(Job *curJob, List **jobs); -static char * GenerateCreateSchemasCommand(List *jobIds, char *schemaOwner); -static char * GenerateJobCommands(List *jobIds, char *templateCommand); -static char * GenerateDeleteJobsCommand(List *jobIds); /* @@ -60,13 +56,8 @@ static char * GenerateDeleteJobsCommand(List *jobIds); List * ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob) { - EnsureNoModificationsHaveBeenDone(); - List *allTasks = CreateTaskListForJobTree(topLevelTasks); - - EnsureCompatibleLocalExecutionState(allTasks); - - List *jobIds = CreateTemporarySchemasForMergeTasks(topLevelJob); + List *jobIds = ExtractJobsInJobTree(topLevelJob); ExecuteTasksInDependencyOrder(allTasks, topLevelTasks, jobIds); @@ -74,40 +65,6 @@ ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob) } -/* - * EnsureCompatibleLocalExecutionState makes sure that the tasks won't have - * any visibility problems because of local execution. - */ -void -EnsureCompatibleLocalExecutionState(List *taskList) -{ - /* - * We have LOCAL_EXECUTION_REQUIRED check here to avoid unnecessarily - * iterating the task list in AnyTaskAccessesLocalNode. - */ - if (GetCurrentLocalExecutionStatus() == LOCAL_EXECUTION_REQUIRED && - AnyTaskAccessesLocalNode(taskList)) - { - ErrorIfTransactionAccessedPlacementsLocally(); - } -} - - -/* - * CreateTemporarySchemasForMergeTasks creates the necessary schemas that will be used - * later in each worker. Single transaction is used to create the schemas. - */ -static List * -CreateTemporarySchemasForMergeTasks(Job *topLeveLJob) -{ - List *jobIds = ExtractJobsInJobTree(topLeveLJob); - char *createSchemasCommand = GenerateCreateSchemasCommand(jobIds, CurrentUserName()); - SendCommandToWorkersInParallel(ALL_SHARD_NODES, createSchemasCommand, - CitusExtensionOwnerName()); - return jobIds; -} - - /* * ExtractJobsInJobTree returns all job ids in the job tree * where the given job is root. @@ -139,67 +96,3 @@ TraverseJobTree(Job *curJob, List **jobIds) TraverseJobTree(childJob, jobIds); } } - - -/* - * GenerateCreateSchemasCommand returns concatanated create schema commands. - */ -static char * -GenerateCreateSchemasCommand(List *jobIds, char *ownerName) -{ - StringInfo createSchemaCommand = makeStringInfo(); - - uint64 *jobIdPointer = NULL; - foreach_ptr(jobIdPointer, jobIds) - { - uint64 jobId = *jobIdPointer; - appendStringInfo(createSchemaCommand, WORKER_CREATE_SCHEMA_QUERY, - jobId, quote_literal_cstr(ownerName)); - } - return createSchemaCommand->data; -} - - -/* - * GenerateJobCommands returns concatenated commands with the given template - * command for each job id from the given job ids. The returned command is - * exactly list_length(jobIds) subcommands. - * E.g create_schema(jobId1); create_schema(jobId2); ... - * This way we can send the command in just one latency to a worker. - */ -static char * -GenerateJobCommands(List *jobIds, char *templateCommand) -{ - StringInfo createSchemaCommand = makeStringInfo(); - - uint64 *jobIdPointer = NULL; - foreach_ptr(jobIdPointer, jobIds) - { - uint64 jobId = *jobIdPointer; - appendStringInfo(createSchemaCommand, templateCommand, jobId); - } - return createSchemaCommand->data; -} - - -/* - * DoRepartitionCleanup removes the temporary job directories and schemas that are - * used for repartition queries for the given job ids. - */ -void -DoRepartitionCleanup(List *jobIds) -{ - SendCommandToWorkersOptionalInParallel(ALL_SHARD_NODES, GenerateDeleteJobsCommand( - jobIds), - CitusExtensionOwnerName()); -} - - -/* - * GenerateDeleteJobsCommand returns concatanated remove job dir commands. - */ -static char * -GenerateDeleteJobsCommand(List *jobIds) -{ - return GenerateJobCommands(jobIds, WORKER_REPARTITION_CLEANUP_QUERY); -} diff --git a/src/backend/distributed/metadata/dependency.c b/src/backend/distributed/metadata/dependency.c index 2d1f67ad3..4b3595575 100644 --- a/src/backend/distributed/metadata/dependency.c +++ b/src/backend/distributed/metadata/dependency.c @@ -122,6 +122,7 @@ typedef struct ViewDependencyNode static List * GetRelationSequenceDependencyList(Oid relationId); +static List * GetRelationFunctionDependencyList(Oid relationId); static List * GetRelationTriggerFunctionDependencyList(Oid relationId); static List * GetRelationStatsSchemaDependencyList(Oid relationId); static List * GetRelationIndicesDependencyList(Oid relationId); @@ -722,7 +723,8 @@ SupportedDependencyByCitus(const ObjectAddress *address) relKind == RELKIND_PARTITIONED_TABLE || relKind == RELKIND_FOREIGN_TABLE || relKind == RELKIND_SEQUENCE || - relKind == RELKIND_INDEX) + relKind == RELKIND_INDEX || + relKind == RELKIND_PARTITIONED_INDEX) { return true; } @@ -1090,9 +1092,15 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe * with them. */ List *sequenceDependencyList = GetRelationSequenceDependencyList(relationId); - result = list_concat(result, sequenceDependencyList); + /* + * Get the dependent functions for tables as columns has default values + * and contraints, then expand dependency list with them. + */ + List *functionDependencyList = GetRelationFunctionDependencyList(relationId); + result = list_concat(result, functionDependencyList); + /* * Tables could have indexes. Indexes themself could have dependencies that * need to be propagated. eg. TEXT SEARCH CONFIGRUATIONS. Here we add the @@ -1133,6 +1141,21 @@ GetRelationSequenceDependencyList(Oid relationId) } +/* + * GetRelationFunctionDependencyList returns the function dependency definition + * list for the given relation. + */ +static List * +GetRelationFunctionDependencyList(Oid relationId) +{ + List *dependentFunctionOids = GetDependentFunctionsWithRelation(relationId); + List *functionDependencyDefList = + CreateObjectAddressDependencyDefList(ProcedureRelationId, dependentFunctionOids); + + return functionDependencyDefList; +} + + /* * GetRelationStatsSchemaDependencyList returns a list of DependencyDefinition * objects for the schemas that statistics' of the relation with relationId depends. diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 490eeb4ca..ef7e64ec1 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -28,9 +28,11 @@ #include "catalog/indexing.h" #include "catalog/pg_am.h" #include "catalog/pg_attrdef.h" +#include "catalog/pg_constraint.h" #include "catalog/pg_depend.h" #include "catalog/pg_foreign_server.h" #include "catalog/pg_namespace.h" +#include "catalog/pg_proc.h" #include "catalog/pg_type.h" #include "commands/async.h" #include "distributed/argutils.h" @@ -85,6 +87,7 @@ char *EnableManualMetadataChangesForUser = ""; static void EnsureObjectMetadataIsSane(int distributionArgumentIndex, int colocationId); +static List * GetFunctionDependenciesForObjects(ObjectAddress *objectAddress); static char * SchemaOwnerName(Oid objectId); static bool HasMetadataWorkers(void); static void CreateShellTableOnWorkers(Oid relationId); @@ -104,7 +107,6 @@ static List * GetObjectsForGrantStmt(ObjectType objectType, Oid objectId); static AccessPriv * GetAccessPrivObjectForGrantStmt(char *permission); static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid, AclItem *aclItem); -static void SetLocalEnableMetadataSync(bool state); static void SetLocalReplicateReferenceTablesOnActivate(bool state); static char * GenerateSetRoleQuery(Oid roleOid); static void MetadataSyncSigTermHandler(SIGNAL_ARGS); @@ -1544,6 +1546,119 @@ GetSequencesFromAttrDef(Oid attrdefOid) } +/* + * GetDependentFunctionsWithRelation returns the dependent functions for the + * given relation id. + */ +List * +GetDependentFunctionsWithRelation(Oid relationId) +{ + List *referencingObjects = NIL; + List *functionOids = NIL; + ScanKeyData key[2]; + HeapTuple tup; + + Relation depRel = table_open(DependRelationId, AccessShareLock); + + ScanKeyInit(&key[0], + Anum_pg_depend_refclassid, + BTEqualStrategyNumber, F_OIDEQ, + ObjectIdGetDatum(RelationRelationId)); + ScanKeyInit(&key[1], + Anum_pg_depend_refobjid, + BTEqualStrategyNumber, F_OIDEQ, + ObjectIdGetDatum(relationId)); + + SysScanDesc scan = systable_beginscan(depRel, DependReferenceIndexId, true, + NULL, 2, key); + + while (HeapTupleIsValid(tup = systable_getnext(scan))) + { + Form_pg_depend deprec = (Form_pg_depend) GETSTRUCT(tup); + + /* + * objsubid is nonzero only for table columns and zero for anything else. + * Since we are trying to find a dependency from the column of a table to + * function we've added deprec->refobjsubid != 0 check. + * + * We are following DEPENDENCY_AUTO for dependencies via column and + * DEPENDENCY_NORMAL anything else. Since only procedure dependencies + * for those dependencies will be obtained in GetFunctionDependenciesForObjects + * following both dependency types are not harmful. + */ + if ((deprec->refobjsubid != 0 && deprec->deptype == DEPENDENCY_AUTO) || + deprec->deptype == DEPENDENCY_NORMAL) + { + ObjectAddress *refAddress = palloc(sizeof(ObjectAddress)); + ObjectAddressSubSet(*refAddress, deprec->classid, + deprec->objid, + deprec->objsubid); + referencingObjects = lappend(referencingObjects, refAddress); + } + } + + systable_endscan(scan); + + table_close(depRel, AccessShareLock); + + ObjectAddress *referencingObject = NULL; + foreach_ptr(referencingObject, referencingObjects) + { + functionOids = list_concat(functionOids, + GetFunctionDependenciesForObjects(referencingObject)); + } + + return functionOids; +} + + +/* + * GetFunctionDependenciesForObjects returns a list of function OIDs that have + * dependency with the given object + */ +static List * +GetFunctionDependenciesForObjects(ObjectAddress *objectAddress) +{ + List *functionOids = NIL; + ScanKeyData key[3]; + HeapTuple tup; + + Relation depRel = table_open(DependRelationId, AccessShareLock); + + ScanKeyInit(&key[0], + Anum_pg_depend_classid, + BTEqualStrategyNumber, F_OIDEQ, + ObjectIdGetDatum(objectAddress->classId)); + ScanKeyInit(&key[1], + Anum_pg_depend_objid, + BTEqualStrategyNumber, F_OIDEQ, + ObjectIdGetDatum(objectAddress->objectId)); + ScanKeyInit(&key[2], + Anum_pg_depend_objsubid, + BTEqualStrategyNumber, F_OIDEQ, + ObjectIdGetDatum(objectAddress->objectSubId)); + + SysScanDesc scan = systable_beginscan(depRel, DependDependerIndexId, true, + NULL, 3, key); + + while (HeapTupleIsValid(tup = systable_getnext(scan))) + { + Form_pg_depend deprec = (Form_pg_depend) GETSTRUCT(tup); + + if (deprec->refclassid == ProcedureRelationId) + { + functionOids = lappend_oid(functionOids, deprec->refobjid); + } + } + + systable_endscan(scan); + + table_close(depRel, AccessShareLock); + + return functionOids; +} + + /* * SequenceDependencyCommandList generates commands to record the dependency * of sequences on tables on the worker. This dependency does not exist by @@ -1832,7 +1947,7 @@ GetAccessPrivObjectForGrantStmt(char *permission) /* * SetLocalEnableMetadataSync sets the enable_metadata_sync locally */ -static void +void SetLocalEnableMetadataSync(bool state) { set_config_option("citus.enable_metadata_sync", state == true ? "on" : "off", diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index 2c5a8b0fe..4ad3f9e48 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -1291,6 +1291,52 @@ NodeGroupHasShardPlacements(int32 groupId, bool onlyConsiderActivePlacements) } +/* + * IsActiveShardPlacement checks if the shard placement is labelled as + * active, and that it is placed in an active worker. + * Expects shard worker to not be NULL. + */ +bool +IsActiveShardPlacement(ShardPlacement *shardPlacement) +{ + WorkerNode *workerNode = + FindWorkerNode(shardPlacement->nodeName, shardPlacement->nodePort); + + if (!workerNode) + { + ereport(ERROR, (errmsg("There is a shard placement on node %s:%d but " + "could not find the node.", shardPlacement->nodeName, + shardPlacement->nodePort))); + } + + return shardPlacement->shardState == SHARD_STATE_ACTIVE && + workerNode->isActive; +} + + +/* + * FilterShardPlacementList filters a list of shard placements based on a filter. + * Keep only the shard for which the filter function returns true. + */ +List * +FilterShardPlacementList(List *shardPlacementList, bool (*filter)(ShardPlacement *)) +{ + List *filteredShardPlacementList = NIL; + ShardPlacement *shardPlacement = NULL; + + foreach_ptr(shardPlacement, shardPlacementList) + { + if (filter(shardPlacement)) + { + filteredShardPlacementList = lappend(filteredShardPlacementList, + shardPlacement); + } + } + + return filteredShardPlacementList; +} + + /* * ActiveShardPlacementListOnGroup returns a list of active shard placements * that are sitting on group with groupId for given shardId. @@ -1323,53 +1369,39 @@ ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId) List * ActiveShardPlacementList(uint64 shardId) { - List *activePlacementList = NIL; List *shardPlacementList = ShardPlacementListIncludingOrphanedPlacements(shardId); - ShardPlacement *shardPlacement = NULL; - foreach_ptr(shardPlacement, shardPlacementList) - { - WorkerNode *workerNode = - FindWorkerNode(shardPlacement->nodeName, shardPlacement->nodePort); - - /* - * We have already resolved the placement to node, so would have - * errored out earlier. - */ - Assert(workerNode != NULL); - - if (shardPlacement->shardState == SHARD_STATE_ACTIVE && - workerNode->isActive) - { - activePlacementList = lappend(activePlacementList, shardPlacement); - } - } + List *activePlacementList = FilterShardPlacementList(shardPlacementList, + IsActiveShardPlacement); return SortList(activePlacementList, CompareShardPlacementsByWorker); } +/* + * IsShardPlacementNotOrphaned checks returns true if a shard placement is not orphaned + * Orphaned shards are shards marked to be deleted at a later point (shardstate = 4). + */ +static inline bool +IsShardPlacementNotOrphaned(ShardPlacement *shardPlacement) +{ + return shardPlacement->shardState != SHARD_STATE_TO_DELETE; +} + + /* * ShardPlacementListWithoutOrphanedPlacements returns shard placements exluding - * the ones that are orphaned, because they are marked to be deleted at a later - * point (shardstate = 4). + * the ones that are orphaned. */ List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId) { - List *activePlacementList = NIL; List *shardPlacementList = ShardPlacementListIncludingOrphanedPlacements(shardId); - ShardPlacement *shardPlacement = NULL; - foreach_ptr(shardPlacement, shardPlacementList) - { - if (shardPlacement->shardState != SHARD_STATE_TO_DELETE) - { - activePlacementList = lappend(activePlacementList, shardPlacement); - } - } + List *activePlacementList = FilterShardPlacementList(shardPlacementList, + IsShardPlacementNotOrphaned); return SortList(activePlacementList, CompareShardPlacementsByWorker); } diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 706f000cb..498497e64 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -188,9 +188,6 @@ citus_set_coordinator_host(PG_FUNCTION_ARGS) Name nodeClusterName = PG_GETARG_NAME(3); nodeMetadata.nodeCluster = NameStr(*nodeClusterName); - /* prevent concurrent modification */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - bool isCoordinatorInMetadata = false; WorkerNode *coordinatorNode = PrimaryNodeForGroup(COORDINATOR_GROUP_ID, &isCoordinatorInMetadata); @@ -1544,7 +1541,7 @@ FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort) * If the node cannot be found this functions errors. */ WorkerNode * -FindNodeWithNodeId(int nodeId) +FindNodeWithNodeId(int nodeId, bool missingOk) { List *workerList = ActiveReadableNodeList(); WorkerNode *workerNode = NULL; @@ -1558,7 +1555,10 @@ FindNodeWithNodeId(int nodeId) } /* there isn't any node with nodeId in pg_dist_node */ - elog(ERROR, "worker node with node id %d could not be found", nodeId); + if (!missingOk) + { + elog(ERROR, "worker node with node id %d could not be found", nodeId); + } return NULL; } @@ -1780,12 +1780,6 @@ AddNodeMetadata(char *nodeName, int32 nodePort, *nodeAlreadyExists = false; - /* - * Prevent / wait for concurrent modification before checking whether - * the worker already exists in pg_dist_node. - */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - WorkerNode *workerNode = FindWorkerNodeAnyCluster(nodeName, nodePort); if (workerNode != NULL) { diff --git a/src/backend/distributed/operations/shard_rebalancer.c b/src/backend/distributed/operations/shard_rebalancer.c index d35427e6b..43dd167b0 100644 --- a/src/backend/distributed/operations/shard_rebalancer.c +++ b/src/backend/distributed/operations/shard_rebalancer.c @@ -34,6 +34,7 @@ #include "distributed/lock_graph.h" #include "distributed/coordinator_protocol.h" #include "distributed/metadata_cache.h" +#include "distributed/metadata_utility.h" #include "distributed/multi_client_executor.h" #include "distributed/multi_progress.h" #include "distributed/multi_server_executor.h" @@ -190,7 +191,7 @@ static void UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent, List *responsiveNodeList, Oid shardReplicationModeOid); /* static declarations for main logic's utility functions */ -static HTAB * ActivePlacementsHash(List *shardPlacementList); +static HTAB * ShardPlacementsListToHash(List *shardPlacementList); static bool PlacementsHashFind(HTAB *placementsHash, uint64 shardId, WorkerNode *workerNode); static void PlacementsHashEnter(HTAB *placementsHash, uint64 shardId, @@ -396,6 +397,7 @@ FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray) placement->shardId = groupPlacement->shardId; placement->shardLength = groupPlacement->shardLength; placement->shardState = groupPlacement->shardState; + placement->nodeId = worker->nodeId; placement->nodeName = pstrdup(worker->workerName); placement->nodePort = worker->workerPort; placement->placementId = groupPlacement->placementId; @@ -446,14 +448,17 @@ GetRebalanceSteps(RebalanceOptions *options) /* sort the lists to make the function more deterministic */ List *activeWorkerList = SortedActiveWorkers(); - List *shardPlacementListList = NIL; + List *activeShardPlacementListList = NIL; Oid relationId = InvalidOid; foreach_oid(relationId, options->relationIdList) { List *shardPlacementList = FullShardPlacementList(relationId, options->excludedShardArray); - shardPlacementListList = lappend(shardPlacementListList, shardPlacementList); + List *activeShardPlacementListForRelation = + FilterShardPlacementList(shardPlacementList, IsActiveShardPlacement); + activeShardPlacementListList = + lappend(activeShardPlacementListList, activeShardPlacementListForRelation); } if (options->threshold < options->rebalanceStrategy->minimumThreshold) @@ -471,7 +476,7 @@ GetRebalanceSteps(RebalanceOptions *options) } return RebalancePlacementUpdates(activeWorkerList, - shardPlacementListList, + activeShardPlacementListList, options->threshold, options->maxShardMoves, options->drainOnly, @@ -795,7 +800,6 @@ rebalance_table_shards(PG_FUNCTION_ARGS) { Oid relationId = PG_GETARG_OID(0); ErrorIfMoveUnsupportedTableType(relationId); - relationIdList = list_make1_oid(relationId); } else @@ -951,9 +955,11 @@ replicate_table_shards(PG_FUNCTION_ARGS) List *activeWorkerList = SortedActiveWorkers(); List *shardPlacementList = FullShardPlacementList(relationId, excludedShardArray); + List *activeShardPlacementList = FilterShardPlacementList(shardPlacementList, + IsActiveShardPlacement); List *placementUpdateList = ReplicationPlacementUpdates(activeWorkerList, - shardPlacementList, + activeShardPlacementList, shardReplicationFactor); placementUpdateList = list_truncate(placementUpdateList, maxShardCopies); @@ -1737,13 +1743,13 @@ ExecuteRebalancerCommandInSeparateTransaction(char *command) * which is placed in the source node but not in the target node as the shard to * move. * - * The shardPlacementListList argument contains a list of lists of shard + * The activeShardPlacementListList argument contains a list of lists of active shard * placements. Each of these lists are balanced independently. This is used to * make sure different colocation groups are balanced separately, so each list * contains the placements of a colocation group. */ List * -RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList, +RebalancePlacementUpdates(List *workerNodeList, List *activeShardPlacementListList, double threshold, int32 maxShardMoves, bool drainOnly, @@ -1755,7 +1761,7 @@ RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList, List *shardPlacementList = NIL; List *placementUpdateList = NIL; - foreach_ptr(shardPlacementList, shardPlacementListList) + foreach_ptr(shardPlacementList, activeShardPlacementListList) { state = InitRebalanceState(workerNodeList, shardPlacementList, functions); @@ -1861,7 +1867,7 @@ InitRebalanceState(List *workerNodeList, List *shardPlacementList, RebalanceState *state = palloc0(sizeof(RebalanceState)); state->functions = functions; - state->placementsHash = ActivePlacementsHash(shardPlacementList); + state->placementsHash = ShardPlacementsListToHash(shardPlacementList); /* create empty fill state for all of the worker nodes */ foreach_ptr(workerNode, workerNodeList) @@ -2413,29 +2419,25 @@ FindAndMoveShardCost(float4 utilizationLowerBound, /* * ReplicationPlacementUpdates returns a list of placement updates which * replicates shard placements that need re-replication. To do this, the - * function loops over the shard placements, and for each shard placement + * function loops over the active shard placements, and for each shard placement * which needs to be re-replicated, it chooses an active worker node with * smallest number of shards as the target node. */ List * -ReplicationPlacementUpdates(List *workerNodeList, List *shardPlacementList, +ReplicationPlacementUpdates(List *workerNodeList, List *activeShardPlacementList, int shardReplicationFactor) { List *placementUpdateList = NIL; ListCell *shardPlacementCell = NULL; uint32 workerNodeIndex = 0; - HTAB *placementsHash = ActivePlacementsHash(shardPlacementList); + HTAB *placementsHash = ShardPlacementsListToHash(activeShardPlacementList); uint32 workerNodeCount = list_length(workerNodeList); /* get number of shards per node */ uint32 *shardCountArray = palloc0(workerNodeCount * sizeof(uint32)); - foreach(shardPlacementCell, shardPlacementList) + foreach(shardPlacementCell, activeShardPlacementList) { ShardPlacement *placement = lfirst(shardPlacementCell); - if (placement->shardState != SHARD_STATE_ACTIVE) - { - continue; - } for (workerNodeIndex = 0; workerNodeIndex < workerNodeCount; workerNodeIndex++) { @@ -2449,7 +2451,7 @@ ReplicationPlacementUpdates(List *workerNodeList, List *shardPlacementList, } } - foreach(shardPlacementCell, shardPlacementList) + foreach(shardPlacementCell, activeShardPlacementList) { WorkerNode *sourceNode = NULL; WorkerNode *targetNode = NULL; @@ -2586,11 +2588,11 @@ ShardActivePlacementCount(HTAB *activePlacementsHash, uint64 shardId, /* - * ActivePlacementsHash creates and returns a hash set for the placements in - * the given list of shard placements which are in active state. + * ShardPlacementsListToHash creates and returns a hash set from a shard + * placement list. */ static HTAB * -ActivePlacementsHash(List *shardPlacementList) +ShardPlacementsListToHash(List *shardPlacementList) { ListCell *shardPlacementCell = NULL; HASHCTL info; @@ -2609,11 +2611,8 @@ ActivePlacementsHash(List *shardPlacementList) foreach(shardPlacementCell, shardPlacementList) { ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell); - if (shardPlacement->shardState == SHARD_STATE_ACTIVE) - { - void *hashKey = (void *) shardPlacement; - hash_search(shardPlacementsHash, hashKey, HASH_ENTER, NULL); - } + void *hashKey = (void *) shardPlacement; + hash_search(shardPlacementsHash, hashKey, HASH_ENTER, NULL); } return shardPlacementsHash; diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 28b750e8f..9f578eac9 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -40,6 +40,7 @@ #include "distributed/colocation_utils.h" #include "distributed/deparse_shard_query.h" #include "distributed/coordinator_protocol.h" +#include "distributed/intermediate_results.h" #include "distributed/metadata_cache.h" #include "distributed/multi_router_planner.h" #include "distributed/multi_join_order.h" @@ -52,15 +53,16 @@ #include "distributed/pg_dist_shard.h" #include "distributed/query_pushdown_planning.h" #include "distributed/query_utils.h" +#include "distributed/recursive_planning.h" #include "distributed/shardinterval_utils.h" #include "distributed/shard_pruning.h" #include "distributed/string_utils.h" - #include "distributed/worker_manager.h" #include "distributed/worker_protocol.h" #include "distributed/version_compat.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" +#include "nodes/print.h" #include "optimizer/clauses.h" #include "nodes/pathnodes.h" #include "optimizer/optimizer.h" @@ -157,8 +159,6 @@ static MapMergeJob * BuildMapMergeJob(Query *jobQuery, List *dependentJobList, Oid baseRelationId, BoundaryNodeJobType boundaryNodeJobType); static uint32 HashPartitionCount(void); -static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray, - uint32 shardIntervalCount); /* Local functions forward declarations for task list creation and helper functions */ static Job * BuildJobTreeTaskList(Job *jobTree, @@ -195,11 +195,11 @@ static bool JoinPrunable(RangeTableFragment *leftFragment, static ShardInterval * FragmentInterval(RangeTableFragment *fragment); static StringInfo FragmentIntervalString(ShardInterval *fragmentInterval); static List * DataFetchTaskList(uint64 jobId, uint32 taskIdIndex, List *fragmentList); -static StringInfo DatumArrayString(Datum *datumArray, uint32 datumCount, Oid datumTypeId); static List * BuildRelationShardList(List *rangeTableList, List *fragmentList); static void UpdateRangeTableAlias(List *rangeTableList, List *fragmentList); static Alias * FragmentAlias(RangeTblEntry *rangeTableEntry, RangeTableFragment *fragment); +static List * FetchTaskResultNameList(List *mapOutputFetchTaskList); static uint64 AnchorShardId(List *fragmentList, uint32 anchorRangeTableId); static List * PruneSqlTaskDependencies(List *sqlTaskList); static List * AssignTaskList(List *sqlTaskList); @@ -218,11 +218,13 @@ static void AssignDataFetchDependencies(List *taskList); static uint32 TaskListHighestTaskId(List *taskList); static List * MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList); static StringInfo CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, - uint32 partitionColumnIndex); + uint32 partitionColumnIndex, bool useBinaryFormat); +static char * PartitionResultNamePrefix(uint64 jobId, int32 taskId); +static char * PartitionResultName(uint64 jobId, uint32 taskId, uint32 partitionId); +static ShardInterval ** RangeIntervalArrayWithNullBucket(ShardInterval **intervalArray, + int intervalCount); static List * MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex); -static StringInfo ColumnNameArrayString(uint32 columnCount, uint64 generatingJobId); -static StringInfo ColumnTypeArrayString(List *targetEntryList); static List * FetchEqualityAttrNumsForRTEOpExpr(OpExpr *opExpr); static List * FetchEqualityAttrNumsForRTEBoolExpr(BoolExpr *boolExpr); @@ -853,10 +855,14 @@ TargetEntryList(List *expressionList) foreach(expressionCell, expressionList) { Expr *expression = (Expr *) lfirst(expressionCell); + int columnNumber = list_length(targetEntryList) + 1; + + StringInfo columnName = makeStringInfo(); + appendStringInfo(columnName, "column%d", columnNumber); + + TargetEntry *targetEntry = makeTargetEntry(expression, columnNumber, + columnName->data, false); - TargetEntry *targetEntry = makeTargetEntry(expression, - list_length(targetEntryList) + 1, - NULL, false); targetEntryList = lappend(targetEntryList, targetEntry); } @@ -2043,45 +2049,6 @@ HashPartitionCount(void) } -/* - * SplitPointObject walks over shard intervals in the given array, extracts each - * shard interval's minimum value, sorts and inserts these minimum values into a - * new array. This sorted array is then used by the MapMerge job. - */ -static ArrayType * -SplitPointObject(ShardInterval **shardIntervalArray, uint32 shardIntervalCount) -{ - Oid typeId = InvalidOid; - bool typeByValue = false; - char typeAlignment = 0; - int16 typeLength = 0; - - /* allocate an array for shard min values */ - uint32 minDatumCount = shardIntervalCount; - Datum *minDatumArray = palloc0(minDatumCount * sizeof(Datum)); - - for (uint32 intervalIndex = 0; intervalIndex < shardIntervalCount; intervalIndex++) - { - ShardInterval *shardInterval = shardIntervalArray[intervalIndex]; - minDatumArray[intervalIndex] = shardInterval->minValue; - Assert(shardInterval->minValueExists); - - /* resolve the datum type on the first pass */ - if (intervalIndex == 0) - { - typeId = shardInterval->valueTypeId; - } - } - - /* construct the split point object from the sorted array */ - get_typlenbyvalalign(typeId, &typeLength, &typeByValue, &typeAlignment); - ArrayType *splitPointObject = construct_array(minDatumArray, minDatumCount, typeId, - typeLength, typeByValue, typeAlignment); - - return splitPointObject; -} - - /* ------------------------------------------------------------ * Functions that relate to building and assigning tasks follow * ------------------------------------------------------------ @@ -4097,34 +4064,6 @@ DataFetchTaskList(uint64 jobId, uint32 taskIdIndex, List *fragmentList) } -/* Helper function to return a datum array's external string representation. */ -static StringInfo -DatumArrayString(Datum *datumArray, uint32 datumCount, Oid datumTypeId) -{ - int16 typeLength = 0; - bool typeByValue = false; - char typeAlignment = 0; - - /* construct the array object from the given array */ - get_typlenbyvalalign(datumTypeId, &typeLength, &typeByValue, &typeAlignment); - ArrayType *arrayObject = construct_array(datumArray, datumCount, datumTypeId, - typeLength, typeByValue, typeAlignment); - Datum arrayObjectDatum = PointerGetDatum(arrayObject); - - /* convert the array object to its string representation */ - FmgrInfo *arrayOutFunction = (FmgrInfo *) palloc0(sizeof(FmgrInfo)); - fmgr_info(F_ARRAY_OUT, arrayOutFunction); - - Datum arrayStringDatum = FunctionCall1(arrayOutFunction, arrayObjectDatum); - char *arrayString = DatumGetCString(arrayStringDatum); - - StringInfo arrayStringInfo = makeStringInfo(); - appendStringInfo(arrayStringInfo, "%s", arrayString); - - return arrayStringInfo; -} - - /* * CreateBasicTask creates a task, initializes fields that are common to each task, * and returns the created task. @@ -4234,19 +4173,26 @@ FragmentAlias(RangeTblEntry *rangeTableEntry, RangeTableFragment *fragment) else if (fragmentType == CITUS_RTE_REMOTE_QUERY) { Task *mergeTask = (Task *) fragment->fragmentReference; - uint64 jobId = mergeTask->jobId; - uint32 taskId = mergeTask->taskId; + List *mapOutputFetchTaskList = mergeTask->dependentTaskList; + List *resultNameList = FetchTaskResultNameList(mapOutputFetchTaskList); + List *mapJobTargetList = mergeTask->mapJobTargetList; - StringInfo jobSchemaName = JobSchemaName(jobId); - StringInfo taskTableName = TaskTableName(taskId); + /* determine whether all types have binary input/output functions */ + bool useBinaryFormat = CanUseBinaryCopyFormatForTargetList(mapJobTargetList); - StringInfo aliasNameString = makeStringInfo(); - appendStringInfo(aliasNameString, "%s.%s", - jobSchemaName->data, taskTableName->data); + /* generate the query on the intermediate result */ + Query *fragmentSetQuery = BuildReadIntermediateResultsArrayQuery(mapJobTargetList, + NIL, + resultNameList, + useBinaryFormat); - aliasName = aliasNameString->data; - fragmentName = taskTableName->data; - schemaName = jobSchemaName->data; + /* we only really care about the function RTE */ + RangeTblEntry *readIntermediateResultsRTE = linitial(fragmentSetQuery->rtable); + + /* crudely override the fragment RTE */ + *rangeTableEntry = *readIntermediateResultsRTE; + + return rangeTableEntry->alias; } /* @@ -4267,6 +4213,30 @@ FragmentAlias(RangeTblEntry *rangeTableEntry, RangeTableFragment *fragment) } +/* + * FetchTaskResultNameList builds a list of result names that reflect + * the output of map-fetch tasks. + */ +static List * +FetchTaskResultNameList(List *mapOutputFetchTaskList) +{ + List *resultNameList = NIL; + Task *mapOutputFetchTask = NULL; + + foreach_ptr(mapOutputFetchTask, mapOutputFetchTaskList) + { + Task *mapTask = linitial(mapOutputFetchTask->dependentTaskList); + int partitionId = mapOutputFetchTask->partitionId; + char *resultName = + PartitionResultName(mapTask->jobId, mapTask->taskId, partitionId); + + resultNameList = lappend(resultNameList, resultName); + } + + return resultNameList; +} + + /* * AnchorShardId walks over each fragment in the given fragment list, finds the * fragment that corresponds to the given anchor range tableId, and returns this @@ -4377,17 +4347,28 @@ MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList) filterQuery->targetList); } + /* determine whether all types have binary input/output functions */ + bool useBinaryFormat = CanUseBinaryCopyFormatForTargetList(filterQuery->targetList); + foreach(filterTaskCell, filterTaskList) { Task *filterTask = (Task *) lfirst(filterTaskCell); StringInfo mapQueryString = CreateMapQueryString(mapMergeJob, filterTask, - partitionColumnResNo); + partitionColumnResNo, + useBinaryFormat); /* convert filter query task into map task */ Task *mapTask = filterTask; SetTaskQueryString(mapTask, mapQueryString->data); mapTask->taskType = MAP_TASK; + /* + * We do not support fail-over in case of map tasks, since we would also + * have to fail over the corresponding merge tasks. We therefore truncate + * the list down to the first element. + */ + mapTask->taskPlacementList = list_truncate(mapTask->taskPlacementList, 1); + mapTaskList = lappend(mapTaskList, mapTask); } @@ -4428,21 +4409,19 @@ PartitionColumnIndex(Var *targetVar, List *targetList) */ static StringInfo CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, - uint32 partitionColumnIndex) + uint32 partitionColumnIndex, bool useBinaryFormat) { uint64 jobId = filterTask->jobId; uint32 taskId = filterTask->taskId; + char *resultNamePrefix = PartitionResultNamePrefix(jobId, taskId); /* wrap repartition query string around filter query string */ StringInfo mapQueryString = makeStringInfo(); char *filterQueryString = TaskQueryString(filterTask); - char *filterQueryEscapedText = quote_literal_cstr(filterQueryString); PartitionType partitionType = mapMergeJob->partitionType; Var *partitionColumn = mapMergeJob->partitionColumn; Oid partitionColumnType = partitionColumn->vartype; - char *partitionColumnTypeFullName = format_type_be_qualified(partitionColumnType); - int32 partitionColumnTypeMod = partitionColumn->vartypmod; ShardInterval **intervalArray = mapMergeJob->sortedShardIntervalArray; uint32 intervalCount = mapMergeJob->partitionCount; @@ -4450,38 +4429,101 @@ CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, if (partitionType == DUAL_HASH_PARTITION_TYPE) { partitionColumnType = INT4OID; - partitionColumnTypeMod = get_typmodin(INT4OID); intervalArray = GenerateSyntheticShardIntervalArray(intervalCount); } else if (partitionType == SINGLE_HASH_PARTITION_TYPE) { partitionColumnType = INT4OID; - partitionColumnTypeMod = get_typmodin(INT4OID); } - - ArrayType *splitPointObject = SplitPointObject(intervalArray, intervalCount); - StringInfo splitPointString = ArrayObjectToString(splitPointObject, - partitionColumnType, - partitionColumnTypeMod); - - char *partitionCommand = NULL; - if (partitionType == RANGE_PARTITION_TYPE) + else if (partitionType == RANGE_PARTITION_TYPE) { - partitionCommand = RANGE_PARTITION_COMMAND; - } - else - { - partitionCommand = HASH_PARTITION_COMMAND; + /* add a partition for NULL values at index 0 */ + intervalArray = RangeIntervalArrayWithNullBucket(intervalArray, intervalCount); + intervalCount++; } - char *partitionColumnIndextText = ConvertIntToString(partitionColumnIndex); - appendStringInfo(mapQueryString, partitionCommand, jobId, taskId, - filterQueryEscapedText, partitionColumnIndextText, - partitionColumnTypeFullName, splitPointString->data); + Oid intervalTypeOutFunc = InvalidOid; + bool intervalTypeVarlena = false; + ArrayType *minValueArray = NULL; + ArrayType *maxValueArray = NULL; + + getTypeOutputInfo(partitionColumnType, &intervalTypeOutFunc, &intervalTypeVarlena); + + ShardMinMaxValueArrays(intervalArray, intervalCount, intervalTypeOutFunc, + &minValueArray, &maxValueArray); + + StringInfo minValuesString = ArrayObjectToString(minValueArray, TEXTOID, + InvalidOid); + StringInfo maxValuesString = ArrayObjectToString(maxValueArray, TEXTOID, + InvalidOid); + + char *partitionMethodString = partitionType == RANGE_PARTITION_TYPE ? + "range" : "hash"; + + /* + * Non-partition columns can easily contain NULL values, so we allow NULL + * values in the column by which we re-partition. They will end up in the + * first partition. + */ + bool allowNullPartitionColumnValue = true; + + /* + * We currently generate empty results for each partition and fetch all of them. + */ + bool generateEmptyResults = true; + + appendStringInfo(mapQueryString, + "SELECT partition_index" + ", %s || '_' || partition_index::text " + ", rows_written " + "FROM pg_catalog.worker_partition_query_result" + "(%s,%s,%d,%s,%s,%s,%s,%s,%s) WHERE rows_written > 0", + quote_literal_cstr(resultNamePrefix), + quote_literal_cstr(resultNamePrefix), + quote_literal_cstr(filterQueryString), + partitionColumnIndex - 1, + quote_literal_cstr(partitionMethodString), + minValuesString->data, + maxValuesString->data, + useBinaryFormat ? "true" : "false", + allowNullPartitionColumnValue ? "true" : "false", + generateEmptyResults ? "true" : "false"); + return mapQueryString; } +/* + * PartitionResultNamePrefix returns the prefix we use for worker_partition_query_result + * results. Each result will have a _ suffix. + */ +static char * +PartitionResultNamePrefix(uint64 jobId, int32 taskId) +{ + StringInfo resultNamePrefix = makeStringInfo(); + + appendStringInfo(resultNamePrefix, "repartition_" UINT64_FORMAT "_%u", jobId, taskId); + + return resultNamePrefix->data; +} + + +/* + * PartitionResultName returns the name of a worker_partition_query_result result for + * a specific partition. + */ +static char * +PartitionResultName(uint64 jobId, uint32 taskId, uint32 partitionId) +{ + StringInfo resultName = makeStringInfo(); + char *resultNamePrefix = PartitionResultNamePrefix(jobId, taskId); + + appendStringInfo(resultName, "%s_%d", resultNamePrefix, partitionId); + + return resultName->data; +} + + /* * GenerateSyntheticShardIntervalArray returns a shard interval pointer array * which has a uniform hash distribution for the given input partitionCount. @@ -4504,6 +4546,12 @@ GenerateSyntheticShardIntervalArray(int partitionCount) int32 shardMinHashToken = PG_INT32_MIN + (shardIndex * hashTokenIncrement); int32 shardMaxHashToken = shardMinHashToken + (hashTokenIncrement - 1); + /* extend the last range to cover the full range of integers */ + if (shardIndex == (partitionCount - 1)) + { + shardMaxHashToken = PG_INT32_MAX; + } + shardInterval->relationId = InvalidOid; shardInterval->minValueExists = true; shardInterval->minValue = Int32GetDatum(shardMinHashToken); @@ -4521,6 +4569,34 @@ GenerateSyntheticShardIntervalArray(int partitionCount) } +/* + * RangeIntervalArrayWithNullBucket prepends an additional bucket for NULL values + * to intervalArray and returns the result. + * + * When we support NULL values in (range-partitioned) shards, we will need to revise + * this logic, since there may already be an interval for NULL values. + */ +static ShardInterval ** +RangeIntervalArrayWithNullBucket(ShardInterval **intervalArray, int intervalCount) +{ + int fullIntervalCount = intervalCount + 1; + ShardInterval **fullIntervalArray = + palloc0(fullIntervalCount * sizeof(ShardInterval *)); + + fullIntervalArray[0] = CitusMakeNode(ShardInterval); + fullIntervalArray[0]->minValueExists = true; + fullIntervalArray[0]->maxValueExists = true; + fullIntervalArray[0]->valueTypeId = intervalArray[0]->valueTypeId; + + for (int intervalIndex = 1; intervalIndex < fullIntervalCount; intervalIndex++) + { + fullIntervalArray[intervalIndex] = intervalArray[intervalIndex - 1]; + } + + return fullIntervalArray; +} + + /* * Determine RowModifyLevel required for given query */ @@ -4598,7 +4674,7 @@ ArrayObjectToString(ArrayType *arrayObject, Oid columnType, int32 columnTypeMod) char *arrayOutputEscapedText = quote_literal_cstr(arrayOutputText); /* add an explicit cast to array's string representation */ - char *arrayOutTypeName = format_type_with_typemod(arrayOutType, columnTypeMod); + char *arrayOutTypeName = format_type_be(arrayOutType); StringInfo arrayString = makeStringInfo(); appendStringInfo(arrayString, "%s::%s", @@ -4660,17 +4736,9 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex) Query *reduceQuery = mapMergeJob->reduceQuery; if (reduceQuery == NULL) { - uint32 columnCount = (uint32) list_length(targetEntryList); - StringInfo columnNames = ColumnNameArrayString(columnCount, jobId); - StringInfo columnTypes = ColumnTypeArrayString(targetEntryList); - - StringInfo mergeQueryString = makeStringInfo(); - appendStringInfo(mergeQueryString, MERGE_FILES_INTO_TABLE_COMMAND, - jobId, taskIdIndex, columnNames->data, columnTypes->data); - - /* create merge task */ + /* create logical merge task (not executed, but useful for bookkeeping) */ mergeTask = CreateBasicTask(jobId, mergeTaskId, MERGE_TASK, - mergeQueryString->data); + ""); } mergeTask->partitionId = partitionId; taskIdIndex++; @@ -4682,26 +4750,35 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex) /* find the node name/port for map task's execution */ List *mapTaskPlacementList = mapTask->taskPlacementList; - ShardPlacement *mapTaskPlacement = linitial(mapTaskPlacementList); - char *mapTaskNodeName = mapTaskPlacement->nodeName; - uint32 mapTaskNodePort = mapTaskPlacement->nodePort; + + char *partitionResultName = + PartitionResultName(jobId, mapTask->taskId, partitionId); + + /* we currently only fetch a single fragment at a time */ + DistributedResultFragment singleFragmentTransfer; + singleFragmentTransfer.resultId = partitionResultName; + singleFragmentTransfer.nodeId = mapTaskPlacement->nodeId; + singleFragmentTransfer.rowCount = 0; + singleFragmentTransfer.targetShardId = INVALID_SHARD_ID; + singleFragmentTransfer.targetShardIndex = partitionId; + + NodeToNodeFragmentsTransfer fragmentsTransfer; + fragmentsTransfer.nodes.sourceNodeId = mapTaskPlacement->nodeId; /* - * We will use the first node even if replication factor is greater than 1 - * When replication factor is greater than 1 and there - * is a connection problem to the node that has done the map task, we will get - * an error in fetch task execution. + * Target node is not yet decided, and not necessary for + * QueryStringForFragmentsTransfer. */ - StringInfo mapFetchQueryString = makeStringInfo(); - appendStringInfo(mapFetchQueryString, MAP_OUTPUT_FETCH_COMMAND, - mapTask->jobId, mapTask->taskId, partitionId, - mergeTaskId, /* fetch results to merge task */ - mapTaskNodeName, mapTaskNodePort); + fragmentsTransfer.nodes.targetNodeId = -1; + + fragmentsTransfer.fragmentList = list_make1(&singleFragmentTransfer); + + char *fetchQueryString = QueryStringForFragmentsTransfer(&fragmentsTransfer); Task *mapOutputFetchTask = CreateBasicTask(jobId, taskIdIndex, MAP_OUTPUT_FETCH_TASK, - mapFetchQueryString->data); + fetchQueryString); mapOutputFetchTask->partitionId = partitionId; mapOutputFetchTask->upstreamTaskId = mergeTaskId; mapOutputFetchTask->dependentTaskList = list_make1(mapTask); @@ -4712,6 +4789,7 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex) /* merge task depends on completion of fetch tasks */ mergeTask->dependentTaskList = mapOutputFetchTaskList; + mergeTask->mapJobTargetList = targetEntryList; /* if single repartitioned, each merge task represents an interval */ if (mapMergeJob->partitionType == RANGE_PARTITION_TYPE) @@ -4738,71 +4816,6 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex) } -/* - * ColumnNameArrayString creates a list of column names for a merged table, and - * outputs this list of column names in their (array) string representation. - */ -static StringInfo -ColumnNameArrayString(uint32 columnCount, uint64 generatingJobId) -{ - Datum *columnNameArray = palloc0(columnCount * sizeof(Datum)); - uint32 columnNameIndex = 0; - - /* build list of intermediate column names, generated by given jobId */ - List *columnNameList = DerivedColumnNameList(columnCount, generatingJobId); - - ListCell *columnNameCell = NULL; - foreach(columnNameCell, columnNameList) - { - Value *columnNameValue = (Value *) lfirst(columnNameCell); - char *columnNameString = strVal(columnNameValue); - Datum columnName = CStringGetDatum(columnNameString); - - columnNameArray[columnNameIndex] = columnName; - columnNameIndex++; - } - - StringInfo columnNameArrayString = DatumArrayString(columnNameArray, columnCount, - CSTRINGOID); - - return columnNameArrayString; -} - - -/* - * ColumnTypeArrayString resolves a list of column types for a merged table, and - * outputs this list of column types in their (array) string representation. - */ -static StringInfo -ColumnTypeArrayString(List *targetEntryList) -{ - ListCell *targetEntryCell = NULL; - - uint32 columnCount = (uint32) list_length(targetEntryList); - Datum *columnTypeArray = palloc0(columnCount * sizeof(Datum)); - uint32 columnTypeIndex = 0; - - foreach(targetEntryCell, targetEntryList) - { - TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); - Node *columnExpression = (Node *) targetEntry->expr; - Oid columnTypeId = exprType(columnExpression); - int32 columnTypeMod = exprTypmod(columnExpression); - - char *columnTypeName = format_type_with_typemod(columnTypeId, columnTypeMod); - Datum columnType = CStringGetDatum(columnTypeName); - - columnTypeArray[columnTypeIndex] = columnType; - columnTypeIndex++; - } - - StringInfo columnTypeArrayString = DatumArrayString(columnTypeArray, columnCount, - CSTRINGOID); - - return columnTypeArrayString; -} - - /* * AssignTaskList assigns locations to given tasks based on dependencies between * tasks and configured task assignment policies. The function also handles the @@ -5392,6 +5405,7 @@ ActiveShardPlacementLists(List *taskList) /* sort shard placements by their creation time */ activeShardPlacementList = SortList(activeShardPlacementList, CompareShardPlacements); + shardPlacementLists = lappend(shardPlacementLists, activeShardPlacementList); } diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 0cedff55f..5e9d01e26 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -218,6 +218,13 @@ static const struct config_enum_entry explain_analyze_sort_method_options[] = { { NULL, 0, false } }; +static const struct config_enum_entry create_object_propagation_options[] = { + {"deferred", CREATE_OBJECT_PROPAGATION_DEFERRED, false}, + {"automatic", CREATE_OBJECT_PROPAGATION_AUTOMATIC, false}, + {"immediate", CREATE_OBJECT_PROPAGATION_IMMEDIATE, false}, + {NULL, 0, false} +}; + /* *INDENT-ON* */ @@ -609,7 +616,7 @@ RegisterCitusConfigVariables(void) false, #endif PGC_SIGHUP, - GUC_STANDARD, + GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomBoolVariable( @@ -675,6 +682,24 @@ RegisterCitusConfigVariables(void) GUC_STANDARD, NULL, NULL, NULL); + DefineCustomEnumVariable( + "citus.create_object_propagation", + gettext_noop("Controls the behavior of CREATE statements in transactions for " + "supported objects"), + gettext_noop("When creating new objects in transactions this setting is used to " + "determine the behavior for propagating. When objects are created " + "in a multi-statement transaction block Citus needs to switch to " + "sequential mode (if not already) to make sure the objects are " + "visible to later statements on shards. The switch to sequential is " + "not always desired. By changing this behavior the user can trade " + "off performance for full transactional consistency on the creation " + "of new objects."), + &CreateObjectPropagationMode, + CREATE_OBJECT_PROPAGATION_DEFERRED, create_object_propagation_options, + PGC_USERSET, + GUC_NO_SHOW_ALL, + NULL, NULL, NULL); + DefineCustomBoolVariable( "citus.defer_drop_after_shard_move", gettext_noop("When enabled a shard move will mark the original shards " diff --git a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql index 849b28761..d836d4b72 100644 --- a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql +++ b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql @@ -42,8 +42,9 @@ GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; #include "udfs/pg_cancel_backend/11.0-1.sql" #include "udfs/pg_terminate_backend/11.0-1.sql" +#include "udfs/worker_partition_query_result/11.0-1.sql" -DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text); +DROP FUNCTION pg_catalog.master_apply_delete_command(text); DROP FUNCTION pg_catalog.master_get_table_metadata(text); DROP FUNCTION pg_catalog.master_append_table_to_shard(bigint, text, text, integer); @@ -79,3 +80,6 @@ BEGIN UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb(partitioned_table_exists), true); END; $$; + +#include "udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql" + diff --git a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql index ba13b134a..f11a9d450 100644 --- a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql +++ b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql @@ -1,6 +1,10 @@ -- citus--11.0-1--10.2-4 DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool); + +DROP FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean, boolean, boolean); +#include "../udfs/worker_partition_query_result/9.2-1.sql" + CREATE FUNCTION pg_catalog.master_apply_delete_command(text) RETURNS integer LANGUAGE C STRICT @@ -21,6 +25,7 @@ CREATE FUNCTION pg_catalog.master_get_table_metadata( AS 'MODULE_PATHNAME', $$master_get_table_metadata$$; COMMENT ON FUNCTION master_get_table_metadata(relation_name text) IS 'fetch metadata values for the table'; + ALTER TABLE pg_catalog.pg_dist_partition DROP COLUMN autoconverted; CREATE FUNCTION master_append_table_to_shard(bigint, text, text, integer) @@ -85,7 +90,6 @@ DROP FUNCTION pg_catalog.citus_shard_indexes_on_worker(); #include "../udfs/create_distributed_function/9.0-1.sql" ALTER TABLE citus.pg_dist_object DROP COLUMN force_delegation; - SET search_path = 'pg_catalog'; @@ -342,4 +346,6 @@ JOIN ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; +DROP FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool); + RESET search_path; diff --git a/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql new file mode 100644 index 000000000..d93dd0f93 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql @@ -0,0 +1,224 @@ +-- citus_finalize_upgrade_to_citus11() is a helper UDF ensures +-- the upgrade to Citus 11 is finished successfully. Upgrade to +-- Citus 11 requires all active primary worker nodes to get the +-- metadata. And, this function's job is to sync the metadata to +-- the nodes that does not already have +-- once the function finishes without any errors and returns true +-- the cluster is ready for running distributed queries from +-- the worker nodes. When debug is enabled, the function provides +-- more information to the user. +CREATE OR REPLACE FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(enforce_version_check bool default true) + RETURNS bool + LANGUAGE plpgsql + AS $$ +BEGIN + + --------------------------------------------- + -- This script consists of N stages + -- Each step is documented, and if log level + -- is reduced to DEBUG1, each step is logged + -- as well + --------------------------------------------- + +------------------------------------------------------------------------------------------ + -- STAGE 0: Ensure no concurrent node metadata changing operation happens while this + -- script is running via acquiring a strong lock on the pg_dist_node +------------------------------------------------------------------------------------------ +BEGIN + LOCK TABLE pg_dist_node IN EXCLUSIVE MODE NOWAIT; + + EXCEPTION WHEN OTHERS THEN + RAISE 'Another node metadata changing operation is in progress, try again.'; +END; + +------------------------------------------------------------------------------------------ + -- STAGE 1: We want all the commands to run in the same transaction block. Without + -- sequential mode, metadata syncing cannot be done in a transaction block along with + -- other commands +------------------------------------------------------------------------------------------ + SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; + +------------------------------------------------------------------------------------------ + -- STAGE 2: Ensure we have the prerequisites + -- (a) only superuser can run this script + -- (b) cannot be executed when enable_ddl_propagation is False + -- (c) can only be executed from the coordinator +------------------------------------------------------------------------------------------ +DECLARE + is_superuser_running boolean := False; + enable_ddl_prop boolean:= False; + local_group_id int := 0; +BEGIN + SELECT rolsuper INTO is_superuser_running FROM pg_roles WHERE rolname = current_user; + IF is_superuser_running IS NOT True THEN + RAISE EXCEPTION 'This operation can only be initiated by superuser'; + END IF; + + SELECT current_setting('citus.enable_ddl_propagation') INTO enable_ddl_prop; + IF enable_ddl_prop IS NOT True THEN + RAISE EXCEPTION 'This operation cannot be completed when citus.enable_ddl_propagation is False.'; + END IF; + + SELECT groupid INTO local_group_id FROM pg_dist_local_group; + + IF local_group_id != 0 THEN + RAISE EXCEPTION 'Operation is not allowed on this node. Connect to the coordinator and run it again.'; + ELSE + RAISE DEBUG 'We are on the coordinator, continue to sync metadata'; + END IF; +END; + + + ------------------------------------------------------------------------------------------ + -- STAGE 3: Ensure all primary nodes are active + ------------------------------------------------------------------------------------------ + DECLARE + primary_disabled_worker_node_count int := 0; + BEGIN + SELECT count(*) INTO primary_disabled_worker_node_count FROM pg_dist_node + WHERE groupid != 0 AND noderole = 'primary' AND NOT isactive; + + IF primary_disabled_worker_node_count != 0 THEN + RAISE EXCEPTION 'There are inactive primary worker nodes, you need to activate the nodes first.' + 'Use SELECT citus_activate_node() to activate the disabled nodes'; + ELSE + RAISE DEBUG 'There are no disabled worker nodes, continue to sync metadata'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 4: Ensure there is no connectivity issues in the cluster + ------------------------------------------------------------------------------------------ + DECLARE + all_nodes_can_connect_to_each_other boolean := False; + BEGIN + SELECT bool_and(coalesce(result, false)) INTO all_nodes_can_connect_to_each_other FROM citus_check_cluster_node_health(); + + IF all_nodes_can_connect_to_each_other != True THEN + RAISE EXCEPTION 'There are unhealth primary nodes, you need to ensure all ' + 'nodes are up and runnnig. Also, make sure that all nodes can connect ' + 'to each other. Use SELECT * FROM citus_check_cluster_node_health(); ' + 'to check the cluster health'; + ELSE + RAISE DEBUG 'Cluster is healthy, all nodes can connect to each other'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 5: Ensure all nodes are on the same version + ------------------------------------------------------------------------------------------ + DECLARE + coordinator_version text := ''; + worker_node_version text := ''; + worker_node_version_count int := 0; + + BEGIN + SELECT extversion INTO coordinator_version from pg_extension WHERE extname = 'citus'; + + -- first, check if all nodes have the same versions + SELECT + count(*) INTO worker_node_version_count + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + IF enforce_version_check AND worker_node_version_count != 1 THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the some of the workers has version different versions'; + ELSE + RAISE DEBUG 'All worker nodes have the same Citus version'; + END IF; + + -- second, check if all nodes have the same versions + SELECT + result INTO worker_node_version + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + + IF enforce_version_check AND coordinator_version != worker_node_version THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the coordinator has version % and the worker(s) has %', + coordinator_version, worker_node_version; + ELSE + RAISE DEBUG 'All nodes have the same Citus version'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 6: Ensure all the partitioned tables have the proper naming structure + -- As described on https://github.com/citusdata/citus/issues/4962 + -- existing indexes on partitioned distributed tables can collide + -- with the index names exists on the shards + -- luckily, we know how to fix it. + -- And, note that we should do this even if the cluster is a basic plan + -- (e.g., single node Citus) such that when cluster scaled out, everything + -- works as intended + -- And, this should be done only ONCE for a cluster as it can be a pretty + -- time consuming operation. Thus, even if the function is called multiple time, + -- we keep track of it and do not re-execute this part if not needed. + ------------------------------------------------------------------------------------------ + DECLARE + partitioned_table_exists_pre_11 boolean:=False; + BEGIN + + -- we recorded if partitioned tables exists during upgrade to Citus 11 + SELECT metadata->>'partitioned_citus_table_exists_pre_11' INTO partitioned_table_exists_pre_11 + FROM pg_dist_node_metadata; + + IF partitioned_table_exists_pre_11 IS NOT NULL AND partitioned_table_exists_pre_11 THEN + + -- this might take long depending on the number of partitions and shards... + RAISE NOTICE 'Preparing all the existing partitioned table indexes'; + PERFORM pg_catalog.fix_all_partition_shard_index_names(); + + -- great, we are done with fixing the existing wrong index names + -- so, lets remove this + UPDATE pg_dist_node_metadata + SET metadata=jsonb_delete(metadata, 'partitioned_citus_table_exists_pre_11'); + ELSE + RAISE DEBUG 'There are no partitioned tables that should be fixed'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 7: Return early if there are no primary worker nodes + -- We don't strictly need this step, but it gives a nicer notice message + ------------------------------------------------------------------------------------------ + DECLARE + primary_worker_node_count bigint :=0; + BEGIN + SELECT count(*) INTO primary_worker_node_count FROM pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + + IF primary_worker_node_count = 0 THEN + RAISE NOTICE 'There are no primary worker nodes, no need to sync metadata to any node'; + RETURN true; + ELSE + RAISE DEBUG 'There are % primary worker nodes, continue to sync metadata', primary_worker_node_count; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 8: Do the actual metadata & object syncing to the worker nodes + -- For the "already synced" metadata nodes, we do not strictly need to + -- sync the objects & metadata, but there is no harm to do it anyway + -- it'll only cost some execution time but makes sure that we have a + -- a consistent metadata & objects across all the nodes + ------------------------------------------------------------------------------------------ + DECLARE + BEGIN + + -- this might take long depending on the number of tables & objects ... + RAISE NOTICE 'Preparing to sync the metadata to all nodes'; + + PERFORM start_metadata_sync_to_node(nodename,nodeport) + FROM + pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + END; + + RETURN true; +END; +$$; +COMMENT ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) + IS 'finalizes upgrade to Citus'; + +REVOKE ALL ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) FROM PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/latest.sql b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/latest.sql new file mode 100644 index 000000000..d93dd0f93 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/latest.sql @@ -0,0 +1,224 @@ +-- citus_finalize_upgrade_to_citus11() is a helper UDF ensures +-- the upgrade to Citus 11 is finished successfully. Upgrade to +-- Citus 11 requires all active primary worker nodes to get the +-- metadata. And, this function's job is to sync the metadata to +-- the nodes that does not already have +-- once the function finishes without any errors and returns true +-- the cluster is ready for running distributed queries from +-- the worker nodes. When debug is enabled, the function provides +-- more information to the user. +CREATE OR REPLACE FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(enforce_version_check bool default true) + RETURNS bool + LANGUAGE plpgsql + AS $$ +BEGIN + + --------------------------------------------- + -- This script consists of N stages + -- Each step is documented, and if log level + -- is reduced to DEBUG1, each step is logged + -- as well + --------------------------------------------- + +------------------------------------------------------------------------------------------ + -- STAGE 0: Ensure no concurrent node metadata changing operation happens while this + -- script is running via acquiring a strong lock on the pg_dist_node +------------------------------------------------------------------------------------------ +BEGIN + LOCK TABLE pg_dist_node IN EXCLUSIVE MODE NOWAIT; + + EXCEPTION WHEN OTHERS THEN + RAISE 'Another node metadata changing operation is in progress, try again.'; +END; + +------------------------------------------------------------------------------------------ + -- STAGE 1: We want all the commands to run in the same transaction block. Without + -- sequential mode, metadata syncing cannot be done in a transaction block along with + -- other commands +------------------------------------------------------------------------------------------ + SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; + +------------------------------------------------------------------------------------------ + -- STAGE 2: Ensure we have the prerequisites + -- (a) only superuser can run this script + -- (b) cannot be executed when enable_ddl_propagation is False + -- (c) can only be executed from the coordinator +------------------------------------------------------------------------------------------ +DECLARE + is_superuser_running boolean := False; + enable_ddl_prop boolean:= False; + local_group_id int := 0; +BEGIN + SELECT rolsuper INTO is_superuser_running FROM pg_roles WHERE rolname = current_user; + IF is_superuser_running IS NOT True THEN + RAISE EXCEPTION 'This operation can only be initiated by superuser'; + END IF; + + SELECT current_setting('citus.enable_ddl_propagation') INTO enable_ddl_prop; + IF enable_ddl_prop IS NOT True THEN + RAISE EXCEPTION 'This operation cannot be completed when citus.enable_ddl_propagation is False.'; + END IF; + + SELECT groupid INTO local_group_id FROM pg_dist_local_group; + + IF local_group_id != 0 THEN + RAISE EXCEPTION 'Operation is not allowed on this node. Connect to the coordinator and run it again.'; + ELSE + RAISE DEBUG 'We are on the coordinator, continue to sync metadata'; + END IF; +END; + + + ------------------------------------------------------------------------------------------ + -- STAGE 3: Ensure all primary nodes are active + ------------------------------------------------------------------------------------------ + DECLARE + primary_disabled_worker_node_count int := 0; + BEGIN + SELECT count(*) INTO primary_disabled_worker_node_count FROM pg_dist_node + WHERE groupid != 0 AND noderole = 'primary' AND NOT isactive; + + IF primary_disabled_worker_node_count != 0 THEN + RAISE EXCEPTION 'There are inactive primary worker nodes, you need to activate the nodes first.' + 'Use SELECT citus_activate_node() to activate the disabled nodes'; + ELSE + RAISE DEBUG 'There are no disabled worker nodes, continue to sync metadata'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 4: Ensure there is no connectivity issues in the cluster + ------------------------------------------------------------------------------------------ + DECLARE + all_nodes_can_connect_to_each_other boolean := False; + BEGIN + SELECT bool_and(coalesce(result, false)) INTO all_nodes_can_connect_to_each_other FROM citus_check_cluster_node_health(); + + IF all_nodes_can_connect_to_each_other != True THEN + RAISE EXCEPTION 'There are unhealth primary nodes, you need to ensure all ' + 'nodes are up and runnnig. Also, make sure that all nodes can connect ' + 'to each other. Use SELECT * FROM citus_check_cluster_node_health(); ' + 'to check the cluster health'; + ELSE + RAISE DEBUG 'Cluster is healthy, all nodes can connect to each other'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 5: Ensure all nodes are on the same version + ------------------------------------------------------------------------------------------ + DECLARE + coordinator_version text := ''; + worker_node_version text := ''; + worker_node_version_count int := 0; + + BEGIN + SELECT extversion INTO coordinator_version from pg_extension WHERE extname = 'citus'; + + -- first, check if all nodes have the same versions + SELECT + count(*) INTO worker_node_version_count + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + IF enforce_version_check AND worker_node_version_count != 1 THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the some of the workers has version different versions'; + ELSE + RAISE DEBUG 'All worker nodes have the same Citus version'; + END IF; + + -- second, check if all nodes have the same versions + SELECT + result INTO worker_node_version + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + + IF enforce_version_check AND coordinator_version != worker_node_version THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the coordinator has version % and the worker(s) has %', + coordinator_version, worker_node_version; + ELSE + RAISE DEBUG 'All nodes have the same Citus version'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 6: Ensure all the partitioned tables have the proper naming structure + -- As described on https://github.com/citusdata/citus/issues/4962 + -- existing indexes on partitioned distributed tables can collide + -- with the index names exists on the shards + -- luckily, we know how to fix it. + -- And, note that we should do this even if the cluster is a basic plan + -- (e.g., single node Citus) such that when cluster scaled out, everything + -- works as intended + -- And, this should be done only ONCE for a cluster as it can be a pretty + -- time consuming operation. Thus, even if the function is called multiple time, + -- we keep track of it and do not re-execute this part if not needed. + ------------------------------------------------------------------------------------------ + DECLARE + partitioned_table_exists_pre_11 boolean:=False; + BEGIN + + -- we recorded if partitioned tables exists during upgrade to Citus 11 + SELECT metadata->>'partitioned_citus_table_exists_pre_11' INTO partitioned_table_exists_pre_11 + FROM pg_dist_node_metadata; + + IF partitioned_table_exists_pre_11 IS NOT NULL AND partitioned_table_exists_pre_11 THEN + + -- this might take long depending on the number of partitions and shards... + RAISE NOTICE 'Preparing all the existing partitioned table indexes'; + PERFORM pg_catalog.fix_all_partition_shard_index_names(); + + -- great, we are done with fixing the existing wrong index names + -- so, lets remove this + UPDATE pg_dist_node_metadata + SET metadata=jsonb_delete(metadata, 'partitioned_citus_table_exists_pre_11'); + ELSE + RAISE DEBUG 'There are no partitioned tables that should be fixed'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 7: Return early if there are no primary worker nodes + -- We don't strictly need this step, but it gives a nicer notice message + ------------------------------------------------------------------------------------------ + DECLARE + primary_worker_node_count bigint :=0; + BEGIN + SELECT count(*) INTO primary_worker_node_count FROM pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + + IF primary_worker_node_count = 0 THEN + RAISE NOTICE 'There are no primary worker nodes, no need to sync metadata to any node'; + RETURN true; + ELSE + RAISE DEBUG 'There are % primary worker nodes, continue to sync metadata', primary_worker_node_count; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 8: Do the actual metadata & object syncing to the worker nodes + -- For the "already synced" metadata nodes, we do not strictly need to + -- sync the objects & metadata, but there is no harm to do it anyway + -- it'll only cost some execution time but makes sure that we have a + -- a consistent metadata & objects across all the nodes + ------------------------------------------------------------------------------------------ + DECLARE + BEGIN + + -- this might take long depending on the number of tables & objects ... + RAISE NOTICE 'Preparing to sync the metadata to all nodes'; + + PERFORM start_metadata_sync_to_node(nodename,nodeport) + FROM + pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + END; + + RETURN true; +END; +$$; +COMMENT ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) + IS 'finalizes upgrade to Citus'; + +REVOKE ALL ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) FROM PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql index 2921de962..fa13dc7bd 100644 --- a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql +++ b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql @@ -12,15 +12,22 @@ BEGIN IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN EXECUTE $cmd$ + -- disable propagation to prevent EnsureCoordinator errors + -- the aggregate created here does not depend on Citus extension (yet) + -- since we add the dependency with the next command + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray); COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; ELSE EXECUTE $cmd$ + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray); COMMENT ON AGGREGATE array_cat_agg(anyarray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; END IF; diff --git a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql index 2921de962..fa13dc7bd 100644 --- a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql @@ -12,15 +12,22 @@ BEGIN IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN EXECUTE $cmd$ + -- disable propagation to prevent EnsureCoordinator errors + -- the aggregate created here does not depend on Citus extension (yet) + -- since we add the dependency with the next command + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray); COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; ELSE EXECUTE $cmd$ + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray); COMMENT ON AGGREGATE array_cat_agg(anyarray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; END IF; diff --git a/src/backend/distributed/sql/udfs/worker_create_or_replace_object/11.0-1.sql b/src/backend/distributed/sql/udfs/worker_create_or_replace_object/11.0-1.sql index d9e21a9b2..6eb26040b 100644 --- a/src/backend/distributed/sql/udfs/worker_create_or_replace_object/11.0-1.sql +++ b/src/backend/distributed/sql/udfs/worker_create_or_replace_object/11.0-1.sql @@ -12,4 +12,4 @@ CREATE OR REPLACE FUNCTION pg_catalog.worker_create_or_replace_object(statements AS 'MODULE_PATHNAME', $$worker_create_or_replace_object_array$$; COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statements text[]) - IS 'takes a lost of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object'; + IS 'takes an array of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object'; diff --git a/src/backend/distributed/sql/udfs/worker_create_or_replace_object/latest.sql b/src/backend/distributed/sql/udfs/worker_create_or_replace_object/latest.sql index d9e21a9b2..6eb26040b 100644 --- a/src/backend/distributed/sql/udfs/worker_create_or_replace_object/latest.sql +++ b/src/backend/distributed/sql/udfs/worker_create_or_replace_object/latest.sql @@ -12,4 +12,4 @@ CREATE OR REPLACE FUNCTION pg_catalog.worker_create_or_replace_object(statements AS 'MODULE_PATHNAME', $$worker_create_or_replace_object_array$$; COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statements text[]) - IS 'takes a lost of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object'; + IS 'takes an array of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object'; diff --git a/src/backend/distributed/sql/udfs/worker_partition_query_result/11.0-1.sql b/src/backend/distributed/sql/udfs/worker_partition_query_result/11.0-1.sql new file mode 100644 index 000000000..d5c6fb8de --- /dev/null +++ b/src/backend/distributed/sql/udfs/worker_partition_query_result/11.0-1.sql @@ -0,0 +1,20 @@ +DROP FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean); + +CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result( + result_prefix text, + query text, + partition_column_index int, + partition_method citus.distribution_type, + partition_min_values text[], + partition_max_values text[], + binary_copy boolean, + allow_null_partition_column boolean DEFAULT false, + generate_empty_results boolean DEFAULT false, + OUT partition_index int, + OUT rows_written bigint, + OUT bytes_written bigint) +RETURNS SETOF record +LANGUAGE C STRICT VOLATILE +AS 'MODULE_PATHNAME', $$worker_partition_query_result$$; +COMMENT ON FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean, boolean, boolean) +IS 'execute a query and partitions its results in set of local result files'; diff --git a/src/backend/distributed/sql/udfs/worker_partition_query_result/latest.sql b/src/backend/distributed/sql/udfs/worker_partition_query_result/latest.sql index bda8384fb..d5c6fb8de 100644 --- a/src/backend/distributed/sql/udfs/worker_partition_query_result/latest.sql +++ b/src/backend/distributed/sql/udfs/worker_partition_query_result/latest.sql @@ -1,3 +1,5 @@ +DROP FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean); + CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result( result_prefix text, query text, @@ -5,12 +7,14 @@ CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result( partition_method citus.distribution_type, partition_min_values text[], partition_max_values text[], - binaryCopy boolean, + binary_copy boolean, + allow_null_partition_column boolean DEFAULT false, + generate_empty_results boolean DEFAULT false, OUT partition_index int, OUT rows_written bigint, OUT bytes_written bigint) RETURNS SETOF record LANGUAGE C STRICT VOLATILE AS 'MODULE_PATHNAME', $$worker_partition_query_result$$; -COMMENT ON FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean) +COMMENT ON FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean, boolean, boolean) IS 'execute a query and partitions its results in set of local result files'; diff --git a/src/backend/distributed/test/shard_rebalancer.c b/src/backend/distributed/test/shard_rebalancer.c index 4cccd851d..ea770cb6e 100644 --- a/src/backend/distributed/test/shard_rebalancer.c +++ b/src/backend/distributed/test/shard_rebalancer.c @@ -20,9 +20,11 @@ #include "distributed/citus_ruleutils.h" #include "distributed/connection_management.h" #include "distributed/listutils.h" +#include "distributed/metadata_utility.h" #include "distributed/multi_physical_planner.h" #include "distributed/shard_cleaner.h" #include "distributed/shard_rebalancer.h" +#include "distributed/relay_utility.h" #include "funcapi.h" #include "miscadmin.h" #include "utils/builtins.h" @@ -85,6 +87,18 @@ run_try_drop_marked_shards(PG_FUNCTION_ARGS) } +/* + * IsActiveTestShardPlacement checks if the dummy shard placement created in tests + * are labelled as active. Note that this function does not check if the worker is also + * active, because the dummy test workers are not registered as actual workers. + */ +static inline bool +IsActiveTestShardPlacement(ShardPlacement *shardPlacement) +{ + return shardPlacement->shardState == SHARD_STATE_ACTIVE; +} + + /* * shard_placement_rebalance_array returns a list of operations which can make a * cluster consisting of given shard placements and worker nodes balanced with @@ -138,7 +152,9 @@ shard_placement_rebalance_array(PG_FUNCTION_ARGS) if (shardPlacementTestInfo->nextColocationGroup) { shardPlacementList = SortList(shardPlacementList, CompareShardPlacements); - shardPlacementListList = lappend(shardPlacementListList, shardPlacementList); + shardPlacementListList = lappend(shardPlacementListList, + FilterShardPlacementList(shardPlacementList, + IsActiveTestShardPlacement)); shardPlacementList = NIL; } shardPlacementList = lappend(shardPlacementList, @@ -290,12 +306,15 @@ shard_placement_replication_array(PG_FUNCTION_ARGS) shardPlacementTestInfo->placement); } + List *activeShardPlacementList = FilterShardPlacementList(shardPlacementList, + IsActiveTestShardPlacement); + /* sort the lists to make the function more deterministic */ workerNodeList = SortList(workerNodeList, CompareWorkerNodes); - shardPlacementList = SortList(shardPlacementList, CompareShardPlacements); + activeShardPlacementList = SortList(activeShardPlacementList, CompareShardPlacements); List *placementUpdateList = ReplicationPlacementUpdates(workerNodeList, - shardPlacementList, + activeShardPlacementList, shardReplicationFactor); ArrayType *placementUpdateJsonArray = PlacementUpdateListToJsonArray( placementUpdateList); @@ -426,6 +445,9 @@ JsonArrayToWorkerTestInfoList(ArrayType *workerNodeJsonArrayObject) workerTestInfo->capacity = JsonFieldValueUInt64Default(workerNodeJson, "capacity", 1); + workerNode->isActive = JsonFieldValueBoolDefault(workerNodeJson, + "isActive", true); + workerTestInfoList = lappend(workerTestInfoList, workerTestInfo); char *disallowedShardsString = JsonFieldValueString( workerNodeJson, "disallowed_shards"); diff --git a/src/backend/distributed/transaction/backend_data.c b/src/backend/distributed/transaction/backend_data.c index 5ca517199..3af20631a 100644 --- a/src/backend/distributed/transaction/backend_data.c +++ b/src/backend/distributed/transaction/backend_data.c @@ -83,6 +83,7 @@ typedef struct BackendManagementShmemData static void StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescriptor); +static bool UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId); static uint64 GenerateGlobalPID(void); static shmem_startup_hook_type prev_shmem_startup_hook = NULL; @@ -114,8 +115,6 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS) { CheckCitusVersion(ERROR); - Oid userId = GetUserId(); - /* prepare data before acquiring spinlock to protect against errors */ int32 initiatorNodeIdentifier = PG_GETARG_INT32(0); uint64 transactionNumber = PG_GETARG_INT64(1); @@ -144,17 +143,11 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS) "transaction id"))); } - MyBackendData->databaseId = MyDatabaseId; - MyBackendData->userId = userId; - MyBackendData->transactionId.initiatorNodeIdentifier = initiatorNodeIdentifier; MyBackendData->transactionId.transactionNumber = transactionNumber; MyBackendData->transactionId.timestamp = timestamp; MyBackendData->transactionId.transactionOriginator = false; - MyBackendData->citusBackend.initiatorNodeIdentifier = - MyBackendData->transactionId.initiatorNodeIdentifier; - SpinLockRelease(&MyBackendData->mutex); PG_RETURN_VOID(); @@ -360,55 +353,48 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto { Datum values[ACTIVE_TRANSACTION_COLUMN_COUNT]; bool isNulls[ACTIVE_TRANSACTION_COLUMN_COUNT]; - bool showAllTransactions = superuser(); + bool showAllBackends = superuser(); const Oid userId = GetUserId(); - /* - * We don't want to initialize memory while spinlock is held so we - * prefer to do it here. This initialization is done only for the first - * row. - */ - memset(values, 0, sizeof(values)); - memset(isNulls, false, sizeof(isNulls)); - - if (is_member_of_role(userId, ROLE_PG_MONITOR)) + if (!showAllBackends && is_member_of_role(userId, ROLE_PG_MONITOR)) { - showAllTransactions = true; + showAllBackends = true; } /* we're reading all distributed transactions, prevent new backends */ LockBackendSharedMemory(LW_SHARED); - for (int backendIndex = 0; backendIndex < MaxBackends; ++backendIndex) + for (int backendIndex = 0; backendIndex < TotalProcCount(); ++backendIndex) { + bool showCurrentBackendDetails = showAllBackends; BackendData *currentBackend = &backendManagementShmemData->backends[backendIndex]; + PGPROC *currentProc = &ProcGlobal->allProcs[backendIndex]; /* to work on data after releasing g spinlock to protect against errors */ - int initiatorNodeIdentifier = -1; uint64 transactionNumber = 0; SpinLockAcquire(¤tBackend->mutex); - if (currentBackend->globalPID == INVALID_CITUS_INTERNAL_BACKEND_GPID) + if (currentProc->pid == 0) { + /* unused PGPROC slot */ SpinLockRelease(¤tBackend->mutex); continue; } /* * Unless the user has a role that allows seeing all transactions (superuser, - * pg_monitor), skip over transactions belonging to other users. + * pg_monitor), we only follow pg_stat_statements owner checks. */ - if (!showAllTransactions && currentBackend->userId != userId) + if (!showCurrentBackendDetails && + UserHasPermissionToViewStatsOf(userId, currentProc->roleId)) { - SpinLockRelease(¤tBackend->mutex); - continue; + showCurrentBackendDetails = true; } Oid databaseId = currentBackend->databaseId; int backendPid = ProcGlobal->allProcs[backendIndex].pid; - initiatorNodeIdentifier = currentBackend->citusBackend.initiatorNodeIdentifier; /* * We prefer to use worker_query instead of distributedCommandOriginator in @@ -423,13 +409,42 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto SpinLockRelease(¤tBackend->mutex); - values[0] = ObjectIdGetDatum(databaseId); - values[1] = Int32GetDatum(backendPid); - values[2] = Int32GetDatum(initiatorNodeIdentifier); - values[3] = !distributedCommandOriginator; - values[4] = UInt64GetDatum(transactionNumber); - values[5] = TimestampTzGetDatum(transactionIdTimestamp); - values[6] = UInt64GetDatum(currentBackend->globalPID); + memset(values, 0, sizeof(values)); + memset(isNulls, false, sizeof(isNulls)); + + /* + * We imitate pg_stat_activity such that if a user doesn't have enough + * privileges, we only show the minimal information including the pid, + * global pid and distributedCommandOriginator. + * + * pid is already can be found in pg_stat_activity for any process, and + * the rest doesn't reveal anything critial for under priviledge users + * but still could be useful for monitoring purposes of Citus. + */ + if (showCurrentBackendDetails) + { + bool missingOk = true; + int initiatorNodeId = + ExtractNodeIdFromGlobalPID(currentBackend->globalPID, missingOk); + + values[0] = ObjectIdGetDatum(databaseId); + values[1] = Int32GetDatum(backendPid); + values[2] = Int32GetDatum(initiatorNodeId); + values[3] = !distributedCommandOriginator; + values[4] = UInt64GetDatum(transactionNumber); + values[5] = TimestampTzGetDatum(transactionIdTimestamp); + values[6] = UInt64GetDatum(currentBackend->globalPID); + } + else + { + isNulls[0] = true; + values[1] = Int32GetDatum(backendPid); + isNulls[2] = true; + values[3] = !distributedCommandOriginator; + isNulls[4] = true; + isNulls[5] = true; + values[6] = UInt64GetDatum(currentBackend->globalPID); + } tuplestore_putvalues(tupleStore, tupleDescriptor, values, isNulls); @@ -446,6 +461,35 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto } +/* + * UserHasPermissionToViewStatsOf returns true if currentUserId can + * see backends of backendOwnedId. + * + * We follow the same approach with pg_stat_activity. + */ +static +bool +UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId) +{ + if (has_privs_of_role(currentUserId, backendOwnedId)) + { + return true; + } + + if (is_member_of_role(currentUserId, +#if PG_VERSION_NUM >= PG_VERSION_14 + ROLE_PG_READ_ALL_STATS)) +#else + DEFAULT_ROLE_READ_ALL_STATS)) +#endif + { + return true; + } + + return false; +} + + /* * InitializeBackendManagement requests the necessary shared memory * from Postgres and sets up the shared memory startup hook. @@ -520,7 +564,6 @@ BackendManagementShmemInit(void) { BackendData *backendData = &backendManagementShmemData->backends[backendIndex]; - backendData->citusBackend.initiatorNodeIdentifier = -1; SpinLockInit(&backendData->mutex); } } @@ -652,16 +695,12 @@ UnSetDistributedTransactionId(void) { SpinLockAcquire(&MyBackendData->mutex); - MyBackendData->databaseId = 0; - MyBackendData->userId = 0; MyBackendData->cancelledDueToDeadlock = false; MyBackendData->transactionId.initiatorNodeIdentifier = 0; MyBackendData->transactionId.transactionOriginator = false; MyBackendData->transactionId.transactionNumber = 0; MyBackendData->transactionId.timestamp = 0; - MyBackendData->citusBackend.initiatorNodeIdentifier = -1; - SpinLockRelease(&MyBackendData->mutex); } } @@ -679,6 +718,8 @@ UnSetGlobalPID(void) SpinLockAcquire(&MyBackendData->mutex); MyBackendData->globalPID = 0; + MyBackendData->databaseId = 0; + MyBackendData->userId = 0; SpinLockRelease(&MyBackendData->mutex); } @@ -760,41 +801,14 @@ AssignDistributedTransactionId(void) uint64 nextTransactionNumber = pg_atomic_fetch_add_u64(transactionNumberSequence, 1); int32 localGroupId = GetLocalGroupId(); TimestampTz currentTimestamp = GetCurrentTimestamp(); - Oid userId = GetUserId(); SpinLockAcquire(&MyBackendData->mutex); - MyBackendData->databaseId = MyDatabaseId; - MyBackendData->userId = userId; - MyBackendData->transactionId.initiatorNodeIdentifier = localGroupId; MyBackendData->transactionId.transactionOriginator = true; MyBackendData->transactionId.transactionNumber = nextTransactionNumber; MyBackendData->transactionId.timestamp = currentTimestamp; - MyBackendData->citusBackend.initiatorNodeIdentifier = localGroupId; - - SpinLockRelease(&MyBackendData->mutex); -} - - -/* - * MarkCitusInitiatedCoordinatorBackend sets that coordinator backend is - * initiated by Citus. - */ -void -MarkCitusInitiatedCoordinatorBackend(void) -{ - /* - * GetLocalGroupId may throw exception which can cause leaving spin lock - * unreleased. Calling GetLocalGroupId function before the lock to avoid this. - */ - int32 localGroupId = GetLocalGroupId(); - - SpinLockAcquire(&MyBackendData->mutex); - - MyBackendData->citusBackend.initiatorNodeIdentifier = localGroupId; - SpinLockRelease(&MyBackendData->mutex); } @@ -821,9 +835,15 @@ AssignGlobalPID(void) globalPID = ExtractGlobalPID(application_name); } + Oid userId = GetUserId(); + SpinLockAcquire(&MyBackendData->mutex); + MyBackendData->globalPID = globalPID; MyBackendData->distributedCommandOriginator = distributedCommandOriginator; + MyBackendData->databaseId = MyDatabaseId; + MyBackendData->userId = userId; + SpinLockRelease(&MyBackendData->mutex); } @@ -926,11 +946,12 @@ ExtractGlobalPID(char *applicationName) * gives us the node id. */ int -ExtractNodeIdFromGlobalPID(uint64 globalPID) +ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk) { int nodeId = (int) (globalPID / GLOBAL_PID_NODE_ID_MULTIPLIER); - if (nodeId == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) + if (!missingOk && + nodeId == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) { ereport(ERROR, (errmsg("originator node of the query with the global pid " "%lu is not in Citus' metadata", globalPID), diff --git a/src/backend/distributed/transaction/citus_dist_stat_activity.c b/src/backend/distributed/transaction/citus_dist_stat_activity.c index ec4f6e8a3..0ee3925fb 100644 --- a/src/backend/distributed/transaction/citus_dist_stat_activity.c +++ b/src/backend/distributed/transaction/citus_dist_stat_activity.c @@ -538,33 +538,35 @@ ReplaceInitiatorNodeIdentifier(int initiator_node_identifier, * transaction. However, we cannot know which node has initiated * the worker query. */ - if (initiator_node_identifier > 0) - { - bool nodeExists = false; - - initiatorWorkerNode = PrimaryNodeForGroup(initiator_node_identifier, &nodeExists); - - /* a query should run on an existing node */ - Assert(nodeExists); - if (initiatorWorkerNode == NULL) - { - ereport(ERROR, (errmsg("no primary node found for group %d", - initiator_node_identifier))); - } - citusDistStat->master_query_host_name = - cstring_to_text(initiatorWorkerNode->workerName); - citusDistStat->master_query_host_port = initiatorWorkerNode->workerPort; - } - else if (initiator_node_identifier == 0 && IsCoordinator()) + if (initiator_node_identifier == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA && + IsCoordinator()) { citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); citusDistStat->master_query_host_port = PostPortNumber; } - else if (initiator_node_identifier == 0) + else if (initiator_node_identifier == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) { citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); citusDistStat->master_query_host_port = 0; } + else if (initiator_node_identifier > 0) + { + /* a query should run on an existing node, but lets be defensive */ + bool missingOk = true; + initiatorWorkerNode = FindNodeWithNodeId(initiator_node_identifier, missingOk); + + if (initiatorWorkerNode) + { + citusDistStat->master_query_host_name = + cstring_to_text(initiatorWorkerNode->workerName); + citusDistStat->master_query_host_port = initiatorWorkerNode->workerPort; + } + else + { + citusDistStat->master_query_host_name = NULL; + citusDistStat->master_query_host_port = 0; + } + } else { citusDistStat->master_query_host_name = NULL; diff --git a/src/include/distributed/adaptive_executor.h b/src/include/distributed/adaptive_executor.h index 0a3768177..0a4735ee7 100644 --- a/src/include/distributed/adaptive_executor.h +++ b/src/include/distributed/adaptive_executor.h @@ -15,6 +15,7 @@ extern bool EnableCostBasedConnectionEstablishment; extern bool PreventIncompleteConnectionEstablishment; extern bool ShouldRunTasksSequentially(List *taskList); +extern uint64 ExecuteTaskList(RowModifyLevel modLevel, List *taskList); extern uint64 ExecuteUtilityTaskList(List *utilityTaskList, bool localExecutionSupported); extern uint64 ExecuteUtilityTaskListExtended(List *utilityTaskList, int poolSize, bool localExecutionSupported); diff --git a/src/include/distributed/backend_data.h b/src/include/distributed/backend_data.h index b463b89f5..c34d94670 100644 --- a/src/include/distributed/backend_data.h +++ b/src/include/distributed/backend_data.h @@ -22,16 +22,6 @@ #include "storage/s_lock.h" -/* - * CitusInitiatedBackend keeps some information about the backends that are - * initiated by Citus. - */ -typedef struct CitusInitiatedBackend -{ - int initiatorNodeIdentifier; -} CitusInitiatedBackend; - - /* * Each backend's active distributed transaction information is tracked via * BackendData in shared memory. @@ -51,7 +41,6 @@ typedef struct BackendData bool cancelledDueToDeadlock; uint64 globalPID; bool distributedCommandOriginator; - CitusInitiatedBackend citusBackend; DistributedTransactionId transactionId; } BackendData; @@ -64,13 +53,12 @@ extern void UnlockBackendSharedMemory(void); extern void UnSetDistributedTransactionId(void); extern void UnSetGlobalPID(void); extern void AssignDistributedTransactionId(void); -extern void MarkCitusInitiatedCoordinatorBackend(void); extern void AssignGlobalPID(void); extern uint64 GetGlobalPID(void); extern void OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator); extern uint64 ExtractGlobalPID(char *applicationName); -extern int ExtractNodeIdFromGlobalPID(uint64 globalPID); +extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk); extern int ExtractProcessIdFromGlobalPID(uint64 globalPID); extern void GetBackendDataForProc(PGPROC *proc, BackendData *result); extern void CancelTransactionDueToDeadlock(PGPROC *proc); diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index c2bf66d5b..8ec3d9e8a 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -122,6 +122,9 @@ typedef enum SearchForeignKeyColumnFlags } SearchForeignKeyColumnFlags; +/* aggregate.c - forward declarations */ +extern List * PostprocessDefineAggregateStmt(Node *node, const char *queryString); + /* cluster.c - forward declarations */ extern List * PreprocessClusterStmt(Node *node, const char *clusterCommand, ProcessUtilityContext processUtilityContext); @@ -264,6 +267,7 @@ extern List * PreprocessCreateFunctionStmt(Node *stmt, const char *queryString, ProcessUtilityContext processUtilityContext); extern List * PostprocessCreateFunctionStmt(Node *stmt, const char *queryString); +extern ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress); extern ObjectAddress CreateFunctionStmtObjectAddress(Node *stmt, bool missing_ok); extern ObjectAddress DefineAggregateStmtObjectAddress(Node *stmt, diff --git a/src/include/distributed/commands/utility_hook.h b/src/include/distributed/commands/utility_hook.h index 7c926fcf1..615a7c6d2 100644 --- a/src/include/distributed/commands/utility_hook.h +++ b/src/include/distributed/commands/utility_hook.h @@ -22,6 +22,13 @@ #include "distributed/version_compat.h" #include "distributed/worker_transaction.h" +typedef enum +{ + CREATE_OBJECT_PROPAGATION_DEFERRED = 0, + CREATE_OBJECT_PROPAGATION_AUTOMATIC = 1, + CREATE_OBJECT_PROPAGATION_IMMEDIATE = 2 +} CreateObjectPropagationOptions; + typedef enum { PROPSETCMD_INVALID = -1, @@ -32,6 +39,7 @@ typedef enum } PropSetCmdBehavior; extern PropSetCmdBehavior PropagateSetCommands; extern bool EnableDDLPropagation; +extern int CreateObjectPropagationMode; extern bool EnableCreateTypePropagation; extern bool EnableAlterRolePropagation; extern bool EnableAlterRoleSetPropagation; diff --git a/src/include/distributed/deparser.h b/src/include/distributed/deparser.h index ebf4a6147..e3b02cdfc 100644 --- a/src/include/distributed/deparser.h +++ b/src/include/distributed/deparser.h @@ -33,6 +33,9 @@ extern void QualifyTreeNode(Node *stmt); extern char * DeparseTreeNode(Node *stmt); extern List * DeparseTreeNodes(List *stmts); +/* forward declarations for qualify_aggregate_stmts.c */ +extern void QualifyDefineAggregateStmt(Node *node); + /* forward declarations for deparse_attribute_stmts.c */ extern char * DeparseRenameAttributeStmt(Node *); diff --git a/src/include/distributed/intermediate_results.h b/src/include/distributed/intermediate_results.h index 6e5a7f640..e40eadba9 100644 --- a/src/include/distributed/intermediate_results.h +++ b/src/include/distributed/intermediate_results.h @@ -48,6 +48,28 @@ typedef struct DistributedResultFragment } DistributedResultFragment; +/* + * NodePair contains the source and destination node in a NodeToNodeFragmentsTransfer. + * It is a separate struct to use it as a key in a hash table. + */ +typedef struct NodePair +{ + uint32 sourceNodeId; + uint32 targetNodeId; +} NodePair; + + +/* + * NodeToNodeFragmentsTransfer contains all fragments that need to be fetched from + * the source node to the destination node in the NodePair. + */ +typedef struct NodeToNodeFragmentsTransfer +{ + NodePair nodes; + List *fragmentList; +} NodeToNodeFragmentsTransfer; + + /* intermediate_results.c */ extern DestReceiver * CreateRemoteFileDestReceiver(const char *resultId, EState *executorState, @@ -72,5 +94,10 @@ extern List * PartitionTasklistResults(const char *resultIdPrefix, List *selectT int partitionColumnIndex, CitusTableCacheEntry *distributionScheme, bool binaryFormat); +extern char * QueryStringForFragmentsTransfer( + NodeToNodeFragmentsTransfer *fragmentsTransfer); +extern void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount, + Oid intervalTypeId, ArrayType **minValueArray, + ArrayType **maxValueArray); #endif /* INTERMEDIATE_RESULTS_H */ diff --git a/src/include/distributed/local_executor.h b/src/include/distributed/local_executor.h index a47dccb17..d2b8cce9c 100644 --- a/src/include/distributed/local_executor.h +++ b/src/include/distributed/local_executor.h @@ -43,6 +43,7 @@ extern void ExecuteUtilityCommand(const char *utilityCommand); extern bool ShouldExecuteTasksLocally(List *taskList); extern bool AnyTaskAccessesLocalNode(List *taskList); extern bool TaskAccessesLocalNode(Task *task); +extern void EnsureCompatibleLocalExecutionState(List *taskList); extern void ErrorIfTransactionAccessedPlacementsLocally(void); extern void DisableLocalExecution(void); extern void SetLocalExecutionStatus(LocalExecutionStatus newStatus); diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index 69d500da4..905d2cab5 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -73,7 +73,9 @@ extern List * GetSequencesFromAttrDef(Oid attrdefOid); extern void GetDependentSequencesWithRelation(Oid relationId, List **attnumList, List **dependentSequenceList, AttrNumber attnum); +extern List * GetDependentFunctionsWithRelation(Oid relationId); extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum); +extern void SetLocalEnableMetadataSync(bool state); #define DELETE_ALL_NODES "DELETE FROM pg_dist_node" #define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement" diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 3e7a3b6f3..c03b3abe7 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -209,6 +209,9 @@ extern ShardInterval * CopyShardInterval(ShardInterval *srcInterval); extern uint64 ShardLength(uint64 shardId); extern bool NodeGroupHasShardPlacements(int32 groupId, bool onlyConsiderActivePlacements); +extern bool IsActiveShardPlacement(ShardPlacement *ShardPlacement); +extern List * FilterShardPlacementList(List *shardPlacementList, bool (*filter)( + ShardPlacement *)); extern List * ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId); extern List * ActiveShardPlacementList(uint64 shardId); extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId); @@ -248,6 +251,7 @@ extern TableConversionReturn * UndistributeTable(TableConversionParameters *para extern void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target); extern List * GetDistributableDependenciesForObject(const ObjectAddress *target); extern bool ShouldPropagate(void); +extern bool ShouldPropagateCreateInCoordinatedTransction(void); extern bool ShouldPropagateObject(const ObjectAddress *address); extern List * ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort); diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 1a3630f81..14fdd7a0c 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -36,15 +36,6 @@ /* Definitions local to the physical planner */ #define NON_PRUNABLE_JOIN -1 #define RESERVED_HASHED_COLUMN_ID MaxAttrNumber -#define MERGE_COLUMN_FORMAT "merge_column_%u" -#define MAP_OUTPUT_FETCH_COMMAND "SELECT worker_fetch_partition_file \ - (" UINT64_FORMAT ", %u, %u, %u, '%s', %u)" -#define RANGE_PARTITION_COMMAND "SELECT worker_range_partition_table \ - (" UINT64_FORMAT ", %d, %s, '%s', '%s'::regtype, %s)" -#define HASH_PARTITION_COMMAND "SELECT worker_hash_partition_table \ - (" UINT64_FORMAT ", %d, %s, '%s', '%s'::regtype, %s)" -#define MERGE_FILES_INTO_TABLE_COMMAND "SELECT worker_merge_files_into_table \ - (" UINT64_FORMAT ", %d, '%s', '%s')" extern int RepartitionJoinBucketCountPerNode; @@ -262,6 +253,10 @@ typedef struct Task uint32 upstreamTaskId; /* only applies to data fetch tasks */ ShardInterval *shardInterval; /* only applies to merge tasks */ bool assignmentConstrained; /* only applies to merge tasks */ + + /* for merge tasks, this is set to the target list of the map task */ + List *mapJobTargetList; + char replicationModel; /* only applies to modify tasks */ /* diff --git a/src/include/distributed/repartition_join_execution.h b/src/include/distributed/repartition_join_execution.h index 596dffc0b..7f2b648cf 100644 --- a/src/include/distributed/repartition_join_execution.h +++ b/src/include/distributed/repartition_join_execution.h @@ -13,8 +13,6 @@ #include "nodes/pg_list.h" extern List * ExecuteDependentTasks(List *taskList, Job *topLevelJob); -extern void EnsureCompatibleLocalExecutionState(List *taskList); -extern void DoRepartitionCleanup(List *jobIds); #endif /* REPARTITION_JOIN_EXECUTION_H */ diff --git a/src/include/distributed/shard_rebalancer.h b/src/include/distributed/shard_rebalancer.h index 3e6d7a8b7..11730492a 100644 --- a/src/include/distributed/shard_rebalancer.h +++ b/src/include/distributed/shard_rebalancer.h @@ -182,7 +182,8 @@ extern Datum init_rebalance_monitor(PG_FUNCTION_ARGS); extern Datum finalize_rebalance_monitor(PG_FUNCTION_ARGS); extern Datum get_rebalance_progress(PG_FUNCTION_ARGS); -extern List * RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementList, +extern List * RebalancePlacementUpdates(List *workerNodeList, + List *shardPlacementListList, double threshold, int32 maxShardMoves, bool drainOnly, diff --git a/src/include/distributed/worker_manager.h b/src/include/distributed/worker_manager.h index db8adaedb..27de1d464 100644 --- a/src/include/distributed/worker_manager.h +++ b/src/include/distributed/worker_manager.h @@ -86,7 +86,7 @@ extern List * ActiveReadableNodeList(void); extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNodeOrError(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort); -extern WorkerNode * FindNodeWithNodeId(int nodeId); +extern WorkerNode * FindNodeWithNodeId(int nodeId, bool missingOk); extern List * ReadDistNode(bool includeNodesFromOtherClusters); extern void EnsureCoordinator(void); extern void InsertCoordinatorIfClusterEmpty(void); diff --git a/src/test/regress/after_citus_upgrade_coord_schedule b/src/test/regress/after_citus_upgrade_coord_schedule index 49b4e73d9..e73837c59 100644 --- a/src/test/regress/after_citus_upgrade_coord_schedule +++ b/src/test/regress/after_citus_upgrade_coord_schedule @@ -4,3 +4,4 @@ test: upgrade_basic_after test: upgrade_partition_constraints_after test: upgrade_pg_dist_object_test_after test: upgrade_columnar_metapage_after +test: upgrade_post_11_after diff --git a/src/test/regress/before_citus_upgrade_coord_schedule b/src/test/regress/before_citus_upgrade_coord_schedule index 006217e8a..169a7f418 100644 --- a/src/test/regress/before_citus_upgrade_coord_schedule +++ b/src/test/regress/before_citus_upgrade_coord_schedule @@ -4,3 +4,4 @@ test: upgrade_basic_before test: upgrade_partition_constraints_before test: upgrade_pg_dist_object_test_before test: upgrade_columnar_metapage_before +test: upgrade_post_11_before diff --git a/src/test/regress/bin/diff b/src/test/regress/bin/diff index 66a558c45..a0b40f328 100755 --- a/src/test/regress/bin/diff +++ b/src/test/regress/bin/diff @@ -19,12 +19,19 @@ test=$(basename "$file1" .out | sed -E "s/_[0-9]+$//") args=${@:1:$#-2} BASEDIR=$(dirname "$0") -# whereis searches for standard unix places before $PATH. So select the first -# entry as the original diff tool. -DIFF=$(whereis diff | sed "s/diff://g" | awk '{print $1}') -if [ -z "$DIFF" ] +DIFF=/usr/bin/diff +if [ ! -f "$DIFF" ] then - DIFF=/usr/bin/diff + # whereis searches for standard unix places before $PATH. So select the + # first entry as the original diff tool. + # With the default WSL2 configuration whereis is very slow though ~400ms, + # so we only use it if /usr/bin/diff does not exist. + DIFF=$(whereis diff | sed "s/diff://g" | awk '{print $1}') + if [ -z "$DIFF" ] + then + echo "ERROR: could not find diff command" 1>&2 + exit 1 + fi fi if test -z "${VANILLATEST:-}" diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index 52b0714c8..beccf8b26 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -227,8 +227,7 @@ class CitusSmallSharedPoolSizeConfig(CitusMXBaseClusterConfig): def __init__(self, arguments): super().__init__(arguments) self.new_settings = { - # can be uncommented after https://github.com/citusdata/citus/issues/5342 - # "citus.local_shared_pool_size": 5, + "citus.local_shared_pool_size": 5, "citus.max_shared_pool_size": 5, } diff --git a/src/test/regress/expected/adaptive_executor_repartition.out b/src/test/regress/expected/adaptive_executor_repartition.out index c5b583bef..a84677a35 100644 --- a/src/test/regress/expected/adaptive_executor_repartition.out +++ b/src/test/regress/expected/adaptive_executor_repartition.out @@ -52,9 +52,12 @@ SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT ROLLBACK; BEGIN; INSERT INTO ab values(1, 2); --- DDL happened before repartition query in a transaction block, so this should error. SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT * FROM ab) second WHERE first.a = second.b; -ERROR: cannot open new connections after the first modification command within a transaction + count +--------------------------------------------------------------------- + 14 +(1 row) + ROLLBACK; SET citus.enable_single_hash_repartition_joins TO ON; CREATE TABLE single_hash_repartition_first (id int, sum int, avg float); diff --git a/src/test/regress/expected/aggregate_support.out b/src/test/regress/expected/aggregate_support.out index bf35d7f4b..57bb03060 100644 --- a/src/test/regress/expected/aggregate_support.out +++ b/src/test/regress/expected/aggregate_support.out @@ -282,6 +282,21 @@ create aggregate binstragg(text, text)( combinefunc=binstragg_combinefunc, stype=text ); +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + select create_distributed_function('binstragg(text,text)'); create_distributed_function --------------------------------------------------------------------- @@ -549,7 +564,6 @@ SELECT id%5, first(val ORDER BY key), last(val ORDER BY key) FROM aggdata GROUP BY id%5 ORDER BY id%5; ERROR: unsupported aggregate function first -- test aggregate with stype which is not a by-value datum --- also test our handling of the aggregate not existing on workers create function sumstring_sfunc(state text, x text) returns text immutable language plpgsql as $$ begin return (state::float8 + x::float8)::text; @@ -561,9 +575,20 @@ create aggregate sumstring(text) ( combinefunc = sumstring_sfunc, initcond = '0' ); -select sumstring(valf::text) from aggdata where valf is not null; -ERROR: function "aggregate_support.sumstring(text)" does not exist -CONTEXT: while executing command on localhost:xxxxx +-- verify that the aggregate is propagated +select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%'; + aggfnoid +--------------------------------------------------------------------- + sumstring +(1 row) + +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.sumstring) + (localhost,57638,t,aggregate_support.sumstring) +(2 rows) + select create_distributed_function('sumstring(text)'); create_distributed_function --------------------------------------------------------------------- @@ -1042,5 +1067,122 @@ LEFT JOIN ref_table ON TRUE; 109 (1 row) +-- try createing aggregate having non-distributable dependency type +create table dummy_tbl (a int); +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; +WARNING: Citus can't distribute function "dummy_fnc" having dependency on non-distributed relation "dummy_tbl" +DETAIL: Function will be created only locally +HINT: To distribute function, distribute dependent relations first. Then, re-create the function +-- should give warning and create aggregate local only +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +WARNING: Citus can't distribute aggregate "dependent_agg" having dependency on non-distributed relation "dummy_tbl" +DETAIL: Aggregate will be created only locally +HINT: To distribute aggregate, distribute dependent relations first. Then, re-create the aggregate +-- clear and try again with distributed table +DROP TABLE dummy_tbl CASCADE; +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to function dummy_fnc(dummy_tbl,double precision) +drop cascades to function dependent_agg(double precision) +create table dummy_tbl (a int); +SELECT create_distributed_table('dummy_tbl','a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; +-- test in tx block +-- shouldn't distribute, as citus.create_object_propagation is set to deferred +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify not distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"") + (localhost,57638,t,"") +(2 rows) + +drop aggregate dependent_agg ( double precision); +-- now try with create_object_propagation = immediate +SET citus.create_object_propagation TO immediate; +-- should distribute, as citus.create_object_propagation is set to immediate +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + +drop aggregate dependent_agg ( double precision); +-- now try with create_object_propagation = automatic +SET citus.create_object_propagation TO automatic; +-- should distribute, as citus.create_object_propagation is set to automatic +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +RESET citus.create_object_propagation; +-- drop and test outside of tx block +drop aggregate dependent_agg (float8); +-- verify that the aggregate is removed from pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist") + (localhost,57638,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist") +(2 rows) + +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +--verify +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + +DROP TABLE dummy_tbl CASCADE; +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to function dummy_fnc(dummy_tbl,double precision) +drop cascades to function dependent_agg(double precision) +SET citus.create_object_propagation TO automatic; +begin; + create type typ1 as (a int); + create or replace function fnagg(a typ1, d double precision) RETURNS typ1 AS $$SELECT 1;$$LANGUAGE sql; + create aggregate dependent_agg (float8) (stype=typ1, sfunc=fnagg); +commit; +RESET citus.create_object_propagation; +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + set client_min_messages to error; drop schema aggregate_support cascade; diff --git a/src/test/regress/expected/coordinator_shouldhaveshards.out b/src/test/regress/expected/coordinator_shouldhaveshards.out index 5d5fa982c..485e7f11b 100644 --- a/src/test/regress/expected/coordinator_shouldhaveshards.out +++ b/src/test/regress/expected/coordinator_shouldhaveshards.out @@ -249,8 +249,30 @@ SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; (1 row) BEGIN; -SET citus.enable_repartition_joins TO ON; +SET citus.enable_unique_job_ids TO off; SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; +NOTICE: executing the command locally: SELECT partition_index, 'repartition_25_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_25_1','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503000 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_25_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_25_4','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503003 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_26_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_26_1','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503000 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_26_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_26_4','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503003 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_1_0']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_2_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_3_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_4_0']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_1_0']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_2_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_3_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_4_0']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_1_3']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_2_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_3_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_4_3']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_1_3']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_2_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_3_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_4_3']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_25_1_0,repartition_25_2_0,repartition_25_3_0,repartition_25_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_26_1_0,repartition_26_2_0,repartition_26_3_0,repartition_26_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_25_1_3,repartition_25_2_3,repartition_25_3_3,repartition_25_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_26_1_3,repartition_26_2_3,repartition_26_3_3,repartition_26_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true count --------------------------------------------------------------------- 100 @@ -268,9 +290,33 @@ NOTICE: executing the command locally: SELECT y FROM coordinator_shouldhaveshar (1 row) SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; -ERROR: cannot execute command because a local execution has accessed a placement in the transaction -DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally -HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" +NOTICE: executing the command locally: SELECT partition_index, 'repartition_29_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_29_1','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503000 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_29_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_29_4','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503003 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_30_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_30_1','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503000 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_30_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_30_4','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503003 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_1_2']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_2_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_3_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_4_2']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_1_2']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_2_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_3_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_4_2']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_1_5']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_2_5']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_3_5']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_4_5']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_1_5']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_2_5']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_3_5']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_4_5']::text[],'localhost',57636) bytes +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_29_1_2,repartition_29_2_2,repartition_29_3_2,repartition_29_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_30_1_2,repartition_30_2_2,repartition_30_3_2,repartition_30_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_29_1_5,repartition_29_2_5,repartition_29_3_5,repartition_29_4_5}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_30_1_5,repartition_30_2_5,repartition_30_3_5,repartition_30_4_5}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true + count +--------------------------------------------------------------------- + 100 +(1 row) + ROLLBACK; CREATE TABLE ref (a int, b int); SELECT create_reference_table('ref'); diff --git a/src/test/regress/expected/distributed_triggers.out b/src/test/regress/expected/distributed_triggers.out index ac038be51..b4ad001f5 100644 --- a/src/test/regress/expected/distributed_triggers.out +++ b/src/test/regress/expected/distributed_triggers.out @@ -386,7 +386,7 @@ ORDER BY shard_key_value, object_id, change_id; -- Triggers (tables) which are not colocated -- CREATE TABLE emptest ( - empname text NOT NULL, + empname text NOT NULL PRIMARY KEY, salary integer ); CREATE TABLE emptest_audit( @@ -394,7 +394,8 @@ CREATE TABLE emptest_audit( stamp timestamp NOT NULL, userid text NOT NULL, empname text NOT NULL, - salary integer + salary integer, + PRIMARY KEY (empname, userid, stamp, operation, salary) ); SELECT create_distributed_table('emptest','empname',colocate_with :='none'); create_distributed_table @@ -477,6 +478,7 @@ CREATE TABLE record_op ( operation_type text not null, stamp timestamp NOT NULL ); +ALTER TABLE record_op REPLICA IDENTITY FULL; SELECT create_distributed_table('record_op', 'empname', colocate_with := 'emptest'); create_distributed_table --------------------------------------------------------------------- @@ -654,7 +656,7 @@ PARTITION BY list (state_code); ALTER TABLE sale ADD CONSTRAINT sale_pk PRIMARY KEY (state_code, sale_date); CREATE TABLE sale_newyork PARTITION OF sale FOR VALUES IN ('NY'); CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA'); -CREATE TABLE record_sale(operation_type text not null, product_sku text, state_code text); +CREATE TABLE record_sale(operation_type text not null, product_sku text, state_code text, units integer, PRIMARY KEY(state_code, product_sku, operation_type, units)); SELECT create_distributed_table('sale', 'state_code'); create_distributed_table --------------------------------------------------------------------- @@ -671,8 +673,8 @@ CREATE OR REPLACE FUNCTION record_sale() RETURNS trigger AS $$ BEGIN - INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code) - VALUES (TG_OP, NEW.product_sku, NEW.state_code); + INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code, units) + VALUES (TG_OP, NEW.product_sku, NEW.state_code, NEW.units); RETURN NULL; END; $$ LANGUAGE plpgsql; @@ -697,7 +699,7 @@ TABLE sale ORDER BY state_code, sale_date; 02-03-2019 | NY | AZ-000A1 | 47 (6 rows) -TABLE record_sale ORDER BY 1,2,3; +SELECT operation_type, product_sku, state_code FROM record_sale ORDER BY 1,2,3; operation_type | product_sku | state_code --------------------------------------------------------------------- INSERT | AZ-000A1 | CA diff --git a/src/test/regress/expected/distributed_types.out b/src/test/regress/expected/distributed_types.out index dcf8dd8b2..2e2ef9c1f 100644 --- a/src/test/regress/expected/distributed_types.out +++ b/src/test/regress/expected/distributed_types.out @@ -453,6 +453,107 @@ SELECT * FROM field_indirection_test_2 ORDER BY 1,2,3; 8 | (10," text10",20) | (40,50) (2 rows) +-- test different ddl propagation modes +SET citus.create_object_propagation TO deferred; +BEGIN; +CREATE TYPE deferred_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +CREATE TABLE deferred_table(a int,b deferred_type); +SELECT create_distributed_table('deferred_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +COMMIT; +SET citus.create_object_propagation TO automatic; +BEGIN; +CREATE TYPE automatic_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +CREATE TABLE automatic_table(a int,b automatic_type); +SELECT create_distributed_table('automatic_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +COMMIT; +SET citus.create_object_propagation TO automatic; +BEGIN; +-- force parallel execution by preceding with a analytical query +SET LOCAL citus.force_max_query_parallelization TO on; +SELECT count(*) FROM automatic_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +CREATE TYPE automatic2_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +CREATE TABLE automatic2_table(a int,b automatic2_type); +SELECT create_distributed_table('automatic2_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +COMMIT; +SET citus.create_object_propagation TO immediate; +BEGIN; +CREATE TYPE immediate_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +CREATE TABLE immediate_table(a int,b immediate_type); +SELECT create_distributed_table('immediate_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +COMMIT; -- clear objects SET client_min_messages TO error; -- suppress cascading objects dropping DROP SCHEMA type_tests CASCADE; diff --git a/src/test/regress/expected/follower_single_node.out b/src/test/regress/expected/follower_single_node.out index de2f88b6e..4c7a03954 100644 --- a/src/test/regress/expected/follower_single_node.out +++ b/src/test/regress/expected/follower_single_node.out @@ -157,25 +157,50 @@ SELECT * FROM ref, local WHERE a = c ORDER BY a; SET citus.enable_repartition_joins TO ON; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + x | y | x | y +--------------------------------------------------------------------- + 2 | 7 | 1 | 2 + 4 | 5 | 3 | 4 + 5 | 6 | 4 | 5 +(3 rows) + SET citus.enable_single_hash_repartition_joins TO ON; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + x | y | x | y +--------------------------------------------------------------------- + 2 | 7 | 1 | 2 + 4 | 5 | 3 | 4 + 5 | 6 | 4 | 5 +(3 rows) + SET citus.task_assignment_policy TO 'round-robin'; SET citus.enable_single_hash_repartition_joins TO ON; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + x | y | x | y +--------------------------------------------------------------------- + 2 | 7 | 1 | 2 + 4 | 5 | 3 | 4 + 5 | 6 | 4 | 5 +(3 rows) + SET citus.task_assignment_policy TO 'greedy'; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + x | y | x | y +--------------------------------------------------------------------- + 2 | 7 | 1 | 2 + 4 | 5 | 3 | 4 + 5 | 6 | 4 | 5 +(3 rows) + SET citus.task_assignment_policy TO 'first-replica'; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + x | y | x | y +--------------------------------------------------------------------- + 2 | 7 | 1 | 2 + 4 | 5 | 3 | 4 + 5 | 6 | 4 | 5 +(3 rows) + RESET citus.enable_repartition_joins; RESET citus.enable_single_hash_repartition_joins; -- Confirm that dummy placements work @@ -319,12 +344,22 @@ SELECT * FROM ref, local WHERE a = c ORDER BY a; SET citus.enable_repartition_joins TO ON; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + x | y | x | y +--------------------------------------------------------------------- + 2 | 7 | 1 | 2 + 4 | 5 | 3 | 4 + 5 | 6 | 4 | 5 +(3 rows) + SET citus.enable_single_hash_repartition_joins TO ON; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + x | y | x | y +--------------------------------------------------------------------- + 2 | 7 | 1 | 2 + 4 | 5 | 3 | 4 + 5 | 6 | 4 | 5 +(3 rows) + RESET citus.enable_repartition_joins; RESET citus.enable_single_hash_repartition_joins; -- Confirm that dummy placements work diff --git a/src/test/regress/expected/function_propagation.out b/src/test/regress/expected/function_propagation.out index a40f1f9a2..ac645c263 100644 --- a/src/test/regress/expected/function_propagation.out +++ b/src/test/regress/expected/function_propagation.out @@ -316,6 +316,710 @@ BEGIN return 1; END; $$; +-- Show that functions are propagated (or not) as a dependency +-- Function as a default column +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Function shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func(id int, col_1 int default func_in_transaction_def()); +SELECT create_distributed_table('table_to_prop_func','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Function should be marked as distributed after distributing the table that depends on it +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_def}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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_def'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_def}",{}) +(2 rows) + +-- Multiple functions as a default column +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_1() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +CREATE OR REPLACE FUNCTION func_in_transaction_2() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::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_2'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2()); +SELECT create_distributed_table('table_to_prop_func_2','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Functions should be marked as distribued after distributing the table that depends on it +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_1}",{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_2}",{}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +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_1'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{}) +(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_2'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_2}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_2}",{}) +(2 rows) + +-- If function has dependency on non-distributed table it should error out +BEGIN; +CREATE TABLE non_dist_table(id int); +CREATE OR REPLACE FUNCTION func_in_transaction_3(param_1 non_dist_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +CREATE TABLE table_to_prop_func_3(id int, col_1 int default func_in_transaction_3(NULL::non_dist_table)); +-- It should error out as there is a non-distributed table dependency +SELECT create_distributed_table('table_to_prop_func_3','id'); +ERROR: type function_propagation_schema.non_dist_table does not exist +CONTEXT: while executing command on localhost:xxxxx +COMMIT; +-- Adding a column with default value should propagate the function +BEGIN; +CREATE TABLE table_to_prop_func_4(id int); +SELECT create_distributed_table('table_to_prop_func_4', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION func_in_transaction_4() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Function shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4(); +-- Function should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_4}",{}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +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_4'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_4}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_4}",{}) +(2 rows) + +-- Adding multiple columns with default values should propagate the function +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_5() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +CREATE OR REPLACE FUNCTION func_in_transaction_6() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::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_6'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_5(id int, col_1 int default func_in_transaction_5(), col_2 int default func_in_transaction_6()); +SELECT create_distributed_table('table_to_prop_func_5', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Functions should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_5}",{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_6}",{}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +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_5'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{}) +(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_6'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_6}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_6}",{}) +(2 rows) + +-- Adding a constraint with function check should propagate the function +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_7(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_6(id int, col_1 int check (function_propagation_schema.func_in_transaction_7(col_1))); +SELECT create_distributed_table('table_to_prop_func_6', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Function should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_7}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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_7'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_7}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_7}",{integer}) +(2 rows) + +-- Adding a constraint with multiple functions check should propagate the function +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_8(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +CREATE OR REPLACE FUNCTION func_in_transaction_9(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::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_9'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_7(id int, col_1 int check (function_propagation_schema.func_in_transaction_8(col_1) and function_propagation_schema.func_in_transaction_9(col_1))); +SELECT create_distributed_table('table_to_prop_func_7', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Function should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_8}",{integer}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_9}",{integer}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +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_8'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer}) +(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_9'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_9}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_9}",{integer}) +(2 rows) + +-- Adding a column with constraint should propagate the function +BEGIN; +CREATE TABLE table_to_prop_func_8(id int, col_1 int); +SELECT create_distributed_table('table_to_prop_func_8', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION func_in_transaction_10(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1)); +-- Function should be marked as distributed after adding the constraint +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_10}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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_10'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_10}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_10}",{integer}) +(2 rows) + +-- If constraint depends on a non-distributed table it should error out +BEGIN; +CREATE TABLE local_table_for_const(id int); +CREATE OR REPLACE FUNCTION func_in_transaction_11(param_1 int, param_2 local_table_for_const) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +CREATE TABLE table_to_prop_func_9(id int, col_1 int check (func_in_transaction_11(col_1, NULL::local_table_for_const))); +-- It should error out since there is non-distributed table dependency exists +SELECT create_distributed_table('table_to_prop_func_9', 'id'); +ERROR: type function_propagation_schema.local_table_for_const does not exist +CONTEXT: while executing command on localhost:xxxxx +COMMIT; +-- Show that function as a part of generated always is supporte +BEGIN; + CREATE OR REPLACE FUNCTION non_sense_func_for_generated_always() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE people ( + id int, + height_cm numeric, + height_in numeric GENERATED ALWAYS AS (height_cm / non_sense_func_for_generated_always()) STORED); + SELECT create_distributed_table('people', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,non_sense_func_for_generated_always}",{}) +(1 row) + +COMMIT; +-- Show that functions depending table via rule are also distributed +BEGIN; +CREATE OR REPLACE FUNCTION func_for_rule() +RETURNS int +LANGUAGE plpgsql STABLE AS +$$ +BEGIN + return 4; +END; +$$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_1_for_rule(id int, col_1 int); +CREATE TABLE table_2_for_rule(id int, col_1 int); +CREATE RULE rule_1 AS ON UPDATE TO table_1_for_rule DO ALSO UPDATE table_2_for_rule SET col_1 = col_1 * func_for_rule(); +SELECT create_distributed_table('table_1_for_rule','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Functions should be distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_for_rule}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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_for_rule'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_for_rule}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_for_rule}",{}) +(2 rows) + +-- Show that functions as partitioning functions are supported +BEGIN; + CREATE OR REPLACE FUNCTION non_sense_func_for_partitioning(int) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE partitioned_table_to_test_func_prop(id INT, a INT) PARTITION BY RANGE (non_sense_func_for_partitioning(id)); + SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer}) +(2 rows) + +-- Test function dependency on citus local table +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_for_local_table() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE citus_local_table_to_test_func(l1 int DEFAULT func_in_transaction_for_local_table()); + SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + + SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_for_local_table}",{}) +(1 row) + +ROLLBACK; +-- Show that having a function dependency on exlude also works +BEGIN; + CREATE OR REPLACE FUNCTION exclude_bool_func() + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return true; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE exclusion_func_prop_table (id int, EXCLUDE USING btree (id WITH =) WHERE (exclude_bool_func())); + SELECT create_distributed_table('exclusion_func_prop_table', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,exclude_bool_func}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,exclude_bool_func}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,exclude_bool_func}",{}) +(2 rows) + +-- Show that having a function dependency for index also works +BEGIN; + CREATE OR REPLACE FUNCTION func_for_index_predicate(col_1 int) + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return col_1 > 5; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE table_to_check_func_index_dep (id int, col_2 int); + CREATE INDEX on table_to_check_func_index_dep(col_2) WHERE (func_for_index_predicate(col_2)); + SELECT create_distributed_table('table_to_check_func_index_dep', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_for_index_predicate}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_for_index_predicate}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_for_index_predicate}",{integer}) +(2 rows) + +-- Test function to function dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_for_func_dep_1() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + CREATE TABLE func_dep_table(a int, b int default func_for_func_dep_1()); + CREATE OR REPLACE FUNCTION func_for_func_dep_2(col_1 func_dep_table) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + SELECT create_distributed_table('func_dep_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_for_func_dep_1}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_for_func_dep_1}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_for_func_dep_1}",{}) +(2 rows) + +-- Test function with SQL language and sequence dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_seq(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT 2 * val; + $$; + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_func(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT func_in_transaction_def_with_seq(val); + $$; + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE SEQUENCE myseq; + CREATE TABLE table_to_prop_seq_func(id int, col_1 bigint default func_in_transaction_def_with_func(func_in_transaction_def_with_seq(nextval('myseq')))); + SELECT create_distributed_table('table_to_prop_seq_func','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +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_def_with_seq'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint}) +(2 rows) + RESET search_path; SET client_min_messages TO WARNING; DROP SCHEMA function_propagation_schema CASCADE; diff --git a/src/test/regress/expected/intermediate_results.out b/src/test/regress/expected/intermediate_results.out index c510b07b3..0bc522ea0 100644 --- a/src/test/regress/expected/intermediate_results.out +++ b/src/test/regress/expected/intermediate_results.out @@ -251,9 +251,6 @@ SELECT * FROM squares ORDER BY x; 5 | 25 (5 rows) --- empty shard interval array should raise error -SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[0]); -ERROR: invalid distribution column value -- cannot use DDL commands select broadcast_intermediate_result('a', 'create table foo(int serial)'); ERROR: cannot execute utility commands @@ -507,10 +504,9 @@ WARNING: Query could not find the intermediate result file "squares_2", it was (0 rows) ROLLBACK TO SAVEPOINT s1; --- fetch from worker 2 should fail -SELECT * FROM fetch_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'localhost', :worker_2_port); -ERROR: could not open file "base/pgsql_job_cache/xx_x_xxx/squares_1.data": No such file or directory -CONTEXT: while executing command on localhost:xxxxx +-- fetch from invalid worker port should fail +SELECT * FROM fetch_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'localhost', 57635); +ERROR: cannot connect to localhost:xxxxx to fetch intermediate results ROLLBACK TO SAVEPOINT s1; -- still, results aren't available on coordinator yet SELECT * FROM read_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'binary') AS res (x int, x2 int); diff --git a/src/test/regress/expected/isolation_citus_dist_activity.out b/src/test/regress/expected/isolation_citus_dist_activity.out index f6cd7da9b..aa536e17f 100644 --- a/src/test/regress/expected/isolation_citus_dist_activity.out +++ b/src/test/regress/expected/isolation_citus_dist_activity.out @@ -42,7 +42,7 @@ query |query_hostname |query_hostport|d (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -112,7 +112,7 @@ query |query_hostname |query_hostport|di (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -176,7 +176,7 @@ query |query_hostname |query_hostport|distribute (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -243,7 +243,7 @@ query |query_hostname |query_ (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- diff --git a/src/test/regress/expected/isolation_distributed_transaction_id.out b/src/test/regress/expected/isolation_distributed_transaction_id.out index 3d44f0069..be52248f8 100644 --- a/src/test/regress/expected/isolation_distributed_transaction_id.out +++ b/src/test/regress/expected/isolation_distributed_transaction_id.out @@ -13,7 +13,7 @@ assign_distributed_transaction_id (1 row) step s1-get-all-transactions: - SELECT initiator_node_identifier, transaction_number, transaction_stamp FROM get_current_transaction_id() ORDER BY 1,2,3; + SELECT initiator_node_identifier, transaction_number, transaction_stamp FROM get_current_transaction_id() ORDER BY 1,2,3; initiator_node_identifier|transaction_number|transaction_stamp --------------------------------------------------------------------- @@ -32,7 +32,7 @@ assign_distributed_transaction_id (1 row) step s2-get-all-transactions: - SELECT initiator_node_identifier, transaction_number, transaction_stamp FROM get_current_transaction_id() ORDER BY 1,2,3; + SELECT initiator_node_identifier, transaction_number, transaction_stamp FROM get_current_transaction_id() ORDER BY 1,2,3; initiator_node_identifier|transaction_number|transaction_stamp --------------------------------------------------------------------- @@ -51,7 +51,7 @@ assign_distributed_transaction_id (1 row) step s3-get-all-transactions: - SELECT initiator_node_identifier, transaction_number, transaction_stamp FROM get_current_transaction_id() ORDER BY 1,2,3; + SELECT initiator_node_identifier, transaction_number, transaction_stamp FROM get_current_transaction_id() ORDER BY 1,2,3; initiator_node_identifier|transaction_number|transaction_stamp --------------------------------------------------------------------- @@ -70,10 +70,10 @@ step s3-commit: starting permutation: s1-create-table s1-begin s1-insert s1-verify-current-xact-is-on-worker s1-drop-table s1-commit step s1-create-table: - -- some tests also use distributed table - CREATE TABLE distributed_transaction_id_table(some_value int, other_value int); - SET citus.shard_count TO 4; - SELECT create_distributed_table('distributed_transaction_id_table', 'some_value'); + -- some tests also use distributed table + CREATE TABLE distributed_transaction_id_table(some_value int, other_value int); + SET citus.shard_count TO 4; + SELECT create_distributed_table('distributed_transaction_id_table', 'some_value'); create_distributed_table --------------------------------------------------------------------- @@ -84,16 +84,16 @@ step s1-begin: BEGIN; step s1-insert: - INSERT INTO distributed_transaction_id_table VALUES (1, 1); + INSERT INTO distributed_transaction_id_table VALUES (1, 1); step s1-verify-current-xact-is-on-worker: - SELECT - remote.nodeport, - remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists - FROM - get_current_transaction_id() as xact, - run_command_on_workers($$ - SELECT row(initiator_node_identifier, transaction_number) + SELECT + remote.nodeport, + remote.result = row(xact.transaction_number)::text AS xact_exists + FROM + get_current_transaction_id() as xact, + run_command_on_workers($$ + SELECT row(transaction_number) FROM get_all_active_transactions() WHERE transaction_number != 0; $$) as remote @@ -106,7 +106,7 @@ nodeport|xact_exists (2 rows) step s1-drop-table: - DROP TABLE distributed_transaction_id_table; + DROP TABLE distributed_transaction_id_table; step s1-commit: COMMIT; diff --git a/src/test/regress/expected/isolation_drop_vs_all.out b/src/test/regress/expected/isolation_drop_vs_all.out index 6009eb41d..e4cfe791a 100644 --- a/src/test/regress/expected/isolation_drop_vs_all.out +++ b/src/test/regress/expected/isolation_drop_vs_all.out @@ -1,12 +1,13 @@ Parsed test spec with 2 sessions -starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-drop s1-commit s2-commit s1-select-count +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-drop s1-commit s2-commit s1-select-count create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-drop: DROP TABLE drop_hash; @@ -23,13 +24,60 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-ddl-create-index s1-commit s2-commit s1-select-count s1-show-indexes +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop-schema s2-drop-schema s1-commit s2-commit s1-select-count create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; +step s1-begin: BEGIN; +step s2-begin: BEGIN; +step s1-drop-schema: DROP SCHEMA drop_tests CASCADE; +step s2-drop-schema: DROP SCHEMA drop_tests CASCADE; +step s1-commit: COMMIT; +step s2-drop-schema: <... completed> +ERROR: schema "drop_tests" does not exist +step s2-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM drop_hash; +ERROR: relation "drop_hash" does not exist +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop-schema s2-drop-schema-2 s1-commit s2-commit s1-select-count +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; +step s1-begin: BEGIN; +step s2-begin: BEGIN; +step s1-drop-schema: DROP SCHEMA drop_tests CASCADE; +step s2-drop-schema-2: DROP SCHEMA drop_tests_2 CASCADE; +step s1-commit: COMMIT; +step s2-commit: COMMIT; +step s1-select-count: SELECT COUNT(*) FROM drop_hash; +ERROR: relation "drop_hash" does not exist +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-ddl-create-index s1-commit s2-commit s1-select-count s1-show-indexes +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-drop: DROP TABLE drop_hash; @@ -53,13 +101,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-ddl-create-index s1-begin s2-begin s1-drop s2-ddl-drop-index s1-commit s2-commit s1-select-count s1-show-indexes +starting permutation: s1-initialize s2-initialize s1-ddl-create-index s1-begin s2-begin s1-drop s2-ddl-drop-index s1-commit s2-commit s1-select-count s1-show-indexes create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id); step s1-begin: BEGIN; step s2-begin: BEGIN; @@ -84,13 +133,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s1-drop s2-ddl-create-index-concurrently s1-commit s1-select-count s1-show-indexes +starting permutation: s1-initialize s2-initialize s1-begin s1-drop s2-ddl-create-index-concurrently s1-commit s1-select-count s1-show-indexes create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s1-drop: DROP TABLE drop_hash; step s2-ddl-create-index-concurrently: CREATE INDEX CONCURRENTLY drop_hash_index ON drop_hash(id); @@ -112,13 +162,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-ddl-add-column s1-commit s2-commit s1-select-count s1-show-columns +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-ddl-add-column s1-commit s2-commit s1-select-count s1-show-columns create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-drop: DROP TABLE drop_hash; @@ -142,13 +193,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-ddl-add-column s1-begin s2-begin s1-drop s2-ddl-drop-column s1-commit s2-commit s1-select-count s1-show-columns +starting permutation: s1-initialize s2-initialize s1-ddl-add-column s1-begin s2-begin s1-drop s2-ddl-drop-column s1-commit s2-commit s1-select-count s1-show-columns create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0; step s1-begin: BEGIN; step s2-begin: BEGIN; @@ -173,13 +225,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-ddl-rename-column s1-commit s2-commit s1-select-count s1-show-columns +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-ddl-rename-column s1-commit s2-commit s1-select-count s1-show-columns create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-drop: DROP TABLE drop_hash; @@ -203,13 +256,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-table-size s1-commit s2-commit s1-select-count +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-table-size s1-commit s2-commit s1-select-count create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-drop: DROP TABLE drop_hash; @@ -226,7 +280,7 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s2-begin s1-drop s2-distribute-table s1-commit s2-commit s1-select-count +starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-distribute-table s1-commit s2-commit s1-select-count create_distributed_table --------------------------------------------------------------------- @@ -234,7 +288,8 @@ create_distributed_table step s1-drop: DROP TABLE drop_hash; step s1-create-non-distributed-table: CREATE TABLE drop_hash(id integer, data text); COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-drop: DROP TABLE drop_hash; @@ -251,13 +306,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-ddl-create-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-ddl-create-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id); @@ -280,13 +336,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-ddl-create-index s1-begin s2-begin s1-ddl-drop-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes +starting permutation: s1-initialize s2-initialize s1-ddl-create-index s1-begin s2-begin s1-ddl-drop-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id); step s1-begin: BEGIN; step s2-begin: BEGIN; @@ -310,13 +367,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-ddl-add-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-ddl-add-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0; @@ -339,13 +397,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-ddl-add-column s1-begin s2-begin s1-ddl-drop-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns +starting permutation: s1-initialize s2-initialize s1-ddl-add-column s1-begin s2-begin s1-ddl-drop-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0; step s1-begin: BEGIN; step s2-begin: BEGIN; @@ -369,13 +428,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-ddl-rename-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-ddl-rename-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-ddl-rename-column: ALTER TABLE drop_hash RENAME data TO new_column; @@ -398,13 +458,14 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-initialize s1-begin s2-begin s1-table-size s2-drop s1-commit s2-commit s1-select-count +starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-table-size s2-drop s1-commit s2-commit s1-select-count create_distributed_table --------------------------------------------------------------------- (1 row) -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-table-size: SELECT citus_total_relation_size('drop_hash'); @@ -424,7 +485,7 @@ restore_isolation_tester_func (1 row) -starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s2-begin s1-distribute-table s2-drop s1-commit s2-commit s1-select-count +starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s2-initialize s1-begin s2-begin s1-distribute-table s2-drop s1-commit s2-commit s1-select-count create_distributed_table --------------------------------------------------------------------- @@ -432,7 +493,8 @@ create_distributed_table step s1-drop: DROP TABLE drop_hash; step s1-create-non-distributed-table: CREATE TABLE drop_hash(id integer, data text); COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; -step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; +step s2-initialize: SET search_path TO 'drop_tests'; step s1-begin: BEGIN; step s2-begin: BEGIN; step s1-distribute-table: SELECT create_distributed_table('drop_hash', 'id'); diff --git a/src/test/regress/expected/isolation_get_all_active_transactions.out b/src/test/regress/expected/isolation_get_all_active_transactions.out index d5c4765b8..87f4e6f33 100644 --- a/src/test/regress/expected/isolation_get_all_active_transactions.out +++ b/src/test/regress/expected/isolation_get_all_active_transactions.out @@ -8,10 +8,10 @@ run_command_on_workers (2 rows) step s1-grant: - GRANT ALL ON test_table TO test_user_1; - SELECT bool_and(success) FROM run_command_on_placements('test_table', 'GRANT ALL ON TABLE %s TO test_user_1'); - GRANT ALL ON test_table TO test_user_2; - SELECT bool_and(success) FROM run_command_on_placements('test_table', 'GRANT ALL ON TABLE %s TO test_user_2'); + GRANT ALL ON test_table TO test_user_1; + SELECT bool_and(success) FROM run_command_on_placements('test_table', 'GRANT ALL ON TABLE %s TO test_user_1'); + GRANT ALL ON test_table TO test_user_2; + SELECT bool_and(success) FROM run_command_on_placements('test_table', 'GRANT ALL ON TABLE %s TO test_user_2'); bool_and --------------------------------------------------------------------- @@ -24,19 +24,19 @@ t (1 row) step s1-begin-insert: - BEGIN; - SET ROLE test_user_1; - INSERT INTO test_table VALUES (100, 100); + BEGIN; + SET ROLE test_user_1; + INSERT INTO test_table VALUES (100, 100); step s2-begin-insert: - BEGIN; - SET ROLE test_user_2; - INSERT INTO test_table VALUES (200, 200); + BEGIN; + SET ROLE test_user_2; + INSERT INTO test_table VALUES (200, 200); step s3-as-admin: - -- Admin should be able to see all transactions - SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; - SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; + -- Admin should be able to see all transactions + SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; + SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; count --------------------------------------------------------------------- @@ -49,26 +49,35 @@ count (1 row) step s3-as-user-1: - -- User should only be able to see its own transactions - SET ROLE test_user_1; - SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; - SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; + -- Eventhough we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert) + -- userId (e.g., PG_PROC->userId) does not change, and hence none of the + -- transactions show up because here we are using test_user_1. This is a + -- limitation of isolation tester, we should be able to re-connect with + -- test_user_1 on s1/2-begin-insert to show that test_user_1 sees only its own processes + SET ROLE test_user_1; + SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; + SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; + +count +--------------------------------------------------------------------- + 0 +(1 row) count --------------------------------------------------------------------- 1 (1 row) -count ---------------------------------------------------------------------- - 2 -(1 row) - step s3-as-readonly: - -- Other user should not see transactions - SET ROLE test_readonly; - SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; - SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; + -- Eventhough we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert) + -- userId (e.g., PG_PROC->userId) does not change, and hence none of the + -- transactions show up because here we are using test_readonly. This is a + -- limitation of isolation tester, we should be able to re-connect with + -- test_readonly on s1/2-begin-insert to show that test_readonly sees only + -- its own processes + SET ROLE test_readonly; + SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; + SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; count --------------------------------------------------------------------- @@ -81,10 +90,10 @@ count (1 row) step s3-as-monitor: - -- Monitor should see all transactions - SET ROLE test_monitor; - SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; - SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; + -- Monitor should see all transactions + SET ROLE test_monitor; + SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; + SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; count --------------------------------------------------------------------- @@ -97,10 +106,10 @@ count (1 row) step s1-commit: - COMMIT; + COMMIT; step s2-commit: - COMMIT; + COMMIT; run_command_on_workers --------------------------------------------------------------------- diff --git a/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out b/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out index cae2222ed..109c61186 100644 --- a/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out +++ b/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out @@ -33,7 +33,7 @@ blocked_statement |current_statement_in_blockin --------------------------------------------------------------------- UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 15; -|localhost |coordinator_host | 57638| 57636 +|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit: @@ -116,7 +116,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -212,7 +212,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |localhost |localhost | 57638| 57637 +UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -308,7 +308,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -404,7 +404,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -594,7 +594,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -878,7 +878,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE |localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE |coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -963,7 +963,7 @@ blocked_statement |current_s --------------------------------------------------------------------- ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); -|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |localhost | 57636| 57638 +|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s2-commit-worker: @@ -1073,7 +1073,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |localhost |localhost | 57637| 57637 +UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -1161,7 +1161,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|localhost |localhost | 57638| 57637 +UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -1225,7 +1225,7 @@ blocked_statement |current_statement_in_blockin --------------------------------------------------------------------- UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 15; -|localhost |coordinator_host | 57638| 57636 +|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit: diff --git a/src/test/regress/expected/isolation_metadata_sync_vs_all.out b/src/test/regress/expected/isolation_metadata_sync_vs_all.out index 087dcbe93..49ce8fa3e 100644 --- a/src/test/regress/expected/isolation_metadata_sync_vs_all.out +++ b/src/test/regress/expected/isolation_metadata_sync_vs_all.out @@ -237,6 +237,126 @@ t (1 row) +starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-schema s1-commit s2-commit s3-compare-snapshot s2-drop-schema +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-create-schema: + CREATE SCHEMA dist_schema + CREATE TABLE dist_table_in_schema(id int, data int); + SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id'); + +step s1-commit: + COMMIT; + +step s2-create-schema: <... completed> +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s2-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + +step s2-drop-schema: + DROP SCHEMA dist_schema CASCADE; + + +starting permutation: s2-create-schema s1-begin s2-begin s1-start-metadata-sync s2-drop-schema s1-commit s2-commit s3-compare-snapshot +step s2-create-schema: + CREATE SCHEMA dist_schema + CREATE TABLE dist_table_in_schema(id int, data int); + SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id'); + +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-drop-schema: + DROP SCHEMA dist_schema CASCADE; + +step s1-commit: + COMMIT; + +step s2-drop-schema: <... completed> +step s2-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + + starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-dist-table s1-commit s2-commit s3-compare-snapshot step s1-begin: BEGIN; @@ -608,6 +728,415 @@ t (1 row) +starting permutation: s2-create-type s1-begin s2-begin s1-start-metadata-sync s2-drop-type s1-commit s2-commit s3-compare-snapshot +step s2-create-type: + CREATE TYPE my_type AS (a int, b int); + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-drop-type: + DROP TYPE my_type; + +step s1-commit: + COMMIT; + +step s2-drop-type: <... completed> +step s2-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + + +starting permutation: s2-create-dist-func s1-begin s2-begin s1-start-metadata-sync s2-drop-dist-func s1-commit s2-commit s3-compare-snapshot +step s2-create-dist-func: + CREATE FUNCTION squares(int) RETURNS SETOF RECORD + AS $$ SELECT i, i * i FROM generate_series(1, $1) i $$ + LANGUAGE SQL; + SELECT create_distributed_function('squares(int)'); + +create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-drop-dist-func: + DROP FUNCTION squares(int); + +step s1-commit: + COMMIT; + +step s2-drop-dist-func: <... completed> +step s2-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + + +starting permutation: s2-create-type s1-begin s1-start-metadata-sync s2-alter-type s1-commit s3-compare-snapshot s3-compare-type-definition +step s2-create-type: + CREATE TYPE my_type AS (a int, b int); + +step s1-begin: + BEGIN; + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-alter-type: + ALTER TYPE my_type ADD ATTRIBUTE x int; + +step s1-commit: + COMMIT; + +step s2-alter-type: <... completed> +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + +step s3-compare-type-definition: + SELECT run_command_on_workers($$SELECT '(1,1,1)'::my_type$$); + +run_command_on_workers +--------------------------------------------------------------------- +(localhost,57637,t,"(1,1,1)") +(localhost,57638,t,"(1,1,1)") +(2 rows) + + +starting permutation: s1-begin s2-begin s2-create-dist-table s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s2-create-dist-table: + CREATE TABLE new_dist_table(id int, data int); + SELECT create_distributed_table('new_dist_table', 'id'); + +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +step s2-commit: + COMMIT; + +step s1-start-metadata-sync: <... completed> +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + + +starting permutation: s2-create-dist-func s1-begin s2-begin s2-drop-dist-func s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot +step s2-create-dist-func: + CREATE FUNCTION squares(int) RETURNS SETOF RECORD + AS $$ SELECT i, i * i FROM generate_series(1, $1) i $$ + LANGUAGE SQL; + SELECT create_distributed_function('squares(int)'); + +create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s2-drop-dist-func: + DROP FUNCTION squares(int); + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +step s2-commit: + COMMIT; + +step s1-start-metadata-sync: <... completed> +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + + +starting permutation: s2-create-schema s1-begin s2-begin s2-drop-schema s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot +step s2-create-schema: + CREATE SCHEMA dist_schema + CREATE TABLE dist_table_in_schema(id int, data int); + SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id'); + +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s2-drop-schema: + DROP SCHEMA dist_schema CASCADE; + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +step s2-commit: + COMMIT; + +step s1-start-metadata-sync: <... completed> +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + + +starting permutation: s2-create-type s1-begin s2-begin s2-alter-type s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot s3-compare-type-definition +step s2-create-type: + CREATE TYPE my_type AS (a int, b int); + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s2-alter-type: + ALTER TYPE my_type ADD ATTRIBUTE x int; + +step s1-start-metadata-sync: + SELECT start_metadata_sync_to_node('localhost', 57638); + +step s2-commit: + COMMIT; + +step s1-start-metadata-sync: <... completed> +start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit: + COMMIT; + +step s3-compare-snapshot: + SELECT count(*) = 0 AS same_metadata_in_workers + FROM + ( + ( + SELECT unnest(activate_node_snapshot()) + EXCEPT + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + ) + UNION + ( + SELECT unnest(result::text[]) AS unnested_result + FROM run_command_on_workers($$SELECT activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) + ) + ) AS foo; + +same_metadata_in_workers +--------------------------------------------------------------------- +t +(1 row) + +step s3-compare-type-definition: + SELECT run_command_on_workers($$SELECT '(1,1,1)'::my_type$$); + +run_command_on_workers +--------------------------------------------------------------------- +(localhost,57637,t,"(1,1,1)") +(localhost,57638,t,"(1,1,1)") +(2 rows) + + starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-type s1-commit s2-commit s3-compare-snapshot step s1-begin: BEGIN; diff --git a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out index 2d49f8586..8fef72010 100644 --- a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out +++ b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out @@ -1,11 +1,19 @@ -Parsed test spec with 3 sessions +Parsed test spec with 4 sessions -starting permutation: s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end +starting permutation: add-node s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end create_distributed_table --------------------------------------------------------------------- (1 row) +step add-node: + SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0); + +?column? +--------------------------------------------------------------------- + 1 +(1 row) + step s1-begin: BEGIN; @@ -59,12 +67,20 @@ master_remove_node (1 row) -starting permutation: s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end +starting permutation: add-node s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end create_distributed_table --------------------------------------------------------------------- (1 row) +step add-node: + SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0); + +?column? +--------------------------------------------------------------------- + 1 +(1 row) + step s1-begin: BEGIN; @@ -83,7 +99,7 @@ pg_sleep (1 row) step s2-view-dist: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC; query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -92,7 +108,7 @@ query |query_hostname |query_hostport|distri |coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression update ref_table set a = a + 1; - |coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression + |coordinator_host| 57636| | 0|idle in transaction|Client |ClientRead|postgres|regression (2 rows) step s2-view-worker: @@ -102,13 +118,15 @@ step s2-view-worker: WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%dump_local_%' AND - query NOT ILIKE '%citus_internal_local_blocked_processes%' + query NOT ILIKE '%citus_internal_local_blocked_processes%' AND + query NOT ILIKE '%add_node%' AND + backend_type = 'client backend' ORDER BY query, query_hostport DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638| | 0|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637| | 0|idle in transaction|Client |ClientRead|postgres|regression (2 rows) step s2-end: @@ -123,12 +141,20 @@ master_remove_node (1 row) -starting permutation: s1-begin s2-begin s1-update-ref-table s2-active-transactions s1-end s2-end +starting permutation: add-node s1-begin s2-begin s1-update-ref-table s2-active-transactions s1-end s2-end create_distributed_table --------------------------------------------------------------------- (1 row) +step add-node: + SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0); + +?column? +--------------------------------------------------------------------- + 1 +(1 row) + step s1-begin: BEGIN; diff --git a/src/test/regress/expected/local_shard_execution.out b/src/test/regress/expected/local_shard_execution.out index 781a9c86c..5cb61b2d5 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -91,6 +91,7 @@ ALTER TABLE abcd DROP COLUMN a; -- connection worker and get ready for the tests \c - - - :worker_1_port SET search_path TO local_shard_execution; +SET citus.enable_unique_job_ids TO off; -- 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 @@ -733,9 +734,51 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar (1 row) SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); -ERROR: cannot execute command because a local execution has accessed a placement in the transaction -DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally -HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" +NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_1','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470001 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_3','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470003 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_66_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_66_1','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470001 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_66_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_66_3','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470003 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_0,repartition_65_2_0,repartition_65_3_0,repartition_65_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_0,repartition_66_2_0,repartition_66_3_0,repartition_66_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_1,repartition_65_2_1,repartition_65_3_1,repartition_65_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_1,repartition_66_2_1,repartition_66_3_1,repartition_66_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_2,repartition_65_2_2,repartition_65_3_2,repartition_65_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_2,repartition_66_2_2,repartition_66_3_2,repartition_66_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_3,repartition_65_2_3,repartition_65_3_3,repartition_65_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_3,repartition_66_2_3,repartition_66_3_3,repartition_66_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + ROLLBACK; -- a local query is followed by an INSERT..SELECT with re-partitioning BEGIN; diff --git a/src/test/regress/expected/local_shard_execution_replicated.out b/src/test/regress/expected/local_shard_execution_replicated.out index 731c825c3..285327095 100644 --- a/src/test/regress/expected/local_shard_execution_replicated.out +++ b/src/test/regress/expected/local_shard_execution_replicated.out @@ -682,6 +682,7 @@ NOTICE: executing the copy locally for shard xxxxx ROLLBACK; BEGIN; SET citus.enable_repartition_joins TO ON; +SET citus.enable_unique_job_ids TO off; SELECT count(*) FROM distributed_table; NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true @@ -693,9 +694,51 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar (1 row) SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); -ERROR: cannot execute command because a local execution has accessed a placement in the transaction -DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally -HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" +NOTICE: executing the command locally: SELECT partition_index, 'repartition_64_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_1','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500001 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_64_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_3','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500003 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_1','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500001 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_3','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500003 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_0,repartition_64_2_0,repartition_64_3_0,repartition_64_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_0,repartition_65_2_0,repartition_65_3_0,repartition_65_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_1,repartition_64_2_1,repartition_64_3_1,repartition_64_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_1,repartition_65_2_1,repartition_65_3_1,repartition_65_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_2,repartition_64_2_2,repartition_64_3_2,repartition_64_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_2,repartition_65_2_2,repartition_65_3_2,repartition_65_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_3,repartition_64_2_3,repartition_64_3_3,repartition_64_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_3,repartition_65_2_3,repartition_65_3_3,repartition_65_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + ROLLBACK; -- a local query is followed by an INSERT..SELECT with re-partitioning BEGIN; diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index e5cbcd994..79d21dd3e 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -966,7 +966,7 @@ DELETE FROM pg_dist_shard WHERE shardid = 1; CREATE TABLE e_transactions(order_id varchar(255) NULL, transaction_id int) PARTITION BY LIST(transaction_id); CREATE TABLE orders_2020_07_01 PARTITION OF e_transactions FOR VALUES IN (1,2,3); -INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', NULL, 7, 's'); +INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', '{VAR :varno 1 :varattno 1 :vartype 1043 :vartypmod 259 :varcollid 100 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1}', 7, 's'); SELECT (metadata->>'partitioned_citus_table_exists_pre_11')::boolean as partitioned_citus_table_exists_pre_11, (metadata->>'partitioned_citus_table_exists_pre_11') IS NULL as is_null @@ -1001,29 +1001,32 @@ SELECT * FROM multi_extension.print_extension_changes(); -- Snapshot of state at 11.0-1 ALTER EXTENSION citus UPDATE TO '11.0-1'; SELECT * FROM multi_extension.print_extension_changes(); - previous_object | current_object + previous_object | current_object --------------------------------------------------------------------- - function citus_disable_node(text,integer) void | - function create_distributed_function(regprocedure,text,text) void | - function master_append_table_to_shard(bigint,text,text,integer) real | - function master_apply_delete_command(text) integer | - function master_get_table_metadata(text) record | - | function citus_check_cluster_node_health() SETOF record - | function citus_check_connection_to_node(text,integer) boolean - | function citus_disable_node(text,integer,boolean) void - | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void - | function citus_internal_global_blocked_processes() SETOF record - | function citus_internal_local_blocked_processes() SETOF record - | function citus_run_local_command(text) void - | function citus_shard_indexes_on_worker() SETOF record - | function citus_shards_on_worker() SETOF record - | function create_distributed_function(regprocedure,text,text,boolean) void - | function pg_cancel_backend(bigint) boolean - | function pg_terminate_backend(bigint,bigint) boolean - | function worker_create_or_replace_object(text[]) boolean - | function worker_drop_sequence_dependency(text) void - | function worker_drop_shell_table(text) void -(20 rows) + function citus_disable_node(text,integer) void | + function create_distributed_function(regprocedure,text,text) void | + function master_append_table_to_shard(bigint,text,text,integer) real | + function master_apply_delete_command(text) integer | + function master_get_table_metadata(text) record | + function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) SETOF record | + | function citus_check_cluster_node_health() SETOF record + | function citus_check_connection_to_node(text,integer) boolean + | function citus_disable_node(text,integer,boolean) void + | function citus_finalize_upgrade_to_citus11(boolean) boolean + | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void + | function citus_internal_global_blocked_processes() SETOF record + | function citus_internal_local_blocked_processes() SETOF record + | function citus_run_local_command(text) void + | function citus_shard_indexes_on_worker() SETOF record + | function citus_shards_on_worker() SETOF record + | function create_distributed_function(regprocedure,text,text,boolean) void + | function pg_cancel_backend(bigint) boolean + | function pg_terminate_backend(bigint,bigint) boolean + | function worker_create_or_replace_object(text[]) boolean + | function worker_drop_sequence_dependency(text) void + | function worker_drop_shell_table(text) void + | function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) SETOF record +(23 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version @@ -1113,8 +1116,7 @@ HINT: Use DROP FUNCTION relation_is_a_known_shard(regclass) first. SET citus.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false'; -- This will fail because of previous function declaration -ALTER EXTENSION citus UPDATE TO '8.1-1'; -NOTICE: version "8.1-1" of extension "citus" is already installed +ALTER EXTENSION citus UPDATE TO '9.1-1'; -- We can DROP problematic function and continue ALTER EXTENSION even when version checks are on SET citus.enable_version_checks TO 'true'; SET columnar.enable_version_checks TO 'true'; @@ -1123,8 +1125,8 @@ ERROR: cannot drop function relation_is_a_known_shard(regclass) because extensi HINT: You can drop extension citus instead. SET citus.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false'; -ALTER EXTENSION citus UPDATE TO '8.1-1'; -NOTICE: version "8.1-1" of extension "citus" is already installed +ALTER EXTENSION citus UPDATE TO '9.1-1'; +NOTICE: version "9.1-1" of extension "citus" is already installed -- Test updating to the latest version without specifying the version number ALTER EXTENSION citus UPDATE; -- re-create in newest version diff --git a/src/test/regress/expected/multi_follower_select_statements.out b/src/test/regress/expected/multi_follower_select_statements.out index 9f92db197..42c3058ee 100644 --- a/src/test/regress/expected/multi_follower_select_statements.out +++ b/src/test/regress/expected/multi_follower_select_statements.out @@ -117,12 +117,18 @@ order by s_i_id; SET citus.enable_repartition_joins TO ON; SELECT count(*) FROM the_table t1 JOIN the_table t2 USING(b); -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + count +--------------------------------------------------------------------- + 2 +(1 row) + SET citus.enable_single_hash_repartition_joins TO ON; SELECT count(*) FROM the_table t1 , the_table t2 WHERE t1.a = t2.b; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + count +--------------------------------------------------------------------- + 2 +(1 row) + SELECT node_name, node_port FROM 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 c9357e954..8e5242607 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -27,11 +27,17 @@ AS 'SELECT hashtext( ($1.i + $1.i2)::text);' LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +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 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -66,7 +72,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -88,7 +95,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), diff --git a/src/test/regress/expected/multi_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index ee6ac8800..437e188ee 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -28,11 +28,17 @@ AS 'SELECT hashtext( ($1.i + $1.i2)::text);' LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +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 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -67,7 +73,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -89,7 +96,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), diff --git a/src/test/regress/expected/prepared_statements_1.out b/src/test/regress/expected/prepared_statements_1.out index e71be3f9d..418453705 100644 --- a/src/test/regress/expected/prepared_statements_1.out +++ b/src/test/regress/expected/prepared_statements_1.out @@ -24,3 +24,25 @@ BEGIN; (1 row) ROLLBACK; +PREPARE xact_repartitioned_prepared AS + SELECT count(*) FROM repartition_prepared_test t1 JOIN repartition_prepared_test t2 USING (b); +BEGIN; + -- Prepared re-partition join in a transaction block after a write + INSERT INTO repartition_prepared_test VALUES (1,2); + EXECUTE xact_repartitioned_prepared; + count +--------------------------------------------------------------------- + 226 +(1 row) + +ROLLBACK; +BEGIN; + -- Prepared re-partition join in a transaction block before a write + EXECUTE xact_repartitioned_prepared; + count +--------------------------------------------------------------------- + 209 +(1 row) + + INSERT INTO repartition_prepared_test VALUES (1,2); +ROLLBACK; diff --git a/src/test/regress/expected/propagate_foreign_servers.out b/src/test/regress/expected/propagate_foreign_servers.out index 2e9b164e7..bebb5a4f6 100644 --- a/src/test/regress/expected/propagate_foreign_servers.out +++ b/src/test/regress/expected/propagate_foreign_servers.out @@ -7,6 +7,15 @@ SELECT citus_remove_node('localhost', :worker_1_port); (1 row) +-- not related, but added here to test propagation of aggregates +-- to newly added nodes +CREATE AGGREGATE array_agg (anynonarray) +( + sfunc = array_agg_transfn, + stype = internal, + finalfunc = array_agg_finalfn, + finalfunc_extra +); -- create schema, extension and foreign server while the worker is removed SET citus.enable_ddl_propagation TO OFF; CREATE SCHEMA test_dependent_schema; @@ -28,6 +37,20 @@ NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipp 1 (1 row) +-- verify that the aggregate is propagated to the new node +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%propagate_foreign_server.array_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57638,t,propagate_foreign_server.array_agg) +(1 row) + +-- verify that the aggregate is added top pg_dist_object on the new node +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57638,t,1) +(1 row) + SELECT citus_add_local_table_to_metadata('foreign_table'); citus_add_local_table_to_metadata --------------------------------------------------------------------- @@ -163,4 +186,6 @@ ERROR: operation is not allowed on this node HINT: Connect to the coordinator and run it again. \c - - - :master_port DROP SCHEMA propagate_foreign_server CASCADE; -NOTICE: drop cascades to extension postgres_fdw +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to function propagate_foreign_server.array_agg(anynonarray) +drop cascades to extension postgres_fdw diff --git a/src/test/regress/expected/shard_rebalancer.out b/src/test/regress/expected/shard_rebalancer.out index c63b50842..b443d99e9 100644 --- a/src/test/regress/expected/shard_rebalancer.out +++ b/src/test/regress/expected/shard_rebalancer.out @@ -2380,3 +2380,64 @@ SELECT rebalance_table_shards(); CALL citus_cleanup_orphaned_shards(); DROP TABLE test_rebalance_with_index CASCADE; +-- Test rebalancer with disabled worker +SET citus.next_shard_id TO 433500; +SET citus.shard_replication_factor TO 2; +DROP TABLE IF EXISTS test_rebalance_with_disabled_worker; +CREATE TABLE test_rebalance_with_disabled_worker (a int); +SELECT create_distributed_table('test_rebalance_with_disabled_worker', 'a', colocate_with:='none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_disable_node('localhost', :worker_2_port); + citus_disable_node +--------------------------------------------------------------------- + +(1 row) + +SELECT public.wait_until_metadata_sync(30000); + wait_until_metadata_sync +--------------------------------------------------------------------- + +(1 row) + +SELECT rebalance_table_shards('test_rebalance_with_disabled_worker'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_activate_node('localhost', :worker_2_port); + citus_activate_node +--------------------------------------------------------------------- + 35 +(1 row) + +DROP TABLE test_rebalance_with_disabled_worker; +-- Test rebalance with all shards excluded +DROP TABLE IF EXISTS test_with_all_shards_excluded; +CREATE TABLE test_with_all_shards_excluded(a int PRIMARY KEY); +SELECT create_distributed_table('test_with_all_shards_excluded', 'a', colocate_with:='none', shard_count:=4); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardid FROM pg_dist_shard; + shardid +--------------------------------------------------------------------- + 433504 + 433505 + 433506 + 433507 +(4 rows) + +SELECT rebalance_table_shards('test_with_all_shards_excluded', excluded_shard_list:='{102073, 102074, 102075, 102076}'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE test_with_all_shards_excluded; diff --git a/src/test/regress/expected/text_search.out b/src/test/regress/expected/text_search.out index 1b4f652c1..a4af57051 100644 --- a/src/test/regress/expected/text_search.out +++ b/src/test/regress/expected/text_search.out @@ -484,6 +484,26 @@ SELECT create_distributed_table('t5', 'name'); (1 row) +-- make sure partial indices propagate their dependencies +-- first have a TEXT SEARCH CONFIGURATION that is not distributed +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION partial_index_test_config ( parser = default ); +RESET citus.enable_ddl_propagation; +CREATE TABLE sensors( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_a_partition PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_search_name ON sensors USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + SET client_min_messages TO 'warning'; DROP SCHEMA text_search, text_search2, "Text Search Requiring Quote's" CASCADE; DROP ROLE text_search_owner; diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index 185bf19c5..5f5ea7b1e 100644 --- a/src/test/regress/expected/upgrade_list_citus_objects.out +++ b/src/test/regress/expected/upgrade_list_citus_objects.out @@ -14,7 +14,7 @@ WHERE refclassid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND deptype = 'e' AND e.extname='citus' ORDER BY 1; - description + description --------------------------------------------------------------------- access method columnar event trigger citus_cascade_to_partition @@ -57,6 +57,7 @@ ORDER BY 1; function citus_drop_trigger() function citus_executor_name(integer) function citus_extradata_container(internal) + function citus_finalize_upgrade_to_citus11(boolean) function citus_finish_pg_upgrade() function citus_get_active_worker_nodes() function citus_internal.columnar_ensure_am_depends_catalog() @@ -225,7 +226,7 @@ ORDER BY 1; function worker_partial_agg(oid,anyelement) function worker_partial_agg_ffunc(internal) function worker_partial_agg_sfunc(internal,oid,anyelement) - function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) + function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) function worker_partitioned_relation_size(regclass) function worker_partitioned_relation_total_size(regclass) function worker_partitioned_table_size(regclass) @@ -272,5 +273,5 @@ ORDER BY 1; view citus_worker_stat_activity view pg_dist_shard_placement view time_partitions -(256 rows) +(257 rows) diff --git a/src/test/regress/expected/upgrade_pg_dist_object_test_after.out b/src/test/regress/expected/upgrade_pg_dist_object_test_after.out index 616b4fc32..443cb6205 100644 --- a/src/test/regress/expected/upgrade_pg_dist_object_test_after.out +++ b/src/test/regress/expected/upgrade_pg_dist_object_test_after.out @@ -39,12 +39,38 @@ drop cascades to table upgrade_basic.t_range SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3; type | object_names | object_args --------------------------------------------------------------------- - database | {postgres} | {} - extension | {isn} | {} - role | {postgres} | {} - schema | {fooschema} | {} - schema | {new_schema} | {} - schema | {public} | {} - type | {fooschema.footype} | {} - (7 rows) + collation | {post_11_upgrade,german_phonebook_unpropagated} | {} + database | {postgres} | {} + extension | {isn} | {} + extension | {plpgsql} | {} + function | {post_11_upgrade,func_in_transaction_def} | {} + role | {postgres} | {} + schema | {fooschema} | {} + schema | {new_schema} | {} + schema | {post_11_upgrade} | {} + schema | {public} | {} + table | {fooschema,footable} | {} + table | {new_schema,another_dist_table} | {} + table | {post_11_upgrade,colocated_dist_table} | {} + table | {post_11_upgrade,colocated_partitioned_table} | {} + table | {post_11_upgrade,colocated_partitioned_table_2020_01_01} | {} + table | {post_11_upgrade,dist} | {} + table | {post_11_upgrade,index_backed_rep_identity} | {} + table | {post_11_upgrade,part_table} | {} + table | {post_11_upgrade,part_table_p202008} | {} + table | {post_11_upgrade,part_table_p202009} | {} + table | {post_11_upgrade,reference_table} | {} + table | {post_11_upgrade,sensors} | {} + table | {post_11_upgrade,sensors_2020_01_01} | {} + table | {post_11_upgrade,sensors_news} | {} + table | {post_11_upgrade,sensors_old} | {} + table | {post_11_upgrade,sensors_parser} | {} + table | {post_11_upgrade,sensors_parser_a_partition} | {} + table | {post_11_upgrade,test_propagate_collate} | {} + table | {public,dist_table} | {} + table | {public,isn_dist_table} | {} + text search configuration | {post_11_upgrade,partial_index_test_config} | {} + type | {fooschema.footype} | {} + type | {post_11_upgrade.my_type} | {} + (33 rows) diff --git a/src/test/regress/expected/upgrade_pg_dist_object_test_after_1.out b/src/test/regress/expected/upgrade_pg_dist_object_test_after_1.out new file mode 100644 index 000000000..743164bfc --- /dev/null +++ b/src/test/regress/expected/upgrade_pg_dist_object_test_after_1.out @@ -0,0 +1,54 @@ +-- run this test only when old citus version is 9.0 +\set upgrade_test_old_citus_version `echo "$CITUS_OLD_VERSION"` +SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int = 9 AND + substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int = 0 +AS upgrade_test_old_citus_version_e_9_0; + upgrade_test_old_citus_version_e_9_0 +--------------------------------------------------------------------- + t +(1 row) + +\gset +\if :upgrade_test_old_citus_version_e_9_0 +\else +\q +\endif +-- drop objects from previous test (uprade_basic_after.sql) for a clean test +-- drop upgrade_basic schema and switch back to public schema +SET search_path to public; +DROP SCHEMA upgrade_basic CASCADE; +NOTICE: drop cascades to 7 other objects +DETAIL: drop cascades to table upgrade_basic.t +drop cascades to table upgrade_basic.tp +drop cascades to table upgrade_basic.t_ab +drop cascades to table upgrade_basic.t2 +drop cascades to table upgrade_basic.r +drop cascades to table upgrade_basic.tr +drop cascades to table upgrade_basic.t_range +-- as we updated citus to available version, +-- "isn" extension +-- "new_schema" schema +-- "public" schema +-- "fooschema" schema +-- "footype" type (under schema 'fooschema') + -- will now be marked as distributed + -- but, + -- "seg" extension + -- will not be marked as distributed +-- see underlying objects +SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3; + type | object_names | object_args +--------------------------------------------------------------------- + database | {postgres} | {} + extension | {isn} | {} + role | {postgres} | {} + schema | {fooschema} | {} + schema | {new_schema} | {} + schema | {public} | {} + table | {fooschema,footable} | {} + table | {new_schema,another_dist_table} | {} + table | {public,dist_table} | {} + table | {public,isn_dist_table} | {} + type | {fooschema.footype} | {} + (11 rows) + diff --git a/src/test/regress/expected/upgrade_post_11_after.out b/src/test/regress/expected/upgrade_post_11_after.out new file mode 100644 index 000000000..2da475c08 --- /dev/null +++ b/src/test/regress/expected/upgrade_post_11_after.out @@ -0,0 +1,76 @@ +SET search_path = post_11_upgrade; +-- make sure that we always (re)sync the metadata +UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb('true'::bool), true); +SELECT citus_finalize_upgrade_to_citus11(enforce_version_check:=false); +NOTICE: Preparing all the existing partitioned table indexes +NOTICE: Preparing to sync the metadata to all nodes + citus_finalize_upgrade_to_citus11 +--------------------------------------------------------------------- + t +(1 row) + +-- tables are objects with Citus 11+ +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{post_11_upgrade,func_in_transaction_def}",{}) + (schema,{post_11_upgrade},{}) + (table,"{post_11_upgrade,part_table}",{}) + (table,"{post_11_upgrade,sensors}",{}) + ("text search configuration","{post_11_upgrade,partial_index_test_config}",{}) + (type,{post_11_upgrade.my_type},{}) +(6 rows) + +-- on all nodes +SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1; + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57636,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}") + (localhost,57637,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}") +(2 rows) + +-- Create the necessary test utility function +CREATE OR REPLACE FUNCTION activate_node_snapshot() + RETURNS text[] + LANGUAGE C STRICT + AS 'citus'; +-- make sure that workers and the coordinator has the same datesyle +SET datestyle = "ISO, YMD"; +SELECT 1 FROM run_command_on_workers($$ALTER SYSTEM SET datestyle = "ISO, YMD";$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +SELECT 1 FROM run_command_on_workers($$SELECT pg_reload_conf()$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +-- make sure that the metadata is consistent across all nodes +-- we exclude the distributed_object_data as they are +-- not sorted in the same order (as OIDs differ on the nodes) +SELECT count(*) = 0 AS same_metadata_in_workers FROM +( + ( + SELECT unnest(activate_node_snapshot()) as command + EXCEPT + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + ) +UNION + ( + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) as command + ) +) AS foo WHERE command NOT ILIKE '%distributed_object_data%'; + same_metadata_in_workers +--------------------------------------------------------------------- + t +(1 row) + diff --git a/src/test/regress/expected/upgrade_post_11_before.out b/src/test/regress/expected/upgrade_post_11_before.out new file mode 100644 index 000000000..37bbab11b --- /dev/null +++ b/src/test/regress/expected/upgrade_post_11_before.out @@ -0,0 +1,184 @@ +-- test cases for #3970 +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +CREATE SCHEMA post_11_upgrade; +SET search_path = post_11_upgrade; +--1. create a partitioned table, and a vanilla table that will be colocated with this table +CREATE TABLE part_table ( + work_ymdt timestamp without time zone NOT NULL, + seq bigint NOT NULL, + my_seq bigint NOT NULL, + work_memo character varying(150), + CONSTRAINT work_memo_check CHECK ((octet_length((work_memo)::text) <= 150)), + PRIMARY KEY(seq, work_ymdt) +) +PARTITION BY RANGE (work_ymdt); +CREATE TABLE dist(seq bigint UNIQUE); +--2. perform create_distributed_table +SELECT create_distributed_table('part_table', 'seq'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist','seq'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +--3. add a partitions +CREATE TABLE part_table_p202008 PARTITION OF part_table FOR VALUES FROM ('2020-08-01 00:00:00') TO ('2020-09-01 00:00:00'); +CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00'); +--3. create indexes +CREATE INDEX i_part_1 ON part_table(seq); +CREATE INDEX i_part_2 ON part_table(my_seq, seq); +CREATE INDEX i_part_3 ON part_table(work_memo, seq); +CREATE TABLE sensors( +measureid integer, +eventdatetime date, +measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_old PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE TABLE sensors_2020_01_01 PARTITION OF sensors FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +CREATE TABLE sensors_news PARTITION OF sensors FOR VALUES FROM ('2020-05-01') TO ('2025-01-01'); +CREATE INDEX index_on_parent ON sensors(lower(measureid::text)); +CREATE INDEX index_on_child ON sensors_2020_01_01(lower(measure_data::text)); +CREATE INDEX hash_index ON sensors USING HASH((measure_data->'IsFailed')); +CREATE INDEX index_with_include ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime); +CREATE STATISTICS s1 (dependencies) ON measureid, eventdatetime FROM sensors; +CREATE STATISTICS s2 (dependencies) ON measureid, eventdatetime FROM sensors_2020_01_01; +ALTER INDEX index_on_parent ALTER COLUMN 1 SET STATISTICS 1000; +ALTER INDEX index_on_child ALTER COLUMN 1 SET STATISTICS 1000; +CLUSTER sensors_2020_01_01 USING index_on_child; +SELECT create_distributed_table('sensors', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- create a colocated distributed tables and create foreign keys FROM/TO +-- the partitions +CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY); +SELECT create_distributed_table('colocated_dist_table', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CLUSTER colocated_dist_table USING colocated_dist_table_pkey; +WARNING: not propagating CLUSTER command to worker nodes +CREATE TABLE colocated_partitioned_table( + measureid integer, + eventdatetime date, + PRIMARY KEY (measureid, eventdatetime)) +PARTITION BY RANGE(eventdatetime); +CREATE TABLE colocated_partitioned_table_2020_01_01 PARTITION OF colocated_partitioned_table FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +SELECT create_distributed_table('colocated_partitioned_table', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CLUSTER colocated_partitioned_table_2020_01_01 USING colocated_partitioned_table_2020_01_01_pkey; +WARNING: not propagating CLUSTER command to worker nodes +CREATE TABLE reference_table (measureid integer PRIMARY KEY); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +-- this table is used to make sure that index backed +-- replica identites can have clustered indexes +-- and no index statistics +CREATE TABLE index_backed_rep_identity(key int NOT NULL); +CREATE UNIQUE INDEX uqx ON index_backed_rep_identity(key); +ALTER TABLE index_backed_rep_identity REPLICA IDENTITY USING INDEX uqx; +CLUSTER index_backed_rep_identity USING uqx; +SELECT create_distributed_table('index_backed_rep_identity', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- from parent to regular dist +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid); +-- from parent to parent +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_parent FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table(measureid, eventdatetime); +-- from parent to child +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_child FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid, eventdatetime); +-- load some data +INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_partitioned_table SELECT i, '2020-01-05' FROM generate_series(0,1000)i; +INSERT INTO sensors SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i; +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default ); +SELECT 1 FROM run_command_on_workers($$CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default );$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +SELECT run_command_on_workers('SET citus.enable_ddl_propagation TO off; +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$;'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57636,t,SET) + (localhost,57637,t,SET) +(2 rows) + +CREATE TYPE post_11_upgrade.my_type AS (a int); +RESET citus.enable_ddl_propagation; +CREATE TABLE sensors_parser( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + col_with_def int DEFAULT post_11_upgrade.func_in_transaction_def(), + col_with_type post_11_upgrade.my_type, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_parser_a_partition PARTITION OF sensors_parser FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_parser_search_name ON sensors_parser USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors_parser', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET citus.enable_ddl_propagation TO off; +CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk'); +SELECT 1 FROM run_command_on_workers($$CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk');$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +SET citus.enable_ddl_propagation TO on; +CREATE TABLE test_propagate_collate(id int, t2 text COLLATE german_phonebook_unpropagated); +SELECT create_distributed_table('test_propagate_collate', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/output/multi_copy.source b/src/test/regress/output/multi_copy.source index d6d72d963..4b35d4465 100644 --- a/src/test/regress/output/multi_copy.source +++ b/src/test/regress/output/multi_copy.source @@ -636,7 +636,7 @@ 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, '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 +DETAIL: There are 115 objects to replicate, depending on your environment this might take a while ?column? --------------------------------------------------------------------- 1 diff --git a/src/test/regress/spec/isolation_citus_dist_activity.spec b/src/test/regress/spec/isolation_citus_dist_activity.spec index c41c671f0..5047a656c 100644 --- a/src/test/regress/spec/isolation_citus_dist_activity.spec +++ b/src/test/regress/spec/isolation_citus_dist_activity.spec @@ -89,7 +89,7 @@ step "s3-rollback" step "s3-view-worker" { - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; } // we prefer to sleep before "s2-view-dist" so that we can ensure diff --git a/src/test/regress/spec/isolation_distributed_transaction_id.spec b/src/test/regress/spec/isolation_distributed_transaction_id.spec index f928918ed..0b626f2c8 100644 --- a/src/test/regress/spec/isolation_distributed_transaction_id.spec +++ b/src/test/regress/spec/isolation_distributed_transaction_id.spec @@ -49,11 +49,11 @@ step "s1-verify-current-xact-is-on-worker" { SELECT remote.nodeport, - remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists + remote.result = row(xact.transaction_number)::text AS xact_exists FROM get_current_transaction_id() as xact, run_command_on_workers($$ - SELECT row(initiator_node_identifier, transaction_number) + SELECT row(transaction_number) FROM get_all_active_transactions() WHERE transaction_number != 0; $$) as remote diff --git a/src/test/regress/spec/isolation_drop_vs_all.spec b/src/test/regress/spec/isolation_drop_vs_all.spec index 3aef01eee..37015b111 100644 --- a/src/test/regress/spec/isolation_drop_vs_all.spec +++ b/src/test/regress/spec/isolation_drop_vs_all.spec @@ -2,30 +2,37 @@ // How we organize this isolation test spec, is explained at README.md file in this directory. // -// create range distributed table to test behavior of DROP in concurrent operations +// create distributed table to test behavior of DROP in concurrent operations setup { SELECT citus_internal.replace_isolation_tester_func(); SELECT citus_internal.refresh_isolation_tester_prepared_statement(); SET citus.shard_replication_factor TO 1; - CREATE TABLE drop_hash(id integer, data text); - SELECT create_distributed_table('drop_hash', 'id'); + CREATE SCHEMA drop_tests + CREATE TABLE drop_hash(id integer, data text); + SELECT create_distributed_table('drop_tests.drop_hash', 'id'); + + CREATE SCHEMA drop_tests_2 + CREATE TABLE drop_hash_2(id integer, data text); + SELECT create_distributed_table('drop_tests_2.drop_hash_2', 'id'); } // drop distributed table teardown { - DROP TABLE IF EXISTS drop_hash CASCADE; + DROP TABLE IF EXISTS drop_tests.drop_hash, drop_tests_2.drop_hash_2 CASCADE; + DROP SCHEMA IF EXISTS drop_tests, drop_tests_2 CASCADE; SELECT citus_internal.restore_isolation_tester_func(); } // session 1 session "s1" -step "s1-initialize" { COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; } +step "s1-initialize" { SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;} step "s1-begin" { BEGIN; } step "s1-drop" { DROP TABLE drop_hash; } +step "s1-drop-schema" { DROP SCHEMA drop_tests CASCADE; } step "s1-ddl-create-index" { CREATE INDEX drop_hash_index ON drop_hash(id); } step "s1-ddl-drop-index" { DROP INDEX drop_hash_index; } step "s1-ddl-add-column" { ALTER TABLE drop_hash ADD new_column int DEFAULT 0; } @@ -41,8 +48,11 @@ step "s1-commit" { COMMIT; } // session 2 session "s2" +step "s2-initialize" { SET search_path TO 'drop_tests'; } step "s2-begin" { BEGIN; } step "s2-drop" { DROP TABLE drop_hash; } +step "s2-drop-schema" { DROP SCHEMA drop_tests CASCADE; } +step "s2-drop-schema-2" { DROP SCHEMA drop_tests_2 CASCADE; } step "s2-ddl-create-index" { CREATE INDEX drop_hash_index ON drop_hash(id); } step "s2-ddl-drop-index" { DROP INDEX drop_hash_index; } step "s2-ddl-create-index-concurrently" { CREATE INDEX CONCURRENTLY drop_hash_index ON drop_hash(id); } @@ -54,23 +64,25 @@ step "s2-distribute-table" { SELECT create_distributed_table('drop_hash', 'id'); step "s2-commit" { COMMIT; } // permutations - DROP vs DROP -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop-schema" "s2-drop-schema" "s1-commit" "s2-commit" "s1-select-count" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop-schema" "s2-drop-schema-2" "s1-commit" "s2-commit" "s1-select-count" // permutations - DROP first -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-create-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" -permutation "s1-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" -permutation "s1-initialize" "s1-begin" "s1-drop" "s2-ddl-create-index-concurrently" "s1-commit" "s1-select-count" "s1-show-indexes" -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-add-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" -permutation "s1-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-rename-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-table-size" "s1-commit" "s2-commit" "s1-select-count" -permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-distribute-table" "s1-commit" "s2-commit" "s1-select-count" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-create-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" +permutation "s1-initialize" "s2-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s1-drop" "s2-ddl-create-index-concurrently" "s1-commit" "s1-select-count" "s1-show-indexes" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-add-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" +permutation "s1-initialize" "s2-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-rename-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-table-size" "s1-commit" "s2-commit" "s1-select-count" +permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-distribute-table" "s1-commit" "s2-commit" "s1-select-count" // permutations - DROP second -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-ddl-create-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" -permutation "s1-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-ddl-drop-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-ddl-add-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" -permutation "s1-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-ddl-drop-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-ddl-rename-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" -permutation "s1-initialize" "s1-begin" "s2-begin" "s1-table-size" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" -permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s1-begin" "s2-begin" "s1-distribute-table" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-ddl-create-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" +permutation "s1-initialize" "s2-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-ddl-drop-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-ddl-add-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" +permutation "s1-initialize" "s2-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-ddl-drop-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-ddl-rename-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" +permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-table-size" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" +permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-distribute-table" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" diff --git a/src/test/regress/spec/isolation_get_all_active_transactions.spec b/src/test/regress/spec/isolation_get_all_active_transactions.spec index fd69c0ac4..685046e63 100644 --- a/src/test/regress/spec/isolation_get_all_active_transactions.spec +++ b/src/test/regress/spec/isolation_get_all_active_transactions.spec @@ -77,7 +77,11 @@ step "s3-as-admin" step "s3-as-user-1" { - -- User should only be able to see its own transactions + -- Even though we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert) + -- userId (e.g., PG_PROC->userId) does not change, and hence none of the + -- transactions show up because here we are using test_user_1. This is a + -- limitation of isolation tester, we should be able to re-connect with + -- test_user_1 on s1/2-begin-insert to show that test_user_1 sees only its own processes SET ROLE test_user_1; SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; @@ -85,7 +89,12 @@ step "s3-as-user-1" step "s3-as-readonly" { - -- Other user should not see transactions + -- Even though we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert) + -- userId (e.g., PG_PROC->userId) does not change, and hence none of the + -- transactions show up because here we are using test_readonly. This is a + -- limitation of isolation tester, we should be able to re-connect with + -- test_readonly on s1/2-begin-insert to show that test_readonly sees only + -- its own processes SET ROLE test_readonly; SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; diff --git a/src/test/regress/spec/isolation_metadata_sync_vs_all.spec b/src/test/regress/spec/isolation_metadata_sync_vs_all.spec index 7ff246365..80cfe2e33 100644 --- a/src/test/regress/spec/isolation_metadata_sync_vs_all.spec +++ b/src/test/regress/spec/isolation_metadata_sync_vs_all.spec @@ -29,7 +29,6 @@ setup teardown { - // drop all distributed tables DROP TABLE IF EXISTS ref_table, dist_table, dist_partitioned_table, @@ -39,7 +38,6 @@ teardown new_ref_table; - // drop all distributed objects DROP FUNCTION activate_node_snapshot(); DROP FUNCTION IF EXISTS squares(int); DROP TYPE IF EXISTS my_type; @@ -110,6 +108,19 @@ step "s2-create-dist-table" SELECT create_distributed_table('new_dist_table', 'id'); } +step "s2-create-schema" +{ + CREATE SCHEMA dist_schema + CREATE TABLE dist_table_in_schema(id int, data int); + + SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id'); +} + +step "s2-drop-schema" +{ + DROP SCHEMA dist_schema CASCADE; +} + step "s2-create-ref-table" { CREATE TABLE new_ref_table(id int, data int); @@ -136,6 +147,16 @@ step "s2-create-type" CREATE TYPE my_type AS (a int, b int); } +step "s2-drop-type" +{ + DROP TYPE my_type; +} + +step "s2-alter-type" +{ + ALTER TYPE my_type ADD ATTRIBUTE x int; +} + step "s2-create-dist-func" { CREATE FUNCTION squares(int) RETURNS SETOF RECORD @@ -145,6 +166,11 @@ step "s2-create-dist-func" SELECT create_distributed_function('squares(int)'); } +step "s2-drop-dist-func" +{ + DROP FUNCTION squares(int); +} + session "s3" step "s3-compare-snapshot" @@ -168,6 +194,11 @@ step "s3-compare-snapshot" ) AS foo; } +step "s3-compare-type-definition" +{ + SELECT run_command_on_workers($$SELECT '(1,1,1)'::my_type$$); +} + step "s3-debug" { SELECT unnest(activate_node_snapshot()); @@ -186,6 +217,8 @@ permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-start-metadata-sy // the following operations get blocked when a concurrent metadata sync is in progress permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-alter-table" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-table" "s1-commit" "s2-commit" "s3-compare-snapshot" +permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-schema" "s1-commit" "s2-commit" "s3-compare-snapshot" "s2-drop-schema" +permutation "s2-create-schema" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-schema" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-table" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-ref-table" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-attach-partition" "s1-commit" "s2-commit" "s3-compare-snapshot" @@ -193,6 +226,16 @@ permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync" permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-partition-of" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-add-fk" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s2-add-fk" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-fk" "s1-commit" "s2-commit" "s3-compare-snapshot" +permutation "s2-create-type" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-type" "s1-commit" "s2-commit" "s3-compare-snapshot" +permutation "s2-create-dist-func" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-dist-func" "s1-commit" "s2-commit" "s3-compare-snapshot" +permutation "s2-create-type" "s1-begin" "s1-start-metadata-sync" "s2-alter-type" "s1-commit" "s3-compare-snapshot" "s3-compare-type-definition" + +// the following operations block concurrent metadata sync calls +permutation "s1-begin" "s2-begin" "s2-create-dist-table" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot" +permutation "s2-create-dist-func" "s1-begin" "s2-begin" "s2-drop-dist-func" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot" +permutation "s2-create-schema" "s1-begin" "s2-begin" "s2-drop-schema" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot" +permutation "s2-create-type" "s1-begin" "s2-begin" "s2-alter-type" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot" "s3-compare-type-definition" + // the following operations do not get blocked permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-type" "s1-commit" "s2-commit" "s3-compare-snapshot" diff --git a/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec b/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec index fa2079ba5..09da5970d 100644 --- a/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec +++ b/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec @@ -3,8 +3,6 @@ setup SELECT citus_internal.replace_isolation_tester_func(); SELECT citus_internal.refresh_isolation_tester_prepared_statement(); - SELECT master_add_node('localhost', 57636, groupid => 0); - CREATE TABLE ref_table(a int primary key); SELECT create_reference_table('ref_table'); INSERT INTO ref_table VALUES (1), (3), (5), (7); @@ -83,7 +81,7 @@ step "s2-lock-ref-table-placement-on-coordinator" step "s2-view-dist" { - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC; } step "s2-view-worker" @@ -94,7 +92,9 @@ step "s2-view-worker" WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%dump_local_%' AND - query NOT ILIKE '%citus_internal_local_blocked_processes%' + query NOT ILIKE '%citus_internal_local_blocked_processes%' AND + query NOT ILIKE '%add_node%' AND + backend_type = 'client backend' ORDER BY query, query_hostport DESC; } @@ -123,14 +123,25 @@ step "deadlock-checker-call" SELECT check_distributed_deadlocks(); } + +// adding node in setup stage prevents getting a gpid with proper nodeid +session "add-node" + +// we issue the checker not only when there are deadlocks to ensure that we never cancel +// backend inappropriately +step "add-node" +{ + SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0); +} + // verify that locks on the placement of the reference table on the coordinator is // taken into account when looking for distributed deadlocks -permutation "s1-begin" "s2-begin" "s1-update-dist-table" "s2-lock-ref-table-placement-on-coordinator" "s1-lock-ref-table-placement-on-coordinator" "s2-update-dist-table" "deadlock-checker-call" "s1-end" "s2-end" +permutation "add-node" "s1-begin" "s2-begin" "s1-update-dist-table" "s2-lock-ref-table-placement-on-coordinator" "s1-lock-ref-table-placement-on-coordinator" "s2-update-dist-table" "deadlock-checker-call" "s1-end" "s2-end" // verify that *_dist_stat_activity() functions return the correct result when query // has a task on the coordinator. -permutation "s1-begin" "s2-begin" "s1-update-ref-table" "s2-sleep" "s2-view-dist" "s2-view-worker" "s2-end" "s1-end" +permutation "add-node" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-sleep" "s2-view-dist" "s2-view-worker" "s2-end" "s1-end" // verify that get_*_active_transactions() functions return the correct result when // the query has a task on the coordinator. -permutation "s1-begin" "s2-begin" "s1-update-ref-table" "s2-active-transactions" "s1-end" "s2-end" +permutation "add-node" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-active-transactions" "s1-end" "s2-end" diff --git a/src/test/regress/sql/adaptive_executor_repartition.sql b/src/test/regress/sql/adaptive_executor_repartition.sql index bb625ae6f..1f2e21951 100644 --- a/src/test/regress/sql/adaptive_executor_repartition.sql +++ b/src/test/regress/sql/adaptive_executor_repartition.sql @@ -24,7 +24,6 @@ ROLLBACK; BEGIN; INSERT INTO ab values(1, 2); --- DDL happened before repartition query in a transaction block, so this should error. SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT * FROM ab) second WHERE first.a = second.b; ROLLBACK; diff --git a/src/test/regress/sql/aggregate_support.sql b/src/test/regress/sql/aggregate_support.sql index 7c82418a7..a83688d82 100644 --- a/src/test/regress/sql/aggregate_support.sql +++ b/src/test/regress/sql/aggregate_support.sql @@ -160,6 +160,10 @@ create aggregate binstragg(text, text)( combinefunc=binstragg_combinefunc, stype=text ); +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$); + +SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$); select create_distributed_function('binstragg(text,text)'); @@ -313,7 +317,6 @@ SELECT id%5, first(val ORDER BY key), last(val ORDER BY key) FROM aggdata GROUP BY id%5 ORDER BY id%5; -- test aggregate with stype which is not a by-value datum --- also test our handling of the aggregate not existing on workers create function sumstring_sfunc(state text, x text) returns text immutable language plpgsql as $$ begin return (state::float8 + x::float8)::text; @@ -326,8 +329,10 @@ create aggregate sumstring(text) ( combinefunc = sumstring_sfunc, initcond = '0' ); +-- verify that the aggregate is propagated +select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%'; +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';$$); -select sumstring(valf::text) from aggdata where valf is not null; select create_distributed_function('sumstring(text)'); select sumstring(valf::text) from aggdata where valf is not null; @@ -533,5 +538,79 @@ SELECT floor(AVG(COALESCE(agg_col, 10))) FROM dist_table LEFT JOIN ref_table ON TRUE; +-- try createing aggregate having non-distributable dependency type +create table dummy_tbl (a int); +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; +-- should give warning and create aggregate local only +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); + +-- clear and try again with distributed table +DROP TABLE dummy_tbl CASCADE; + +create table dummy_tbl (a int); +SELECT create_distributed_table('dummy_tbl','a'); +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; + +-- test in tx block +-- shouldn't distribute, as citus.create_object_propagation is set to deferred +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify not distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +drop aggregate dependent_agg ( double precision); + +-- now try with create_object_propagation = immediate +SET citus.create_object_propagation TO immediate; +-- should distribute, as citus.create_object_propagation is set to immediate +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; + +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +drop aggregate dependent_agg ( double precision); + +-- now try with create_object_propagation = automatic +SET citus.create_object_propagation TO automatic; +-- should distribute, as citus.create_object_propagation is set to automatic +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; + +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); + +RESET citus.create_object_propagation; + +-- drop and test outside of tx block +drop aggregate dependent_agg (float8); +-- verify that the aggregate is removed from pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +--verify +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +DROP TABLE dummy_tbl CASCADE; + +SET citus.create_object_propagation TO automatic; +begin; + create type typ1 as (a int); + create or replace function fnagg(a typ1, d double precision) RETURNS typ1 AS $$SELECT 1;$$LANGUAGE sql; + create aggregate dependent_agg (float8) (stype=typ1, sfunc=fnagg); +commit; +RESET citus.create_object_propagation; + +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + set client_min_messages to error; drop schema aggregate_support cascade; diff --git a/src/test/regress/sql/coordinator_shouldhaveshards.sql b/src/test/regress/sql/coordinator_shouldhaveshards.sql index bc79513c3..c5e2b6177 100644 --- a/src/test/regress/sql/coordinator_shouldhaveshards.sql +++ b/src/test/regress/sql/coordinator_shouldhaveshards.sql @@ -112,7 +112,7 @@ SET citus.enable_repartition_joins TO ON; SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; BEGIN; -SET citus.enable_repartition_joins TO ON; +SET citus.enable_unique_job_ids TO off; SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; END; diff --git a/src/test/regress/sql/distributed_triggers.sql b/src/test/regress/sql/distributed_triggers.sql index 9f8a2e3b7..33ecdbb8c 100644 --- a/src/test/regress/sql/distributed_triggers.sql +++ b/src/test/regress/sql/distributed_triggers.sql @@ -218,7 +218,7 @@ ORDER BY shard_key_value, object_id, change_id; -- Triggers (tables) which are not colocated -- CREATE TABLE emptest ( - empname text NOT NULL, + empname text NOT NULL PRIMARY KEY, salary integer ); @@ -227,7 +227,8 @@ CREATE TABLE emptest_audit( stamp timestamp NOT NULL, userid text NOT NULL, empname text NOT NULL, - salary integer + salary integer, + PRIMARY KEY (empname, userid, stamp, operation, salary) ); SELECT create_distributed_table('emptest','empname',colocate_with :='none'); @@ -282,6 +283,8 @@ CREATE TABLE record_op ( operation_type text not null, stamp timestamp NOT NULL ); +ALTER TABLE record_op REPLICA IDENTITY FULL; + SELECT create_distributed_table('record_op', 'empname', colocate_with := 'emptest'); CREATE OR REPLACE FUNCTION record_emp() RETURNS TRIGGER AS $rec_audit$ BEGIN @@ -376,7 +379,7 @@ ALTER TABLE sale ADD CONSTRAINT sale_pk PRIMARY KEY (state_code, sale_date); CREATE TABLE sale_newyork PARTITION OF sale FOR VALUES IN ('NY'); CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA'); -CREATE TABLE record_sale(operation_type text not null, product_sku text, state_code text); +CREATE TABLE record_sale(operation_type text not null, product_sku text, state_code text, units integer, PRIMARY KEY(state_code, product_sku, operation_type, units)); SELECT create_distributed_table('sale', 'state_code'); SELECT create_distributed_table('record_sale', 'state_code', colocate_with := 'sale'); @@ -385,8 +388,8 @@ CREATE OR REPLACE FUNCTION record_sale() RETURNS trigger AS $$ BEGIN - INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code) - VALUES (TG_OP, NEW.product_sku, NEW.state_code); + INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code, units) + VALUES (TG_OP, NEW.product_sku, NEW.state_code, NEW.units); RETURN NULL; END; $$ LANGUAGE plpgsql; @@ -403,7 +406,7 @@ INSERT INTO sale(sale_date,state_code,product_sku,units) VALUES ('2019-02-03', 'NY', 'AZ-000A1', 47); TABLE sale ORDER BY state_code, sale_date; -TABLE record_sale ORDER BY 1,2,3; +SELECT operation_type, product_sku, state_code FROM record_sale ORDER BY 1,2,3; -- --Test ALTER TRIGGER diff --git a/src/test/regress/sql/distributed_types.sql b/src/test/regress/sql/distributed_types.sql index a136e2fa8..23d84d26b 100644 --- a/src/test/regress/sql/distributed_types.sql +++ b/src/test/regress/sql/distributed_types.sql @@ -281,6 +281,47 @@ UPDATE field_indirection_test_2 SET (ct2_col, ct1_col) = ('(10, "text10", 20)', SELECT * FROM field_indirection_test_2 ORDER BY 1,2,3; +-- test different ddl propagation modes +SET citus.create_object_propagation TO deferred; +BEGIN; +CREATE TYPE deferred_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE deferred_table(a int,b deferred_type); +SELECT create_distributed_table('deferred_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + +SET citus.create_object_propagation TO automatic; +BEGIN; +CREATE TYPE automatic_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE automatic_table(a int,b automatic_type); +SELECT create_distributed_table('automatic_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + +SET citus.create_object_propagation TO automatic; +BEGIN; +-- force parallel execution by preceding with a analytical query +SET LOCAL citus.force_max_query_parallelization TO on; +SELECT count(*) FROM automatic_table; + +CREATE TYPE automatic2_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE automatic2_table(a int,b automatic2_type); +SELECT create_distributed_table('automatic2_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + +SET citus.create_object_propagation TO immediate; +BEGIN; +CREATE TYPE immediate_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE immediate_table(a int,b immediate_type); +SELECT create_distributed_table('immediate_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + -- clear objects SET client_min_messages TO error; -- suppress cascading objects dropping DROP SCHEMA type_tests CASCADE; diff --git a/src/test/regress/sql/function_propagation.sql b/src/test/regress/sql/function_propagation.sql index 79168497b..a9a6d04d8 100644 --- a/src/test/regress/sql/function_propagation.sql +++ b/src/test/regress/sql/function_propagation.sql @@ -188,6 +188,483 @@ BEGIN END; $$; +-- Show that functions are propagated (or not) as a dependency + +-- Function as a default column +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_def() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; + + CREATE TABLE table_to_prop_func(id int, col_1 int default func_in_transaction_def()); + SELECT create_distributed_table('table_to_prop_func','id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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_def'::regproc::oid;$$) ORDER BY 1,2; + + +-- Multiple functions as a default column +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_1() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_2() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid; + + CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2()); + SELECT create_distributed_table('table_to_prop_func_2','id'); + + -- Functions should be marked as distribued after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +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_1'::regproc::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_2'::regproc::oid;$$) ORDER BY 1,2; + + +-- If function has dependency on non-distributed table it should error out +BEGIN; + CREATE TABLE non_dist_table(id int); + + CREATE OR REPLACE FUNCTION func_in_transaction_3(param_1 non_dist_table) + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + CREATE TABLE table_to_prop_func_3(id int, col_1 int default func_in_transaction_3(NULL::non_dist_table)); + + -- It should error out as there is a non-distributed table dependency + SELECT create_distributed_table('table_to_prop_func_3','id'); +COMMIT; + + +-- Adding a column with default value should propagate the function +BEGIN; + CREATE TABLE table_to_prop_func_4(id int); + SELECT create_distributed_table('table_to_prop_func_4', 'id'); + + CREATE OR REPLACE FUNCTION func_in_transaction_4() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; + + ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4(); + + -- Function should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +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_4'::regproc::oid;$$) ORDER BY 1,2; + + +-- Adding multiple columns with default values should propagate the function +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_5() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_6() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid; + + CREATE TABLE table_to_prop_func_5(id int, col_1 int default func_in_transaction_5(), col_2 int default func_in_transaction_6()); + SELECT create_distributed_table('table_to_prop_func_5', 'id'); + + -- Functions should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +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_5'::regproc::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_6'::regproc::oid;$$) ORDER BY 1,2; + +-- Adding a constraint with function check should propagate the function +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_7(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; + + CREATE TABLE table_to_prop_func_6(id int, col_1 int check (function_propagation_schema.func_in_transaction_7(col_1))); + SELECT create_distributed_table('table_to_prop_func_6', 'id'); + + -- Function should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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_7'::regproc::oid;$$) ORDER BY 1,2; + + +-- Adding a constraint with multiple functions check should propagate the function +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_8(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_9(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid; + + CREATE TABLE table_to_prop_func_7(id int, col_1 int check (function_propagation_schema.func_in_transaction_8(col_1) and function_propagation_schema.func_in_transaction_9(col_1))); + SELECT create_distributed_table('table_to_prop_func_7', 'id'); + + -- Function should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +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_8'::regproc::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_9'::regproc::oid;$$) ORDER BY 1,2; + + +-- Adding a column with constraint should propagate the function +BEGIN; + CREATE TABLE table_to_prop_func_8(id int, col_1 int); + SELECT create_distributed_table('table_to_prop_func_8', 'id'); + + CREATE OR REPLACE FUNCTION func_in_transaction_10(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; + + ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1)); + + -- Function should be marked as distributed after adding the constraint + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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_10'::regproc::oid;$$) ORDER BY 1,2; + + +-- If constraint depends on a non-distributed table it should error out +BEGIN; + CREATE TABLE local_table_for_const(id int); + + CREATE OR REPLACE FUNCTION func_in_transaction_11(param_1 int, param_2 local_table_for_const) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + CREATE TABLE table_to_prop_func_9(id int, col_1 int check (func_in_transaction_11(col_1, NULL::local_table_for_const))); + + -- It should error out since there is non-distributed table dependency exists + SELECT create_distributed_table('table_to_prop_func_9', 'id'); +COMMIT; + + +-- Show that function as a part of generated always is supporte +BEGIN; + + CREATE OR REPLACE FUNCTION non_sense_func_for_generated_always() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; + + CREATE TABLE people ( + id int, + height_cm numeric, + height_in numeric GENERATED ALWAYS AS (height_cm / non_sense_func_for_generated_always()) STORED); + + SELECT create_distributed_table('people', 'id'); + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; +COMMIT; + + +-- Show that functions depending table via rule are also distributed +BEGIN; + CREATE OR REPLACE FUNCTION func_for_rule() + RETURNS int + LANGUAGE plpgsql STABLE AS + $$ + BEGIN + return 4; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; + + CREATE TABLE table_1_for_rule(id int, col_1 int); + CREATE TABLE table_2_for_rule(id int, col_1 int); + + CREATE RULE rule_1 AS ON UPDATE TO table_1_for_rule DO ALSO UPDATE table_2_for_rule SET col_1 = col_1 * func_for_rule(); + + SELECT create_distributed_table('table_1_for_rule','id'); + + -- Functions should be distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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_for_rule'::regproc::oid;$$) ORDER BY 1,2; + + +-- Show that functions as partitioning functions are supported +BEGIN; + + CREATE OR REPLACE FUNCTION non_sense_func_for_partitioning(int) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; + + CREATE TABLE partitioned_table_to_test_func_prop(id INT, a INT) PARTITION BY RANGE (non_sense_func_for_partitioning(id)); + + SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id'); + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2; + + +-- Test function dependency on citus local table +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_for_local_table() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; + + CREATE TABLE citus_local_table_to_test_func(l1 int DEFAULT func_in_transaction_for_local_table()); + SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; +ROLLBACK; + +-- Show that having a function dependency on exlude also works +BEGIN; + CREATE OR REPLACE FUNCTION exclude_bool_func() + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return true; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; + + CREATE TABLE exclusion_func_prop_table (id int, EXCLUDE USING btree (id WITH =) WHERE (exclude_bool_func())); + SELECT create_distributed_table('exclusion_func_prop_table', 'id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2; + + +-- Show that having a function dependency for index also works +BEGIN; + CREATE OR REPLACE FUNCTION func_for_index_predicate(col_1 int) + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return col_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; + + CREATE TABLE table_to_check_func_index_dep (id int, col_2 int); + CREATE INDEX on table_to_check_func_index_dep(col_2) WHERE (func_for_index_predicate(col_2)); + + SELECT create_distributed_table('table_to_check_func_index_dep', 'id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2; + + +-- Test function to function dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_for_func_dep_1() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + + CREATE TABLE func_dep_table(a int, b int default func_for_func_dep_1()); + + CREATE OR REPLACE FUNCTION func_for_func_dep_2(col_1 func_dep_table) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + + SELECT create_distributed_table('func_dep_table', 'a'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2; + + +-- Test function with SQL language and sequence dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_seq(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT 2 * val; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_func(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT func_in_transaction_def_with_seq(val); + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; + + CREATE SEQUENCE myseq; + CREATE TABLE table_to_prop_seq_func(id int, col_1 bigint default func_in_transaction_def_with_func(func_in_transaction_def_with_seq(nextval('myseq')))); + + SELECT create_distributed_table('table_to_prop_seq_func','id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +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_def_with_seq'::regproc::oid;$$) ORDER BY 1,2; + RESET search_path; SET client_min_messages TO WARNING; DROP SCHEMA function_propagation_schema CASCADE; diff --git a/src/test/regress/sql/intermediate_results.sql b/src/test/regress/sql/intermediate_results.sql index ae5402345..67e80d1ee 100644 --- a/src/test/regress/sql/intermediate_results.sql +++ b/src/test/regress/sql/intermediate_results.sql @@ -132,9 +132,6 @@ WITH (FORMAT text); SELECT * FROM squares ORDER BY x; --- empty shard interval array should raise error -SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[0]); - -- cannot use DDL commands select broadcast_intermediate_result('a', 'create table foo(int serial)'); select broadcast_intermediate_result('a', 'prepare foo as select 1'); @@ -233,8 +230,8 @@ SAVEPOINT s1; -- results aren't available on coordinator yet SELECT * FROM read_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'binary') AS res (x int, x2 int); ROLLBACK TO SAVEPOINT s1; --- fetch from worker 2 should fail -SELECT * FROM fetch_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'localhost', :worker_2_port); +-- fetch from invalid worker port should fail +SELECT * FROM fetch_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'localhost', 57635); ROLLBACK TO SAVEPOINT s1; -- still, results aren't available on coordinator yet SELECT * FROM read_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'binary') AS res (x int, x2 int); diff --git a/src/test/regress/sql/local_shard_execution.sql b/src/test/regress/sql/local_shard_execution.sql index c7dad6f00..b68863a7f 100644 --- a/src/test/regress/sql/local_shard_execution.sql +++ b/src/test/regress/sql/local_shard_execution.sql @@ -67,6 +67,7 @@ ALTER TABLE abcd DROP COLUMN a; -- connection worker and get ready for the tests \c - - - :worker_1_port SET search_path TO local_shard_execution; +SET citus.enable_unique_job_ids TO off; -- returns true of the distribution key filter -- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard diff --git a/src/test/regress/sql/local_shard_execution_replicated.sql b/src/test/regress/sql/local_shard_execution_replicated.sql index 01af172a3..81b47cfc8 100644 --- a/src/test/regress/sql/local_shard_execution_replicated.sql +++ b/src/test/regress/sql/local_shard_execution_replicated.sql @@ -348,6 +348,7 @@ ROLLBACK; BEGIN; SET citus.enable_repartition_joins TO ON; +SET citus.enable_unique_job_ids TO off; SELECT count(*) FROM distributed_table; SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); ROLLBACK; diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index aaed45803..0bac6d067 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -430,7 +430,7 @@ DELETE FROM pg_dist_shard WHERE shardid = 1; CREATE TABLE e_transactions(order_id varchar(255) NULL, transaction_id int) PARTITION BY LIST(transaction_id); CREATE TABLE orders_2020_07_01 PARTITION OF e_transactions FOR VALUES IN (1,2,3); -INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', NULL, 7, 's'); +INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', '{VAR :varno 1 :varattno 1 :vartype 1043 :vartypmod 259 :varcollid 100 :varlevelsup 0 :varnosyn 1 :varattnosyn 1 :location -1}', 7, 's'); SELECT (metadata->>'partitioned_citus_table_exists_pre_11')::boolean as partitioned_citus_table_exists_pre_11, @@ -526,7 +526,7 @@ $function$; SET citus.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false'; -- This will fail because of previous function declaration -ALTER EXTENSION citus UPDATE TO '8.1-1'; +ALTER EXTENSION citus UPDATE TO '9.1-1'; -- We can DROP problematic function and continue ALTER EXTENSION even when version checks are on SET citus.enable_version_checks TO 'true'; @@ -535,7 +535,7 @@ DROP FUNCTION pg_catalog.relation_is_a_known_shard(regclass); SET citus.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false'; -ALTER EXTENSION citus UPDATE TO '8.1-1'; +ALTER EXTENSION citus UPDATE TO '9.1-1'; -- Test updating to the latest version without specifying the version number ALTER EXTENSION citus UPDATE; 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 779f4c24e..962f77318 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql @@ -35,12 +35,18 @@ LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +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 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -84,7 +90,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -112,7 +119,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS diff --git a/src/test/regress/sql/multi_repartition_udt.sql b/src/test/regress/sql/multi_repartition_udt.sql index 2e5c1de24..2b71ec116 100644 --- a/src/test/regress/sql/multi_repartition_udt.sql +++ b/src/test/regress/sql/multi_repartition_udt.sql @@ -36,12 +36,18 @@ LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +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 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -85,7 +91,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -113,7 +120,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS diff --git a/src/test/regress/sql/prepared_statements_1.sql b/src/test/regress/sql/prepared_statements_1.sql index ec3f521ad..aee511696 100644 --- a/src/test/regress/sql/prepared_statements_1.sql +++ b/src/test/regress/sql/prepared_statements_1.sql @@ -17,3 +17,18 @@ BEGIN; CREATE TEMP TABLE repartition_prepared_tmp AS EXECUTE repartition_prepared(1); SELECT count(*) from repartition_prepared_tmp; ROLLBACK; + +PREPARE xact_repartitioned_prepared AS + SELECT count(*) FROM repartition_prepared_test t1 JOIN repartition_prepared_test t2 USING (b); + +BEGIN; + -- Prepared re-partition join in a transaction block after a write + INSERT INTO repartition_prepared_test VALUES (1,2); + EXECUTE xact_repartitioned_prepared; +ROLLBACK; + +BEGIN; + -- Prepared re-partition join in a transaction block before a write + EXECUTE xact_repartitioned_prepared; + INSERT INTO repartition_prepared_test VALUES (1,2); +ROLLBACK; diff --git a/src/test/regress/sql/propagate_foreign_servers.sql b/src/test/regress/sql/propagate_foreign_servers.sql index f27a10e5d..cd64f7c9b 100644 --- a/src/test/regress/sql/propagate_foreign_servers.sql +++ b/src/test/regress/sql/propagate_foreign_servers.sql @@ -4,6 +4,16 @@ SET search_path TO propagate_foreign_server; -- remove node to add later SELECT citus_remove_node('localhost', :worker_1_port); +-- not related, but added here to test propagation of aggregates +-- to newly added nodes +CREATE AGGREGATE array_agg (anynonarray) +( + sfunc = array_agg_transfn, + stype = internal, + finalfunc = array_agg_finalfn, + finalfunc_extra +); + -- create schema, extension and foreign server while the worker is removed SET citus.enable_ddl_propagation TO OFF; CREATE SCHEMA test_dependent_schema; @@ -20,6 +30,13 @@ CREATE FOREIGN TABLE foreign_table ( OPTIONS (schema_name 'test_dependent_schema', table_name 'foreign_table_test'); SELECT 1 FROM citus_add_node('localhost', :master_port, groupId=>0); + +-- verify that the aggregate is propagated to the new node +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%propagate_foreign_server.array_agg%';$$); + +-- verify that the aggregate is added top pg_dist_object on the new node +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$); + SELECT citus_add_local_table_to_metadata('foreign_table'); ALTER TABLE foreign_table OWNER TO pg_monitor; diff --git a/src/test/regress/sql/shard_rebalancer.sql b/src/test/regress/sql/shard_rebalancer.sql index 8a9ae16d9..bad28f1e9 100644 --- a/src/test/regress/sql/shard_rebalancer.sql +++ b/src/test/regress/sql/shard_rebalancer.sql @@ -1419,3 +1419,34 @@ UPDATE pg_dist_node SET shouldhaveshards=true WHERE nodeport = :worker_2_port; SELECT rebalance_table_shards(); CALL citus_cleanup_orphaned_shards(); DROP TABLE test_rebalance_with_index CASCADE; + + +-- Test rebalancer with disabled worker + +SET citus.next_shard_id TO 433500; +SET citus.shard_replication_factor TO 2; + +DROP TABLE IF EXISTS test_rebalance_with_disabled_worker; +CREATE TABLE test_rebalance_with_disabled_worker (a int); +SELECT create_distributed_table('test_rebalance_with_disabled_worker', 'a', colocate_with:='none'); + +SELECT citus_disable_node('localhost', :worker_2_port); +SELECT public.wait_until_metadata_sync(30000); + +SELECT rebalance_table_shards('test_rebalance_with_disabled_worker'); + +SELECT citus_activate_node('localhost', :worker_2_port); + +DROP TABLE test_rebalance_with_disabled_worker; + +-- Test rebalance with all shards excluded + +DROP TABLE IF EXISTS test_with_all_shards_excluded; +CREATE TABLE test_with_all_shards_excluded(a int PRIMARY KEY); +SELECT create_distributed_table('test_with_all_shards_excluded', 'a', colocate_with:='none', shard_count:=4); + +SELECT shardid FROM pg_dist_shard; + +SELECT rebalance_table_shards('test_with_all_shards_excluded', excluded_shard_list:='{102073, 102074, 102075, 102076}'); + +DROP TABLE test_with_all_shards_excluded; diff --git a/src/test/regress/sql/text_search.sql b/src/test/regress/sql/text_search.sql index 916644fd6..bb8254f0c 100644 --- a/src/test/regress/sql/text_search.sql +++ b/src/test/regress/sql/text_search.sql @@ -258,6 +258,23 @@ CREATE TABLE t5(id int, name text); CREATE INDEX t5_search_name ON t5 USING gin (to_tsvector('"Text Search Requiring Quote''s"."Quoted Config Name"'::regconfig, (COALESCE(name, ''::character varying))::text)); SELECT create_distributed_table('t5', 'name'); +-- make sure partial indices propagate their dependencies +-- first have a TEXT SEARCH CONFIGURATION that is not distributed +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION partial_index_test_config ( parser = default ); +RESET citus.enable_ddl_propagation; + +CREATE TABLE sensors( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_a_partition PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_search_name ON sensors USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors', 'measureid'); + SET client_min_messages TO 'warning'; DROP SCHEMA text_search, text_search2, "Text Search Requiring Quote's" CASCADE; DROP ROLE text_search_owner; diff --git a/src/test/regress/sql/upgrade_post_11_after.sql b/src/test/regress/sql/upgrade_post_11_after.sql new file mode 100644 index 000000000..5e08ca8b1 --- /dev/null +++ b/src/test/regress/sql/upgrade_post_11_after.sql @@ -0,0 +1,42 @@ +SET search_path = post_11_upgrade; + +-- make sure that we always (re)sync the metadata +UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb('true'::bool), true); +SELECT citus_finalize_upgrade_to_citus11(enforce_version_check:=false); + +-- tables are objects with Citus 11+ +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1; + +-- on all nodes +SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1; + +-- Create the necessary test utility function +CREATE OR REPLACE FUNCTION activate_node_snapshot() + RETURNS text[] + LANGUAGE C STRICT + AS 'citus'; + +-- make sure that workers and the coordinator has the same datesyle +SET datestyle = "ISO, YMD"; +SELECT 1 FROM run_command_on_workers($$ALTER SYSTEM SET datestyle = "ISO, YMD";$$); +SELECT 1 FROM run_command_on_workers($$SELECT pg_reload_conf()$$); + +-- make sure that the metadata is consistent across all nodes +-- we exclude the distributed_object_data as they are +-- not sorted in the same order (as OIDs differ on the nodes) +SELECT count(*) = 0 AS same_metadata_in_workers FROM +( + ( + SELECT unnest(activate_node_snapshot()) as command + EXCEPT + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + ) +UNION + ( + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) as command + ) +) AS foo WHERE command NOT ILIKE '%distributed_object_data%'; diff --git a/src/test/regress/sql/upgrade_post_11_before.sql b/src/test/regress/sql/upgrade_post_11_before.sql new file mode 100644 index 000000000..959b026f8 --- /dev/null +++ b/src/test/regress/sql/upgrade_post_11_before.sql @@ -0,0 +1,155 @@ + +-- test cases for #3970 +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; + +CREATE SCHEMA post_11_upgrade; +SET search_path = post_11_upgrade; + +--1. create a partitioned table, and a vanilla table that will be colocated with this table +CREATE TABLE part_table ( + work_ymdt timestamp without time zone NOT NULL, + seq bigint NOT NULL, + my_seq bigint NOT NULL, + work_memo character varying(150), + CONSTRAINT work_memo_check CHECK ((octet_length((work_memo)::text) <= 150)), + PRIMARY KEY(seq, work_ymdt) +) +PARTITION BY RANGE (work_ymdt); + +CREATE TABLE dist(seq bigint UNIQUE); + +--2. perform create_distributed_table +SELECT create_distributed_table('part_table', 'seq'); +SELECT create_distributed_table('dist','seq'); + +--3. add a partitions +CREATE TABLE part_table_p202008 PARTITION OF part_table FOR VALUES FROM ('2020-08-01 00:00:00') TO ('2020-09-01 00:00:00'); +CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00'); + +--3. create indexes +CREATE INDEX i_part_1 ON part_table(seq); +CREATE INDEX i_part_2 ON part_table(my_seq, seq); +CREATE INDEX i_part_3 ON part_table(work_memo, seq); + + +CREATE TABLE sensors( +measureid integer, +eventdatetime date, +measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); + +CREATE TABLE sensors_old PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE TABLE sensors_2020_01_01 PARTITION OF sensors FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +CREATE TABLE sensors_news PARTITION OF sensors FOR VALUES FROM ('2020-05-01') TO ('2025-01-01'); + +CREATE INDEX index_on_parent ON sensors(lower(measureid::text)); +CREATE INDEX index_on_child ON sensors_2020_01_01(lower(measure_data::text)); +CREATE INDEX hash_index ON sensors USING HASH((measure_data->'IsFailed')); +CREATE INDEX index_with_include ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime); + +CREATE STATISTICS s1 (dependencies) ON measureid, eventdatetime FROM sensors; +CREATE STATISTICS s2 (dependencies) ON measureid, eventdatetime FROM sensors_2020_01_01; + +ALTER INDEX index_on_parent ALTER COLUMN 1 SET STATISTICS 1000; +ALTER INDEX index_on_child ALTER COLUMN 1 SET STATISTICS 1000; + +CLUSTER sensors_2020_01_01 USING index_on_child; +SELECT create_distributed_table('sensors', 'measureid'); + + +-- create a colocated distributed tables and create foreign keys FROM/TO +-- the partitions +CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY); +SELECT create_distributed_table('colocated_dist_table', 'measureid'); + +CLUSTER colocated_dist_table USING colocated_dist_table_pkey; + +CREATE TABLE colocated_partitioned_table( + measureid integer, + eventdatetime date, + PRIMARY KEY (measureid, eventdatetime)) +PARTITION BY RANGE(eventdatetime); + +CREATE TABLE colocated_partitioned_table_2020_01_01 PARTITION OF colocated_partitioned_table FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +SELECT create_distributed_table('colocated_partitioned_table', 'measureid'); + +CLUSTER colocated_partitioned_table_2020_01_01 USING colocated_partitioned_table_2020_01_01_pkey; + +CREATE TABLE reference_table (measureid integer PRIMARY KEY); +SELECT create_reference_table('reference_table'); + +-- this table is used to make sure that index backed +-- replica identites can have clustered indexes +-- and no index statistics +CREATE TABLE index_backed_rep_identity(key int NOT NULL); +CREATE UNIQUE INDEX uqx ON index_backed_rep_identity(key); +ALTER TABLE index_backed_rep_identity REPLICA IDENTITY USING INDEX uqx; +CLUSTER index_backed_rep_identity USING uqx; +SELECT create_distributed_table('index_backed_rep_identity', 'key'); + +-- from parent to regular dist +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid); + +-- from parent to parent +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_parent FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table(measureid, eventdatetime); + +-- from parent to child +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_child FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid, eventdatetime); + +-- load some data +INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_partitioned_table SELECT i, '2020-01-05' FROM generate_series(0,1000)i; +INSERT INTO sensors SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i; + + +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default ); +SELECT 1 FROM run_command_on_workers($$CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default );$$); + +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +SELECT run_command_on_workers('SET citus.enable_ddl_propagation TO off; +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$;'); + +CREATE TYPE post_11_upgrade.my_type AS (a int); + +RESET citus.enable_ddl_propagation; + +CREATE TABLE sensors_parser( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + col_with_def int DEFAULT post_11_upgrade.func_in_transaction_def(), + col_with_type post_11_upgrade.my_type, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_parser_a_partition PARTITION OF sensors_parser FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_parser_search_name ON sensors_parser USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors_parser', 'measureid'); + + +SET citus.enable_ddl_propagation TO off; +CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk'); +SELECT 1 FROM run_command_on_workers($$CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk');$$); +SET citus.enable_ddl_propagation TO on; + +CREATE TABLE test_propagate_collate(id int, t2 text COLLATE german_phonebook_unpropagated); +SELECT create_distributed_table('test_propagate_collate', 'id');