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_add_local_table_to_metadata.c b/src/backend/distributed/commands/citus_add_local_table_to_metadata.c index f8c2a3042..6752a3ee3 100644 --- a/src/backend/distributed/commands/citus_add_local_table_to_metadata.c +++ b/src/backend/distributed/commands/citus_add_local_table_to_metadata.c @@ -31,6 +31,7 @@ #include "distributed/commands/sequence.h" #include "distributed/commands/utility_hook.h" #include "distributed/metadata/distobject.h" +#include "distributed/metadata/dependency.h" #include "distributed/foreign_key_relationship.h" #include "distributed/listutils.h" #include "distributed/local_executor.h" @@ -317,6 +318,7 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys, bool autoConve * Ensure dependencies exist as we will create shell table on the other nodes * in the MX case. */ + EnsureRelationDependenciesCanBeDistributed(&tableAddress); EnsureDependenciesExistOnAllNodes(&tableAddress); /* 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/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index b96882649..64734fff8 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -443,6 +443,8 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName, */ ObjectAddress tableAddress = { 0 }; ObjectAddressSet(tableAddress, RelationRelationId, relationId); + + EnsureRelationDependenciesCanBeDistributed(&tableAddress); EnsureDependenciesExistOnAllNodes(&tableAddress); char replicationModel = DecideReplicationModel(distributionMethod, 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/drop_distributed_table.c b/src/backend/distributed/commands/drop_distributed_table.c index 99c2cc2ab..d4718aab8 100644 --- a/src/backend/distributed/commands/drop_distributed_table.c +++ b/src/backend/distributed/commands/drop_distributed_table.c @@ -11,11 +11,13 @@ #include "postgres.h" #include "miscadmin.h" + #include "distributed/commands/utility_hook.h" #include "distributed/commands.h" #include "distributed/metadata_utility.h" #include "distributed/coordinator_protocol.h" #include "distributed/metadata_sync.h" +#include "distributed/multi_partitioning_utils.h" #include "distributed/worker_transaction.h" #include "utils/builtins.h" #include "utils/lsyscache.h" @@ -123,6 +125,10 @@ master_remove_distributed_table_metadata_from_workers(PG_FUNCTION_ARGS) * The function is a no-op for non-distributed tables and clusters that don't * have any workers with metadata. Also, the function errors out if called * from a worker node. + * + * This function assumed that it is called via a trigger. But we cannot do the + * typical CALLED_AS_TRIGGER check because this is called via another trigger, + * which CALLED_AS_TRIGGER does not cover. */ static void MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName, @@ -146,6 +152,16 @@ MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName return; } + if (PartitionTable(relationId)) + { + /* + * MasterRemoveDistributedTableMetadataFromWorkers is only called from drop trigger. + * When parent is dropped in a drop trigger, we remove all the corresponding + * partitions via the parent, mostly for performance reasons. + */ + return; + } + /* drop the distributed table metadata on the workers */ char *deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName); SendCommandToWorkersWithMetadata(deleteDistributionCommand); 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..11729a21f 100644 --- a/src/backend/distributed/commands/function.c +++ b/src/backend/distributed/commands/function.c @@ -69,6 +69,10 @@ (strncmp(arg, prefix, strlen(prefix)) == 0) /* forward declaration for helper functions*/ +static bool RecreateSameNonColocatedFunction(ObjectAddress functionAddress, + char *distributionArgumentName, + bool colocateWithTableNameDefault, + bool *forceDelegationAddress); static void ErrorIfAnyNodeDoesNotHaveMetadata(void); static char * GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace); static char * GetFunctionAlterOwnerCommand(const RegProcedure funcOid); @@ -82,7 +86,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); @@ -129,6 +132,7 @@ create_distributed_function(PG_FUNCTION_ARGS) char *distributionArgumentName = NULL; char *colocateWithTableName = NULL; + bool colocateWithTableNameDefault = false; bool *forceDelegationAddress = NULL; bool forceDelegation = false; ObjectAddress extensionAddress = { 0 }; @@ -168,8 +172,13 @@ create_distributed_function(PG_FUNCTION_ARGS) colocateWithText = PG_GETARG_TEXT_P(2); colocateWithTableName = text_to_cstring(colocateWithText); + if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0) + { + colocateWithTableNameDefault = true; + } + /* check if the colocation belongs to a reference table */ - if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0) + if (!colocateWithTableNameDefault) { Oid colocationRelationId = ResolveRelationId(colocateWithText, false); colocatedWithReferenceTable = IsCitusTableType(colocationRelationId, @@ -193,6 +202,20 @@ create_distributed_function(PG_FUNCTION_ARGS) ObjectAddressSet(functionAddress, ProcedureRelationId, funcOid); + if (RecreateSameNonColocatedFunction(functionAddress, + distributionArgumentName, + colocateWithTableNameDefault, + forceDelegationAddress)) + { + char *schemaName = get_namespace_name(get_func_namespace(funcOid)); + char *functionName = get_func_name(funcOid); + char *qualifiedName = quote_qualified_identifier(schemaName, functionName); + ereport(NOTICE, (errmsg("procedure %s is already distributed", qualifiedName), + errdetail("Citus distributes procedures with CREATE " + "[PROCEDURE|FUNCTION|AGGREGATE] commands"))); + PG_RETURN_VOID(); + } + /* * If the function is owned by an extension, only update the * pg_dist_object, and not propagate the CREATE FUNCTION. Function @@ -260,6 +283,55 @@ create_distributed_function(PG_FUNCTION_ARGS) } +/* + * RecreateSameNonColocatedFunction returns true if the given parameters of + * create_distributed_function will not change anything on the given function. + * Returns false otherwise. + */ +static bool +RecreateSameNonColocatedFunction(ObjectAddress functionAddress, + char *distributionArgumentName, + bool colocateWithTableNameDefault, + bool *forceDelegationAddress) +{ + DistObjectCacheEntry *cacheEntry = + LookupDistObjectCacheEntry(ProcedureRelationId, + functionAddress.objectId, + InvalidOid); + + if (cacheEntry == NULL || !cacheEntry->isValid || !cacheEntry->isDistributed) + { + return false; + } + + /* + * If the colocationId, forceDelegation and distributionArgIndex fields of a + * pg_dist_object entry of a distributed function are all set to zero, it means + * that function is either automatically distributed by ddl propagation, without + * calling create_distributed_function. Or, it could be distributed via + * create_distributed_function, but with no parameters. + * + * For these cases, calling create_distributed_function for that function, + * without parameters would be idempotent. Hence we can simply early return here, + * by providing a notice message to the user. + */ + + /* are pg_dist_object fields set to zero? */ + bool functionDistributedWithoutParams = + cacheEntry->colocationId == 0 && + cacheEntry->forceDelegation == 0 && + cacheEntry->distributionArgIndex == 0; + + /* called create_distributed_function without parameters? */ + bool distributingAgainWithNoParams = + distributionArgumentName == NULL && + colocateWithTableNameDefault && + forceDelegationAddress == NULL; + + return functionDistributedWithoutParams && distributingAgainWithNoParams; +} + + /* * ErrorIfAnyNodeDoesNotHaveMetadata throws error if any * of the worker nodes does not have the metadata. @@ -1348,51 +1420,6 @@ 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 * -GetUndistributableDependency(ObjectAddress *objectAddress) -{ - List *dependencies = GetAllDependenciesForObject(objectAddress); - ObjectAddress *dependency = NULL; - foreach_ptr(dependency, dependencies) - { - if (IsObjectDistributed(dependency)) - { - continue; - } - - if (!SupportedDependencyByCitus(dependency)) - { - /* - * Since roles should be handled manually with Citus community, skip them. - */ - if (getObjectClass(dependency) != OCLASS_ROLE) - { - return dependency; - } - } - - if (getObjectClass(dependency) == OCLASS_CLASS) - { - /* - * Citus can only distribute dependent non-distributed sequence - * and composite types. - */ - char relKind = get_rel_relkind(dependency->objectId); - if (relKind != RELKIND_SEQUENCE && relKind != RELKIND_COMPOSITE_TYPE) - { - return dependency; - } - } - } - - return NULL; -} - - /* * CreateFunctionStmtObjectAddress returns the ObjectAddress for the subject of the * CREATE [OR REPLACE] FUNCTION statement. If missing_ok is false it will error with the @@ -1443,10 +1470,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 +2054,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/table.c b/src/backend/distributed/commands/table.c index 4bf1ff373..fe1e21daf 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -1955,6 +1955,7 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement) /* changing a relation could introduce new dependencies */ ObjectAddress tableAddress = { 0 }; ObjectAddressSet(tableAddress, RelationRelationId, relationId); + EnsureRelationDependenciesCanBeDistributed(&tableAddress); EnsureDependenciesExistOnAllNodes(&tableAddress); } 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/connection/shared_connection_stats.c b/src/backend/distributed/connection/shared_connection_stats.c index 7592a6feb..4cdd065d7 100644 --- a/src/backend/distributed/connection/shared_connection_stats.c +++ b/src/backend/distributed/connection/shared_connection_stats.c @@ -106,6 +106,9 @@ int MaxSharedPoolSize = 0; */ int LocalSharedPoolSize = 0; +/* number of connections reserved for Citus */ +int MaxClientConnections = ALLOW_ALL_EXTERNAL_CONNECTIONS; + /* the following two structs are used for accessing shared memory */ static HTAB *SharedConnStatsHash = NULL; @@ -192,6 +195,25 @@ StoreAllRemoteConnectionStats(Tuplestorestate *tupleStore, TupleDesc tupleDescri } +/* + * GetMaxClientConnections returns the value of citus.max_client_connections, + * or max_connections when it is -1 or when connecting as superuser. + * + * The latter is done because citus.max_client_connections does not apply to + * superuser. + */ +int +GetMaxClientConnections(void) +{ + if (MaxClientConnections == ALLOW_ALL_EXTERNAL_CONNECTIONS || superuser()) + { + return MaxConnections; + } + + return MaxClientConnections; +} + + /* * GetMaxSharedPoolSize is a wrapper around MaxSharedPoolSize which is controlled * via a GUC. @@ -204,7 +226,7 @@ GetMaxSharedPoolSize(void) { if (MaxSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY) { - return MaxConnections; + return GetMaxClientConnections(); } return MaxSharedPoolSize; @@ -223,7 +245,7 @@ GetLocalSharedPoolSize(void) { if (LocalSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY) { - return MaxConnections * 0.5; + return GetMaxClientConnections() * 0.5; } return LocalSharedPoolSize; @@ -318,7 +340,7 @@ TryToIncrementSharedConnectionCounter(const char *hostname, int port) return false; } - activeBackendCount = GetAllActiveClientBackendCount(); + activeBackendCount = GetExternalClientBackendCount(); } LockConnectionSharedMemory(LW_EXCLUSIVE); 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..5994c5d51 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; } @@ -739,6 +741,142 @@ SupportedDependencyByCitus(const ObjectAddress *address) } +/* + * EnsureRelationDependenciesCanBeDistributed ensures all dependencies of the relation + * can be distributed. + */ +void +EnsureRelationDependenciesCanBeDistributed(ObjectAddress *relationAddress) +{ + ObjectAddress *undistributableDependency = + GetUndistributableDependency(relationAddress); + + if (undistributableDependency != NULL) + { + char *tableName = get_rel_name(relationAddress->objectId); + + 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 + * + * For now the relations are the only type that is supported by Citus + * but can not be distributed as dependency, though we've added an + * explicit check below as well to not to break the logic here in case + * GetUndistributableDependency changes. + */ + if (getObjectClass(undistributableDependency) == OCLASS_CLASS) + { + char *dependentRelationName = get_rel_name( + undistributableDependency->objectId); + + ereport(ERROR, (errmsg("Relation \"%s\" has dependency to a table" + " \"%s\" that is not in Citus' metadata", + tableName, dependentRelationName), + errhint("Distribute dependent relation first."))); + } + } + + char *objectType = NULL; + #if PG_VERSION_NUM >= PG_VERSION_14 + objectType = getObjectDescription(undistributableDependency, false); + #else + objectType = getObjectDescription(undistributableDependency); + #endif + ereport(ERROR, (errmsg("Relation \"%s\" has dependency on unsupported " + "object \"%s\"", tableName, objectType))); + } +} + + +/* + * GetUndistributableDependency checks whether object has any non-distributable + * dependency. If any one found, it will be returned. + */ +ObjectAddress * +GetUndistributableDependency(ObjectAddress *objectAddress) +{ + List *dependencies = GetAllDependenciesForObject(objectAddress); + ObjectAddress *dependency = NULL; + + /* + * Users can disable metadata sync by their own risk. If it is disabled, Citus + * doesn't propagate dependencies. So, if it is disabled, there is no undistributable + * dependency. + */ + if (!EnableMetadataSync) + { + return NULL; + } + + foreach_ptr(dependency, dependencies) + { + /* + * Objects with the id smaller than FirstNormalObjectId should be created within + * initdb. Citus needs to have such objects as distributed, so we can not add + * such check to dependency resolution logic. Though, Citus shouldn't error + * out if such dependency is not supported. So, skip them here. + */ + if (dependency->objectId < FirstNormalObjectId) + { + continue; + } + + /* + * If object is distributed already, ignore it. + */ + if (IsObjectDistributed(dependency)) + { + continue; + } + + /* + * If the dependency is not supported with Citus, return the dependency. + */ + if (!SupportedDependencyByCitus(dependency)) + { + /* + * Since roles should be handled manually with Citus community, skip them. + */ + if (getObjectClass(dependency) != OCLASS_ROLE) + { + return dependency; + } + } + + if (getObjectClass(dependency) == OCLASS_CLASS) + { + char relKind = get_rel_relkind(dependency->objectId); + + if (relKind == RELKIND_SEQUENCE || relKind == RELKIND_COMPOSITE_TYPE) + { + /* citus knows how to auto-distribute these dependencies */ + continue; + } + else if (relKind == RELKIND_INDEX || relKind == RELKIND_PARTITIONED_INDEX) + { + /* + * Indexes are only qualified for distributed objects for dependency + * tracking purposes, so we can ignore those. + */ + continue; + } + else + { + /* + * Citus doesn't know how to auto-distribute the rest of the RELKINDs + * via dependency resolution + */ + return dependency; + } + } + } + + return NULL; +} + + /* * IsTableOwnedByExtension returns whether the table with the given relation ID is * owned by an extension. @@ -1090,9 +1228,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 +1277,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/distobject.c b/src/backend/distributed/metadata/distobject.c index 41b3b372d..b345210af 100644 --- a/src/backend/distributed/metadata/distobject.c +++ b/src/backend/distributed/metadata/distobject.c @@ -209,8 +209,9 @@ MarkObjectDistributedLocally(const ObjectAddress *distAddress) ObjectIdGetDatum(distAddress->objectId), Int32GetDatum(distAddress->objectSubId) }; - char *insertQuery = "INSERT INTO citus.pg_dist_object (classid, objid, objsubid) " - "VALUES ($1, $2, $3) ON CONFLICT DO NOTHING"; + char *insertQuery = + "INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid) " + "VALUES ($1, $2, $3) ON CONFLICT DO NOTHING"; int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes, paramValues); if (spiStatus < 0) @@ -324,7 +325,7 @@ UnmarkObjectDistributed(const ObjectAddress *address) Int32GetDatum(address->objectSubId) }; - char *deleteQuery = "DELETE FROM citus.pg_dist_object WHERE classid = $1 AND " + char *deleteQuery = "DELETE FROM pg_catalog.pg_dist_object WHERE classid = $1 AND " "objid = $2 AND objsubid = $3"; int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes, diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 2265ca691..6cce0688b 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -247,8 +247,13 @@ static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMe Oid *intervalTypeId, int32 *intervalTypeMod); static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid); static void CachedRelationLookup(const char *relationName, Oid *cachedOid); +static void CachedRelationLookupExtended(const char *relationName, Oid *cachedOid, + bool missing_ok); static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace, Oid *cachedOid); +static void CachedRelationNamespaceLookupExtended(const char *relationName, + Oid renamespace, Oid *cachedOid, + bool missing_ok); static ShardPlacement * ResolveGroupShardPlacement( GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry, int shardIndex); @@ -2321,8 +2326,37 @@ CitusCatalogNamespaceId(void) Oid DistObjectRelationId(void) { - CachedRelationNamespaceLookup("pg_dist_object", CitusCatalogNamespaceId(), - &MetadataCache.distObjectRelationId); + /* + * In older versions pg_dist_object was living in the `citus` namespace, With Citus 11 + * this has been moved to pg_dist_catalog. + * + * During upgrades it could therefore be that we simply need to look in the old + * catalog. Since we expect to find it most of the time in the pg_catalog schema from + * now on we will start there. + * + * even after the table has been moved, the oid's stay the same, so we don't have to + * invalidate the cache after a move + * + * Note: during testing we also up/downgrade the extension, and sometimes interact + * with the database when the schema and the binary are not in sync. Hance we always + * allow the catalog to be missing on our first lookup. The error message might + * therefore become misleading as it will complain about citus.pg_dist_object not + * being found when called too early. + */ + CachedRelationLookupExtended("pg_dist_object", + &MetadataCache.distObjectRelationId, + true); + if (!OidIsValid(MetadataCache.distObjectRelationId)) + { + /* + * We can only ever reach here while we are creating/altering our extension before + * the table is moved to pg_catalog. + */ + CachedRelationNamespaceLookupExtended("pg_dist_object", + CitusCatalogNamespaceId(), + &MetadataCache.distObjectRelationId, + false); + } return MetadataCache.distObjectRelationId; } @@ -2332,9 +2366,38 @@ DistObjectRelationId(void) Oid DistObjectPrimaryKeyIndexId(void) { - CachedRelationNamespaceLookup("pg_dist_object_pkey", - CitusCatalogNamespaceId(), - &MetadataCache.distObjectPrimaryKeyIndexId); + /* + * In older versions pg_dist_object was living in the `citus` namespace, With Citus 11 + * this has been moved to pg_dist_catalog. + * + * During upgrades it could therefore be that we simply need to look in the old + * catalog. Since we expect to find it most of the time in the pg_catalog schema from + * now on we will start there. + * + * even after the table has been moved, the oid's stay the same, so we don't have to + * invalidate the cache after a move + * + * Note: during testing we also up/downgrade the extension, and sometimes interact + * with the database when the schema and the binary are not in sync. Hance we always + * allow the catalog to be missing on our first lookup. The error message might + * therefore become misleading as it will complain about citus.pg_dist_object not + * being found when called too early. + */ + CachedRelationLookupExtended("pg_dist_object_pkey", + &MetadataCache.distObjectPrimaryKeyIndexId, + true); + + if (!OidIsValid(MetadataCache.distObjectPrimaryKeyIndexId)) + { + /* + * We can only ever reach here while we are creating/altering our extension before + * the table is moved to pg_catalog. + */ + CachedRelationNamespaceLookupExtended("pg_dist_object_pkey", + CitusCatalogNamespaceId(), + &MetadataCache.distObjectPrimaryKeyIndexId, + false); + } return MetadataCache.distObjectPrimaryKeyIndexId; } @@ -4591,9 +4654,30 @@ CachedRelationLookup(const char *relationName, Oid *cachedOid) } +/* + * CachedRelationLookupExtended performs a cached lookup for the relation + * relationName, with the result cached in *cachedOid. Will _not_ throw an error when + * missing_ok is set to true. + */ +static void +CachedRelationLookupExtended(const char *relationName, Oid *cachedOid, bool missing_ok) +{ + CachedRelationNamespaceLookupExtended(relationName, PG_CATALOG_NAMESPACE, cachedOid, + missing_ok); +} + + static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace, Oid *cachedOid) +{ + CachedRelationNamespaceLookupExtended(relationName, relnamespace, cachedOid, false); +} + + +static void +CachedRelationNamespaceLookupExtended(const char *relationName, Oid relnamespace, + Oid *cachedOid, bool missing_ok) { /* force callbacks to be registered, so we always get notified upon changes */ InitializeCaches(); @@ -4602,7 +4686,7 @@ CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace, { *cachedOid = get_relname_relid(relationName, relnamespace); - if (*cachedOid == InvalidOid) + if (*cachedOid == InvalidOid && !missing_ok) { ereport(ERROR, (errmsg( "cache lookup failed for %s, called too early?", diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 490eeb4ca..dc501923e 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -28,9 +28,12 @@ #include "catalog/indexing.h" #include "catalog/pg_am.h" #include "catalog/pg_attrdef.h" +#include "catalog/pg_collation.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" @@ -46,12 +49,14 @@ #include "distributed/maintenanced.h" #include "distributed/metadata_cache.h" #include "distributed/metadata_sync.h" +#include "distributed/metadata_utility.h" #include "distributed/metadata/distobject.h" #include "distributed/metadata/pg_dist_object.h" #include "distributed/multi_executor.h" #include "distributed/multi_join_order.h" #include "distributed/multi_partitioning_utils.h" #include "distributed/multi_physical_planner.h" +#include "distributed/pg_dist_colocation.h" #include "distributed/pg_dist_node.h" #include "distributed/pg_dist_shard.h" #include "distributed/relation_access_tracking.h" @@ -85,6 +90,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 +110,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); @@ -122,6 +127,14 @@ static void EnsureShardMetadataIsSane(Oid relationId, int64 shardId, char storag static void EnsureShardPlacementMetadataIsSane(Oid relationId, int64 shardId, int64 placementId, int32 shardState, int64 shardLength, int32 groupId); +static char * ColocationGroupCreateCommand(uint32 colocationId, int shardCount, + int replicationFactor, + Oid distributionColumnType, + Oid distributionColumnCollation); +static char * ColocationGroupDeleteCommand(uint32 colocationId); +static char * RemoteTypeIdExpression(Oid typeId); +static char * RemoteCollationIdExpression(Oid colocationId); + PG_FUNCTION_INFO_V1(start_metadata_sync_to_node); PG_FUNCTION_INFO_V1(stop_metadata_sync_to_node); @@ -140,6 +153,8 @@ PG_FUNCTION_INFO_V1(citus_internal_update_placement_metadata); PG_FUNCTION_INFO_V1(citus_internal_delete_shard_metadata); PG_FUNCTION_INFO_V1(citus_internal_update_relation_colocation); PG_FUNCTION_INFO_V1(citus_internal_add_object_metadata); +PG_FUNCTION_INFO_V1(citus_internal_add_colocation_metadata); +PG_FUNCTION_INFO_V1(citus_internal_delete_colocation_metadata); static bool got_SIGTERM = false; @@ -556,6 +571,7 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode) dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PLACEMENTS); dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_DISTRIBUTED_OBJECTS); + dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_COLOCATION); Assert(superuser()); SendOptionalMetadataCommandListToWorkerInCoordinatedTransaction( @@ -1544,6 +1560,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 +1961,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", @@ -3046,3 +3175,311 @@ citus_internal_update_relation_colocation(PG_FUNCTION_ARGS) PG_RETURN_VOID(); } + + +/* + * citus_internal_add_colocation_metadata is an internal UDF to + * add a row to pg_dist_colocation. + */ +Datum +citus_internal_add_colocation_metadata(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + EnsureSuperUser(); + + int colocationId = PG_GETARG_INT32(0); + int shardCount = PG_GETARG_INT32(1); + int replicationFactor = PG_GETARG_INT32(2); + Oid distributionColumnType = PG_GETARG_INT32(3); + Oid distributionColumnCollation = PG_GETARG_INT32(4); + + if (!ShouldSkipMetadataChecks()) + { + /* this UDF is not allowed allowed for executing as a separate command */ + EnsureCoordinatorInitiatedOperation(); + } + + InsertColocationGroupLocally(colocationId, shardCount, replicationFactor, + distributionColumnType, distributionColumnCollation); + + PG_RETURN_VOID(); +} + + +/* + * citus_internal_delete_colocation_metadata is an internal UDF to + * delte row from pg_dist_colocation. + */ +Datum +citus_internal_delete_colocation_metadata(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + EnsureSuperUser(); + + int colocationId = PG_GETARG_INT32(0); + + if (!ShouldSkipMetadataChecks()) + { + /* this UDF is not allowed allowed for executing as a separate command */ + EnsureCoordinatorInitiatedOperation(); + } + + DeleteColocationGroupLocally(colocationId); + + PG_RETURN_VOID(); +} + + +/* + * SyncNewColocationGroup synchronizes a new pg_dist_colocation entry to a worker. + */ +void +SyncNewColocationGroupToNodes(uint32 colocationId, int shardCount, int replicationFactor, + Oid distributionColumnType, Oid distributionColumnCollation) +{ + char *command = ColocationGroupCreateCommand(colocationId, shardCount, + replicationFactor, + distributionColumnType, + distributionColumnCollation); + + /* + * We require superuser for all pg_dist_colocation operations because we have + * no reasonable way of restricting access. + */ + SendCommandToWorkersWithMetadataViaSuperUser(command); +} + + +/* + * ColocationGroupCreateCommand returns a command for creating a colocation group. + */ +static char * +ColocationGroupCreateCommand(uint32 colocationId, int shardCount, int replicationFactor, + Oid distributionColumnType, Oid distributionColumnCollation) +{ + StringInfo insertColocationCommand = makeStringInfo(); + + appendStringInfo(insertColocationCommand, + "SELECT pg_catalog.citus_internal_add_colocation_metadata(" + "%d, %d, %d, %s, %s)", + colocationId, + shardCount, + replicationFactor, + RemoteTypeIdExpression(distributionColumnType), + RemoteCollationIdExpression(distributionColumnCollation)); + + return insertColocationCommand->data; +} + + +/* + * RemoteTypeIdExpression returns an expression in text form that can + * be used to obtain the OID of a type on a different node when included + * in a query string. + */ +static char * +RemoteTypeIdExpression(Oid typeId) +{ + /* by default, use 0 (InvalidOid) */ + char *expression = "0"; + + /* we also have pg_dist_colocation entries for reference tables */ + if (typeId != InvalidOid) + { + char *typeName = format_type_extended(typeId, -1, + FORMAT_TYPE_FORCE_QUALIFY | + FORMAT_TYPE_ALLOW_INVALID); + + /* format_type_extended returns ??? in case of an unknown type */ + if (strcmp(typeName, "???") != 0) + { + StringInfo regtypeExpression = makeStringInfo(); + + appendStringInfo(regtypeExpression, + "%s::regtype", + quote_literal_cstr(typeName)); + + expression = regtypeExpression->data; + } + } + + return expression; +} + + +/* + * RemoteCollationIdExpression returns an expression in text form that can + * be used to obtain the OID of a type on a different node when included + * in a query string. Currently this is a sublink because regcollation type + * is not available in PG12. + */ +static char * +RemoteCollationIdExpression(Oid colocationId) +{ + /* by default, use 0 (InvalidOid) */ + char *expression = "0"; + + if (colocationId != InvalidOid) + { + Datum collationIdDatum = ObjectIdGetDatum(colocationId); + HeapTuple collationTuple = SearchSysCache1(COLLOID, collationIdDatum); + + if (HeapTupleIsValid(collationTuple)) + { + Form_pg_collation collationform = + (Form_pg_collation) GETSTRUCT(collationTuple); + char *collationName = NameStr(collationform->collname); + char *collationSchemaName = get_namespace_name(collationform->collnamespace); + + StringInfo colocationIdQuery = makeStringInfo(); + appendStringInfo(colocationIdQuery, + "(select oid from pg_collation" + " where collname = %s" + " and collnamespace = %s::regnamespace)", + quote_literal_cstr(collationName), + quote_literal_cstr(collationSchemaName)); + + expression = colocationIdQuery->data; + } + + ReleaseSysCache(collationTuple); + } + + return expression; +} + + +/* + * SyncDeleteColocationGroupToNodes deletes a pg_dist_colocation record from workers. + */ +void +SyncDeleteColocationGroupToNodes(uint32 colocationId) +{ + char *command = ColocationGroupDeleteCommand(colocationId); + + /* + * We require superuser for all pg_dist_colocation operations because we have + * no reasonable way of restricting access. + */ + SendCommandToWorkersWithMetadataViaSuperUser(command); +} + + +/* + * ColocationGroupDeleteCommand returns a command for deleting a colocation group. + */ +static char * +ColocationGroupDeleteCommand(uint32 colocationId) +{ + StringInfo deleteColocationCommand = makeStringInfo(); + + appendStringInfo(deleteColocationCommand, + "SELECT pg_catalog.citus_internal_delete_colocation_metadata(%d)", + colocationId); + + return deleteColocationCommand->data; +} + + +/* + * ColocationGroupCreateCommandList returns the full list of commands for syncing + * pg_dist_colocation. + */ +List * +ColocationGroupCreateCommandList(void) +{ + bool hasColocations = false; + + StringInfo colocationGroupCreateCommand = makeStringInfo(); + appendStringInfo(colocationGroupCreateCommand, + "WITH colocation_group_data (colocationid, shardcount, " + "replicationfactor, distributioncolumntype, " + "distributioncolumncollationname, " + "distributioncolumncollationschema) AS (VALUES "); + + Relation pgDistColocation = table_open(DistColocationRelationId(), AccessShareLock); + + bool indexOK = false; + SysScanDesc scanDescriptor = systable_beginscan(pgDistColocation, InvalidOid, indexOK, + NULL, 0, NULL); + + HeapTuple colocationTuple = systable_getnext(scanDescriptor); + + while (HeapTupleIsValid(colocationTuple)) + { + if (hasColocations) + { + appendStringInfo(colocationGroupCreateCommand, ", "); + } + + hasColocations = true; + + Form_pg_dist_colocation colocationForm = + (Form_pg_dist_colocation) GETSTRUCT(colocationTuple); + + appendStringInfo(colocationGroupCreateCommand, + "(%d, %d, %d, %s, ", + colocationForm->colocationid, + colocationForm->shardcount, + colocationForm->replicationfactor, + RemoteTypeIdExpression(colocationForm->distributioncolumntype)); + + /* + * For collations, include the names in the VALUES section and then + * join with pg_collation. + */ + Oid distributionColumCollation = colocationForm->distributioncolumncollation; + if (distributionColumCollation != InvalidOid) + { + Datum collationIdDatum = ObjectIdGetDatum(distributionColumCollation); + HeapTuple collationTuple = SearchSysCache1(COLLOID, collationIdDatum); + + if (HeapTupleIsValid(collationTuple)) + { + Form_pg_collation collationform = + (Form_pg_collation) GETSTRUCT(collationTuple); + char *collationName = NameStr(collationform->collname); + char *collationSchemaName = get_namespace_name( + collationform->collnamespace); + + appendStringInfo(colocationGroupCreateCommand, + "%s, %s)", + quote_literal_cstr(collationName), + quote_literal_cstr(collationSchemaName)); + + ReleaseSysCache(collationTuple); + } + else + { + appendStringInfo(colocationGroupCreateCommand, + "NULL, NULL)"); + } + } + else + { + appendStringInfo(colocationGroupCreateCommand, + "NULL, NULL)"); + } + + colocationTuple = systable_getnext(scanDescriptor); + } + + systable_endscan(scanDescriptor); + table_close(pgDistColocation, AccessShareLock); + + if (!hasColocations) + { + return NIL; + } + + appendStringInfo(colocationGroupCreateCommand, + ") SELECT pg_catalog.citus_internal_add_colocation_metadata(" + "colocationid, shardcount, replicationfactor, " + "distributioncolumntype, coalesce(c.oid, 0)) " + "FROM colocation_group_data d LEFT JOIN pg_collation c " + "ON (d.distributioncolumncollationname = c.collname " + "AND d.distributioncolumncollationschema::regnamespace" + " = c.collnamespace)"); + + return list_make1(colocationGroupCreateCommand->data); +} 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..f9f070166 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -146,6 +146,8 @@ PG_FUNCTION_INFO_V1(master_activate_node); PG_FUNCTION_INFO_V1(citus_update_node); PG_FUNCTION_INFO_V1(master_update_node); PG_FUNCTION_INFO_V1(get_shard_id_for_distribution_column); +PG_FUNCTION_INFO_V1(citus_nodename_for_nodeid); +PG_FUNCTION_INFO_V1(citus_nodeport_for_nodeid); /* @@ -188,9 +190,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); @@ -656,6 +655,8 @@ PgDistTableMetadataSyncCommandList(void) DELETE_ALL_PLACEMENTS); metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, DELETE_ALL_DISTRIBUTED_OBJECTS); + metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, + DELETE_ALL_COLOCATION); /* create pg_dist_partition, pg_dist_shard and pg_dist_placement entries */ foreach_ptr(cacheEntry, propagatedTableList) @@ -667,6 +668,11 @@ PgDistTableMetadataSyncCommandList(void) tableMetadataCreateCommandList); } + /* commands to insert pg_dist_colocation entries */ + List *colocationGroupSyncCommandList = ColocationGroupCreateCommandList(); + metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList, + colocationGroupSyncCommandList); + /* As the last step, propagate the pg_dist_object entities */ Assert(ShouldPropagate()); List *distributedObjectSyncCommandList = DistributedObjectMetadataSyncCommandList(); @@ -1469,6 +1475,50 @@ get_shard_id_for_distribution_column(PG_FUNCTION_ARGS) } +/* + * citus_nodename_for_nodeid returns the node name for the node with given node id + */ +Datum +citus_nodename_for_nodeid(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + int nodeId = PG_GETARG_INT32(0); + + bool missingOk = true; + WorkerNode *node = FindNodeWithNodeId(nodeId, missingOk); + + if (node == NULL) + { + PG_RETURN_NULL(); + } + + PG_RETURN_TEXT_P(cstring_to_text(node->workerName)); +} + + +/* + * citus_nodeport_for_nodeid returns the node port for the node with given node id + */ +Datum +citus_nodeport_for_nodeid(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + int nodeId = PG_GETARG_INT32(0); + + bool missingOk = true; + WorkerNode *node = FindNodeWithNodeId(nodeId, missingOk); + + if (node == NULL) + { + PG_RETURN_NULL(); + } + + PG_RETURN_INT32(node->workerPort); +} + + /* * FindWorkerNode searches over the worker nodes and returns the workerNode * if it already exists. Else, the function returns NULL. @@ -1544,21 +1594,24 @@ 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; + List *nodeList = ActiveReadableNodeList(); + WorkerNode *node = NULL; - foreach_ptr(workerNode, workerList) + foreach_ptr(node, nodeList) { - if (workerNode->nodeId == nodeId) + if (node->nodeId == nodeId) { - return workerNode; + return node; } } /* 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, "node with node id %d could not be found", nodeId); + } return NULL; } @@ -1780,12 +1833,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 a3a6cb3c7..5c319da53 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -23,6 +23,7 @@ #include "safe_lib.h" +#include "catalog/pg_authid.h" #include "citus_version.h" #include "commands/explain.h" #include "common/string.h" @@ -84,12 +85,14 @@ #include "libpq/auth.h" #include "port/atomics.h" #include "postmaster/postmaster.h" +#include "replication/walsender.h" #include "storage/ipc.h" #include "optimizer/planner.h" #include "optimizer/paths.h" #include "tcop/tcopprot.h" #include "utils/guc.h" #include "utils/guc_tables.h" +#include "utils/syscache.h" #include "utils/varlena.h" #include "columnar/mod.h" @@ -113,9 +116,9 @@ static void DoInitialCleanup(void); static void ResizeStackToMaximumDepth(void); static void multi_log_hook(ErrorData *edata); static void RegisterConnectionCleanup(void); -static void RegisterClientBackendCounterDecrement(void); +static void RegisterExternalClientBackendCounterDecrement(void); static void CitusCleanupConnectionsAtExit(int code, Datum arg); -static void DecrementClientBackendCounterAtExit(int code, Datum arg); +static void DecrementExternalClientBackendCounterAtExit(int code, Datum arg); static void CreateRequiredDirectories(void); static void RegisterCitusConfigVariables(void); static void OverridePostgresConfigAssignHooks(void); @@ -135,6 +138,7 @@ static const char * LocalPoolSizeGucShowHook(void); static bool StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource source); static void CitusAuthHook(Port *port, int status); +static bool IsSuperuser(char *userName); static ClientAuthentication_hook_type original_client_auth_hook = NULL; @@ -218,6 +222,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* */ @@ -481,16 +492,16 @@ RegisterConnectionCleanup(void) /* - * RegisterClientBackendCounterDecrement is called when the backend terminates. + * RegisterExternalClientBackendCounterDecrement is called when the backend terminates. * For all client backends, we register a callback that will undo */ static void -RegisterClientBackendCounterDecrement(void) +RegisterExternalClientBackendCounterDecrement(void) { static bool registeredCleanup = false; if (registeredCleanup == false) { - before_shmem_exit(DecrementClientBackendCounterAtExit, 0); + before_shmem_exit(DecrementExternalClientBackendCounterAtExit, 0); registeredCleanup = true; } @@ -520,13 +531,13 @@ CitusCleanupConnectionsAtExit(int code, Datum arg) /* - * DecrementClientBackendCounterAtExit is called before_shmem_exit() of the + * DecrementExternalClientBackendCounterAtExit is called before_shmem_exit() of the * backend for the purposes decrementing */ static void -DecrementClientBackendCounterAtExit(int code, Datum arg) +DecrementExternalClientBackendCounterAtExit(int code, Datum arg) { - DecrementClientBackendCounter(); + DecrementExternalClientBackendCounter(); } @@ -603,7 +614,7 @@ RegisterCitusConfigVariables(void) false, #endif PGC_SIGHUP, - GUC_STANDARD, + GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomBoolVariable( @@ -669,6 +680,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 " @@ -1315,6 +1344,23 @@ RegisterCitusConfigVariables(void) GUC_STANDARD, NULL, NULL, NULL); + DefineCustomIntVariable( + "citus.max_client_connections", + gettext_noop("Sets the maximum number of connections regular clients can make"), + gettext_noop("To ensure that a Citus cluster has a sufficient number of " + "connection slots to serve queries internally, it can be " + "useful to reserve connection slots for Citus internal " + "connections. When max_client_connections is set to a value " + "below max_connections, the remaining connections are reserved " + "for connections between Citus nodes. This does not affect " + "superuser_reserved_connections. If set to -1, no connections " + "are reserved."), + &MaxClientConnections, + -1, -1, MaxConnections, + PGC_SUSET, + GUC_STANDARD, + NULL, NULL, NULL); + DefineCustomIntVariable( "citus.max_intermediate_result_size", gettext_noop("Sets the maximum size of the intermediate results in KB for " @@ -2146,12 +2192,86 @@ StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource source) static void CitusAuthHook(Port *port, int status) { + uint64 gpid = ExtractGlobalPID(port->application_name); + + /* external connections to not have a GPID immediately */ + if (gpid == INVALID_CITUS_INTERNAL_BACKEND_GPID) + { + /* + * We raise the shared connection counter pre-emptively. As a result, we may + * have scenarios in which a few simultaneous connection attempts prevent + * each other from succeeding, but we avoid scenarios where we oversubscribe + * the system. + * + * By also calling RegisterExternalClientBackendCounterDecrement here, we + * immediately lower the counter if we throw a FATAL error below. The client + * connection counter may temporarily exceed maxClientConnections in between. + */ + RegisterExternalClientBackendCounterDecrement(); + + uint32 externalClientCount = IncrementExternalClientBackendCounter(); + + /* + * Limit non-superuser client connections if citus.max_client_connections + * is set. + */ + if (MaxClientConnections >= 0 && + !IsSuperuser(port->user_name) && + externalClientCount > MaxClientConnections) + { + ereport(FATAL, (errcode(ERRCODE_TOO_MANY_CONNECTIONS), + errmsg("remaining connection slots are reserved for " + "non-replication superuser connections"), + errdetail("the server is configured to accept up to %d " + "regular client connections", + MaxClientConnections))); + } + + /* + * Right after this, before we assign global pid, this backend + * might get blocked by a DDL as that happens during parsing. + * + * That's why, lets mark the backend as an external backend + * which is likely to execute a distributed command. + * + * We do this so that this backend gets the chance to show + * up in citus_lock_waits. + */ + InitializeBackendData(); + SetBackendDataDistributedCommandOriginator(true); + } + /* let other authentication hooks to kick in first */ if (original_client_auth_hook) { original_client_auth_hook(port, status); } - - RegisterClientBackendCounterDecrement(); - IncrementClientBackendCounter(); +} + + +/* + * IsSuperuser returns whether the role with the given name is superuser. + */ +static bool +IsSuperuser(char *roleName) +{ + if (roleName == NULL) + { + return false; + } + + HeapTuple roleTuple = SearchSysCache1(AUTHNAME, CStringGetDatum(roleName)); + if (!HeapTupleIsValid(roleTuple)) + { + ereport(FATAL, + (errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION), + errmsg("role \"%s\" does not exist", roleName))); + } + + Form_pg_authid rform = (Form_pg_authid) GETSTRUCT(roleTuple); + bool isSuperuser = rform->rolsuper; + + ReleaseSysCache(roleTuple); + + return isSuperuser; } 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..591092a0d 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 @@ -10,6 +10,8 @@ #include "udfs/citus_shard_indexes_on_worker/11.0-1.sql" #include "udfs/citus_internal_add_object_metadata/11.0-1.sql" +#include "udfs/citus_internal_add_colocation_metadata/11.0-1.sql" +#include "udfs/citus_internal_delete_colocation_metadata/11.0-1.sql" #include "udfs/citus_run_local_command/11.0-1.sql" #include "udfs/worker_drop_sequence_dependency/11.0-1.sql" #include "udfs/worker_drop_shell_table/11.0-1.sql" @@ -20,30 +22,36 @@ #include "udfs/citus_internal_local_blocked_processes/11.0-1.sql" #include "udfs/citus_internal_global_blocked_processes/11.0-1.sql" -#include "udfs/citus_worker_stat_activity/11.0-1.sql" +#include "udfs/run_command_on_all_nodes/11.0-1.sql" +#include "udfs/citus_stat_activity/11.0-1.sql" + #include "udfs/worker_create_or_replace_object/11.0-1.sql" #include "udfs/citus_isolation_test_session_is_blocked/11.0-1.sql" #include "udfs/citus_blocking_pids/11.0-1.sql" +#include "udfs/citus_calculate_gpid/11.0-1.sql" +#include "udfs/citus_backend_gpid/11.0-1.sql" -CREATE VIEW citus.citus_worker_stat_activity AS -SELECT * FROM pg_catalog.citus_worker_stat_activity(); -ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog; -GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC; - +DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity() CASCADE; +DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity() CASCADE; #include "udfs/citus_dist_stat_activity/11.0-1.sql" -CREATE VIEW citus.citus_dist_stat_activity AS -SELECT * FROM pg_catalog.citus_dist_stat_activity(); -ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog; -GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; +-- a very simple helper function defined for citus_lock_waits +CREATE OR REPLACE FUNCTION get_nodeid_for_groupid(groupIdInput int) RETURNS int AS $$ +DECLARE + returnNodeNodeId int := 0; +begin + SELECT nodeId into returnNodeNodeId FROM pg_dist_node WHERE groupid = groupIdInput and nodecluster = current_setting('citus.cluster_name'); + RETURN returnNodeNodeId; +end +$$ LANGUAGE plpgsql; --- we have to recreate this view because recreated citus_dist_stat_activity that this view depends #include "udfs/citus_lock_waits/11.0-1.sql" #include "udfs/pg_cancel_backend/11.0-1.sql" #include "udfs/pg_terminate_backend/11.0-1.sql" +#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 +87,16 @@ 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" + +ALTER TABLE citus.pg_dist_object SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.pg_dist_object TO public; +#include "udfs/citus_prepare_pg_upgrade/11.0-1.sql" +#include "udfs/citus_finish_pg_upgrade/11.0-1.sql" + +#include "udfs/citus_nodename_for_nodeid/11.0-1.sql" +#include "udfs/citus_nodeport_for_nodeid/11.0-1.sql" + +#include "udfs/citus_nodeid_for_gpid/11.0-1.sql" +#include "udfs/citus_pid_for_gpid/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..19d5bb22d 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,12 @@ -- citus--11.0-1--10.2-4 +REVOKE SELECT ON pg_catalog.pg_dist_object FROM public; +ALTER TABLE pg_catalog.pg_dist_object SET SCHEMA citus; 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 +27,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) @@ -45,6 +52,8 @@ DROP FUNCTION pg_catalog.citus_check_connection_to_node (text, integer); DROP FUNCTION pg_catalog.citus_check_cluster_node_health (); DROP FUNCTION pg_catalog.citus_internal_add_object_metadata(text, text[], text[], integer, integer, boolean); +DROP FUNCTION pg_catalog.citus_internal_add_colocation_metadata(int, int, int, regtype, oid); +DROP FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(int); DROP FUNCTION pg_catalog.citus_run_local_command(text); DROP FUNCTION pg_catalog.worker_drop_sequence_dependency(text); DROP FUNCTION pg_catalog.worker_drop_shell_table(table_name text); @@ -85,7 +94,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'; @@ -113,10 +121,13 @@ CREATE FUNCTION get_global_active_transactions(OUT datid oid, OUT process_id int RESET search_path; -DROP FUNCTION citus_internal_local_blocked_processes CASCADE; -DROP FUNCTION citus_internal_global_blocked_processes CASCADE; +DROP VIEW pg_catalog.citus_lock_waits; -DROP FUNCTION pg_catalog.citus_dist_stat_activity CASCADE; +DROP FUNCTION citus_internal_local_blocked_processes; +DROP FUNCTION citus_internal_global_blocked_processes; + +DROP VIEW IF EXISTS pg_catalog.citus_dist_stat_activity; +DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity; CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, @@ -142,7 +153,8 @@ ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; SET search_path = 'pg_catalog'; -DROP FUNCTION citus_worker_stat_activity CASCADE; +DROP VIEW IF EXISTS citus_worker_stat_activity; +DROP FUNCTION IF EXISTS citus_worker_stat_activity; CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, @@ -165,10 +177,10 @@ IS 'returns distributed transaction activity on shards of distributed tables'; DROP FUNCTION pg_catalog.worker_create_or_replace_object(text[]); #include "../udfs/worker_create_or_replace_object/9.0-1.sql" -DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(bigint) CASCADE; -DROP FUNCTION IF EXISTS pg_catalog.pg_terminate_backend(bigint, bigint) CASCADE; +DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(bigint); +DROP FUNCTION IF EXISTS pg_catalog.pg_terminate_backend(bigint, bigint); -DROP FUNCTION pg_catalog.dump_local_wait_edges CASCADE; +DROP FUNCTION pg_catalog.dump_local_wait_edges; CREATE FUNCTION pg_catalog.dump_local_wait_edges( OUT waiting_pid int4, OUT waiting_node_id int4, @@ -185,7 +197,7 @@ AS $$MODULE_PATHNAME$$, $$dump_local_wait_edges$$; COMMENT ON FUNCTION pg_catalog.dump_local_wait_edges() IS 'returns all local lock wait chains, that start from distributed transactions'; -DROP FUNCTION pg_catalog.dump_global_wait_edges CASCADE; +DROP FUNCTION pg_catalog.dump_global_wait_edges; CREATE FUNCTION pg_catalog.dump_global_wait_edges( OUT waiting_pid int4, OUT waiting_node_id int4, @@ -342,4 +354,19 @@ 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); +DROP FUNCTION pg_catalog.citus_calculate_gpid(integer,integer); +DROP FUNCTION pg_catalog.citus_backend_gpid(); +DROP FUNCTION get_nodeid_for_groupid(integer); + RESET search_path; + +DROP VIEW pg_catalog.citus_stat_activity; +DROP FUNCTION pg_catalog.citus_stat_activity; +DROP FUNCTION pg_catalog.run_command_on_all_nodes; + +DROP FUNCTION pg_catalog.citus_nodename_for_nodeid(integer); +DROP FUNCTION pg_catalog.citus_nodeport_for_nodeid(integer); + +DROP FUNCTION pg_catalog.citus_nodeid_for_gpid(bigint); +DROP FUNCTION pg_catalog.citus_pid_for_gpid(bigint); diff --git a/src/backend/distributed/sql/udfs/citus_backend_gpid/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_backend_gpid/11.0-1.sql new file mode 100644 index 000000000..fd7222c04 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_backend_gpid/11.0-1.sql @@ -0,0 +1,8 @@ +CREATE FUNCTION pg_catalog.citus_backend_gpid() + RETURNS BIGINT + LANGUAGE C STRICT + AS 'MODULE_PATHNAME',$$citus_backend_gpid$$; +COMMENT ON FUNCTION pg_catalog.citus_backend_gpid() + IS 'returns gpid of the current backend'; + +GRANT EXECUTE ON FUNCTION pg_catalog.citus_backend_gpid() TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_backend_gpid/latest.sql b/src/backend/distributed/sql/udfs/citus_backend_gpid/latest.sql new file mode 100644 index 000000000..fd7222c04 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_backend_gpid/latest.sql @@ -0,0 +1,8 @@ +CREATE FUNCTION pg_catalog.citus_backend_gpid() + RETURNS BIGINT + LANGUAGE C STRICT + AS 'MODULE_PATHNAME',$$citus_backend_gpid$$; +COMMENT ON FUNCTION pg_catalog.citus_backend_gpid() + IS 'returns gpid of the current backend'; + +GRANT EXECUTE ON FUNCTION pg_catalog.citus_backend_gpid() TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_calculate_gpid/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_calculate_gpid/11.0-1.sql new file mode 100644 index 000000000..a5eecc7ae --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_calculate_gpid/11.0-1.sql @@ -0,0 +1,9 @@ +CREATE FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer, + pid integer) + RETURNS BIGINT + LANGUAGE C STRICT + AS 'MODULE_PATHNAME',$$citus_calculate_gpid$$; +COMMENT ON FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer, pid integer) + IS 'calculate gpid of a backend running on any node'; + +GRANT EXECUTE ON FUNCTION pg_catalog.citus_calculate_gpid(integer, integer) TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_calculate_gpid/latest.sql b/src/backend/distributed/sql/udfs/citus_calculate_gpid/latest.sql new file mode 100644 index 000000000..a5eecc7ae --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_calculate_gpid/latest.sql @@ -0,0 +1,9 @@ +CREATE FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer, + pid integer) + RETURNS BIGINT + LANGUAGE C STRICT + AS 'MODULE_PATHNAME',$$citus_calculate_gpid$$; +COMMENT ON FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer, pid integer) + IS 'calculate gpid of a backend running on any node'; + +GRANT EXECUTE ON FUNCTION pg_catalog.citus_calculate_gpid(integer, integer) TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_dist_stat_activity/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_dist_stat_activity/11.0-1.sql index 7b38f627d..97dc387b3 100644 --- a/src/backend/distributed/sql/udfs/citus_dist_stat_activity/11.0-1.sql +++ b/src/backend/distributed/sql/udfs/citus_dist_stat_activity/11.0-1.sql @@ -1,19 +1,8 @@ -DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity CASCADE; +DROP VIEW IF EXISTS pg_catalog.citus_dist_stat_activity; -CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -RETURNS SETOF RECORD -LANGUAGE C STRICT AS 'MODULE_PATHNAME', -$$citus_dist_stat_activity$$; +CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS +SELECT * FROM citus_stat_activity +WHERE is_worker_query = false; -COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -IS 'returns distributed transaction activity on distributed tables'; +ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_dist_stat_activity/latest.sql b/src/backend/distributed/sql/udfs/citus_dist_stat_activity/latest.sql index 7b38f627d..97dc387b3 100644 --- a/src/backend/distributed/sql/udfs/citus_dist_stat_activity/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_dist_stat_activity/latest.sql @@ -1,19 +1,8 @@ -DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity CASCADE; +DROP VIEW IF EXISTS pg_catalog.citus_dist_stat_activity; -CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -RETURNS SETOF RECORD -LANGUAGE C STRICT AS 'MODULE_PATHNAME', -$$citus_dist_stat_activity$$; +CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS +SELECT * FROM citus_stat_activity +WHERE is_worker_query = false; -COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -IS 'returns distributed transaction activity on distributed tables'; +ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; 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/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/11.0-1.sql new file mode 100644 index 000000000..2c2635687 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/11.0-1.sql @@ -0,0 +1,151 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_finish_pg_upgrade() + RETURNS void + LANGUAGE plpgsql + SET search_path = pg_catalog + AS $cppu$ +DECLARE + table_name regclass; + command text; + trigger_name text; +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; + + -- + -- Citus creates the array_cat_agg but because of a compatibility + -- issue between pg13-pg14, we drop and create it during upgrade. + -- And as Citus creates it, there needs to be a dependency to the + -- Citus extension, so we create that dependency here. + -- We are not using: + -- ALTER EXENSION citus DROP/CREATE AGGREGATE array_cat_agg + -- because we don't have an easy way to check if the aggregate + -- exists with anyarray type or anycompatiblearray type. + + INSERT INTO pg_depend + SELECT + 'pg_proc'::regclass::oid as classid, + (SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') as objid, + 0 as objsubid, + 'pg_extension'::regclass::oid as refclassid, + (select oid from pg_extension where extname = 'citus') as refobjid, + 0 as refobjsubid , + 'e' as deptype; + + -- + -- restore citus catalog tables + -- + INSERT INTO pg_catalog.pg_dist_partition SELECT * FROM public.pg_dist_partition; + INSERT INTO pg_catalog.pg_dist_shard SELECT * FROM public.pg_dist_shard; + INSERT INTO pg_catalog.pg_dist_placement SELECT * FROM public.pg_dist_placement; + INSERT INTO pg_catalog.pg_dist_node_metadata SELECT * FROM public.pg_dist_node_metadata; + INSERT INTO pg_catalog.pg_dist_node SELECT * FROM public.pg_dist_node; + INSERT INTO pg_catalog.pg_dist_local_group SELECT * FROM public.pg_dist_local_group; + INSERT INTO pg_catalog.pg_dist_transaction SELECT * FROM public.pg_dist_transaction; + INSERT INTO pg_catalog.pg_dist_colocation SELECT * FROM public.pg_dist_colocation; + -- enterprise catalog tables + INSERT INTO pg_catalog.pg_dist_authinfo SELECT * FROM public.pg_dist_authinfo; + INSERT INTO pg_catalog.pg_dist_poolinfo SELECT * FROM public.pg_dist_poolinfo; + + INSERT INTO pg_catalog.pg_dist_rebalance_strategy SELECT + name, + default_strategy, + shard_cost_function::regprocedure::regproc, + node_capacity_function::regprocedure::regproc, + shard_allowed_on_node_function::regprocedure::regproc, + default_threshold, + minimum_threshold, + improvement_threshold + FROM public.pg_dist_rebalance_strategy; + + -- + -- drop backup tables + -- + DROP TABLE public.pg_dist_authinfo; + DROP TABLE public.pg_dist_colocation; + DROP TABLE public.pg_dist_local_group; + DROP TABLE public.pg_dist_node; + DROP TABLE public.pg_dist_node_metadata; + DROP TABLE public.pg_dist_partition; + DROP TABLE public.pg_dist_placement; + DROP TABLE public.pg_dist_poolinfo; + DROP TABLE public.pg_dist_shard; + DROP TABLE public.pg_dist_transaction; + DROP TABLE public.pg_dist_rebalance_strategy; + + -- + -- reset sequences + -- + PERFORM setval('pg_catalog.pg_dist_shardid_seq', (SELECT MAX(shardid)+1 AS max_shard_id FROM pg_dist_shard), false); + PERFORM setval('pg_catalog.pg_dist_placement_placementid_seq', (SELECT MAX(placementid)+1 AS max_placement_id FROM pg_dist_placement), false); + PERFORM setval('pg_catalog.pg_dist_groupid_seq', (SELECT MAX(groupid)+1 AS max_group_id FROM pg_dist_node), false); + PERFORM setval('pg_catalog.pg_dist_node_nodeid_seq', (SELECT MAX(nodeid)+1 AS max_node_id FROM pg_dist_node), false); + PERFORM setval('pg_catalog.pg_dist_colocationid_seq', (SELECT MAX(colocationid)+1 AS max_colocation_id FROM pg_dist_colocation), false); + + -- + -- register triggers + -- + FOR table_name IN SELECT logicalrelid FROM pg_catalog.pg_dist_partition + LOOP + trigger_name := 'truncate_trigger_' || table_name::oid; + command := 'create trigger ' || trigger_name || ' after truncate on ' || table_name || ' execute procedure pg_catalog.citus_truncate_trigger()'; + EXECUTE command; + command := 'update pg_trigger set tgisinternal = true where tgname = ' || quote_literal(trigger_name); + EXECUTE command; + END LOOP; + + -- + -- set dependencies + -- + INSERT INTO pg_depend + SELECT + 'pg_class'::regclass::oid as classid, + p.logicalrelid::regclass::oid as objid, + 0 as objsubid, + 'pg_extension'::regclass::oid as refclassid, + (select oid from pg_extension where extname = 'citus') as refobjid, + 0 as refobjsubid , + 'n' as deptype + FROM pg_catalog.pg_dist_partition p; + + -- set dependencies for columnar table access method + PERFORM citus_internal.columnar_ensure_am_depends_catalog(); + + -- restore pg_dist_object from the stable identifiers + TRUNCATE pg_catalog.pg_dist_object; + INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid) + SELECT + address.classid, + address.objid, + address.objsubid, + naming.distribution_argument_index, + naming.colocationid + FROM + public.pg_dist_object naming, + pg_catalog.pg_get_object_address(naming.type, naming.object_names, naming.object_args) address; + + DROP TABLE public.pg_dist_object; +END; +$cppu$; + +COMMENT ON FUNCTION pg_catalog.citus_finish_pg_upgrade() + IS 'perform tasks to restore citus settings from a location that has been prepared before pg_upgrade'; 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..2c2635687 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; @@ -124,8 +131,8 @@ BEGIN PERFORM citus_internal.columnar_ensure_am_depends_catalog(); -- restore pg_dist_object from the stable identifiers - TRUNCATE citus.pg_dist_object; - INSERT INTO citus.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid) + TRUNCATE pg_catalog.pg_dist_object; + INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid) SELECT address.classid, address.objid, diff --git a/src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/11.0-1.sql new file mode 100644 index 000000000..823f45569 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/11.0-1.sql @@ -0,0 +1,13 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_colocation_metadata( + colocation_id int, + shard_count int, + replication_factor int, + distribution_column_type regtype, + distribution_column_collation oid) + RETURNS void + LANGUAGE C + STRICT + AS 'MODULE_PATHNAME'; + +COMMENT ON FUNCTION pg_catalog.citus_internal_add_colocation_metadata(int,int,int,regtype,oid) IS + 'Inserts a co-location group into pg_dist_colocation'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/latest.sql new file mode 100644 index 000000000..823f45569 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/latest.sql @@ -0,0 +1,13 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_colocation_metadata( + colocation_id int, + shard_count int, + replication_factor int, + distribution_column_type regtype, + distribution_column_collation oid) + RETURNS void + LANGUAGE C + STRICT + AS 'MODULE_PATHNAME'; + +COMMENT ON FUNCTION pg_catalog.citus_internal_add_colocation_metadata(int,int,int,regtype,oid) IS + 'Inserts a co-location group into pg_dist_colocation'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/11.0-1.sql new file mode 100644 index 000000000..d4c3f1be9 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/11.0-1.sql @@ -0,0 +1,9 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_delete_colocation_metadata( + colocation_id int) + RETURNS void + LANGUAGE C + STRICT + AS 'MODULE_PATHNAME'; + +COMMENT ON FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(int) IS + 'deletes a co-location group from pg_dist_colocation'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/latest.sql new file mode 100644 index 000000000..d4c3f1be9 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/latest.sql @@ -0,0 +1,9 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_delete_colocation_metadata( + colocation_id int) + RETURNS void + LANGUAGE C + STRICT + AS 'MODULE_PATHNAME'; + +COMMENT ON FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(int) IS + 'deletes a co-location group from pg_dist_colocation'; diff --git a/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql index 779341657..b3de12632 100644 --- a/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql +++ b/src/backend/distributed/sql/udfs/citus_lock_waits/11.0-1.sql @@ -2,43 +2,44 @@ SET search_path = 'pg_catalog'; CREATE VIEW citus.citus_lock_waits AS WITH -citus_dist_stat_activity AS -( - SELECT * FROM citus_dist_stat_activity +unique_global_wait_edges_with_calculated_gpids AS ( +SELECT + -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL + -- also for legacy reasons citus_internal_global_blocked_processes() returns groupId, we replace that with nodeIds + case WHEN waiting_global_pid !=0 THEN waiting_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(waiting_node_id), waiting_pid) END waiting_global_pid, + case WHEN blocking_global_pid !=0 THEN blocking_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(blocking_node_id), blocking_pid) END blocking_global_pid, + + -- citus_internal_global_blocked_processes returns groupId, we replace it here with actual + -- nodeId to be consisten with the other views + get_nodeid_for_groupid(blocking_node_id) as blocking_node_id, + get_nodeid_for_groupid(waiting_node_id) as waiting_node_id, + + blocking_transaction_waiting + + FROM citus_internal_global_blocked_processes() ), unique_global_wait_edges AS ( - SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() + SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM unique_global_wait_edges_with_calculated_gpids ), -citus_dist_stat_activity_with_node_id AS +citus_dist_stat_activity_with_calculated_gpids AS ( - SELECT - citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id - FROM - citus_dist_stat_activity LEFT JOIN pg_dist_node - ON - citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND - citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport + -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL + SELECT CASE WHEN global_pid != 0 THEN global_pid ELSE citus_calculate_gpid(nodeid, pid) END global_pid, nodeid, pid, query FROM citus_dist_stat_activity ) SELECT - waiting.global_pid as waiting_gpid, - blocking.global_pid as blocking_gpid, - waiting.pid AS waiting_pid, - blocking.pid AS blocking_pid, - waiting.query AS blocked_statement, - blocking.query AS current_statement_in_blocking_process, - waiting.initiator_node_id AS waiting_node_id, - blocking.initiator_node_id AS blocking_node_id, - waiting.distributed_query_host_name AS waiting_node_name, - blocking.distributed_query_host_name AS blocking_node_name, - waiting.distributed_query_host_port AS waiting_node_port, - blocking.distributed_query_host_port AS blocking_node_port + waiting.global_pid as waiting_gpid, + blocking.global_pid as blocking_gpid, + waiting.query AS blocked_statement, + blocking.query AS current_statement_in_blocking_process, + waiting.nodeid AS waiting_nodeid, + blocking.nodeid AS blocking_nodeid FROM - unique_global_wait_edges -JOIN - citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) -JOIN - citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); + unique_global_wait_edges + JOIN + citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) + JOIN + citus_dist_stat_activity_with_calculated_gpids blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql b/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql index 779341657..b3de12632 100644 --- a/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_lock_waits/latest.sql @@ -2,43 +2,44 @@ SET search_path = 'pg_catalog'; CREATE VIEW citus.citus_lock_waits AS WITH -citus_dist_stat_activity AS -( - SELECT * FROM citus_dist_stat_activity +unique_global_wait_edges_with_calculated_gpids AS ( +SELECT + -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL + -- also for legacy reasons citus_internal_global_blocked_processes() returns groupId, we replace that with nodeIds + case WHEN waiting_global_pid !=0 THEN waiting_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(waiting_node_id), waiting_pid) END waiting_global_pid, + case WHEN blocking_global_pid !=0 THEN blocking_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(blocking_node_id), blocking_pid) END blocking_global_pid, + + -- citus_internal_global_blocked_processes returns groupId, we replace it here with actual + -- nodeId to be consisten with the other views + get_nodeid_for_groupid(blocking_node_id) as blocking_node_id, + get_nodeid_for_groupid(waiting_node_id) as waiting_node_id, + + blocking_transaction_waiting + + FROM citus_internal_global_blocked_processes() ), unique_global_wait_edges AS ( - SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() + SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM unique_global_wait_edges_with_calculated_gpids ), -citus_dist_stat_activity_with_node_id AS +citus_dist_stat_activity_with_calculated_gpids AS ( - SELECT - citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id - FROM - citus_dist_stat_activity LEFT JOIN pg_dist_node - ON - citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND - citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport + -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL + SELECT CASE WHEN global_pid != 0 THEN global_pid ELSE citus_calculate_gpid(nodeid, pid) END global_pid, nodeid, pid, query FROM citus_dist_stat_activity ) SELECT - waiting.global_pid as waiting_gpid, - blocking.global_pid as blocking_gpid, - waiting.pid AS waiting_pid, - blocking.pid AS blocking_pid, - waiting.query AS blocked_statement, - blocking.query AS current_statement_in_blocking_process, - waiting.initiator_node_id AS waiting_node_id, - blocking.initiator_node_id AS blocking_node_id, - waiting.distributed_query_host_name AS waiting_node_name, - blocking.distributed_query_host_name AS blocking_node_name, - waiting.distributed_query_host_port AS waiting_node_port, - blocking.distributed_query_host_port AS blocking_node_port + waiting.global_pid as waiting_gpid, + blocking.global_pid as blocking_gpid, + waiting.query AS blocked_statement, + blocking.query AS current_statement_in_blocking_process, + waiting.nodeid AS waiting_nodeid, + blocking.nodeid AS blocking_nodeid FROM - unique_global_wait_edges -JOIN - citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) -JOIN - citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); + unique_global_wait_edges + JOIN + citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) + JOIN + citus_dist_stat_activity_with_calculated_gpids blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_nodeid_for_gpid/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_nodeid_for_gpid/11.0-1.sql new file mode 100644 index 000000000..c77dd94df --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_nodeid_for_gpid/11.0-1.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint) + RETURNS integer + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_nodeid_for_gpid$$; + +COMMENT ON FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint) + IS 'returns node id for the global process with given global pid'; diff --git a/src/backend/distributed/sql/udfs/citus_nodeid_for_gpid/latest.sql b/src/backend/distributed/sql/udfs/citus_nodeid_for_gpid/latest.sql new file mode 100644 index 000000000..c77dd94df --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_nodeid_for_gpid/latest.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint) + RETURNS integer + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_nodeid_for_gpid$$; + +COMMENT ON FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint) + IS 'returns node id for the global process with given global pid'; diff --git a/src/backend/distributed/sql/udfs/citus_nodename_for_nodeid/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_nodename_for_nodeid/11.0-1.sql new file mode 100644 index 000000000..4032029c2 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_nodename_for_nodeid/11.0-1.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer) + RETURNS text + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_nodename_for_nodeid$$; + +COMMENT ON FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer) + IS 'returns node name for the node with given node id'; diff --git a/src/backend/distributed/sql/udfs/citus_nodename_for_nodeid/latest.sql b/src/backend/distributed/sql/udfs/citus_nodename_for_nodeid/latest.sql new file mode 100644 index 000000000..4032029c2 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_nodename_for_nodeid/latest.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer) + RETURNS text + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_nodename_for_nodeid$$; + +COMMENT ON FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer) + IS 'returns node name for the node with given node id'; diff --git a/src/backend/distributed/sql/udfs/citus_nodeport_for_nodeid/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_nodeport_for_nodeid/11.0-1.sql new file mode 100644 index 000000000..d543949d8 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_nodeport_for_nodeid/11.0-1.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer) + RETURNS integer + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_nodeport_for_nodeid$$; + +COMMENT ON FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer) + IS 'returns node port for the node with given node id'; diff --git a/src/backend/distributed/sql/udfs/citus_nodeport_for_nodeid/latest.sql b/src/backend/distributed/sql/udfs/citus_nodeport_for_nodeid/latest.sql new file mode 100644 index 000000000..d543949d8 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_nodeport_for_nodeid/latest.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer) + RETURNS integer + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_nodeport_for_nodeid$$; + +COMMENT ON FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer) + IS 'returns node port for the node with given node id'; diff --git a/src/backend/distributed/sql/udfs/citus_pid_for_gpid/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_pid_for_gpid/11.0-1.sql new file mode 100644 index 000000000..c6564d119 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_pid_for_gpid/11.0-1.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint) + RETURNS integer + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_pid_for_gpid$$; + +COMMENT ON FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint) + IS 'returns process id for the global process with given global pid'; diff --git a/src/backend/distributed/sql/udfs/citus_pid_for_gpid/latest.sql b/src/backend/distributed/sql/udfs/citus_pid_for_gpid/latest.sql new file mode 100644 index 000000000..c6564d119 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_pid_for_gpid/latest.sql @@ -0,0 +1,7 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint) + RETURNS integer + LANGUAGE C STABLE STRICT + AS 'MODULE_PATHNAME', $$citus_pid_for_gpid$$; + +COMMENT ON FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint) + IS 'returns process id for the global process with given global pid'; diff --git a/src/backend/distributed/sql/udfs/citus_prepare_pg_upgrade/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_prepare_pg_upgrade/11.0-1.sql new file mode 100644 index 000000000..ff7e5d43e --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_prepare_pg_upgrade/11.0-1.sql @@ -0,0 +1,74 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_prepare_pg_upgrade() + RETURNS void + LANGUAGE plpgsql + SET search_path = pg_catalog + AS $cppu$ +BEGIN + + DELETE FROM pg_depend WHERE + objid IN (SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') AND + refobjid IN (select oid from pg_extension where extname = 'citus'); + -- + -- We are dropping the aggregates because postgres 14 changed + -- array_cat type from anyarray to anycompatiblearray. When + -- upgrading to pg14, specifically when running pg_restore on + -- array_cat_agg we would get an error. So we drop the aggregate + -- and create the right one on citus_finish_pg_upgrade. + + DROP AGGREGATE IF EXISTS array_cat_agg(anyarray); + DROP AGGREGATE IF EXISTS array_cat_agg(anycompatiblearray); + -- + -- Drop existing backup tables + -- + DROP TABLE IF EXISTS public.pg_dist_partition; + DROP TABLE IF EXISTS public.pg_dist_shard; + DROP TABLE IF EXISTS public.pg_dist_placement; + DROP TABLE IF EXISTS public.pg_dist_node_metadata; + DROP TABLE IF EXISTS public.pg_dist_node; + DROP TABLE IF EXISTS public.pg_dist_local_group; + DROP TABLE IF EXISTS public.pg_dist_transaction; + DROP TABLE IF EXISTS public.pg_dist_colocation; + DROP TABLE IF EXISTS public.pg_dist_authinfo; + DROP TABLE IF EXISTS public.pg_dist_poolinfo; + DROP TABLE IF EXISTS public.pg_dist_rebalance_strategy; + DROP TABLE IF EXISTS public.pg_dist_object; + + -- + -- backup citus catalog tables + -- + CREATE TABLE public.pg_dist_partition AS SELECT * FROM pg_catalog.pg_dist_partition; + CREATE TABLE public.pg_dist_shard AS SELECT * FROM pg_catalog.pg_dist_shard; + CREATE TABLE public.pg_dist_placement AS SELECT * FROM pg_catalog.pg_dist_placement; + CREATE TABLE public.pg_dist_node_metadata AS SELECT * FROM pg_catalog.pg_dist_node_metadata; + CREATE TABLE public.pg_dist_node AS SELECT * FROM pg_catalog.pg_dist_node; + CREATE TABLE public.pg_dist_local_group AS SELECT * FROM pg_catalog.pg_dist_local_group; + CREATE TABLE public.pg_dist_transaction AS SELECT * FROM pg_catalog.pg_dist_transaction; + CREATE TABLE public.pg_dist_colocation AS SELECT * FROM pg_catalog.pg_dist_colocation; + -- enterprise catalog tables + CREATE TABLE public.pg_dist_authinfo AS SELECT * FROM pg_catalog.pg_dist_authinfo; + CREATE TABLE public.pg_dist_poolinfo AS SELECT * FROM pg_catalog.pg_dist_poolinfo; + CREATE TABLE public.pg_dist_rebalance_strategy AS SELECT + name, + default_strategy, + shard_cost_function::regprocedure::text, + node_capacity_function::regprocedure::text, + shard_allowed_on_node_function::regprocedure::text, + default_threshold, + minimum_threshold, + improvement_threshold + FROM pg_catalog.pg_dist_rebalance_strategy; + + -- store upgrade stable identifiers on pg_dist_object catalog + CREATE TABLE public.pg_dist_object AS SELECT + address.type, + address.object_names, + address.object_args, + objects.distribution_argument_index, + objects.colocationid + FROM pg_catalog.pg_dist_object objects, + pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address; +END; +$cppu$; + +COMMENT ON FUNCTION pg_catalog.citus_prepare_pg_upgrade() + IS 'perform tasks to copy citus settings to a location that could later be restored after pg_upgrade is done'; diff --git a/src/backend/distributed/sql/udfs/citus_prepare_pg_upgrade/latest.sql b/src/backend/distributed/sql/udfs/citus_prepare_pg_upgrade/latest.sql index 8b87c8f38..ff7e5d43e 100644 --- a/src/backend/distributed/sql/udfs/citus_prepare_pg_upgrade/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_prepare_pg_upgrade/latest.sql @@ -65,7 +65,7 @@ BEGIN address.object_args, objects.distribution_argument_index, objects.colocationid - FROM citus.pg_dist_object objects, + FROM pg_catalog.pg_dist_object objects, pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address; END; $cppu$; diff --git a/src/backend/distributed/sql/udfs/citus_stat_activity/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_stat_activity/11.0-1.sql new file mode 100644 index 000000000..5dea5a903 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_stat_activity/11.0-1.sql @@ -0,0 +1,42 @@ +-- citus_stat_activity combines the pg_stat_activity views from all nodes and adds global_pid, nodeid and is_worker_query columns. +-- The columns of citus_stat_activity don't change based on the Postgres version, however the pg_stat_activity's columns do. +-- Both Postgres 13 and 14 added one more column to pg_stat_activity (leader_pid and query_id). +-- citus_stat_activity has the most expansive column set, including the newly added columns. +-- If citus_stat_activity is queried in a Postgres version where pg_stat_activity doesn't have some columns citus_stat_activity has +-- the values for those columns will be NULL + +CREATE OR REPLACE FUNCTION pg_catalog.citus_stat_activity(OUT global_pid bigint, OUT nodeid int, OUT is_worker_query boolean, OUT datid oid, OUT datname name, OUT pid integer, + OUT leader_pid integer, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr inet, OUT client_hostname text, + OUT client_port integer, OUT backend_start timestamp with time zone, OUT xact_start timestamp with time zone, + OUT query_start timestamp with time zone, OUT state_change timestamp with time zone, OUT wait_event_type text, OUT wait_event text, + OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query_id bigint, OUT query text, OUT backend_type text) + RETURNS SETOF record + LANGUAGE plpgsql + AS $function$ +BEGIN + RETURN QUERY SELECT * FROM jsonb_to_recordset(( + SELECT jsonb_agg(all_csa_rows_as_jsonb.csa_row_as_jsonb)::JSONB FROM ( + SELECT jsonb_array_elements(run_command_on_all_nodes.result::JSONB)::JSONB || ('{"nodeid":' || run_command_on_all_nodes.nodeid || '}')::JSONB AS csa_row_as_jsonb + FROM run_command_on_all_nodes($$ + SELECT coalesce(to_jsonb(array_agg(csa_from_one_node.*)), '[{}]'::JSONB) + FROM ( + SELECT global_pid, worker_query AS is_worker_query, pg_stat_activity.* FROM + pg_stat_activity LEFT JOIN get_all_active_transactions() ON process_id = pid + ) AS csa_from_one_node; + $$, parallel:=true, give_warning_for_connection_errors:=true) + WHERE success = 't' + ) AS all_csa_rows_as_jsonb + )) + AS (global_pid bigint, nodeid int, is_worker_query boolean, datid oid, datname name, pid integer, + leader_pid integer, usesysid oid, usename name, application_name text, client_addr inet, client_hostname text, + client_port integer, backend_start timestamp with time zone, xact_start timestamp with time zone, + query_start timestamp with time zone, state_change timestamp with time zone, wait_event_type text, wait_event text, + state text, backend_xid xid, backend_xmin xid, query_id bigint, query text, backend_type text); +END; +$function$; + +CREATE OR REPLACE VIEW citus.citus_stat_activity AS +SELECT * FROM pg_catalog.citus_stat_activity(); + +ALTER VIEW citus.citus_stat_activity SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.citus_stat_activity TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_stat_activity/latest.sql b/src/backend/distributed/sql/udfs/citus_stat_activity/latest.sql new file mode 100644 index 000000000..5dea5a903 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_stat_activity/latest.sql @@ -0,0 +1,42 @@ +-- citus_stat_activity combines the pg_stat_activity views from all nodes and adds global_pid, nodeid and is_worker_query columns. +-- The columns of citus_stat_activity don't change based on the Postgres version, however the pg_stat_activity's columns do. +-- Both Postgres 13 and 14 added one more column to pg_stat_activity (leader_pid and query_id). +-- citus_stat_activity has the most expansive column set, including the newly added columns. +-- If citus_stat_activity is queried in a Postgres version where pg_stat_activity doesn't have some columns citus_stat_activity has +-- the values for those columns will be NULL + +CREATE OR REPLACE FUNCTION pg_catalog.citus_stat_activity(OUT global_pid bigint, OUT nodeid int, OUT is_worker_query boolean, OUT datid oid, OUT datname name, OUT pid integer, + OUT leader_pid integer, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr inet, OUT client_hostname text, + OUT client_port integer, OUT backend_start timestamp with time zone, OUT xact_start timestamp with time zone, + OUT query_start timestamp with time zone, OUT state_change timestamp with time zone, OUT wait_event_type text, OUT wait_event text, + OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query_id bigint, OUT query text, OUT backend_type text) + RETURNS SETOF record + LANGUAGE plpgsql + AS $function$ +BEGIN + RETURN QUERY SELECT * FROM jsonb_to_recordset(( + SELECT jsonb_agg(all_csa_rows_as_jsonb.csa_row_as_jsonb)::JSONB FROM ( + SELECT jsonb_array_elements(run_command_on_all_nodes.result::JSONB)::JSONB || ('{"nodeid":' || run_command_on_all_nodes.nodeid || '}')::JSONB AS csa_row_as_jsonb + FROM run_command_on_all_nodes($$ + SELECT coalesce(to_jsonb(array_agg(csa_from_one_node.*)), '[{}]'::JSONB) + FROM ( + SELECT global_pid, worker_query AS is_worker_query, pg_stat_activity.* FROM + pg_stat_activity LEFT JOIN get_all_active_transactions() ON process_id = pid + ) AS csa_from_one_node; + $$, parallel:=true, give_warning_for_connection_errors:=true) + WHERE success = 't' + ) AS all_csa_rows_as_jsonb + )) + AS (global_pid bigint, nodeid int, is_worker_query boolean, datid oid, datname name, pid integer, + leader_pid integer, usesysid oid, usename name, application_name text, client_addr inet, client_hostname text, + client_port integer, backend_start timestamp with time zone, xact_start timestamp with time zone, + query_start timestamp with time zone, state_change timestamp with time zone, wait_event_type text, wait_event text, + state text, backend_xid xid, backend_xmin xid, query_id bigint, query text, backend_type text); +END; +$function$; + +CREATE OR REPLACE VIEW citus.citus_stat_activity AS +SELECT * FROM pg_catalog.citus_stat_activity(); + +ALTER VIEW citus.citus_stat_activity SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.citus_stat_activity TO PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_worker_stat_activity/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_worker_stat_activity/11.0-1.sql deleted file mode 100644 index 6f585b2e8..000000000 --- a/src/backend/distributed/sql/udfs/citus_worker_stat_activity/11.0-1.sql +++ /dev/null @@ -1,19 +0,0 @@ -DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity CASCADE; - -CREATE OR REPLACE FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -RETURNS SETOF RECORD -LANGUAGE C STRICT AS 'MODULE_PATHNAME', -$$citus_worker_stat_activity$$; - -COMMENT ON FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -IS 'returns distributed transaction activity on shards of distributed tables'; diff --git a/src/backend/distributed/sql/udfs/citus_worker_stat_activity/latest.sql b/src/backend/distributed/sql/udfs/citus_worker_stat_activity/latest.sql deleted file mode 100644 index 6f585b2e8..000000000 --- a/src/backend/distributed/sql/udfs/citus_worker_stat_activity/latest.sql +++ /dev/null @@ -1,19 +0,0 @@ -DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity CASCADE; - -CREATE OR REPLACE FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -RETURNS SETOF RECORD -LANGUAGE C STRICT AS 'MODULE_PATHNAME', -$$citus_worker_stat_activity$$; - -COMMENT ON FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, - OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, - OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, - OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz, - OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text, - OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8) -IS 'returns distributed transaction activity on shards of distributed tables'; diff --git a/src/backend/distributed/sql/udfs/run_command_on_all_nodes/11.0-1.sql b/src/backend/distributed/sql/udfs/run_command_on_all_nodes/11.0-1.sql new file mode 100644 index 000000000..1361ef6aa --- /dev/null +++ b/src/backend/distributed/sql/udfs/run_command_on_all_nodes/11.0-1.sql @@ -0,0 +1,60 @@ +DROP FUNCTION IF EXISTS pg_catalog.run_command_on_all_nodes; + +CREATE FUNCTION pg_catalog.run_command_on_all_nodes(command text, parallel bool default true, give_warning_for_connection_errors bool default false, + OUT nodeid int, OUT success bool, OUT result text) + RETURNS SETOF record + LANGUAGE plpgsql + AS $function$ +DECLARE + nodenames text[]; + ports int[]; + commands text[]; + current_node_is_in_metadata boolean; + command_result_of_current_node text; +BEGIN + WITH citus_nodes AS ( + SELECT * FROM pg_dist_node + WHERE isactive = 't' AND nodecluster = current_setting('citus.cluster_name') + AND ( + (current_setting('citus.use_secondary_nodes') = 'never' AND noderole = 'primary') + OR + (current_setting('citus.use_secondary_nodes') = 'always' AND noderole = 'secondary') + ) + ORDER BY nodename, nodeport + ) + SELECT array_agg(citus_nodes.nodename), array_agg(citus_nodes.nodeport), array_agg(command) + INTO nodenames, ports, commands + FROM citus_nodes; + + SELECT count(*) > 0 FROM pg_dist_node + WHERE isactive = 't' + AND nodecluster = current_setting('citus.cluster_name') + AND groupid IN (SELECT groupid FROM pg_dist_local_group) + INTO current_node_is_in_metadata; + + -- This will happen when we call this function on coordinator and + -- the coordinator is not added to the metadata. + -- We'll manually add current node to the lists to actually run on all nodes. + -- But when the coordinator is not added to metadata and this function + -- is called from a worker node, this will not be enough and we'll + -- not be able run on all nodes. + IF NOT current_node_is_in_metadata THEN + SELECT + array_append(nodenames, current_setting('citus.local_hostname')), + array_append(ports, current_setting('port')::int), + array_append(commands, command) + INTO nodenames, ports, commands; + END IF; + + FOR nodeid, success, result IN + SELECT coalesce(pg_dist_node.nodeid, 0) AS nodeid, mrow.success, mrow.result + FROM master_run_on_worker(nodenames, ports, commands, parallel) mrow + LEFT JOIN pg_dist_node ON mrow.node_name = pg_dist_node.nodename AND mrow.node_port = pg_dist_node.nodeport + LOOP + IF give_warning_for_connection_errors AND NOT success THEN + RAISE WARNING 'Error on node with node id %: %', nodeid, result; + END IF; + RETURN NEXT; + END LOOP; +END; +$function$; diff --git a/src/backend/distributed/sql/udfs/run_command_on_all_nodes/latest.sql b/src/backend/distributed/sql/udfs/run_command_on_all_nodes/latest.sql new file mode 100644 index 000000000..1361ef6aa --- /dev/null +++ b/src/backend/distributed/sql/udfs/run_command_on_all_nodes/latest.sql @@ -0,0 +1,60 @@ +DROP FUNCTION IF EXISTS pg_catalog.run_command_on_all_nodes; + +CREATE FUNCTION pg_catalog.run_command_on_all_nodes(command text, parallel bool default true, give_warning_for_connection_errors bool default false, + OUT nodeid int, OUT success bool, OUT result text) + RETURNS SETOF record + LANGUAGE plpgsql + AS $function$ +DECLARE + nodenames text[]; + ports int[]; + commands text[]; + current_node_is_in_metadata boolean; + command_result_of_current_node text; +BEGIN + WITH citus_nodes AS ( + SELECT * FROM pg_dist_node + WHERE isactive = 't' AND nodecluster = current_setting('citus.cluster_name') + AND ( + (current_setting('citus.use_secondary_nodes') = 'never' AND noderole = 'primary') + OR + (current_setting('citus.use_secondary_nodes') = 'always' AND noderole = 'secondary') + ) + ORDER BY nodename, nodeport + ) + SELECT array_agg(citus_nodes.nodename), array_agg(citus_nodes.nodeport), array_agg(command) + INTO nodenames, ports, commands + FROM citus_nodes; + + SELECT count(*) > 0 FROM pg_dist_node + WHERE isactive = 't' + AND nodecluster = current_setting('citus.cluster_name') + AND groupid IN (SELECT groupid FROM pg_dist_local_group) + INTO current_node_is_in_metadata; + + -- This will happen when we call this function on coordinator and + -- the coordinator is not added to the metadata. + -- We'll manually add current node to the lists to actually run on all nodes. + -- But when the coordinator is not added to metadata and this function + -- is called from a worker node, this will not be enough and we'll + -- not be able run on all nodes. + IF NOT current_node_is_in_metadata THEN + SELECT + array_append(nodenames, current_setting('citus.local_hostname')), + array_append(ports, current_setting('port')::int), + array_append(commands, command) + INTO nodenames, ports, commands; + END IF; + + FOR nodeid, success, result IN + SELECT coalesce(pg_dist_node.nodeid, 0) AS nodeid, mrow.success, mrow.result + FROM master_run_on_worker(nodenames, ports, commands, parallel) mrow + LEFT JOIN pg_dist_node ON mrow.node_name = pg_dist_node.nodename AND mrow.node_port = pg_dist_node.nodeport + LOOP + IF give_warning_for_connection_errors AND NOT success THEN + RAISE WARNING 'Error on node with node id %: %', nodeid, result; + END IF; + RETURN NEXT; + END LOOP; +END; +$function$; 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/backend_counter.c b/src/backend/distributed/test/backend_counter.c index c63a45543..1b9984ac9 100644 --- a/src/backend/distributed/test/backend_counter.c +++ b/src/backend/distributed/test/backend_counter.c @@ -27,5 +27,5 @@ PG_FUNCTION_INFO_V1(get_all_active_client_backend_count); Datum get_all_active_client_backend_count(PG_FUNCTION_ARGS) { - PG_RETURN_UINT32(GetAllActiveClientBackendCount()); + PG_RETURN_UINT32(GetExternalClientBackendCount()); } diff --git a/src/backend/distributed/test/global_pid.c b/src/backend/distributed/test/global_pid.c new file mode 100644 index 000000000..e5ef60a96 --- /dev/null +++ b/src/backend/distributed/test/global_pid.c @@ -0,0 +1,33 @@ +/*------------------------------------------------------------------------- + * + * test/src/global_pid.c + * + * This file contains functions to test the global pid. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" +#include "fmgr.h" + +#include "distributed/backend_data.h" +#include "distributed/metadata_cache.h" + +PG_FUNCTION_INFO_V1(test_assign_global_pid); + + +/* + * test_assign_global_pid is the wrapper UDF for AssignGlobalPID and is only meant for use + * in tests. + */ +Datum +test_assign_global_pid(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + AssignGlobalPID(); + + PG_RETURN_VOID(); +} diff --git a/src/backend/distributed/test/make_external_connection.c b/src/backend/distributed/test/make_external_connection.c new file mode 100644 index 000000000..424793dea --- /dev/null +++ b/src/backend/distributed/test/make_external_connection.c @@ -0,0 +1,67 @@ +/*------------------------------------------------------------------------- + * + * test/src/make_external_connection.c + * + * This file contains UDF to connect to a node without using the Citus + * internal application_name. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" +#include "miscadmin.h" +#include "libpq-fe.h" + +#include "access/xact.h" +#include "distributed/connection_management.h" +#include "distributed/coordinator_protocol.h" +#include "distributed/function_utils.h" +#include "distributed/intermediate_result_pruning.h" +#include "distributed/lock_graph.h" +#include "distributed/metadata_cache.h" +#include "distributed/remote_commands.h" +#include "distributed/run_from_same_connection.h" + +#include "distributed/version_compat.h" +#include "executor/spi.h" +#include "lib/stringinfo.h" +#include "postmaster/postmaster.h" +#include "utils/builtins.h" +#include "utils/memutils.h" + + +PG_FUNCTION_INFO_V1(make_external_connection_to_node); + + +/* + * make_external_connection_to_node opens a conneciton to a node + * and keeps it until the end of the session. + */ +Datum +make_external_connection_to_node(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + char *nodeName = text_to_cstring(PG_GETARG_TEXT_P(0)); + uint32 nodePort = PG_GETARG_UINT32(1); + char *userName = text_to_cstring(PG_GETARG_TEXT_P(2)); + char *databaseName = text_to_cstring(PG_GETARG_TEXT_P(3)); + + StringInfo connectionString = makeStringInfo(); + appendStringInfo(connectionString, + "host=%s port=%d user=%s dbname=%s", + nodeName, nodePort, userName, databaseName); + + PGconn *pgConn = PQconnectdb(connectionString->data); + + if (PQstatus(pgConn) != CONNECTION_OK) + { + PQfinish(pgConn); + + ereport(ERROR, (errmsg("connection failed"))); + } + + PG_RETURN_VOID(); +} diff --git a/src/backend/distributed/test/run_from_same_connection.c b/src/backend/distributed/test/run_from_same_connection.c index ad39664bf..657c1fd82 100644 --- a/src/backend/distributed/test/run_from_same_connection.c +++ b/src/backend/distributed/test/run_from_same_connection.c @@ -188,7 +188,7 @@ run_commands_on_session_level_connection_to_node(PG_FUNCTION_ARGS) /* * override_backend_data_command_originator is a wrapper around - * OverrideBackendDataDistributedCommandOriginator(). + * SetBackendDataDistributedCommandOriginator(). */ Datum override_backend_data_command_originator(PG_FUNCTION_ARGS) @@ -197,7 +197,7 @@ override_backend_data_command_originator(PG_FUNCTION_ARGS) bool distributedCommandOriginator = PG_GETARG_BOOL(0); - OverrideBackendDataDistributedCommandOriginator(distributedCommandOriginator); + SetBackendDataDistributedCommandOriginator(distributedCommandOriginator); PG_RETURN_VOID(); } 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..925071c35 100644 --- a/src/backend/distributed/transaction/backend_data.c +++ b/src/backend/distributed/transaction/backend_data.c @@ -68,14 +68,12 @@ typedef struct BackendManagementShmemData pg_atomic_uint64 nextTransactionNumber; /* - * Total number of client backends that are authenticated. - * We only care about activeClientBackendCounter when adaptive - * connection management is enabled, otherwise always zero. + * Total number of external client backends that are authenticated. * * Note that the counter does not consider any background workers - * or such, it only counts client_backends. + * or such, and also exludes internal connections between nodes. */ - pg_atomic_uint32 activeClientBackendCounter; + pg_atomic_uint32 externalClientBackendCounter; BackendData backends[FLEXIBLE_ARRAY_MEMBER]; } BackendManagementShmemData; @@ -83,6 +81,8 @@ typedef struct BackendManagementShmemData static void StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescriptor); +static bool UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId); +static uint64 CalculateGlobalPID(int32 nodeId, pid_t pid); static uint64 GenerateGlobalPID(void); static shmem_startup_hook_type prev_shmem_startup_hook = NULL; @@ -98,6 +98,10 @@ PG_FUNCTION_INFO_V1(assign_distributed_transaction_id); PG_FUNCTION_INFO_V1(get_current_transaction_id); PG_FUNCTION_INFO_V1(get_global_active_transactions); PG_FUNCTION_INFO_V1(get_all_active_transactions); +PG_FUNCTION_INFO_V1(citus_calculate_gpid); +PG_FUNCTION_INFO_V1(citus_backend_gpid); +PG_FUNCTION_INFO_V1(citus_nodeid_for_gpid); +PG_FUNCTION_INFO_V1(citus_pid_for_gpid); /* @@ -114,8 +118,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 +146,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 +356,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 +412,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 +464,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. @@ -504,7 +551,7 @@ BackendManagementShmemInit(void) pg_atomic_init_u64(&backendManagementShmemData->nextTransactionNumber, 1); /* there are no active backends yet, so start with zero */ - pg_atomic_init_u32(&backendManagementShmemData->activeClientBackendCounter, 0); + pg_atomic_init_u32(&backendManagementShmemData->externalClientBackendCounter, 0); /* * We need to init per backend's spinlock before any backend @@ -520,7 +567,6 @@ BackendManagementShmemInit(void) { BackendData *backendData = &backendManagementShmemData->backends[backendIndex]; - backendData->citusBackend.initiatorNodeIdentifier = -1; SpinLockInit(&backendData->mutex); } } @@ -652,16 +698,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 +721,9 @@ UnSetGlobalPID(void) SpinLockAcquire(&MyBackendData->mutex); MyBackendData->globalPID = 0; + MyBackendData->databaseId = 0; + MyBackendData->userId = 0; + MyBackendData->distributedCommandOriginator = false; SpinLockRelease(&MyBackendData->mutex); } @@ -760,41 +805,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,20 +839,30 @@ 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); } /* - * OverrideBackendDataDistributedCommandOriginator should only be used for isolation testing. - * See how it is used in the relevant functions. + * SetBackendDataDistributedCommandOriginator is used to set the distributedCommandOriginator + * field on MyBackendData. */ void -OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator) +SetBackendDataDistributedCommandOriginator(bool distributedCommandOriginator) { + if (!MyBackendData) + { + return; + } SpinLockAcquire(&MyBackendData->mutex); MyBackendData->distributedCommandOriginator = distributedCommandOriginator; @@ -862,10 +890,27 @@ GetGlobalPID(void) /* - * GenerateGlobalPID generates the global process id for the current backend. + * citus_calculate_gpid calculates the gpid for any given process on any + * given node. + */ +Datum +citus_calculate_gpid(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + int32 nodeId = PG_GETARG_INT32(0); + int32 pid = PG_GETARG_INT32(1); + + PG_RETURN_UINT64(CalculateGlobalPID(nodeId, pid)); +} + + +/* + * CalculateGlobalPID gets a nodeId and pid, and returns the global pid + * that can be assigned for a process with the given input. */ static uint64 -GenerateGlobalPID(void) +CalculateGlobalPID(int32 nodeId, pid_t pid) { /* * We try to create a human readable global pid that consists of node id and process id. @@ -876,7 +921,60 @@ GenerateGlobalPID(void) * node ids might cause overflow. But even for the applications that scale around 50 nodes every * day it'd take about 100K years. So we are not worried. */ - return (((uint64) GetLocalNodeId()) * GLOBAL_PID_NODE_ID_MULTIPLIER) + getpid(); + return (((uint64) nodeId) * GLOBAL_PID_NODE_ID_MULTIPLIER) + pid; +} + + +/* + * GenerateGlobalPID generates the global process id for the current backend. + * See CalculateGlobalPID for the details. + */ +static uint64 +GenerateGlobalPID(void) +{ + return CalculateGlobalPID(GetLocalNodeId(), getpid()); +} + + +/* + * citus_backend_gpid similar to pg_backend_pid, but returns Citus + * assigned gpid. + */ +Datum +citus_backend_gpid(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + PG_RETURN_UINT64(GetGlobalPID()); +} + + +/* + * citus_nodeid_for_gpid returns node id for the global process with given global pid + */ +Datum +citus_nodeid_for_gpid(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + uint64 globalPID = PG_GETARG_INT64(0); + + bool missingOk = false; + PG_RETURN_INT32(ExtractNodeIdFromGlobalPID(globalPID, missingOk)); +} + + +/* + * citus_pid_for_gpid returns process id for the global process with given global pid + */ +Datum +citus_pid_for_gpid(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + uint64 globalPID = PG_GETARG_INT64(0); + + PG_RETURN_INT32(ExtractProcessIdFromGlobalPID(globalPID)); } @@ -926,11 +1024,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), @@ -1145,36 +1244,37 @@ GetMyProcLocalTransactionId(void) /* - * GetAllActiveClientBackendCount returns activeClientBackendCounter in + * GetExternalClientBackendCount returns externalClientBackendCounter in * the shared memory. */ int -GetAllActiveClientBackendCount(void) +GetExternalClientBackendCount(void) { uint32 activeBackendCount = - pg_atomic_read_u32(&backendManagementShmemData->activeClientBackendCounter); + pg_atomic_read_u32(&backendManagementShmemData->externalClientBackendCounter); return activeBackendCount; } /* - * IncrementClientBackendCounter increments activeClientBackendCounter in + * IncrementExternalClientBackendCounter increments externalClientBackendCounter in * the shared memory by one. */ -void -IncrementClientBackendCounter(void) +uint32 +IncrementExternalClientBackendCounter(void) { - pg_atomic_add_fetch_u32(&backendManagementShmemData->activeClientBackendCounter, 1); + return pg_atomic_add_fetch_u32( + &backendManagementShmemData->externalClientBackendCounter, 1); } /* - * DecrementClientBackendCounter decrements activeClientBackendCounter in + * DecrementExternalClientBackendCounter decrements externalClientBackendCounter in * the shared memory by one. */ void -DecrementClientBackendCounter(void) +DecrementExternalClientBackendCounter(void) { - pg_atomic_sub_fetch_u32(&backendManagementShmemData->activeClientBackendCounter, 1); + pg_atomic_sub_fetch_u32(&backendManagementShmemData->externalClientBackendCounter, 1); } diff --git a/src/backend/distributed/transaction/citus_dist_stat_activity.c b/src/backend/distributed/transaction/citus_dist_stat_activity.c index ec4f6e8a3..3aa6372e6 100644 --- a/src/backend/distributed/transaction/citus_dist_stat_activity.c +++ b/src/backend/distributed/transaction/citus_dist_stat_activity.c @@ -2,8 +2,7 @@ * * citus_dist_stat_activity.c * - * This file contains functions for monitoring the distributed transactions - * across the cluster. + * The methods in the file are deprecated. * * Copyright (c) Citus Data, Inc. * @@ -11,1104 +10,27 @@ */ #include "postgres.h" -#include "libpq-fe.h" #include "miscadmin.h" -#include "fmgr.h" -#include "postmaster/postmaster.h" - #include "funcapi.h" -#include "access/htup_details.h" -#include "catalog/pg_type.h" -#include "datatype/timestamp.h" -#include "distributed/backend_data.h" -#include "distributed/connection_management.h" -#include "distributed/listutils.h" -#include "distributed/lock_graph.h" -#include "distributed/coordinator_protocol.h" -#include "distributed/metadata_cache.h" -#include "distributed/remote_commands.h" -#include "distributed/transaction_identifier.h" -#include "distributed/tuplestore.h" -#include "executor/spi.h" -#include "nodes/execnodes.h" -#include "storage/ipc.h" -#include "storage/lwlock.h" -#include "storage/proc.h" -#include "storage/spin.h" -#include "storage/s_lock.h" -#include "utils/builtins.h" -#include "utils/fmgrprotos.h" -#include "utils/inet.h" -#include "utils/timestamp.h" - - -/* - * citus_dist_stat_activity() and citus_worker_stat_activity() is similar to - * pg_stat_activity. Those functions basically return join of - * pg_stat_activity and get_all_active_transactions() on each node - * in the cluster. The only difference is that citus_dist_stat_activity() - * gets transactions where worker_query = false and citus_worker_stat_activity() - * gets transactions where worker_query = true. - * - * In other words, citus_dist_stat_activity returns only the queries that are the - * distributed queries. citus_worker_stat_activity returns only the queries that - * are worker queries (e.g., queries on the shards) initiated by those distributed - * queries. To understand this better, let us give an example. If a users starts - * a query like "UPDATE table SET value = 1", this query would show up on - * citus_dist_stat_activity. The same query would generate #shard worker queries, - * all of which would show up on citus_worker_stat_activity. - * - * An important note on this views is that they only show the activity - * that are inside distributed transactions. Distributed transactions - * cover the following: - * - All multi-shard modifications (DDLs, COPY, UPDATE, DELETE, INSERT .. SELECT) - * - All multi-shard queries with CTEs (modifying CTEs, read-only CTEs) - * - All recursively planned subqueries - * - All queries within transaction blocks (BEGIN; query; COMMIT;) - * - * In other words, the following types of queries won't be observed in these - * views: - * - Single-shard queries that are not inside transaction blocks - * - Multi-shard select queries that are not inside transaction blocks - * - Task-tracker queries - * - * - * The following information for all the distributed transactions: - * query_host_name text - * query_host_port int - * database_id oid - * databaese_name name - * process_id integer - * initiator_node_host text - * initiator_node_port int - * distributed_transaction_number bigint - * distributed_transaction_stamp timestamp with time zone - * usesysid oid - * usename name - * application_name text - * client_addr inet - * client_hostname text - * client_port integer - * backend_start timestamp with time zone - * xact_start timestamp with time zone - * query_start timestamp with time zone - * state_change timestamp with time zone - * wait_event_type text - * wait_event text - * state text - * backend_xid xid - * backend_xmin xid - * query text - * backend_type text - */ - -/* - * We get CITUS_DIST_STAT_ACTIVITY_QUERY_COLS from workers and manually add - * CITUS_DIST_STAT_ADDITIONAL_COLS for hostname and hostport. Also, instead of - * showing the initiator_node_id we expand it to initiator_node_host and - * initiator_node_port. - */ -#define CITUS_DIST_STAT_ACTIVITY_QUERY_COLS 24 -#define CITUS_DIST_STAT_ADDITIONAL_COLS 3 -#define CITUS_DIST_STAT_ACTIVITY_COLS \ - CITUS_DIST_STAT_ACTIVITY_QUERY_COLS + CITUS_DIST_STAT_ADDITIONAL_COLS - - -#define coordinator_host_name "coordinator_host" - -/* - * We get the query_host_name and query_host_port while opening the connection to - * the node. We also replace initiator_node_identifier with initiator_node_host - * and initiator_node_port. Thus, they are not in the query below. - */ - -#define CITUS_DIST_STAT_ACTIVITY_QUERY \ - "\ -SELECT \ - dist_txs.initiator_node_identifier, \ - dist_txs.transaction_number, \ - dist_txs.transaction_stamp, \ - pg_stat_activity.datid, \ - pg_stat_activity.datname, \ - pg_stat_activity.pid, \ - pg_stat_activity.usesysid, \ - pg_stat_activity.usename, \ - pg_stat_activity.application_name, \ - pg_stat_activity.client_addr, \ - pg_stat_activity.client_hostname, \ - pg_stat_activity.client_port, \ - pg_stat_activity.backend_start, \ - pg_stat_activity.xact_start, \ - pg_stat_activity.query_start, \ - pg_stat_activity.state_change, \ - pg_stat_activity.wait_event_type, \ - pg_stat_activity.wait_event, \ - pg_stat_activity.state, \ - pg_stat_activity.backend_xid, \ - pg_stat_activity.backend_xmin, \ - pg_stat_activity.query, \ - pg_stat_activity.backend_type, \ - dist_txs.global_pid \ -FROM \ - pg_stat_activity \ - INNER JOIN \ - get_all_active_transactions() AS dist_txs(database_id, process_id, initiator_node_identifier, worker_query, transaction_number, transaction_stamp, global_pid) \ - ON pg_stat_activity.pid = dist_txs.process_id \ -WHERE \ - backend_type = 'client backend' \ - AND \ - worker_query = False \ - AND \ - pg_stat_activity.query NOT ILIKE '%stat_activity%';" - -#define CITUS_WORKER_STAT_ACTIVITY_QUERY \ - "\ -SELECT \ - dist_txs.initiator_node_identifier, \ - dist_txs.transaction_number, \ - dist_txs.transaction_stamp, \ - pg_stat_activity.datid, \ - pg_stat_activity.datname, \ - pg_stat_activity.pid, \ - pg_stat_activity.usesysid, \ - pg_stat_activity.usename, \ - pg_stat_activity.application_name, \ - pg_stat_activity.client_addr, \ - pg_stat_activity.client_hostname, \ - pg_stat_activity.client_port, \ - pg_stat_activity.backend_start, \ - pg_stat_activity.xact_start, \ - pg_stat_activity.query_start, \ - pg_stat_activity.state_change, \ - pg_stat_activity.wait_event_type, \ - pg_stat_activity.wait_event, \ - pg_stat_activity.state, \ - pg_stat_activity.backend_xid, \ - pg_stat_activity.backend_xmin, \ - pg_stat_activity.query, \ - pg_stat_activity.backend_type, \ - dist_txs.global_id \ -FROM \ - pg_stat_activity \ - JOIN \ - get_all_active_transactions() AS dist_txs(database_id, process_id, initiator_node_identifier, worker_query, transaction_number, transaction_stamp, global_id) \ - ON pg_stat_activity.pid = dist_txs.process_id \ -WHERE \ - worker_query = True \ - AND \ - pg_stat_activity.query NOT ILIKE '%stat_activity%';" - -typedef struct CitusDistStat -{ - text *query_host_name; - int query_host_port; - - text *master_query_host_name; - int master_query_host_port; - uint64 distributed_transaction_number; - TimestampTz distributed_transaction_stamp; - - /* fields from pg_stat_statement */ - Oid database_id; - Name databaese_name; - int process_id; - Oid usesysid; - Name usename; - text *application_name; - inet *client_addr; - text *client_hostname; - int client_port; - TimestampTz backend_start; - TimestampTz xact_start; - TimestampTz query_start; - TimestampTz state_change; - text *wait_event_type; - text *wait_event; - text *state; - TransactionId backend_xid; - TransactionId backend_xmin; - text *query; - text *backend_type; - uint64 global_pid; -} CitusDistStat; - - -/* local forward declarations */ -static List * CitusStatActivity(const char *statQuery); -static void ReturnCitusDistStats(List *citusStatsList, FunctionCallInfo fcinfo); -static CitusDistStat * ParseCitusDistStat(PGresult *result, int64 rowIndex); -static void ReplaceInitiatorNodeIdentifier(int initiator_node_identifier, - CitusDistStat *citusDistStat); - -/* utility functions to parse the fields from PGResult */ -static text * ParseTextField(PGresult *result, int rowIndex, int colIndex); -static Name ParseNameField(PGresult *result, int rowIndex, int colIndex); -static inet * ParseInetField(PGresult *result, int rowIndex, int colIndex); -static TransactionId ParseXIDField(PGresult *result, int rowIndex, int colIndex); - -/* utility functions to fetch the fields from heapTuple */ -static List * GetLocalNodeCitusDistStat(const char *statQuery); -static List * LocalNodeCitusDistStat(const char *statQuery, const char *hostname, int - port); -static CitusDistStat * HeapTupleToCitusDistStat(HeapTuple result, TupleDesc - rowDescriptor); -static int64 ParseIntFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex); -static text * ParseTextFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int - colIndex); -static Name ParseNameFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex); -static inet * ParseInetFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int - colIndex); -static TimestampTz ParseTimestampTzFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, - int colIndex); -static TransactionId ParseXIDFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int - colIndex); - PG_FUNCTION_INFO_V1(citus_dist_stat_activity); PG_FUNCTION_INFO_V1(citus_worker_stat_activity); - -/* - * citus_dist_stat_activity connects to all nodes in the cluster and returns - * pg_stat_activity like result set but only consisting of queries that are - * on the distributed tables and inside distributed transactions. - */ +/* This UDF is deprecated. */ Datum citus_dist_stat_activity(PG_FUNCTION_ARGS) { - CheckCitusVersion(ERROR); + ereport(ERROR, (errmsg("This UDF is deprecated."))); - List *citusDistStatStatements = CitusStatActivity(CITUS_DIST_STAT_ACTIVITY_QUERY); - - ReturnCitusDistStats(citusDistStatStatements, fcinfo); - - PG_RETURN_VOID(); + PG_RETURN_NULL(); } -/* - * citus_worker_stat_activity connects to all nodes in the cluster and returns - * pg_stat_activity like result set but only consisting of queries that are - * on the shards of distributed tables and inside distributed transactions. - */ +/* This UDF is deprecated. */ Datum citus_worker_stat_activity(PG_FUNCTION_ARGS) { - CheckCitusVersion(ERROR); + ereport(ERROR, (errmsg("This UDF is deprecated."))); - List *citusWorkerStatStatements = CitusStatActivity(CITUS_WORKER_STAT_ACTIVITY_QUERY); - - ReturnCitusDistStats(citusWorkerStatStatements, fcinfo); - - PG_RETURN_VOID(); -} - - -/* - * CitusStatActivity gets the stats query, connects to each node in the - * cluster, executes the query and parses the results. The function returns - * list of CitusDistStat struct for further processing. - * - * The function connects to each active primary node in the pg_dist_node. Plus, - * if the query is being executed on the coordinator, the function connects to - * localhost as well. The implication of this is that whenever the query is executed - * from a MX worker node, it wouldn't be able to get information from the queries - * executed on the coordinator given that there is not metadata information about that. - */ -static List * -CitusStatActivity(const char *statQuery) -{ - List *workerNodeList = ActivePrimaryNonCoordinatorNodeList(NoLock); - List *connectionList = NIL; - - /* - * For the local node, we can avoid opening connections. This might be - * important when we're on the coordinator since it requires configuring - * the authentication for self-connection via any user who calls the citus - * stat activity functions. - */ - List *citusStatsList = GetLocalNodeCitusDistStat(statQuery); - - /* - * We prefer to connect with the current user to the remote nodes. This will - * ensure that we have the same privilage restrictions that pg_stat_activity - * enforces. - */ - char *nodeUser = CurrentUserName(); - - int32 localGroupId = GetLocalGroupId(); - - /* open connections in parallel */ - WorkerNode *workerNode = NULL; - foreach_ptr(workerNode, workerNodeList) - { - const char *nodeName = workerNode->workerName; - int nodePort = workerNode->workerPort; - int connectionFlags = 0; - - if (workerNode->groupId == localGroupId) - { - /* we already get these stats via GetLocalNodeCitusDistStat() */ - continue; - } - - MultiConnection *connection = StartNodeUserDatabaseConnection(connectionFlags, - nodeName, nodePort, - nodeUser, NULL); - - connectionList = lappend(connectionList, connection); - } - - FinishConnectionListEstablishment(connectionList); - - /* send commands in parallel */ - MultiConnection *connection = NULL; - foreach_ptr(connection, connectionList) - { - int querySent = SendRemoteCommand(connection, statQuery); - if (querySent == 0) - { - ReportConnectionError(connection, WARNING); - } - } - - /* receive query results */ - foreach_ptr(connection, connectionList) - { - bool raiseInterrupts = true; - - PGresult *result = GetRemoteCommandResult(connection, raiseInterrupts); - if (!IsResponseOK(result)) - { - ReportResultError(connection, result, WARNING); - continue; - } - - int64 rowCount = PQntuples(result); - int64 colCount = PQnfields(result); - - if (colCount != CITUS_DIST_STAT_ACTIVITY_QUERY_COLS) - { - /* - * We don't expect to hit this error, but keep it here in case there - * is a version mistmatch. - */ - ereport(WARNING, (errmsg("unexpected number of columns from " - "citus stat query"))); - continue; - } - - for (int64 rowIndex = 0; rowIndex < rowCount; rowIndex++) - { - CitusDistStat *citusDistStat = ParseCitusDistStat(result, rowIndex); - - /* - * Add the query_host_name and query_host_port which denote where - * the query is being running. - */ - citusDistStat->query_host_name = cstring_to_text(connection->hostname); - citusDistStat->query_host_port = connection->port; - - citusStatsList = lappend(citusStatsList, citusDistStat); - } - - PQclear(result); - ForgetResults(connection); - } - - return citusStatsList; -} - - -/* - * GetLocalNodeCitusDistStat simple executes the given query with SPI to get - * the result of the given stat query on the local node. - */ -static List * -GetLocalNodeCitusDistStat(const char *statQuery) -{ - List *citusStatsList = NIL; - - if (IsCoordinator()) - { - /* - * Coordinator's nodename and nodeport doesn't show-up in the metadata, - * so mark it manually as executing from the coordinator. - */ - citusStatsList = LocalNodeCitusDistStat(statQuery, coordinator_host_name, - PostPortNumber); - - return citusStatsList; - } - - int32 localGroupId = GetLocalGroupId(); - - /* get the current worker's node stats */ - List *workerNodeList = ActivePrimaryNonCoordinatorNodeList(NoLock); - WorkerNode *workerNode = NULL; - foreach_ptr(workerNode, workerNodeList) - { - if (workerNode->groupId == localGroupId) - { - const char *nodeName = workerNode->workerName; - int nodePort = workerNode->workerPort; - - citusStatsList = LocalNodeCitusDistStat(statQuery, nodeName, nodePort); - - break; - } - } - - return citusStatsList; -} - - -/* - * ParseCitusDistStat is a helper function which basically gets a PGresult - * and parses the results for rowIndex. Finally, returns CitusDistStat for - * further processing of the data retrieved. - * - * HeapTupleToCitusDistStat() and ParseCitusDistStat() are doing the same thing on - * different input data structures. Thus, any change to here should be reflected in - * the other function as well. - */ -static CitusDistStat * -ParseCitusDistStat(PGresult *result, int64 rowIndex) -{ - CitusDistStat *citusDistStat = (CitusDistStat *) palloc0(sizeof(CitusDistStat)); - - - int initiator_node_identifier = - PQgetisnull(result, rowIndex, 0) ? -1 : ParseIntField(result, rowIndex, 0); - - ReplaceInitiatorNodeIdentifier(initiator_node_identifier, citusDistStat); - - citusDistStat->distributed_transaction_number = ParseIntField(result, rowIndex, 1); - citusDistStat->distributed_transaction_stamp = - ParseTimestampTzField(result, rowIndex, 2); - - /* fields from pg_stat_statement */ - citusDistStat->database_id = ParseIntField(result, rowIndex, 3); - citusDistStat->databaese_name = ParseNameField(result, rowIndex, 4); - citusDistStat->process_id = ParseIntField(result, rowIndex, 5); - citusDistStat->usesysid = ParseIntField(result, rowIndex, 6); - citusDistStat->usename = ParseNameField(result, rowIndex, 7); - citusDistStat->application_name = ParseTextField(result, rowIndex, 8); - citusDistStat->client_addr = ParseInetField(result, rowIndex, 9); - citusDistStat->client_hostname = ParseTextField(result, rowIndex, 10); - citusDistStat->client_port = ParseIntField(result, rowIndex, 11); - citusDistStat->backend_start = ParseTimestampTzField(result, rowIndex, 12); - citusDistStat->xact_start = ParseTimestampTzField(result, rowIndex, 13); - citusDistStat->query_start = ParseTimestampTzField(result, rowIndex, 14); - citusDistStat->state_change = ParseTimestampTzField(result, rowIndex, 15); - citusDistStat->wait_event_type = ParseTextField(result, rowIndex, 16); - citusDistStat->wait_event = ParseTextField(result, rowIndex, 17); - citusDistStat->state = ParseTextField(result, rowIndex, 18); - citusDistStat->backend_xid = ParseXIDField(result, rowIndex, 19); - citusDistStat->backend_xmin = ParseXIDField(result, rowIndex, 20); - citusDistStat->query = ParseTextField(result, rowIndex, 21); - citusDistStat->backend_type = ParseTextField(result, rowIndex, 22); - citusDistStat->global_pid = ParseIntField(result, rowIndex, 23); - - return citusDistStat; -} - - -static void -ReplaceInitiatorNodeIdentifier(int initiator_node_identifier, - CitusDistStat *citusDistStat) -{ - WorkerNode *initiatorWorkerNode = NULL; - - /* - * Replace initiator_node_identifier with initiator_node_hostname - * and initiator_node_port given that those are a lot more useful. - * - * The rules are following: - * - If initiator_node_identifier belongs to a worker, simply get it - * from the metadata - * - If the initiator_node_identifier belongs to the coordinator and - * we're executing the function on the coordinator, get the localhost - * and port - * - If the initiator_node_identifier belongs to the coordinator and - * we're executing the function on a worker node, manually mark it - * as "coordinator_host" given that we cannot know the host and port - * - If the initiator_node_identifier doesn't equal to zero, we know that - * it is a worker query initiated outside of a distributed - * 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()) - { - citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); - citusDistStat->master_query_host_port = PostPortNumber; - } - else if (initiator_node_identifier == 0) - { - citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); - citusDistStat->master_query_host_port = 0; - } - else - { - citusDistStat->master_query_host_name = NULL; - citusDistStat->master_query_host_port = 0; - } -} - - -/* - * LocalNodeCitusDistStat simply executes the given query via SPI and parses - * the results back in a list for further processing. - * - * hostname and port is provided for filling the fields on the return list, obviously - * not for executing the SPI. - */ -static List * -LocalNodeCitusDistStat(const char *statQuery, const char *hostname, int port) -{ - List *localNodeCitusDistStatList = NIL; - bool readOnly = true; - - MemoryContext upperContext = CurrentMemoryContext, oldContext = NULL; - - int spiConnectionResult = SPI_connect(); - if (spiConnectionResult != SPI_OK_CONNECT) - { - ereport(WARNING, (errmsg("could not connect to SPI manager to get " - "the local stat activity"))); - - SPI_finish(); - - return NIL; - } - - int spiQueryResult = SPI_execute(statQuery, readOnly, 0); - if (spiQueryResult != SPI_OK_SELECT) - { - ereport(WARNING, (errmsg("execution was not successful while trying to get " - "the local stat activity"))); - - SPI_finish(); - - return NIL; - } - - /* - * SPI_connect switches to its own memory context, which is destroyed by - * the call to SPI_finish. SPI_palloc is provided to allocate memory in - * the previous ("upper") context, but that is inadequate when we need to - * call other functions that themselves use the normal palloc (such as - * lappend). So we switch to the upper context ourselves as needed. - */ - oldContext = MemoryContextSwitchTo(upperContext); - - for (uint64 rowIndex = 0; rowIndex < SPI_processed; rowIndex++) - { - TupleDesc rowDescriptor = SPI_tuptable->tupdesc; - - /* we use pointers from the tuple, so copy it before processing */ - HeapTuple row = SPI_copytuple(SPI_tuptable->vals[rowIndex]); - CitusDistStat *citusDistStat = HeapTupleToCitusDistStat(row, rowDescriptor); - - /* - * Add the query_host_name and query_host_port which denote where - * the query is being running. - */ - citusDistStat->query_host_name = cstring_to_text(hostname); - citusDistStat->query_host_port = port; - - localNodeCitusDistStatList = lappend(localNodeCitusDistStatList, citusDistStat); - } - - MemoryContextSwitchTo(oldContext); - - SPI_finish(); - - return localNodeCitusDistStatList; -} - - -/* - * HeapTupleToCitusDistStat is a helper function which basically gets a heapTuple - * and fetches the results for the given tuple. Finally, returns CitusDistStat for - * further processing of the data retrieved. - * - * HeapTupleToCitusDistStat() and ParseCitusDistStat() are doing the same thing on - * different input data structures. Thus, any change to here should be reflected in - * the other function as well. - */ -static CitusDistStat * -HeapTupleToCitusDistStat(HeapTuple result, TupleDesc rowDescriptor) -{ - CitusDistStat *citusDistStat = (CitusDistStat *) palloc0(sizeof(CitusDistStat)); - - int initiator_node_identifier = ParseIntFieldFromHeapTuple(result, rowDescriptor, 1); - - ReplaceInitiatorNodeIdentifier(initiator_node_identifier, citusDistStat); - - citusDistStat->distributed_transaction_number = - ParseIntFieldFromHeapTuple(result, rowDescriptor, 2); - citusDistStat->distributed_transaction_stamp = - ParseTimestampTzFieldFromHeapTuple(result, rowDescriptor, 3); - - /* fields from pg_stat_statement */ - citusDistStat->database_id = ParseIntFieldFromHeapTuple(result, rowDescriptor, 4); - citusDistStat->databaese_name = ParseNameFieldFromHeapTuple(result, rowDescriptor, 5); - citusDistStat->process_id = ParseIntFieldFromHeapTuple(result, rowDescriptor, 6); - citusDistStat->usesysid = ParseIntFieldFromHeapTuple(result, rowDescriptor, 7); - citusDistStat->usename = ParseNameFieldFromHeapTuple(result, rowDescriptor, 8); - citusDistStat->application_name = - ParseTextFieldFromHeapTuple(result, rowDescriptor, 9); - citusDistStat->client_addr = ParseInetFieldFromHeapTuple(result, rowDescriptor, 10); - citusDistStat->client_hostname = - ParseTextFieldFromHeapTuple(result, rowDescriptor, 11); - citusDistStat->client_port = ParseIntFieldFromHeapTuple(result, rowDescriptor, 12); - citusDistStat->backend_start = - ParseTimestampTzFieldFromHeapTuple(result, rowDescriptor, 13); - citusDistStat->xact_start = - ParseTimestampTzFieldFromHeapTuple(result, rowDescriptor, 14); - citusDistStat->query_start = - ParseTimestampTzFieldFromHeapTuple(result, rowDescriptor, 15); - citusDistStat->state_change = - ParseTimestampTzFieldFromHeapTuple(result, rowDescriptor, 16); - citusDistStat->wait_event_type = - ParseTextFieldFromHeapTuple(result, rowDescriptor, 17); - citusDistStat->wait_event = ParseTextFieldFromHeapTuple(result, rowDescriptor, 18); - citusDistStat->state = ParseTextFieldFromHeapTuple(result, rowDescriptor, 19); - citusDistStat->backend_xid = ParseXIDFieldFromHeapTuple(result, rowDescriptor, 20); - citusDistStat->backend_xmin = ParseXIDFieldFromHeapTuple(result, rowDescriptor, 21); - citusDistStat->query = ParseTextFieldFromHeapTuple(result, rowDescriptor, 22); - citusDistStat->backend_type = ParseTextFieldFromHeapTuple(result, rowDescriptor, 23); - citusDistStat->global_pid = ParseIntFieldFromHeapTuple(result, rowDescriptor, 24); - - return citusDistStat; -} - - -/* - * ParseIntFieldFromHeapTuple fetches an int64 from a heapTuple or returns 0 if the - * result is NULL. - */ -static int64 -ParseIntFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex) -{ - bool isNull = false; - - Datum resultDatum = SPI_getbinval(tuple, tupdesc, colIndex, &isNull); - if (isNull) - { - return 0; - } - - return DatumGetInt64(resultDatum); -} - - -/* - * ParseTextFieldFromHeapTuple parses a text from a heapTuple or returns - * NULL if the result is NULL. - */ -static text * -ParseTextFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex) -{ - bool isNull = false; - - Datum resultDatum = SPI_getbinval(tuple, tupdesc, colIndex, &isNull); - if (isNull) - { - return NULL; - } - - return (text *) DatumGetPointer(resultDatum); -} - - -/* - * ParseNameFieldFromHeapTuple fetches a name from a heapTuple result or returns NULL if the - * result is NULL. - */ -static Name -ParseNameFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex) -{ - bool isNull = false; - - Datum resultDatum = SPI_getbinval(tuple, tupdesc, colIndex, &isNull); - if (isNull) - { - return NULL; - } - - return (Name) DatumGetPointer(resultDatum); -} - - -/* - * ParseInetFieldFromHeapTuple fetcges an inet from a heapTuple or returns NULL if the - * result is NULL. - */ -static inet * -ParseInetFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex) -{ - bool isNull = false; - - Datum resultDatum = SPI_getbinval(tuple, tupdesc, colIndex, &isNull); - if (isNull) - { - return NULL; - } - - return DatumGetInetP(resultDatum); -} - - -/* - * ParseTimestampTzFieldFromHeapTuple parses a timestamptz from a heapTuple or returns - * DT_NOBEGIN if the result is NULL. - */ -static TimestampTz -ParseTimestampTzFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex) -{ - bool isNull = false; - - Datum resultDatum = SPI_getbinval(tuple, tupdesc, colIndex, &isNull); - if (isNull) - { - return DT_NOBEGIN; - } - - return DatumGetTimestampTz(resultDatum); -} - - -/* - * ParseXIDFieldFromHeapTuple parses a XID from a heapTuple or returns - * PG_UINT32_MAX if the result is NULL. - */ -static TransactionId -ParseXIDFieldFromHeapTuple(HeapTuple tuple, TupleDesc tupdesc, int colIndex) -{ - bool isNull = false; - - Datum resultDatum = SPI_getbinval(tuple, tupdesc, colIndex, &isNull); - if (isNull) - { - /* - * We'd show NULL if user hits the max transaction id, but that should be - * one of the minor problems they'd probably hit. - */ - return PG_UINT32_MAX; - } - - return DatumGetTransactionId(resultDatum); -} - - -/* - * ParseTextField parses a text from a remote result or returns NULL if the - * result is NULL. - */ -static text * -ParseTextField(PGresult *result, int rowIndex, int colIndex) -{ - if (PQgetisnull(result, rowIndex, colIndex)) - { - return NULL; - } - - char *resultString = PQgetvalue(result, rowIndex, colIndex); - Datum resultStringDatum = CStringGetDatum(resultString); - Datum textDatum = DirectFunctionCall1(textin, resultStringDatum); - - return (text *) DatumGetPointer(textDatum); -} - - -/* - * ParseNameField parses a name from a remote result or returns NULL if the - * result is NULL. - */ -static Name -ParseNameField(PGresult *result, int rowIndex, int colIndex) -{ - Datum nameDatum = 0; - - if (PQgetisnull(result, rowIndex, colIndex)) - { - return (Name) nameDatum; - } - - char *resultString = PQgetvalue(result, rowIndex, colIndex); - Datum resultStringDatum = CStringGetDatum(resultString); - nameDatum = DirectFunctionCall1(namein, resultStringDatum); - - return (Name) DatumGetPointer(nameDatum); -} - - -/* - * ParseInetField parses an inet from a remote result or returns NULL if the - * result is NULL. - */ -static inet * -ParseInetField(PGresult *result, int rowIndex, int colIndex) -{ - if (PQgetisnull(result, rowIndex, colIndex)) - { - return NULL; - } - - char *resultString = PQgetvalue(result, rowIndex, colIndex); - Datum resultStringDatum = CStringGetDatum(resultString); - Datum inetDatum = DirectFunctionCall1(inet_in, resultStringDatum); - - return DatumGetInetP(inetDatum); -} - - -/* - * ParseXIDField parses an XID from a remote result or returns 0 if the - * result is NULL. - */ -static TransactionId -ParseXIDField(PGresult *result, int rowIndex, int colIndex) -{ - if (PQgetisnull(result, rowIndex, colIndex)) - { - /* - * We'd show NULL if user hits the max transaction id, but that should be - * one of the minor problems they'd probably hit. - */ - return PG_UINT32_MAX; - } - - char *resultString = PQgetvalue(result, rowIndex, colIndex); - Datum resultStringDatum = CStringGetDatum(resultString); - Datum XIDDatum = DirectFunctionCall1(xidin, resultStringDatum); - - return DatumGetTransactionId(XIDDatum); -} - - -/* - * ReturnCitusDistStats returns the stats for a set returning function. - */ -static void -ReturnCitusDistStats(List *citusStatsList, FunctionCallInfo fcinfo) -{ - TupleDesc tupleDesc; - Tuplestorestate *tupleStore = SetupTuplestore(fcinfo, &tupleDesc); - - CitusDistStat *citusDistStat = NULL; - foreach_ptr(citusDistStat, citusStatsList) - { - Datum values[CITUS_DIST_STAT_ACTIVITY_COLS]; - bool nulls[CITUS_DIST_STAT_ACTIVITY_COLS]; - - memset(values, 0, sizeof(values)); - memset(nulls, 0, sizeof(nulls)); - - if (citusDistStat->query_host_name != NULL) - { - values[0] = PointerGetDatum(citusDistStat->query_host_name); - } - else - { - nulls[0] = true; - } - - values[1] = Int32GetDatum(citusDistStat->query_host_port); - - if (citusDistStat->master_query_host_name != NULL) - { - values[2] = PointerGetDatum(citusDistStat->master_query_host_name); - } - else - { - nulls[2] = true; - } - - values[3] = Int32GetDatum(citusDistStat->master_query_host_port); - values[4] = UInt64GetDatum(citusDistStat->distributed_transaction_number); - - if (citusDistStat->distributed_transaction_stamp != DT_NOBEGIN) - { - values[5] = TimestampTzGetDatum(citusDistStat->distributed_transaction_stamp); - } - else - { - nulls[5] = true; - } - - values[6] = ObjectIdGetDatum(citusDistStat->database_id); - - if (citusDistStat->databaese_name != NULL) - { - values[7] = CStringGetDatum(NameStr(*citusDistStat->databaese_name)); - } - else - { - nulls[7] = true; - } - - values[8] = Int32GetDatum(citusDistStat->process_id); - values[9] = ObjectIdGetDatum(citusDistStat->usesysid); - - if (citusDistStat->usename != NULL) - { - values[10] = CStringGetDatum(NameStr(*citusDistStat->usename)); - } - else - { - nulls[10] = true; - } - - if (citusDistStat->application_name != NULL) - { - values[11] = PointerGetDatum(citusDistStat->application_name); - } - else - { - nulls[11] = true; - } - - if (citusDistStat->client_addr != NULL) - { - values[12] = InetPGetDatum(citusDistStat->client_addr); - } - else - { - nulls[12] = true; - } - - if (citusDistStat->client_hostname != NULL) - { - values[13] = PointerGetDatum(citusDistStat->client_hostname); - } - else - { - nulls[13] = true; - } - - values[14] = Int32GetDatum(citusDistStat->client_port); - - if (citusDistStat->backend_start != DT_NOBEGIN) - { - values[15] = TimestampTzGetDatum(citusDistStat->backend_start); - } - else - { - nulls[15] = true; - } - - if (citusDistStat->xact_start != DT_NOBEGIN) - { - values[16] = TimestampTzGetDatum(citusDistStat->xact_start); - } - else - { - nulls[16] = true; - } - - if (citusDistStat->query_start != DT_NOBEGIN) - { - values[17] = TimestampTzGetDatum(citusDistStat->query_start); - } - else - { - nulls[17] = true; - } - - if (citusDistStat->state_change != DT_NOBEGIN) - { - values[18] = TimestampTzGetDatum(citusDistStat->state_change); - } - else - { - nulls[18] = true; - } - - if (citusDistStat->wait_event_type != NULL) - { - values[19] = PointerGetDatum(citusDistStat->wait_event_type); - } - else - { - nulls[19] = true; - } - - if (citusDistStat->wait_event != NULL) - { - values[20] = PointerGetDatum(citusDistStat->wait_event); - } - else - { - nulls[20] = true; - } - - if (citusDistStat->state != NULL) - { - values[21] = PointerGetDatum(citusDistStat->state); - } - else - { - nulls[21] = true; - } - - if (citusDistStat->backend_xid != PG_UINT32_MAX) - { - values[22] = TransactionIdGetDatum(citusDistStat->backend_xid); - } - else - { - nulls[22] = true; - } - - if (citusDistStat->backend_xmin != PG_UINT32_MAX) - { - values[23] = TransactionIdGetDatum(citusDistStat->backend_xmin); - } - else - { - nulls[23] = true; - } - - if (citusDistStat->query != NULL) - { - values[24] = PointerGetDatum(citusDistStat->query); - } - else - { - nulls[24] = true; - } - - if (citusDistStat->backend_type != NULL) - { - values[25] = PointerGetDatum(citusDistStat->backend_type); - } - else - { - nulls[25] = true; - } - - values[26] = Int32GetDatum(citusDistStat->global_pid); - - tuplestore_putvalues(tupleStore, tupleDesc, values, nulls); - } + PG_RETURN_NULL(); } diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index 4edc9e424..9fb616ad8 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -583,6 +583,25 @@ CreateColocationGroup(int shardCount, int replicationFactor, Oid distributionCol Oid distributionColumnCollation) { uint32 colocationId = GetNextColocationId(); + + InsertColocationGroupLocally(colocationId, shardCount, replicationFactor, + distributionColumnType, distributionColumnCollation); + + SyncNewColocationGroupToNodes(colocationId, shardCount, replicationFactor, + distributionColumnType, distributionColumnCollation); + + return colocationId; +} + + +/* + * InsertColocationGroupLocally inserts a record into pg_dist_colocation. + */ +void +InsertColocationGroupLocally(uint32 colocationId, int shardCount, int replicationFactor, + Oid distributionColumnType, + Oid distributionColumnCollation) +{ Datum values[Natts_pg_dist_colocation]; bool isNulls[Natts_pg_dist_colocation]; @@ -610,8 +629,6 @@ CreateColocationGroup(int shardCount, int replicationFactor, Oid distributionCol /* increment the counter so that next command can see the row */ CommandCounterIncrement(); table_close(pgDistColocation, RowExclusiveLock); - - return colocationId; } @@ -1215,10 +1232,22 @@ DeleteColocationGroupIfNoTablesBelong(uint32 colocationId) /* - * DeleteColocationGroup deletes the colocation group from pg_dist_colocation. + * DeleteColocationGroup deletes the colocation group from pg_dist_colocation + * throughout the cluster. */ static void DeleteColocationGroup(uint32 colocationId) +{ + DeleteColocationGroupLocally(colocationId); + SyncDeleteColocationGroupToNodes(colocationId); +} + + +/* + * DeleteColocationGroupLocally deletes the colocation group from pg_dist_colocation. + */ +void +DeleteColocationGroupLocally(uint32 colocationId) { int scanKeyCount = 1; ScanKeyData scanKey[1]; diff --git a/src/backend/distributed/worker/worker_drop_protocol.c b/src/backend/distributed/worker/worker_drop_protocol.c index f2ae3597f..0f425583b 100644 --- a/src/backend/distributed/worker/worker_drop_protocol.c +++ b/src/backend/distributed/worker/worker_drop_protocol.c @@ -29,6 +29,7 @@ #include "distributed/coordinator_protocol.h" #include "distributed/metadata_cache.h" #include "distributed/metadata/distobject.h" +#include "distributed/multi_partitioning_utils.h" #include "foreign/foreign.h" #include "utils/builtins.h" #include "utils/fmgroids.h" @@ -37,12 +38,13 @@ PG_FUNCTION_INFO_V1(worker_drop_distributed_table); PG_FUNCTION_INFO_V1(worker_drop_shell_table); PG_FUNCTION_INFO_V1(worker_drop_sequence_dependency); - +static void WorkerDropDistributedTable(Oid relationId); #if PG_VERSION_NUM < PG_VERSION_13 static long deleteDependencyRecordsForSpecific(Oid classId, Oid objectId, char deptype, Oid refclassId, Oid refobjectId); #endif + /* * worker_drop_distributed_table drops the distributed table with the given oid, * then, removes the associated rows from pg_dist_partition, pg_dist_shard and @@ -64,8 +66,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS) text *relationName = PG_GETARG_TEXT_P(0); Oid relationId = ResolveRelationId(relationName, true); - ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 }; - if (!OidIsValid(relationId)) { ereport(NOTICE, (errmsg("relation %s does not exist, skipping", @@ -75,8 +75,45 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS) EnsureTableOwner(relationId); - List *shardList = LoadShardList(relationId); + if (PartitionedTable(relationId)) + { + /* + * When "DROP SCHEMA .. CASCADE" happens, we rely on Postgres' drop trigger + * to send the individual DROP TABLE commands for tables. + * + * In case of partitioned tables, we have no control on the order of DROP + * commands that is sent to the extension. We can try to sort while processing + * on the coordinator, but we prefer to handle it in a more flexible manner. + * + * That's why, whenever we see a partitioned table, we drop all the corresponding + * partitions first. Otherwise, WorkerDropDistributedTable() would already drop + * the shell tables of the partitions (e.g., due to performDeletion(..CASCADE), + * and further WorkerDropDistributedTable() on the partitions would become no-op. + * + * If, say one partition has already been dropped earlier, that should also be fine + * because we read the existing partitions. + */ + List *partitionList = PartitionList(relationId); + Oid partitionOid = InvalidOid; + foreach_oid(partitionOid, partitionList) + { + WorkerDropDistributedTable(partitionOid); + } + } + WorkerDropDistributedTable(relationId); + + PG_RETURN_VOID(); +} + + +/* + * WorkerDropDistributedTable is a helper function for worker_drop_distributed_table, see + * tha function for the details. + */ +static void +WorkerDropDistributedTable(Oid relationId) +{ /* first check the relation type */ Relation distributedRelation = relation_open(relationId, AccessShareLock); @@ -86,9 +123,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS) relation_close(distributedRelation, AccessShareLock); /* prepare distributedTableObject for dropping the table */ - distributedTableObject.classId = RelationRelationId; - distributedTableObject.objectId = relationId; - distributedTableObject.objectSubId = 0; + ObjectAddress distributedTableObject = { RelationRelationId, relationId, 0 }; /* Drop dependent sequences from pg_dist_object */ #if PG_VERSION_NUM >= PG_VERSION_13 @@ -121,6 +156,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS) } /* iterate over shardList to delete the corresponding rows */ + List *shardList = LoadShardList(relationId); uint64 *shardIdPointer = NULL; foreach_ptr(shardIdPointer, shardList) { @@ -140,8 +176,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS) /* delete the row from pg_dist_partition */ DeletePartitionRow(relationId); - - PG_RETURN_VOID(); } 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..ccb4da535 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 void SetBackendDataDistributedCommandOriginator(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); @@ -78,9 +66,9 @@ extern bool MyBackendGotCancelledDueToDeadlock(bool clearState); extern bool MyBackendIsInDisributedTransaction(void); extern List * ActiveDistributedTransactionNumbers(void); extern LocalTransactionId GetMyProcLocalTransactionId(void); -extern int GetAllActiveClientBackendCount(void); -extern void IncrementClientBackendCounter(void); -extern void DecrementClientBackendCounter(void); +extern int GetExternalClientBackendCount(void); +extern uint32 IncrementExternalClientBackendCounter(void); +extern void DecrementExternalClientBackendCounter(void); extern bool ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort, char *queryString, StringInfo queryResultString, diff --git a/src/include/distributed/colocation_utils.h b/src/include/distributed/colocation_utils.h index 0ce4d8fb1..0095ac427 100644 --- a/src/include/distributed/colocation_utils.h +++ b/src/include/distributed/colocation_utils.h @@ -31,6 +31,10 @@ uint32 ColocationId(int shardCount, int replicationFactor, Oid distributionColum extern uint32 CreateColocationGroup(int shardCount, int replicationFactor, Oid distributionColumnType, Oid distributionColumnCollation); +extern void InsertColocationGroupLocally(uint32 colocationId, int shardCount, + int replicationFactor, + Oid distributionColumnType, + Oid distributionColumnCollation); extern bool IsColocateWithNone(char *colocateWithTableName); extern uint32 GetNextColocationId(void); extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId); @@ -43,5 +47,6 @@ extern void UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colo bool localOnly); extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId); extern List * ColocationGroupTableList(uint32 colocationId, uint32 count); +extern void DeleteColocationGroupLocally(uint32 colocationId); #endif /* COLOCATION_UTILS_H_ */ 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/dependency.h b/src/include/distributed/metadata/dependency.h index 141b2a628..92714f6cb 100644 --- a/src/include/distributed/metadata/dependency.h +++ b/src/include/distributed/metadata/dependency.h @@ -21,6 +21,8 @@ extern List * GetUniqueDependenciesList(List *objectAddressesList); extern List * GetDependenciesForObject(const ObjectAddress *target); extern List * GetAllSupportedDependenciesForObject(const ObjectAddress *target); extern List * GetAllDependenciesForObject(const ObjectAddress *target); +extern void EnsureRelationDependenciesCanBeDistributed(ObjectAddress *relationAddress); +extern ObjectAddress * GetUndistributableDependency(ObjectAddress *target); extern List * OrderObjectAddressListInDependencyOrder(List *objectAddressList); extern bool SupportedDependencyByCitus(const ObjectAddress *address); extern List * GetPgDependTuplesForDependingObjects(Oid targetObjectClassId, diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index 69d500da4..e67726bfc 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -38,6 +38,7 @@ extern bool ShouldSyncTableMetadata(Oid relationId); extern bool ShouldSyncTableMetadataViaCatalog(Oid relationId); extern List * NodeMetadataCreateCommands(void); extern List * DistributedObjectMetadataSyncCommandList(void); +extern List * ColocationGroupCreateCommandList(void); extern List * CitusTableMetadataCreateCommandList(Oid relationId); extern List * NodeMetadataDropCommands(void); extern char * MarkObjectsDistributedCreateCommand(List *addresses, @@ -73,13 +74,21 @@ 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); +extern void SyncNewColocationGroupToNodes(uint32 colocationId, int shardCount, + int replicationFactor, + Oid distributionColumType, + Oid distributionColumnCollation); +extern void SyncDeleteColocationGroupToNodes(uint32 colocationId); #define DELETE_ALL_NODES "DELETE FROM pg_dist_node" #define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement" #define DELETE_ALL_SHARDS "DELETE FROM pg_dist_shard" -#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM citus.pg_dist_object" +#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM pg_catalog.pg_dist_object" #define DELETE_ALL_PARTITIONS "DELETE FROM pg_dist_partition" +#define DELETE_ALL_COLOCATION "DELETE FROM pg_catalog.pg_dist_colocation" #define REMOVE_ALL_SHELL_TABLES_COMMAND \ "SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition" #define REMOVE_ALL_CITUS_TABLES_COMMAND \ 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/shared_connection_stats.h b/src/include/distributed/shared_connection_stats.h index 1efb37d28..638d42f63 100644 --- a/src/include/distributed/shared_connection_stats.h +++ b/src/include/distributed/shared_connection_stats.h @@ -14,15 +14,18 @@ #define ADJUST_POOLSIZE_AUTOMATICALLY 0 #define DISABLE_CONNECTION_THROTTLING -1 #define DISABLE_REMOTE_CONNECTIONS_FOR_LOCAL_QUERIES -1 +#define ALLOW_ALL_EXTERNAL_CONNECTIONS -1 extern int MaxSharedPoolSize; extern int LocalSharedPoolSize; +extern int MaxClientConnections; extern void InitializeSharedConnectionStats(void); extern void WaitForSharedConnection(void); extern void WakeupWaiterBackendsForSharedConnection(void); +extern int GetMaxClientConnections(void); extern int GetMaxSharedPoolSize(void); extern int GetLocalSharedPoolSize(void); extern bool TryToIncrementSharedConnectionCounter(const char *hostname, int port); 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/after_pg_upgrade_schedule b/src/test/regress/after_pg_upgrade_schedule index c8a1b6b40..f8e4e66ae 100644 --- a/src/test/regress/after_pg_upgrade_schedule +++ b/src/test/regress/after_pg_upgrade_schedule @@ -1,4 +1,4 @@ -test: upgrade_basic_after upgrade_type_after upgrade_ref2ref_after upgrade_distributed_function_after upgrade_rebalance_strategy_after upgrade_list_citus_objects upgrade_autoconverted_after +test: upgrade_basic_after upgrade_type_after upgrade_ref2ref_after upgrade_distributed_function_after upgrade_rebalance_strategy_after upgrade_list_citus_objects upgrade_autoconverted_after upgrade_citus_stat_activity # This attempts dropping citus extension (and rollbacks), so please do # not run in parallel with any other tests. 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/before_pg_upgrade_schedule b/src/test/regress/before_pg_upgrade_schedule index a5292e0ce..880b25e0a 100644 --- a/src/test/regress/before_pg_upgrade_schedule +++ b/src/test/regress/before_pg_upgrade_schedule @@ -6,6 +6,7 @@ test: upgrade_ref2ref_before test: upgrade_type_before test: upgrade_distributed_function_before upgrade_rebalance_strategy_before test: upgrade_autoconverted_before +test: upgrade_citus_stat_activity # upgrade_columnar_before renames public schema to citus_schema, so let's # run this test as the last one. 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/bin/normalize.sed b/src/test/regress/bin/normalize.sed index 916d62afa..d7ffe31c3 100644 --- a/src/test/regress/bin/normalize.sed +++ b/src/test/regress/bin/normalize.sed @@ -260,3 +260,6 @@ s/NOTICE: issuing WITH placement_data\(shardid, shardstate, shardlength, groupi # global_pid when pg_cancel_backend is sent to workers s/pg_cancel_backend\('[0-9]+'::bigint\)/pg_cancel_backend('xxxxx'::bigint)/g s/issuing SELECT pg_cancel_backend\([0-9]+::integer\)/issuing SELECT pg_cancel_backend(xxxxx::integer)/g + +# node id in run_command_on_all_nodes warning +s/Error on node with node id [0-9]+/Error on node with node id xxxxx/g diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index 52b0714c8..205be5aed 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -190,6 +190,9 @@ class PostgresConfig(CitusDefaultClusterConfig): super().__init__(arguments) self.worker_amount = 0 self.is_citus = False + self.new_settings = { + "citus.use_citus_managed_tables": False, + } class CitusSingleNodeClusterConfig(CitusDefaultClusterConfig): @@ -227,8 +230,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..80ac1bb7a 100644 --- a/src/test/regress/expected/aggregate_support.out +++ b/src/test/regress/expected/aggregate_support.out @@ -40,12 +40,16 @@ create aggregate sum2_strict (int) ( combinefunc = sum2_sfunc_strict ); select create_distributed_function('sum2(int)'); +NOTICE: procedure aggregate_support.sum2 is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('sum2_strict(int)'); +NOTICE: procedure aggregate_support.sum2_strict is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -96,12 +100,16 @@ create aggregate psum_strict(int, int)( initcond=0 ); select create_distributed_function('psum(int,int)'); +NOTICE: procedure aggregate_support.psum is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('psum_strict(int,int)'); +NOTICE: procedure aggregate_support.psum_strict is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -282,7 +290,24 @@ 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 pg_catalog.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)'); +NOTICE: procedure aggregate_support.binstragg is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -512,12 +537,16 @@ CREATE AGGREGATE last ( combinefunc = last_agg ); SELECT create_distributed_function('first(anyelement)'); +NOTICE: procedure aggregate_support.first is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) SELECT create_distributed_function('last(anyelement)'); +NOTICE: procedure aggregate_support.last is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -549,7 +578,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,10 +589,23 @@ 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)'); +NOTICE: procedure aggregate_support.sumstring is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -589,6 +630,8 @@ create aggregate array_collect_sort(el int) ( initcond = '{}' ); select create_distributed_function('array_collect_sort(int)'); +NOTICE: procedure aggregate_support.array_collect_sort is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -1042,5 +1085,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 pg_catalog.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 pg_catalog.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/alter_table_set_access_method.out b/src/test/regress/expected/alter_table_set_access_method.out index 209999667..75ddac37e 100644 --- a/src/test/regress/expected/alter_table_set_access_method.out +++ b/src/test/regress/expected/alter_table_set_access_method.out @@ -474,11 +474,8 @@ SELECT c.relname, a.amname FROM pg_class c, pg_am a where c.relname SIMILAR TO ' table_type_ref | heap (4 rows) -SELECT alter_table_set_access_method('table_type_dist', 'fake_am'); +SELECT alter_table_set_access_method('table_type_dist', 'columnar'); NOTICE: creating a new table for alter_table_set_access_method.table_type_dist -WARNING: fake_scan_getnextslot -CONTEXT: SQL statement "SELECT TRUE FROM alter_table_set_access_method.table_type_dist_1533505599 LIMIT 1" -WARNING: fake_scan_getnextslot NOTICE: moving the data of alter_table_set_access_method.table_type_dist NOTICE: dropping the old alter_table_set_access_method.table_type_dist NOTICE: renaming the new table to alter_table_set_access_method.table_type_dist @@ -487,11 +484,8 @@ NOTICE: renaming the new table to alter_table_set_access_method.table_type_dist (1 row) -SELECT alter_table_set_access_method('table_type_ref', 'fake_am'); +SELECT alter_table_set_access_method('table_type_ref', 'columnar'); NOTICE: creating a new table for alter_table_set_access_method.table_type_ref -WARNING: fake_scan_getnextslot -CONTEXT: SQL statement "SELECT TRUE FROM alter_table_set_access_method.table_type_ref_1037855087 LIMIT 1" -WARNING: fake_scan_getnextslot NOTICE: moving the data of alter_table_set_access_method.table_type_ref NOTICE: dropping the old alter_table_set_access_method.table_type_ref NOTICE: renaming the new table to alter_table_set_access_method.table_type_ref @@ -500,7 +494,7 @@ NOTICE: renaming the new table to alter_table_set_access_method.table_type_ref (1 row) -SELECT alter_table_set_access_method('table_type_pg_local', 'fake_am'); +SELECT alter_table_set_access_method('table_type_pg_local', 'columnar'); NOTICE: creating a new table for alter_table_set_access_method.table_type_pg_local NOTICE: moving the data of alter_table_set_access_method.table_type_pg_local NOTICE: dropping the old alter_table_set_access_method.table_type_pg_local @@ -510,7 +504,7 @@ NOTICE: renaming the new table to alter_table_set_access_method.table_type_pg_l (1 row) -SELECT alter_table_set_access_method('table_type_citus_local', 'fake_am'); +SELECT alter_table_set_access_method('table_type_citus_local', 'columnar'); NOTICE: creating a new table for alter_table_set_access_method.table_type_citus_local NOTICE: moving the data of alter_table_set_access_method.table_type_citus_local NOTICE: dropping the old alter_table_set_access_method.table_type_citus_local @@ -523,17 +517,17 @@ NOTICE: renaming the new table to alter_table_set_access_method.table_type_citu SELECT table_name, citus_table_type, distribution_column, shard_count, access_method FROM public.citus_tables WHERE table_name::text LIKE 'table\_type%' ORDER BY 1; table_name | citus_table_type | distribution_column | shard_count | access_method --------------------------------------------------------------------- - table_type_dist | distributed | a | 4 | fake_am - table_type_ref | reference | | 1 | fake_am + table_type_dist | distributed | a | 4 | columnar + table_type_ref | reference | | 1 | columnar (2 rows) SELECT c.relname, a.amname FROM pg_class c, pg_am a where c.relname SIMILAR TO 'table_type\D*' AND c.relnamespace = 'alter_table_set_access_method'::regnamespace AND c.relam = a.oid; relname | amname --------------------------------------------------------------------- - table_type_citus_local | fake_am - table_type_dist | fake_am - table_type_pg_local | fake_am - table_type_ref | fake_am + table_type_citus_local | columnar + table_type_dist | columnar + table_type_pg_local | columnar + table_type_ref | columnar (4 rows) -- test when the parent of a partition has foreign key to a reference table diff --git a/src/test/regress/expected/citus_local_tables.out b/src/test/regress/expected/citus_local_tables.out index 4e58021dd..9dac373cd 100644 --- a/src/test/regress/expected/citus_local_tables.out +++ b/src/test/regress/expected/citus_local_tables.out @@ -224,12 +224,16 @@ CREATE FOREIGN TABLE foreign_table ( ) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true'); -- observe that we do not create fdw server for shell table, both shard relation -- & shell relation points to the same same server object +-- Disable metadata sync since citus doesn't support distributing +-- foreign data wrappers for now. +SET citus.enable_metadata_sync TO OFF; SELECT citus_add_local_table_to_metadata('foreign_table'); citus_add_local_table_to_metadata --------------------------------------------------------------------- (1 row) +RESET citus.enable_metadata_sync; DROP FOREIGN TABLE foreign_table; NOTICE: executing the command locally: DROP FOREIGN TABLE IF EXISTS citus_local_tables_test_schema.foreign_table_xxxxx CASCADE -- drop them for next tests diff --git a/src/test/regress/expected/coordinator_evaluation.out b/src/test/regress/expected/coordinator_evaluation.out index f675c316f..a0ed7ea51 100644 --- a/src/test/regress/expected/coordinator_evaluation.out +++ b/src/test/regress/expected/coordinator_evaluation.out @@ -11,6 +11,8 @@ BEGIN RETURN localGroupId; END; $$ language plpgsql VOLATILE; SELECT create_distributed_function('get_local_node_id_volatile()'); +NOTICE: procedure coordinator_evaluation.get_local_node_id_volatile is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -24,6 +26,8 @@ BEGIN RETURN localGroupId; END; $$ language plpgsql VOLATILE; SELECT create_distributed_function('get_local_node_id_volatile_sum_with_param(int)'); +NOTICE: procedure coordinator_evaluation.get_local_node_id_volatile_sum_with_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -586,6 +590,8 @@ SELECT count(*) FROM coordinator_evaluation_table_2 WHERE key = 101; CREATE TYPE comptype_int as (int_a int); CREATE DOMAIN domain_comptype_int AS comptype_int CHECK ((VALUE).int_a > 0); -- citus does not propagate domain types +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. SELECT run_command_on_workers( $$ CREATE DOMAIN coordinator_evaluation.domain_comptype_int AS coordinator_evaluation.comptype_int CHECK ((VALUE).int_a > 0) @@ -597,12 +603,16 @@ $$); (2 rows) CREATE TABLE reference_table(column_a coordinator_evaluation.domain_comptype_int); +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_reference_table('reference_table'); create_reference_table --------------------------------------------------------------------- (1 row) +RESET citus.enable_metadata_sync; INSERT INTO reference_table (column_a) VALUES ('(1)'); INSERT INTO reference_table (column_a) VALUES ('(2)'), ('(3)'); INSERT INTO reference_table VALUES ('(4)'), ('(5)'); diff --git a/src/test/regress/expected/coordinator_evaluation_modify.out b/src/test/regress/expected/coordinator_evaluation_modify.out index e75ac60b1..0b14c109d 100644 --- a/src/test/regress/expected/coordinator_evaluation_modify.out +++ b/src/test/regress/expected/coordinator_evaluation_modify.out @@ -21,6 +21,8 @@ BEGIN RETURN localGroupId; END; $$ language plpgsql STABLE; SELECT create_distributed_function('get_local_node_id_stable()'); +NOTICE: procedure coordinator_evaluation_combinations_modify.get_local_node_id_stable is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/coordinator_evaluation_select.out b/src/test/regress/expected/coordinator_evaluation_select.out index 0fa70f066..e387de2be 100644 --- a/src/test/regress/expected/coordinator_evaluation_select.out +++ b/src/test/regress/expected/coordinator_evaluation_select.out @@ -21,6 +21,8 @@ BEGIN RETURN localGroupId; END; $$ language plpgsql VOLATILE; SELECT create_distributed_function('get_local_node_id_volatile()'); +NOTICE: procedure coordinator_evaluation_combinations.get_local_node_id_volatile is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- 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/disable_object_propagation.out b/src/test/regress/expected/disable_object_propagation.out index 8429fe301..6ebe15089 100644 --- a/src/test/regress/expected/disable_object_propagation.out +++ b/src/test/regress/expected/disable_object_propagation.out @@ -77,7 +77,7 @@ SELECT create_distributed_table('t4','a'); DROP TABLE t4; -- as long as the table is using the type some operations are hard to force COMMIT; -- verify the type is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid; +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid; count --------------------------------------------------------------------- 1 diff --git a/src/test/regress/expected/distributed_functions.out b/src/test/regress/expected/distributed_functions.out index 614f19288..3534b12a2 100644 --- a/src/test/regress/expected/distributed_functions.out +++ b/src/test/regress/expected/distributed_functions.out @@ -33,6 +33,8 @@ BEGIN END; $$; SELECT create_distributed_function('notice(text)'); +NOTICE: procedure function_tests.notice is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -193,7 +195,7 @@ SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)') (1 row) -SELECT distribution_argument_index is NULL, colocationid is NULL from citus.pg_dist_object +SELECT distribution_argument_index is NULL, colocationid is NULL from pg_catalog.pg_dist_object WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure; ?column? | ?column? --------------------------------------------------------------------- @@ -486,6 +488,8 @@ AS 'select $1 = $2;' IMMUTABLE RETURNS NULL ON NULL INPUT; select create_distributed_function('eq(macaddr,macaddr)'); +NOTICE: procedure function_tests.eq is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -648,7 +652,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', '$1' -- show that the colocationIds are the same SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; table_and_function_colocated @@ -665,7 +669,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val (1 row) SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; table_and_function_colocated @@ -700,7 +704,7 @@ SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_wit (1 row) SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure; table_and_function_colocated @@ -715,7 +719,7 @@ SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:= (1 row) SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'add_text(text, text)'::regprocedure; table_and_function_colocated @@ -797,6 +801,8 @@ BEGIN END; $$; SELECT create_distributed_function('func_with_return_table(int)'); +NOTICE: procedure function_tests.func_with_return_table is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -824,6 +830,8 @@ CREATE OR REPLACE FUNCTION func_with_out_param(a int, out b int) RETURNS int LANGUAGE sql AS $$ select 1; $$; SELECT create_distributed_function('func_with_out_param(int)'); +NOTICE: procedure function_tests.func_with_out_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -869,6 +877,8 @@ SELECT create_distributed_function('func_with_inout_param(int)'); ERROR: function "func_with_inout_param(int)" does not exist -- this should work SELECT create_distributed_function('func_with_inout_param(int,int)'); +NOTICE: procedure function_tests.func_with_inout_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -893,6 +903,8 @@ CREATE OR REPLACE FUNCTION func_with_variadic_param(a int, variadic b int[]) LANGUAGE sql AS $$ select 1; $$; -- this should work SELECT create_distributed_function('func_with_variadic_param(int,int[])'); +NOTICE: procedure function_tests.func_with_variadic_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -923,6 +935,8 @@ $BODY$ LANGUAGE plpgsql VOLATILE COST 100; SELECT create_distributed_function('func_returning_setof_int(date,interval)'); +NOTICE: procedure function_tests.func_returning_setof_int is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -961,6 +975,8 @@ $BODY$ LANGUAGE plpgsql VOLATILE COST 100; SELECT create_distributed_function('func_returning_setof_int_with_variadic_param(date,int[])'); +NOTICE: procedure function_tests.func_returning_setof_int_with_variadic_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -997,6 +1013,8 @@ SELECT create_distributed_function('proc_with_variadic_param(date)'); ERROR: function "proc_with_variadic_param(date)" does not exist -- this should work SELECT create_distributed_function('proc_with_variadic_param(date,int[])'); +NOTICE: procedure function_tests.proc_with_variadic_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -1028,6 +1046,8 @@ SELECT create_distributed_function('proc_with_inout_param(date)'); ERROR: function "proc_with_inout_param(date)" does not exist -- this should work SELECT create_distributed_function('proc_with_inout_param(date,int)'); +NOTICE: procedure function_tests.proc_with_inout_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/distributed_functions_conflict.out b/src/test/regress/expected/distributed_functions_conflict.out index 354aea9c0..8101cf0f1 100644 --- a/src/test/regress/expected/distributed_functions_conflict.out +++ b/src/test/regress/expected/distributed_functions_conflict.out @@ -26,6 +26,8 @@ CREATE AGGREGATE existing_agg(int) ( STYPE = int ); SELECT create_distributed_function('existing_agg(int)'); +NOTICE: procedure proc_conflict.existing_agg is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -87,6 +89,8 @@ CREATE AGGREGATE existing_agg(int) ( STYPE = int ); SELECT create_distributed_function('existing_agg(int)'); +NOTICE: procedure proc_conflict.existing_agg is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- 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..c20326820 100644 --- a/src/test/regress/expected/distributed_types.out +++ b/src/test/regress/expected/distributed_types.out @@ -415,6 +415,8 @@ HINT: Use the column name to insert or update the composite type as a single va CREATE TYPE two_ints as (if1 int, if2 int); CREATE DOMAIN domain AS two_ints CHECK ((VALUE).if1 > 0); -- citus does not propagate domain objects +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. SELECT run_command_on_workers( $$ CREATE DOMAIN type_tests.domain AS type_tests.two_ints CHECK ((VALUE).if1 > 0); @@ -426,12 +428,16 @@ $$); (2 rows) CREATE TABLE domain_indirection_test (f1 int, f3 domain, domain_array domain[]); +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_table('domain_indirection_test', 'f1'); create_distributed_table --------------------------------------------------------------------- (1 row) +RESET citus.enable_metadata_sync; -- not supported (field indirection to underlying composite type) INSERT INTO domain_indirection_test (f1,f3.if1, f3.if2) VALUES (0, 1, 2); ERROR: inserting or modifying composite type fields is not supported @@ -453,6 +459,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/drop_partitioned_table.out b/src/test/regress/expected/drop_partitioned_table.out index edc535020..846656aaf 100644 --- a/src/test/regress/expected/drop_partitioned_table.out +++ b/src/test/regress/expected/drop_partitioned_table.out @@ -395,3 +395,117 @@ NOTICE: issuing ROLLBACK DROP SCHEMA drop_partitioned_table CASCADE; NOTICE: drop cascades to 3 other objects SET search_path TO public; +-- dropping the schema should drop the metadata on the workers +CREATE SCHEMA partitioning_schema; +SET search_path TO partitioning_schema; +CREATE TABLE part_table ( + col timestamp + ) PARTITION BY RANGE (col); +CREATE TABLE part_table_1 + PARTITION OF part_table + FOR VALUES FROM ('2010-01-01') TO ('2015-01-01'); +SELECT create_distributed_table('part_table', 'col'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- show we have pg_dist_partition entries on the workers +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,2) + (localhost,57638,t,2) +(2 rows) + +-- show we have pg_dist_object entries on the workers +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,2) + (localhost,57638,t,2) +(2 rows) + +DROP SCHEMA partitioning_schema CASCADE; +NOTICE: drop cascades to table part_table +-- show we don't have pg_dist_partition entries on the workers after dropping the schema +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,0) + (localhost,57638,t,0) +(2 rows) + +-- show we don't have pg_dist_object entries on the workers after dropping the schema +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,0) + (localhost,57638,t,0) +(2 rows) + +-- dropping the parent should drop the metadata on the workers +CREATE SCHEMA partitioning_schema; +SET search_path TO partitioning_schema; +CREATE TABLE part_table ( + col timestamp + ) PARTITION BY RANGE (col); +CREATE TABLE part_table_1 + PARTITION OF part_table + FOR VALUES FROM ('2010-01-01') TO ('2015-01-01'); +SELECT create_distributed_table('part_table', 'col'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE part_table; +-- show we don't have pg_dist_partition entries on the workers after dropping the parent +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,0) + (localhost,57638,t,0) +(2 rows) + +-- show we don't have pg_dist_object entries on the workers after dropping the parent +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,0) + (localhost,57638,t,0) +(2 rows) + +SET search_path TO partitioning_schema; +CREATE TABLE part_table ( + col timestamp + ) PARTITION BY RANGE (col); +CREATE TABLE part_table_1 + PARTITION OF part_table + FOR VALUES FROM ('2010-01-01') TO ('2015-01-01'); +SELECT create_distributed_table('part_table', 'col'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE part_table_1; +-- show we have pg_dist_partition entries for the parent on the workers after dropping the partition +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +-- show we have pg_dist_object entries for the parent on the workers after dropping the partition +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +-- clean-up +DROP SCHEMA partitioning_schema CASCADE; +NOTICE: drop cascades to table part_table 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/forcedelegation_functions.out b/src/test/regress/expected/forcedelegation_functions.out index c26f7b75b..1bb6b8ba7 100644 --- a/src/test/regress/expected/forcedelegation_functions.out +++ b/src/test/regress/expected/forcedelegation_functions.out @@ -307,8 +307,8 @@ $$ LANGUAGE plpgsql; DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('func_calls_forcepush_func()'); -DEBUG: switching to sequential query execution mode -DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +NOTICE: procedure forcepushdown_schema.func_calls_forcepush_func is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/function_propagation.out b/src/test/regress/expected/function_propagation.out index a40f1f9a2..cf87f43fd 100644 --- a/src/test/regress/expected/function_propagation.out +++ b/src/test/regress/expected/function_propagation.out @@ -14,39 +14,39 @@ BEGIN END; $$; -- Check all dependent objects and function depends on all nodes -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid; pg_identify_object_as_address --------------------------------------------------------------------- (schema,{function_propagation_schema},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (type,{function_propagation_schema.function_prop_type},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (schema,{function_propagation_schema},{}) localhost | 57638 | t | (schema,{function_propagation_schema},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type},{}) localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type}) @@ -64,26 +64,26 @@ BEGIN return 1; END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (type,{function_propagation_schema.function_prop_type_2},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{function_propagation_schema,func_2}",{integer}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type_2},{}) localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type_2},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{function_propagation_schema,func_2}",{integer}) @@ -105,18 +105,18 @@ BEGIN return 1; END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{function_propagation_schema,func_3}",{integer}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{function_propagation_schema,func_3}",{integer}) @@ -134,7 +134,7 @@ BEGIN return 1; END; $$; -WARNING: Citus can't distribute function "func_4" having dependency on non-distributed relation "function_prop_table" +WARNING: Citus can't distribute function "func_4" having dependency on non-distributed relation "function_prop_table" DETAIL: Function will be created only locally HINT: To distribute function, distribute dependent relations first. Then, re-create the function CREATE OR REPLACE FUNCTION func_5(param_1 int) @@ -163,13 +163,13 @@ BEGIN return 1; END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table}) @@ -210,12 +210,12 @@ BEGIN; END; $$; -- Within transaction functions are not distributed - SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) - SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) @@ -230,26 +230,26 @@ BEGIN return 1; END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (type,{function_propagation_schema.type_in_transaction},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (type,{function_propagation_schema.type_in_transaction},{}) localhost | 57638 | t | (type,{function_propagation_schema.type_in_transaction},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction}) @@ -265,18 +265,18 @@ $$ SELECT max(id) FROM table_in_sql_body $$; -- Show that only function has propagated, since the table is not resolved as dependency -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{function_propagation_schema,max_of_table}",{}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{function_propagation_schema,max_of_table}",{}) @@ -316,6 +316,878 @@ 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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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: Relation "table_to_prop_func_3" has dependency to a table "non_dist_table" that is not in Citus' metadata +HINT: Distribute dependent relation first. +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 pg_catalog.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 pg_catalog.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 pg_catalog.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 a column with default function depending on non-distributable table should fail +BEGIN; + CREATE TABLE non_dist_table_for_function(id int); + CREATE OR REPLACE FUNCTION non_dist_func(col_1 non_dist_table_for_function) + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + CREATE TABLE table_to_dist(id int); + SELECT create_distributed_table('table_to_dist', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + ALTER TABLE table_to_dist ADD COLUMN col_1 int default function_propagation_schema.non_dist_func(NULL::non_dist_table_for_function); +ERROR: Relation "table_to_dist" has dependency to a table "non_dist_table_for_function" that is not in Citus' metadata +HINT: Distribute dependent relation first. +ROLLBACK; +-- 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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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: Relation "table_to_prop_func_9" has dependency to a table "local_table_for_const" that is not in Citus' metadata +HINT: Distribute dependent relation first. +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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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) + +-- Show that having a dependency on another dist table work out tx +CREATE TABLE loc_for_func_dist ( + product_no integer, + name text, + price numeric CONSTRAINT positive_price CHECK (price > 0)); +SELECT create_distributed_table('loc_for_func_dist', 'product_no'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION non_sense_func_for_default_val(loc_for_func_dist) +RETURNS int +LANGUAGE plpgsql IMMUTABLE AS +$$ +BEGIN +return 1; +END; +$$; +CREATE TABLE table_non_for_func_dist ( + a int, + b int DEFAULT non_sense_func_for_default_val(NULL::loc_for_func_dist)); +SELECT create_distributed_table('table_non_for_func_dist', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET citus.shard_replication_factor = 1; +-- test creating a colocated function +CREATE TABLE tbl_to_colocate (a int); +SELECT create_distributed_table('tbl_to_colocate', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- first test colocating function with a ref table +CREATE TABLE tbl_to_colocate_ref (a int); +SELECT create_reference_table('tbl_to_colocate_ref'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql; +-- see the empty pg_dist_object entries +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + | | +(1 row) + +-- colocate the function with ref table +SELECT create_distributed_function('func_to_colocate(int)', colocate_with:='tbl_to_colocate_ref'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- see the pg_dist_object entry +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + | 10003 | +(1 row) + +-- convert to non-delegated +SELECT create_distributed_function('func_to_colocate(int)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- show that the pg_dist_object fields are gone +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + | | +(1 row) + +-- colocate the function with distributed table +SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- see the pg_dist_object entry +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + 0 | 10005 | +(1 row) + +-- try create or replace the same func +CREATE OR REPLACE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql; +-- verify the pg_dist_object entry is the same +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + 0 | 10005 | +(1 row) + +-- convert to non-delegated +SELECT create_distributed_function('func_to_colocate(int)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- show that the pg_dist_object fields are gone +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + | | +(1 row) + +-- force delegate +SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate', true); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- show pg_dist_object fields +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + 0 | 10005 | t +(1 row) + +-- convert to non-delegated +SELECT create_distributed_function('func_to_colocate(int)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- show that the pg_dist_object fields are gone +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + distribution_argument_index | colocationid | force_delegation +--------------------------------------------------------------------- + | | +(1 row) + RESET search_path; SET client_min_messages TO WARNING; DROP SCHEMA function_propagation_schema CASCADE; diff --git a/src/test/regress/expected/global_cancel.out b/src/test/regress/expected/global_cancel.out index 5ebc4098d..ede4bc7e3 100644 --- a/src/test/regress/expected/global_cancel.out +++ b/src/test/regress/expected/global_cancel.out @@ -45,7 +45,7 @@ SELECT 1 FROM run_command_on_workers('CREATE USER global_cancel_user'); (2 rows) RESET client_min_messages; -SET ROLE global_cancel_user; +\c - global_cancel_user - :master_port SELECT pg_typeof(:maintenance_daemon_gpid); pg_typeof --------------------------------------------------------------------- @@ -58,7 +58,10 @@ CONTEXT: while executing command on localhost:xxxxx SELECT pg_terminate_backend(:maintenance_daemon_gpid); ERROR: must be a superuser to terminate superuser process CONTEXT: while executing command on localhost:xxxxx -RESET ROLE; +-- we can cancel our own backend +SELECT pg_cancel_backend(citus_backend_gpid()); +ERROR: canceling statement due to user request +\c - postgres - :master_port SELECT nodeid AS coordinator_node_id FROM pg_dist_node WHERE nodeport = :master_port \gset SET client_min_messages TO DEBUG; -- 10000000000 is the node id multiplier for global pid @@ -79,5 +82,33 @@ DETAIL: from localhost:xxxxx (1 row) RESET client_min_messages; +SELECT citus_backend_gpid() = citus_calculate_gpid(:coordinator_node_id, pg_backend_pid()); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT nodename = citus_nodename_for_nodeid(nodeid) AND nodeport = citus_nodeport_for_nodeid(nodeid) +FROM pg_dist_node +WHERE isactive = true AND noderole = 'primary'; + ?column? +--------------------------------------------------------------------- + t + t + t +(3 rows) + +SELECT citus_nodeid_for_gpid(10000000000 * 2 + 3); + citus_nodeid_for_gpid +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT citus_pid_for_gpid(10000000000 * 2 + 3); + citus_pid_for_gpid +--------------------------------------------------------------------- + 3 +(1 row) + DROP SCHEMA global_cancel CASCADE; -NOTICE: drop cascades to table dist_table +NOTICE: drop cascades to table global_cancel.dist_table diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index afa54b7e8..856690159 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -1182,6 +1182,7 @@ END; $$ LANGUAGE plpgsql STABLE; SELECT create_distributed_function('dist_func(int, int)'); +NOTICE: procedure insert_select_repartition.dist_func is already distributed create_distributed_function --------------------------------------------------------------------- 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..171d2faef 100644 --- a/src/test/regress/expected/isolation_citus_dist_activity.out +++ b/src/test/regress/expected/isolation_citus_dist_activity.out @@ -1,4 +1,25 @@ -Parsed test spec with 3 sessions +Parsed test spec with 4 sessions + +starting permutation: add-coordinator-to-metadata +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step add-coordinator-to-metadata: + SELECT 1 FROM citus_add_node('localhost', 57636, groupid:=0); + SELECT test_assign_global_pid(); + +?column? +--------------------------------------------------------------------- + 1 +(1 row) + +test_assign_global_pid +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-alter-table s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback create_distributed_table @@ -32,32 +53,32 @@ 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 '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; + SELECT query, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%'), ('%citus_add_node%')) 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- ALTER TABLE test_table ADD COLUMN x INT; -|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (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, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -SELECT worker_apply_shard_ddl_command (1300004, 'public', ' +SELECT worker_apply_shard_ddl_command (1300009, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; -')|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -SELECT worker_apply_shard_ddl_command (1300003, 'public', ' +')|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT worker_apply_shard_ddl_command (1300008, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; -')|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -SELECT worker_apply_shard_ddl_command (1300002, 'public', ' +')|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT worker_apply_shard_ddl_command (1300007, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; -')|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -SELECT worker_apply_shard_ddl_command (1300001, 'public', ' +')|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT worker_apply_shard_ddl_command (1300006, 'public', ' ALTER TABLE test_table ADD COLUMN x INT; -')|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +')|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (4 rows) step s2-rollback: @@ -102,21 +123,21 @@ 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 '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; + SELECT query, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%'), ('%citus_add_node%')) 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- INSERT INTO test_table VALUES (100, 100); -|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (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, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -INSERT INTO public.test_table_1300008 (column1, column2) VALUES (100, 100)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +INSERT INTO public.test_table_1300013 (column1, column2) VALUES (100, 100)|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (1 row) step s2-rollback: @@ -166,24 +187,24 @@ 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 '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; + SELECT query, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%'), ('%citus_add_node%')) 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- SELECT count(*) FROM test_table; -|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (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, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -SELECT count(*) AS count FROM public.test_table_1300014 test_table WHERE true|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -SELECT count(*) AS count FROM public.test_table_1300013 test_table WHERE true|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -SELECT count(*) AS count FROM public.test_table_1300012 test_table WHERE true|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -SELECT count(*) AS count FROM public.test_table_1300011 test_table WHERE true|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT count(*) AS count FROM public.test_table_1300019 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT count(*) AS count FROM public.test_table_1300018 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT count(*) AS count FROM public.test_table_1300017 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT count(*) AS count FROM public.test_table_1300016 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (4 rows) step s2-rollback: @@ -233,21 +254,21 @@ 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 '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; + SELECT query, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%'), ('%citus_add_node%')) 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- SELECT count(*) FROM test_table WHERE column1 = 55; -|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (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, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true 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 +query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -SELECT count(*) AS count FROM public.test_table_1300017 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +SELECT count(*) AS count FROM public.test_table_1300022 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression (1 row) step s2-rollback: @@ -259,3 +280,18 @@ step s1-commit: step s3-rollback: ROLLBACK; + +starting permutation: remove-coordinator-from-metadata +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step remove-coordinator-from-metadata: + SELECT citus_remove_node('localhost', 57636); + +citus_remove_node +--------------------------------------------------------------------- + +(1 row) + 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_ensure_dependency_activate_node.out b/src/test/regress/expected/isolation_ensure_dependency_activate_node.out index d8c1a5a74..fda0ead1d 100644 --- a/src/test/regress/expected/isolation_ensure_dependency_activate_node.out +++ b/src/test/regress/expected/isolation_ensure_dependency_activate_node.out @@ -9,7 +9,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -102,7 +102,7 @@ create_distributed_table step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -180,7 +180,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -279,7 +279,7 @@ step s2-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -357,7 +357,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-public-s step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -456,7 +456,7 @@ step s1-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -534,7 +534,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-cre step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -628,7 +628,7 @@ create_distributed_table step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -707,7 +707,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -807,7 +807,7 @@ step s2-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -886,7 +886,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -986,7 +986,7 @@ step s1-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1065,7 +1065,7 @@ starting permutation: s1-print-distributed-objects s2-create-schema s1-begin s2- step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1165,7 +1165,7 @@ step s2-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1244,7 +1244,7 @@ starting permutation: s1-print-distributed-objects s1-add-worker s2-create-schem step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1358,7 +1358,7 @@ step s3-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1438,7 +1438,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1 step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1560,7 +1560,7 @@ step s2-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1641,7 +1641,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1726,7 +1726,7 @@ step s1-commit: step s2-create-type: <... completed> step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1804,7 +1804,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2- step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1888,7 +1888,7 @@ step s1-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -1966,7 +1966,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -2069,7 +2069,7 @@ step s1-commit: step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -2149,7 +2149,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -2254,7 +2254,7 @@ wait_until_metadata_sync step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -2332,7 +2332,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2- step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -2444,7 +2444,7 @@ wait_until_metadata_sync step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -2522,7 +2522,7 @@ starting permutation: s1-print-distributed-objects s2-begin s2-create-schema s2- step s1-print-distributed-objects: SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); @@ -2635,7 +2635,7 @@ wait_until_metadata_sync step s2-print-distributed-objects: -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); diff --git a/src/test/regress/expected/isolation_extension_commands.out b/src/test/regress/expected/isolation_extension_commands.out index 711c854c7..b8ab7bc94 100644 --- a/src/test/regress/expected/isolation_extension_commands.out +++ b/src/test/regress/expected/isolation_extension_commands.out @@ -20,7 +20,7 @@ step s1-commit: step s2-create-extension-version-11: <... completed> step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -81,7 +81,7 @@ step s1-commit: step s2-alter-extension-update-to-version-12: <... completed> step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -150,7 +150,7 @@ step s1-commit: step s2-drop-extension: <... completed> step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -206,7 +206,7 @@ step s1-commit: step s2-create-extension-with-schema1: <... completed> step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -267,7 +267,7 @@ step s1-commit: step s2-drop-extension: <... completed> step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -338,7 +338,7 @@ step s1-commit: step s2-alter-extension-set-schema3: <... completed> step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -406,7 +406,7 @@ step s1-commit: step s2-create-extension-with-schema1: <... completed> step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -481,7 +481,7 @@ step s2-commit: COMMIT; step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -564,7 +564,7 @@ step s1-add-node-1: <... completed> (1 row) step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -633,7 +633,7 @@ step s1-remove-node-1: <... completed> (1 row) step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -688,7 +688,7 @@ step s2-commit: COMMIT; step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -763,7 +763,7 @@ step s1-remove-node-1: <... completed> (1 row) step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -830,7 +830,7 @@ step s2-commit: COMMIT; step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); @@ -909,7 +909,7 @@ step s1-add-node-1: <... completed> (1 row) step s1-print: - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); 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..8cbedbf37 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 @@ -1,4 +1,4 @@ -Parsed test spec with 4 sessions +Parsed test spec with 8 sessions starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-commit-worker s2-stop-connection step s1-begin: @@ -27,13 +27,13 @@ step s2-update-ref-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 15; -|localhost |coordinator_host | 57638| 57636 + (1 row) step s1-commit: @@ -61,7 +61,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -112,11 +112,11 @@ step s2-update-ref-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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 (1 row) step s1-commit-worker: @@ -157,7 +157,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -208,11 +208,11 @@ step s2-update-dist-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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 (1 row) step s1-commit-worker: @@ -253,7 +253,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -304,11 +304,11 @@ step s2-update-ref-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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 (1 row) step s1-commit-worker: @@ -349,7 +349,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -400,11 +400,11 @@ step s2-update-ref-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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) (1 row) step s1-commit-worker: @@ -445,7 +445,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -501,9 +501,9 @@ run_commands_on_session_level_connection_to_node (1 row) step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement|current_statement_in_blocking_process --------------------------------------------------------------------- (0 rows) @@ -539,7 +539,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -590,11 +590,11 @@ step s2-update-ref-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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 (1 row) step s1-commit-worker: @@ -635,7 +635,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -691,9 +691,9 @@ run_commands_on_session_level_connection_to_node (1 row) step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement|current_statement_in_blocking_process --------------------------------------------------------------------- (0 rows) @@ -729,7 +729,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -785,9 +785,9 @@ run_commands_on_session_level_connection_to_node (1 row) step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement|current_statement_in_blocking_process --------------------------------------------------------------------- (0 rows) @@ -823,7 +823,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -874,11 +874,11 @@ step s2-update-ref-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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 (1 row) step s1-commit-worker: @@ -919,7 +919,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -957,13 +957,13 @@ step s1-alter-table: ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- 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) (1 row) step s2-commit-worker: @@ -986,7 +986,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -1003,22 +1003,22 @@ step s2-update-on-the-coordinator: UPDATE tt1 SET value_1 = 4; step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- UPDATE tt1 SET value_1 = 4; | UPDATE tt1 SET value_1 = 4; - |coordinator_host |coordinator_host | 57636| 57636 + (1 row) step s1-commit: COMMIT; step s2-update-on-the-coordinator: <... completed> -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -1069,11 +1069,11 @@ step s4-update-dist-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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 (1 row) step s1-commit-worker: @@ -1114,7 +1114,7 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) @@ -1157,11 +1157,11 @@ step s2-update-dist-table-id-1: SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- -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 (1 row) step s1-commit-worker: @@ -1194,13 +1194,13 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) -starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-stop-connection +starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-stop-connection s5-begin s5-alter s6-select s3-select-distributed-waiting-queries s3-show-actual-gpids s5-rollback s8-begin s8-select s7-alter s3-select-distributed-waiting-queries s3-show-actual-gpids s8-rollback step s1-begin: BEGIN; @@ -1219,13 +1219,13 @@ step s2-update-ref-table: SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); step s3-select-distributed-waiting-queries: - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; -blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port +blocked_statement |current_statement_in_blocking_process --------------------------------------------------------------------- UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 15; -|localhost |coordinator_host | 57638| 57636 + (1 row) step s1-commit: @@ -1245,7 +1245,87 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +step s5-begin: + BEGIN; + +step s5-alter: + ALTER TABLE tt1 ADD COLUMN new_column INT; + +step s6-select: + SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1; + +step s3-select-distributed-waiting-queries: + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + +blocked_statement |current_statement_in_blocking_process +--------------------------------------------------------------------- + + SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1; +| + ALTER TABLE tt1 ADD COLUMN new_column INT; + +(1 row) + +step s3-show-actual-gpids: + SELECT global_pid > 0 as gpid_exists, query FROM citus_stat_activity WHERE state = 'active' AND query IN (SELECT blocked_statement FROM citus_lock_waits UNION SELECT current_statement_in_blocking_process FROM citus_lock_waits) ORDER BY 1 DESC; + +gpid_exists|query +--------------------------------------------------------------------- +f | + SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1; + +(1 row) + +step s5-rollback: + ROLLBACK; + +step s6-select: <... completed> +user_id +--------------------------------------------------------------------- + 7 +(1 row) + +step s8-begin: + BEGIN; + +step s8-select: + SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1; + +user_id +--------------------------------------------------------------------- + 7 +(1 row) + +step s7-alter: + ALTER TABLE tt1 ADD COLUMN new_column INT; + +step s3-select-distributed-waiting-queries: + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + +blocked_statement |current_statement_in_blocking_process +--------------------------------------------------------------------- + + ALTER TABLE tt1 ADD COLUMN new_column INT; +| + SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1; + +(1 row) + +step s3-show-actual-gpids: + SELECT global_pid > 0 as gpid_exists, query FROM citus_stat_activity WHERE state = 'active' AND query IN (SELECT blocked_statement FROM citus_lock_waits UNION SELECT current_statement_in_blocking_process FROM citus_lock_waits) ORDER BY 1 DESC; + +gpid_exists|query +--------------------------------------------------------------------- +t | + ALTER TABLE tt1 ADD COLUMN new_column INT; + +(1 row) + +step s8-rollback: + ROLLBACK; + +step s7-alter: <... completed> +citus_remove_node --------------------------------------------------------------------- (1 row) diff --git a/src/test/regress/expected/isolation_global_pid.out b/src/test/regress/expected/isolation_global_pid.out index 99ca48693..21573fd16 100644 --- a/src/test/regress/expected/isolation_global_pid.out +++ b/src/test/regress/expected/isolation_global_pid.out @@ -1,6 +1,6 @@ Parsed test spec with 2 sessions -starting permutation: s1-start-session-level-connection s1-worker-begin s1-worker-select s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_worker_stat_activity s1-worker-commit s1-stop-session-level-connection +starting permutation: s1-start-session-level-connection s1-worker-begin s1-worker-select s2-coordinator-citus_stat_activity s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_stat_activity-in-workers s1-worker-commit s1-stop-session-level-connection create_distributed_table --------------------------------------------------------------------- @@ -30,18 +30,32 @@ run_commands_on_session_level_connection_to_node (1 row) -step s2-coordinator-citus_dist_stat_activity: - SELECT global_pid != 0 FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' and query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; +step s2-coordinator-citus_stat_activity: + SELECT global_pid != 0 FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; ?column? --------------------------------------------------------------------- t (1 row) -step s2-coordinator-citus_worker_stat_activity: - SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN ( - SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' +step s2-coordinator-citus_dist_stat_activity: + SELECT query FROM citus_dist_stat_activity WHERE global_pid IN ( + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' ) + AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%' + ORDER BY 1; + +query +--------------------------------------------------------------------- +SET citus.enable_local_execution TO off; SET citus.force_max_query_parallelization TO ON; SELECT * FROM dist_table +(1 row) + +step s2-coordinator-citus_stat_activity-in-workers: + SELECT query FROM citus_stat_activity WHERE global_pid IN ( + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' + ) + AND is_worker_query = true + AND backend_type = 'client backend' ORDER BY 1; query @@ -68,13 +82,13 @@ stop_session_level_connection_to_node (1 row) -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) -starting permutation: s1-coordinator-begin s1-coordinator-select s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_worker_stat_activity s2-coordinator-get_all_active_transactions s2-coordinator-get_global_active_transactions s1-coordinator-commit +starting permutation: s1-coordinator-begin s1-coordinator-select s2-coordinator-citus_stat_activity s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_stat_activity-in-workers s2-coordinator-get_all_active_transactions s2-coordinator-get_global_active_transactions s1-coordinator-commit create_distributed_table --------------------------------------------------------------------- @@ -92,18 +106,36 @@ a|b --------------------------------------------------------------------- (0 rows) -step s2-coordinator-citus_dist_stat_activity: - SELECT global_pid != 0 FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' and query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; +step s2-coordinator-citus_stat_activity: + SELECT global_pid != 0 FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; ?column? --------------------------------------------------------------------- t (1 row) -step s2-coordinator-citus_worker_stat_activity: - SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN ( - SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' +step s2-coordinator-citus_dist_stat_activity: + SELECT query FROM citus_dist_stat_activity WHERE global_pid IN ( + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' ) + AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%' + ORDER BY 1; + +query +--------------------------------------------------------------------- + + SET citus.enable_local_execution TO off; + SET citus.force_max_query_parallelization TO ON; + SELECT * FROM dist_table; + +(1 row) + +step s2-coordinator-citus_stat_activity-in-workers: + SELECT query FROM citus_stat_activity WHERE global_pid IN ( + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' + ) + AND is_worker_query = true + AND backend_type = 'client backend' ORDER BY 1; query @@ -116,7 +148,7 @@ SELECT a, b FROM public.dist_table_12345003 dist_table WHERE true step s2-coordinator-get_all_active_transactions: SELECT count(*) FROM get_all_active_transactions() WHERE global_pid IN ( - SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' ); count @@ -126,7 +158,7 @@ count step s2-coordinator-get_global_active_transactions: SELECT count(*) FROM get_global_active_transactions() WHERE global_pid IN ( - SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' ) AND transaction_number != 0; @@ -138,7 +170,7 @@ count step s1-coordinator-commit: COMMIT; -restore_isolation_tester_func +citus_remove_node --------------------------------------------------------------------- (1 row) diff --git a/src/test/regress/expected/isolation_max_client_connections.out b/src/test/regress/expected/isolation_max_client_connections.out new file mode 100644 index 000000000..ada303511 --- /dev/null +++ b/src/test/regress/expected/isolation_max_client_connections.out @@ -0,0 +1,55 @@ +Parsed test spec with 3 sessions + +starting permutation: s1-grant s1-connect s2-connect s2-connect-superuser s3-select +run_command_on_workers +--------------------------------------------------------------------- +(localhost,57637,t,t) +(localhost,57638,t,t) +(2 rows) + +step s1-grant: + SELECT result FROM run_command_on_placements('my_table', 'GRANT SELECT ON TABLE %s TO my_user'); + +result +--------------------------------------------------------------------- +GRANT +GRANT +GRANT +GRANT +(4 rows) + +step s1-connect: + SELECT make_external_connection_to_node('localhost', 57637, 'my_user', current_database()); + +make_external_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-connect: + SELECT make_external_connection_to_node('localhost', 57637, 'my_user', current_database()); + +ERROR: connection failed +step s2-connect-superuser: + SELECT make_external_connection_to_node('localhost', 57637, 'postgres', current_database()); + +make_external_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s3-select: + SET ROLE my_user; + SELECT count(*) FROM my_table; + +count +--------------------------------------------------------------------- + 0 +(1 row) + +run_command_on_workers +--------------------------------------------------------------------- +(localhost,57637,t,t) +(localhost,57638,t,t) +(2 rows) + 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..a2792d0c4 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,32 +99,37 @@ 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, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE backend_type = 'client backend' AND query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%pg_isolation_test_session_is_blocked%'), ('%BEGIN%'), ('%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 +query |state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- SELECT check_distributed_deadlocks(); -|coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression +|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 + |idle in transaction|Client |ClientRead|postgres|regression (2 rows) step s2-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%' AND - query NOT ILIKE '%dump_local_%' AND - query NOT ILIKE '%citus_internal_local_blocked_processes%' - ORDER BY query, query_hostport DESC; + SELECT query, state, wait_event_type, wait_event, usename, datname + FROM citus_stat_activity + WHERE query NOT ILIKE ALL(VALUES + ('%pg_prepared_xacts%'), + ('%COMMIT%'), + ('%dump_local_%'), + ('%citus_internal_local_blocked_processes%'), + ('%add_node%'), + ('%csa_from_one_node%')) + AND is_worker_query = true + AND backend_type = 'client backend' + AND query != '' + 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 +query |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)|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|idle in transaction|Client |ClientRead|postgres|regression (2 rows) step s2-end: @@ -123,12 +144,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..5d29a4e71 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -55,6 +55,8 @@ BEGIN RETURN localGroupId; END; $$ language plpgsql VOLATILE; SELECT create_distributed_function('get_local_node_id_volatile()'); +NOTICE: procedure local_shard_execution.get_local_node_id_volatile is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -91,6 +93,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 +736,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..c297f0a99 100644 --- a/src/test/regress/expected/local_shard_execution_replicated.out +++ b/src/test/regress/expected/local_shard_execution_replicated.out @@ -60,6 +60,8 @@ BEGIN RETURN localGroupId; END; $$ language plpgsql VOLATILE; SELECT create_distributed_function('get_local_node_id_volatile()'); +NOTICE: procedure local_shard_execution_replicated.get_local_node_id_volatile is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -682,6 +684,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 +696,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; @@ -2241,6 +2286,8 @@ BEGIN END; $fn$; SELECT create_distributed_function('register_for_event(int,int,invite_resp)'); +NOTICE: procedure local_shard_execution_replicated.register_for_event is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/master_copy_shard_placement.out b/src/test/regress/expected/master_copy_shard_placement.out index 2e2f23e11..35ca01498 100644 --- a/src/test/regress/expected/master_copy_shard_placement.out +++ b/src/test/regress/expected/master_copy_shard_placement.out @@ -110,8 +110,8 @@ SELECT count(*) FROM history; -- test we can replicate MX tables SET citus.shard_replication_factor TO 1; -- metadata sync will succeed even if we have rep > 1 tables -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0); SELECT start_metadata_sync_to_node('localhost', :worker_1_port); start_metadata_sync_to_node --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_cluster_management.out b/src/test/regress/expected/multi_cluster_management.out index 5a5317e74..0903e0e36 100644 --- a/src/test/regress/expected/multi_cluster_management.out +++ b/src/test/regress/expected/multi_cluster_management.out @@ -101,6 +101,12 @@ SELECT * FROM rebalance_table_shards(); -- TODO: Figure out why this is necessary, rebalance_table_shards shouldn't -- insert stuff into pg_dist_colocation TRUNCATE pg_dist_colocation; +SELECT run_command_on_workers('TRUNCATE pg_dist_colocation'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"TRUNCATE TABLE") +(1 row) + ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000; SELECT 1 FROM citus_activate_node('localhost', :worker_2_port); ?column? @@ -236,8 +242,8 @@ SELECT master_remove_node('localhost', :worker_2_port); (1 row) -- Removing public schema from pg_dist_object because it breaks the next tests -DELETE FROM citus.pg_dist_object WHERE objid = 'public'::regnamespace::oid; -DELETE FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql'); +DELETE FROM pg_catalog.pg_dist_object WHERE objid = 'public'::regnamespace::oid; +DELETE FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql'); -- try to manipulate node metadata via non-super user SET ROLE non_super_user; SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port + 1); diff --git a/src/test/regress/expected/multi_colocation_utils.out b/src/test/regress/expected/multi_colocation_utils.out index 815908799..1fa58264c 100644 --- a/src/test/regress/expected/multi_colocation_utils.out +++ b/src/test/regress/expected/multi_colocation_utils.out @@ -2,6 +2,13 @@ SET citus.next_shard_id TO 1300000; ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 4; -- Delete orphaned entries from pg_dist_colocation DELETE FROM pg_dist_colocation where colocationid = 5 or colocationid = 6; +SELECT 1 FROM run_command_on_workers('DELETE FROM pg_dist_colocation where colocationid = 5 or colocationid = 6'); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + -- =================================================================== -- create test utility function -- =================================================================== @@ -54,16 +61,6 @@ CREATE FUNCTION find_shard_interval_index(bigint) RETURNS int AS 'citus' LANGUAGE C STRICT; --- remove tables from pg_dist_partition, if they don't exist i.e not found in pg_class -delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid); -select 1 from run_command_on_workers($$ - delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid);$$); - ?column? ---------------------------------------------------------------------- - 1 - 1 -(2 rows) - -- =================================================================== -- test co-location util functions -- =================================================================== @@ -356,6 +353,13 @@ SELECT count(*) FROM pg_dist_partition WHERE colocationid IN (4, 5); (1 row) DELETE FROM pg_dist_colocation WHERE colocationid IN (4, 5); +SELECT 1 FROM run_command_on_workers('DELETE FROM pg_dist_colocation WHERE colocationid IN (4, 5)'); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + SET citus.shard_count = 2; CREATE TABLE table1_groupA ( id int ); SELECT create_distributed_table('table1_groupA', 'id'); @@ -449,6 +453,23 @@ SELECT * FROM pg_dist_colocation 7 | 8 | 2 | 23 | 0 (4 rows) +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c) ORDER BY colocationid) FROM pg_dist_colocation c + WHERE colocationid >= 1 AND colocationid < 1000 +$$); + nodeport | unnest +--------------------------------------------------------------------- + 57637 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 5, "replicationfactor": 1, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 6, "replicationfactor": 2, "distributioncolumntype": "25", "distributioncolumncollation": "100"} + 57637 | {"shardcount": 8, "colocationid": 7, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 5, "replicationfactor": 1, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 6, "replicationfactor": 2, "distributioncolumntype": "25", "distributioncolumncollation": "100"} + 57638 | {"shardcount": 8, "colocationid": 7, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} +(8 rows) + SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY logicalrelid; @@ -472,6 +493,16 @@ SELECT * FROM pg_dist_colocation WHERE colocationid = 4; 4 | 2 | 2 | 23 | 0 (1 row) +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c)) FROM pg_dist_colocation c WHERE colocationid = 4 +$$); + nodeport | unnest +--------------------------------------------------------------------- + 57637 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} +(2 rows) + -- dropping all tables in a colocation group also deletes the colocation group DROP TABLE table2_groupA; SELECT * FROM pg_dist_colocation WHERE colocationid = 4; @@ -480,6 +511,16 @@ SELECT * FROM pg_dist_colocation WHERE colocationid = 4; 4 | 2 | 2 | 23 | 0 (1 row) +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c)) FROM pg_dist_colocation c WHERE colocationid = 4 +$$); + nodeport | unnest +--------------------------------------------------------------------- + 57637 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} +(2 rows) + -- create dropped colocation group again SET citus.shard_count = 2; CREATE TABLE table1_groupE ( id int ); @@ -820,6 +861,13 @@ ORDER BY ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1; DELETE FROM pg_dist_colocation WHERE colocationid >= 1 AND colocationid < 1000; +SELECT 1 FROM run_command_on_workers('DELETE FROM pg_dist_colocation WHERE colocationid >= 1 AND colocationid < 1000'); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + UPDATE pg_dist_partition SET colocationid = 0 WHERE colocationid >= 1 AND colocationid < 1000; -- check metadata @@ -933,6 +981,25 @@ SELECT * FROM pg_dist_colocation 5 | 2 | 2 | 23 | 0 (5 rows) +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c) ORDER BY colocationid) FROM pg_dist_colocation c + WHERE colocationid >= 1 AND colocationid < 1000 +$$); + nodeport | unnest +--------------------------------------------------------------------- + 57637 | {"shardcount": 2, "colocationid": 1, "replicationfactor": 1, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 2, "replicationfactor": 2, "distributioncolumntype": "25", "distributioncolumncollation": "100"} + 57637 | {"shardcount": 8, "colocationid": 3, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 5, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 1, "replicationfactor": 1, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 2, "replicationfactor": 2, "distributioncolumntype": "25", "distributioncolumncollation": "100"} + 57638 | {"shardcount": 8, "colocationid": 3, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 5, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} +(10 rows) + SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY colocationid, logicalrelid; @@ -971,11 +1038,11 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with => (1 row) -- activate nodes to get rid of inconsistencies in pg_dist tables -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0); SELECT 1 FROM citus_activate_node('localhost', :worker_1_port); ?column? --------------------------------------------------------------------- @@ -1008,6 +1075,25 @@ SELECT * FROM pg_dist_colocation 5 | 2 | 2 | 23 | 0 (5 rows) +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c) ORDER BY colocationid) FROM pg_dist_colocation c + WHERE colocationid >= 1 AND colocationid < 1000 +$$); + nodeport | unnest +--------------------------------------------------------------------- + 57637 | {"shardcount": 2, "colocationid": 1, "replicationfactor": 1, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 2, "replicationfactor": 2, "distributioncolumntype": "25", "distributioncolumncollation": "100"} + 57637 | {"shardcount": 8, "colocationid": 3, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57637 | {"shardcount": 2, "colocationid": 5, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 1, "replicationfactor": 1, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 2, "replicationfactor": 2, "distributioncolumntype": "25", "distributioncolumncollation": "100"} + 57638 | {"shardcount": 8, "colocationid": 3, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 4, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} + 57638 | {"shardcount": 2, "colocationid": 5, "replicationfactor": 2, "distributioncolumntype": "23", "distributioncolumncollation": "0"} +(10 rows) + SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY colocationid, logicalrelid; diff --git a/src/test/regress/expected/multi_deparse_function.out b/src/test/regress/expected/multi_deparse_function.out index 656c89c40..6ac5dcd17 100644 --- a/src/test/regress/expected/multi_deparse_function.out +++ b/src/test/regress/expected/multi_deparse_function.out @@ -683,6 +683,8 @@ CREATE FUNCTION func_custom_param(IN param intpair, OUT total INT) LANGUAGE SQL; SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_function('func_custom_param(intpair)'); +NOTICE: procedure function_tests.func_custom_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -697,6 +699,8 @@ CREATE FUNCTION func_returns_table(IN count INT) LANGUAGE SQL; SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_function('func_returns_table(INT)'); +NOTICE: procedure function_tests.func_returns_table is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index e5cbcd994..31f79c83a 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -77,8 +77,6 @@ END $func$ LANGUAGE plpgsql; CREATE SCHEMA test; :create_function_test_maintenance_worker -WARNING: Citus can't distribute functions having dependency on unsupported object of type "view" -DETAIL: Function will be created only locally -- check maintenance daemon is started SELECT datname, current_database(), usename, (SELECT extowner::regrole::text FROM pg_extension WHERE extname = 'citus') @@ -428,20 +426,20 @@ SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDE ALTER EXTENSION citus UPDATE TO '9.4-2'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -469,20 +467,20 @@ SELECT * FROM multi_extension.print_extension_changes(); ALTER EXTENSION citus UPDATE TO '9.4-1'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -494,30 +492,17 @@ SELECT * FROM multi_extension.print_extension_changes(); -- Test downgrade to 9.4-1 from 9.5-1 ALTER EXTENSION citus UPDATE TO '9.5-1'; -BEGIN; - SET citus.enable_metadata_sync TO on; - SELECT master_add_node('localhost', :master_port, groupId=>0); -NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata - master_add_node ---------------------------------------------------------------------- - 1 -(1 row) - - CREATE TABLE citus_local_table (a int); - SELECT create_citus_local_table('citus_local_table'); -NOTICE: create_citus_local_table is deprecated in favour of citus_add_local_table_to_metadata - create_citus_local_table ---------------------------------------------------------------------- - -(1 row) - - RESET citus.enable_metadata_sync; - -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table - ALTER EXTENSION citus UPDATE TO '9.4-1'; -ERROR: citus local tables are introduced in Citus 9.5 -HINT: To downgrade Citus to an older version, you should first convert each citus local table to a postgres table by executing SELECT undistribute_table("%s") -CONTEXT: PL/pgSQL function inline_code_block line XX at RAISE -ROLLBACK; +-- TODO: This test should be moved to a valid downgrade testing suite where the downgrade is done, both on the schema and the binaries. Later changes in Citus made a C vs Schema discrepancy error here +-- BEGIN; +-- SET citus.enable_metadata_sync TO on; +-- SELECT master_add_node('localhost', :master_port, groupId=>0); +-- CREATE TABLE citus_local_table (a int); +-- SELECT create_citus_local_table('citus_local_table'); +-- RESET citus.enable_metadata_sync; +-- +-- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table +-- ALTER EXTENSION citus UPDATE TO '9.4-1'; +-- ROLLBACK; -- now we can downgrade as there is no citus local table ALTER EXTENSION citus UPDATE TO '9.4-1'; -- Should be empty result since upgrade+downgrade should be a no-op @@ -578,20 +563,20 @@ SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDE ALTER EXTENSION citus UPDATE TO '9.5-2'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -619,20 +604,20 @@ SELECT * FROM multi_extension.print_extension_changes(); ALTER EXTENSION citus UPDATE TO '9.5-1'; -- should see the old source code SELECT prosrc FROM pg_proc WHERE proname = 'master_update_table_statistics' ORDER BY 1; - prosrc + prosrc --------------------------------------------------------------------- - + - DECLARE + - colocated_tables regclass[]; + - BEGIN + - SELECT get_colocated_table_array(relation) INTO colocated_tables;+ - PERFORM + - master_update_shard_statistics(shardid) + - FROM + - pg_dist_shard + - WHERE + - logicalrelid = ANY (colocated_tables); + - END; + + + + DECLARE + + colocated_tables regclass[]; + + BEGIN + + SELECT get_colocated_table_array(relation) INTO colocated_tables;+ + PERFORM + + master_update_shard_statistics(shardid) + + FROM + + pg_dist_shard + + WHERE + + logicalrelid = ANY (colocated_tables); + + END; + (1 row) @@ -966,7 +951,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 +986,49 @@ 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 citus_dist_stat_activity() SETOF record | + function citus_worker_stat_activity() SETOF record | + 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 | + table citus.pg_dist_object | + view citus_worker_stat_activity | + | function citus_backend_gpid() bigint + | function citus_calculate_gpid(integer,integer) bigint + | 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_colocation_metadata(integer,integer,integer,regtype,oid) void + | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void + | function citus_internal_delete_colocation_metadata(integer) void + | function citus_internal_global_blocked_processes() SETOF record + | function citus_internal_local_blocked_processes() SETOF record + | function citus_nodeid_for_gpid(bigint) integer + | function citus_nodename_for_nodeid(integer) text + | function citus_nodeport_for_nodeid(integer) integer + | function citus_pid_for_gpid(bigint) integer + | function citus_run_local_command(text) void + | function citus_shard_indexes_on_worker() SETOF record + | function citus_shards_on_worker() SETOF record + | function citus_stat_activity() SETOF record + | function create_distributed_function(regprocedure,text,text,boolean) void + | function get_nodeid_for_groupid(integer) integer + | function pg_cancel_backend(bigint) boolean + | function pg_terminate_backend(bigint,bigint) boolean + | function run_command_on_all_nodes(text,boolean,boolean) SETOF record + | 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 + | table pg_dist_object + | view citus_stat_activity +(40 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version @@ -1113,8 +1118,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 +1127,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 @@ -1215,8 +1219,6 @@ HINT: You can manually create a database and its extensions on workers. CREATE EXTENSION citus; CREATE SCHEMA test; :create_function_test_maintenance_worker -WARNING: Citus can't distribute functions having dependency on unsupported object of type "view" -DETAIL: Function will be created only locally -- see that the daemon started SELECT datname, current_database(), usename, (SELECT extowner::regrole::text FROM pg_extension WHERE extname = 'citus') diff --git a/src/test/regress/expected/multi_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_function_evaluation.out b/src/test/regress/expected/multi_function_evaluation.out index 1b2531060..93fdfbffa 100644 --- a/src/test/regress/expected/multi_function_evaluation.out +++ b/src/test/regress/expected/multi_function_evaluation.out @@ -169,6 +169,8 @@ BEGIN END; $function$; SELECT create_distributed_function('stable_squared(int)'); +NOTICE: procedure multi_function_evaluation.stable_squared is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_index_statements.out b/src/test/regress/expected/multi_index_statements.out index aab013acf..59a2626af 100644 --- a/src/test/regress/expected/multi_index_statements.out +++ b/src/test/regress/expected/multi_index_statements.out @@ -84,6 +84,8 @@ BEGIN END; $$ LANGUAGE plpgsql; SELECT create_distributed_function('value_plus_one(int)'); +NOTICE: procedure multi_index_statements.value_plus_one is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -96,6 +98,8 @@ BEGIN END; $$ LANGUAGE plpgsql; SELECT create_distributed_function('multi_index_statements_2.value_plus_one(int)'); +NOTICE: procedure multi_index_statements_2.value_plus_one is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_metadata_attributes.out b/src/test/regress/expected/multi_metadata_attributes.out index bf75e23ef..3ce512c2b 100644 --- a/src/test/regress/expected/multi_metadata_attributes.out +++ b/src/test/regress/expected/multi_metadata_attributes.out @@ -9,7 +9,7 @@ FROM pg_attribute WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, 'pg_dist_rebalance_strategy'::regclass, 'pg_dist_partition'::regclass, - 'citus.pg_dist_object'::regclass) + 'pg_dist_object'::regclass) ORDER BY attrelid, attname; attrelid | attname | atthasmissing | attmissingval --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_metadata_sync.out b/src/test/regress/expected/multi_metadata_sync.out index be923eee3..e85620826 100644 --- a/src/test/regress/expected/multi_metadata_sync.out +++ b/src/test/regress/expected/multi_metadata_sync.out @@ -59,10 +59,11 @@ ALTER ROLE CURRENT_USER WITH PASSWORD 'dummypassword'; -- Show that, with no MX tables, activate node snapshot contains only the delete commands, -- pg_dist_node entries, pg_dist_object entries and roles. SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres - DELETE FROM citus.pg_dist_object + DELETE FROM pg_catalog.pg_dist_colocation + DELETE FROM pg_catalog.pg_dist_object DELETE FROM pg_dist_node DELETE FROM pg_dist_partition DELETE FROM pg_dist_placement @@ -87,8 +88,9 @@ SELECT unnest(activate_node_snapshot()) order by 1; SET citus.enable_ddl_propagation TO 'on' SET citus.enable_ddl_propagation TO 'on' UPDATE pg_dist_local_group SET groupid = 1 + WITH colocation_group_data (colocationid, shardcount, replicationfactor, distributioncolumntype, distributioncolumncollationname, distributioncolumncollationschema) AS (VALUES (1, 1, -1, 0, NULL, NULL)) SELECT pg_catalog.citus_internal_add_colocation_metadata(colocationid, shardcount, replicationfactor, distributioncolumntype, coalesce(c.oid, 0)) FROM colocation_group_data d LEFT JOIN pg_collation c ON (d.distributioncolumncollationname = c.collname AND d.distributioncolumncollationschema::regnamespace = c.collnamespace) WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; -(27 rows) +(29 rows) -- this function is dropped in Citus10, added here for tests SET citus.enable_metadata_sync TO OFF; @@ -121,7 +123,7 @@ reset citus.shard_replication_factor; UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid='mx_test_table'::regclass; -- Show that the created MX table is and its sequences are included in the activate node snapshot SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres ALTER SEQUENCE public.user_defined_seq OWNER TO postgres @@ -129,7 +131,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; ALTER TABLE public.mx_test_table OWNER TO postgres CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass)) - DELETE FROM citus.pg_dist_object + DELETE FROM pg_catalog.pg_dist_colocation + DELETE FROM pg_catalog.pg_dist_object DELETE FROM pg_dist_node DELETE FROM pg_dist_partition DELETE FROM pg_dist_placement @@ -159,15 +162,16 @@ SELECT unnest(activate_node_snapshot()) order by 1; SET citus.enable_ddl_propagation TO 'on' SET citus.enable_ddl_propagation TO 'on' UPDATE pg_dist_local_group SET groupid = 1 + WITH colocation_group_data (colocationid, shardcount, replicationfactor, distributioncolumntype, distributioncolumncollationname, distributioncolumncollationschema) AS (VALUES (1, 1, -1, 0, NULL, NULL), (2, 8, 1, 'integer'::regtype, NULL, NULL)) SELECT pg_catalog.citus_internal_add_colocation_metadata(colocationid, shardcount, replicationfactor, distributioncolumntype, coalesce(c.oid, 0)) FROM colocation_group_data d LEFT JOIN pg_collation c ON (d.distributioncolumncollationname = c.collname AND d.distributioncolumncollationschema::regnamespace = c.collnamespace) WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; -(39 rows) +(41 rows) -- Show that CREATE INDEX commands are included in the activate node snapshot CREATE INDEX mx_index ON mx_test_table(col_2); SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres ALTER SEQUENCE public.user_defined_seq OWNER TO postgres @@ -176,7 +180,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; CREATE INDEX mx_index ON public.mx_test_table USING btree (col_2) CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass)) - DELETE FROM citus.pg_dist_object + DELETE FROM pg_catalog.pg_dist_colocation + DELETE FROM pg_catalog.pg_dist_object DELETE FROM pg_dist_node DELETE FROM pg_dist_partition DELETE FROM pg_dist_placement @@ -206,16 +211,17 @@ SELECT unnest(activate_node_snapshot()) order by 1; SET citus.enable_ddl_propagation TO 'on' SET citus.enable_ddl_propagation TO 'on' UPDATE pg_dist_local_group SET groupid = 1 + WITH colocation_group_data (colocationid, shardcount, replicationfactor, distributioncolumntype, distributioncolumncollationname, distributioncolumncollationschema) AS (VALUES (1, 1, -1, 0, NULL, NULL), (2, 8, 1, 'integer'::regtype, NULL, NULL)) SELECT pg_catalog.citus_internal_add_colocation_metadata(colocationid, shardcount, replicationfactor, distributioncolumntype, coalesce(c.oid, 0)) FROM colocation_group_data d LEFT JOIN pg_collation c ON (d.distributioncolumncollationname = c.collname AND d.distributioncolumncollationschema::regnamespace = c.collnamespace) WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; -(40 rows) +(42 rows) -- Show that schema changes are included in the activate node snapshot CREATE SCHEMA mx_testing_schema; ALTER TABLE mx_test_table SET SCHEMA mx_testing_schema; SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres ALTER SEQUENCE public.user_defined_seq OWNER TO postgres @@ -225,7 +231,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass)) - DELETE FROM citus.pg_dist_object + DELETE FROM pg_catalog.pg_dist_colocation + DELETE FROM pg_catalog.pg_dist_object DELETE FROM pg_dist_node DELETE FROM pg_dist_partition DELETE FROM pg_dist_placement @@ -255,10 +262,11 @@ SELECT unnest(activate_node_snapshot()) order by 1; SET citus.enable_ddl_propagation TO 'on' SET citus.enable_ddl_propagation TO 'on' UPDATE pg_dist_local_group SET groupid = 1 + WITH colocation_group_data (colocationid, shardcount, replicationfactor, distributioncolumntype, distributioncolumncollationname, distributioncolumncollationschema) AS (VALUES (1, 1, -1, 0, NULL, NULL), (2, 8, 1, 'integer'::regtype, NULL, NULL)) SELECT pg_catalog.citus_internal_add_colocation_metadata(colocationid, shardcount, replicationfactor, distributioncolumntype, coalesce(c.oid, 0)) FROM colocation_group_data d LEFT JOIN pg_collation c ON (d.distributioncolumncollationname = c.collname AND d.distributioncolumncollationschema::regnamespace = c.collnamespace) WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; -(41 rows) +(43 rows) -- Show that append distributed tables are not included in the activate node snapshot CREATE TABLE non_mx_test_table (col_1 int, col_2 text); @@ -270,7 +278,7 @@ SELECT master_create_distributed_table('non_mx_test_table', 'col_1', 'append'); UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid='non_mx_test_table'::regclass; SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres ALTER SEQUENCE public.user_defined_seq OWNER TO postgres @@ -280,7 +288,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass)) - DELETE FROM citus.pg_dist_object + DELETE FROM pg_catalog.pg_dist_colocation + DELETE FROM pg_catalog.pg_dist_object DELETE FROM pg_dist_node DELETE FROM pg_dist_partition DELETE FROM pg_dist_placement @@ -310,15 +319,16 @@ SELECT unnest(activate_node_snapshot()) order by 1; SET citus.enable_ddl_propagation TO 'on' SET citus.enable_ddl_propagation TO 'on' UPDATE pg_dist_local_group SET groupid = 1 + WITH colocation_group_data (colocationid, shardcount, replicationfactor, distributioncolumntype, distributioncolumncollationname, distributioncolumncollationschema) AS (VALUES (1, 1, -1, 0, NULL, NULL), (2, 8, 1, 'integer'::regtype, NULL, NULL)) SELECT pg_catalog.citus_internal_add_colocation_metadata(colocationid, shardcount, replicationfactor, distributioncolumntype, coalesce(c.oid, 0)) FROM colocation_group_data d LEFT JOIN pg_collation c ON (d.distributioncolumncollationname = c.collname AND d.distributioncolumncollationschema::regnamespace = c.collnamespace) WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; -(41 rows) +(43 rows) -- Show that range distributed tables are not included in the activate node snapshot UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass; SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres ALTER SEQUENCE public.user_defined_seq OWNER TO postgres @@ -328,7 +338,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass)) - DELETE FROM citus.pg_dist_object + DELETE FROM pg_catalog.pg_dist_colocation + DELETE FROM pg_catalog.pg_dist_object DELETE FROM pg_dist_node DELETE FROM pg_dist_partition DELETE FROM pg_dist_placement @@ -358,10 +369,11 @@ SELECT unnest(activate_node_snapshot()) order by 1; SET citus.enable_ddl_propagation TO 'on' SET citus.enable_ddl_propagation TO 'on' UPDATE pg_dist_local_group SET groupid = 1 + WITH colocation_group_data (colocationid, shardcount, replicationfactor, distributioncolumntype, distributioncolumncollationname, distributioncolumncollationschema) AS (VALUES (1, 1, -1, 0, NULL, NULL), (2, 8, 1, 'integer'::regtype, NULL, NULL)) SELECT pg_catalog.citus_internal_add_colocation_metadata(colocationid, shardcount, replicationfactor, distributioncolumntype, coalesce(c.oid, 0)) FROM colocation_group_data d LEFT JOIN pg_collation c ON (d.distributioncolumncollationname = c.collname AND d.distributioncolumncollationschema::regnamespace = c.collnamespace) WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; -(41 rows) +(43 rows) -- Test start_metadata_sync_to_node and citus_activate_node UDFs -- Ensure that hasmetadata=false for all nodes @@ -497,11 +509,13 @@ SELECT "Column", "Type", "Definition" FROM index_attrs WHERE col_2 | text | col_2 (1 row) --- Check that pg_dist_colocation is not synced +-- Check that pg_dist_colocation is synced SELECT * FROM pg_dist_colocation ORDER BY colocationid; colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation --------------------------------------------------------------------- -(0 rows) + 1 | 1 | -1 | 0 | 0 + 2 | 8 | 1 | 23 | 0 +(2 rows) -- Make sure that truncate trigger has been set for the MX table on worker SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table'::regclass; @@ -1365,10 +1379,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence; -- owner CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement; CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition; -CREATE TABLE pg_dist_object_temp AS SELECT * FROM citus.pg_dist_object; +CREATE TABLE pg_dist_object_temp AS SELECT * FROM pg_catalog.pg_dist_object; DELETE FROM pg_dist_placement; DELETE FROM pg_dist_partition; -DELETE FROM citus.pg_dist_object; +DELETE FROM pg_catalog.pg_dist_object; SELECT groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset SELECT master_remove_node('localhost', :worker_2_port); master_remove_node @@ -1445,7 +1459,7 @@ DROP TABLE mx_table; \c - postgres - :master_port INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp; INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_temp; -INSERT INTO citus.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING; +INSERT INTO pg_catalog.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING; DROP TABLE pg_dist_placement_temp; DROP TABLE pg_dist_partition_temp; DROP TABLE pg_dist_object_temp; @@ -1522,6 +1536,13 @@ ORDER BY (2 rows) SELECT shardid AS ref_table_shardid FROM pg_dist_shard WHERE logicalrelid='mx_ref'::regclass \gset +-- make sure we have the pg_dist_colocation record on the worker +SELECT count(*) FROM pg_dist_colocation WHERE distributioncolumntype = 0; + count +--------------------------------------------------------------------- + 1 +(1 row) + -- Check that DDL commands are propagated to reference tables on workers \c - - - :master_port ALTER TABLE mx_ref ADD COLUMN col_3 NUMERIC DEFAULT 0; @@ -1823,7 +1844,7 @@ ALTER TABLE test_table ADD COLUMN id2 int DEFAULT nextval('mx_test_sequence_1'); ALTER TABLE test_table ALTER COLUMN id2 DROP DEFAULT; ALTER TABLE test_table ALTER COLUMN id2 SET DEFAULT nextval('mx_test_sequence_1'); SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres ALTER SEQUENCE public.mx_test_sequence_0 OWNER TO postgres @@ -1854,7 +1875,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; CREATE TABLE public.dist_table_1 (a integer) CREATE TABLE public.mx_ref (col_1 integer, col_2 text) CREATE TABLE public.test_table (id integer DEFAULT worker_nextval('public.mx_test_sequence_0'::regclass), id2 integer DEFAULT worker_nextval('public.mx_test_sequence_1'::regclass)) - DELETE FROM citus.pg_dist_object + DELETE FROM pg_catalog.pg_dist_colocation + DELETE FROM pg_catalog.pg_dist_object DELETE FROM pg_dist_node DELETE FROM pg_dist_partition DELETE FROM pg_dist_placement @@ -1896,6 +1918,7 @@ SELECT unnest(activate_node_snapshot()) order by 1; SET citus.enable_ddl_propagation TO 'on' SET citus.enable_ddl_propagation TO 'on' UPDATE pg_dist_local_group SET groupid = 1 + WITH colocation_group_data (colocationid, shardcount, replicationfactor, distributioncolumntype, distributioncolumncollationname, distributioncolumncollationschema) AS (VALUES (10002, 7, 1, 'integer'::regtype, NULL, NULL), (10003, 1, -1, 0, NULL, NULL), (10004, 3, 1, 'integer'::regtype, NULL, NULL), (10005, 4, 1, 'integer'::regtype, NULL, NULL)) SELECT pg_catalog.citus_internal_add_colocation_metadata(colocationid, shardcount, replicationfactor, distributioncolumntype, coalesce(c.oid, 0)) FROM colocation_group_data d LEFT JOIN pg_collation c ON (d.distributioncolumncollationname = c.collname AND d.distributioncolumncollationschema::regnamespace = c.collnamespace) WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_test_schema_1', 'mx_table_1']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_test_schema_2', 'mx_table_2']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'mx_ref']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'dist_table_1']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_0']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_1']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 5, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 5, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 5, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 5, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310020, 1, 0, 1, 100020), (1310021, 1, 0, 5, 100021), (1310022, 1, 0, 1, 100022), (1310023, 1, 0, 5, 100023), (1310024, 1, 0, 1, 100024)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data; @@ -1909,7 +1932,7 @@ SELECT unnest(activate_node_snapshot()) order by 1; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310074, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310075, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310076, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310073, 't'::"char", NULL, NULL)) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310083, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310084, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310085, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310086, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; -(84 rows) +(86 rows) -- shouldn't work since test_table is MX ALTER TABLE test_table ADD COLUMN id3 bigserial; @@ -1928,8 +1951,8 @@ ALTER TABLE test_table ADD COLUMN id4 bigserial CHECK (id4 > 0); ERROR: cannot execute ADD COLUMN commands involving serial pseudotypes when metadata is synchronized to workers \c - - - :worker_1_port \ds - List of relations - Schema | Name | Type | Owner + List of relations + Schema | Name | Type | Owner --------------------------------------------------------------------- public | mx_test_sequence_0 | sequence | postgres public | mx_test_sequence_1 | sequence | postgres @@ -1949,8 +1972,8 @@ DETAIL: drop cascades to default value for column id2 of table test_table drop cascades to default value for column id of table test_table \c - - - :worker_1_port \ds - List of relations - Schema | Name | Type | Owner + List of relations + Schema | Name | Type | Owner --------------------------------------------------------------------- public | mx_test_table_col_3_seq | sequence | postgres public | sequence_rollback | sequence | postgres @@ -2080,13 +2103,13 @@ NOTICE: dropping metadata on the node (localhost,57637) stop_metadata_sync_to_node --------------------------------------------------------------------- - (1 row) +(1 row) SELECT start_metadata_sync_to_node('localhost', :worker_1_port); start_metadata_sync_to_node --------------------------------------------------------------------- - (1 row) +(1 row) RESET citus.shard_count; RESET citus.shard_replication_factor; diff --git a/src/test/regress/expected/multi_multiuser.out b/src/test/regress/expected/multi_multiuser.out index 0e25a38d2..f2ccf29e7 100644 --- a/src/test/regress/expected/multi_multiuser.out +++ b/src/test/regress/expected/multi_multiuser.out @@ -471,6 +471,8 @@ SELECT create_distributed_function('usage_access_func(usage_access_type,int[])') ERROR: must be owner of function usage_access_func SET ROLE usage_access; SELECT create_distributed_function('usage_access_func(usage_access_type,int[])'); +NOTICE: procedure public.usage_access_func is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_mx_alter_distributed_table.out b/src/test/regress/expected/multi_mx_alter_distributed_table.out index 5960c1c14..427dfe027 100644 --- a/src/test/regress/expected/multi_mx_alter_distributed_table.out +++ b/src/test/regress/expected/multi_mx_alter_distributed_table.out @@ -191,7 +191,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410002 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); proname | colocationid --------------------------------------------------------------------- proc_0 | 1410002 @@ -226,7 +226,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410003 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); proname | colocationid --------------------------------------------------------------------- proc_0 | 1410003 @@ -272,7 +272,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410004 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); proname | colocationid --------------------------------------------------------------------- proc_0 | 1410004 @@ -305,7 +305,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410003 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); proname | colocationid --------------------------------------------------------------------- proc_0 | 1410003 @@ -359,7 +359,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410005 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); proname | colocationid --------------------------------------------------------------------- proc_0 | 1410005 @@ -389,7 +389,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410005 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; proname | colocationid --------------------------------------------------------------------- proc_0 | 1410005 @@ -440,7 +440,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410003 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; proname | colocationid --------------------------------------------------------------------- proc_0 | 1410003 @@ -465,7 +465,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg test_proc_colocation_0 | 1410006 (1 row) -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; proname | colocationid --------------------------------------------------------------------- proc_0 | 1410003 diff --git a/src/test/regress/expected/multi_mx_call.out b/src/test/regress/expected/multi_mx_call.out index 5d440e67b..a8fb95b5f 100644 --- a/src/test/regress/expected/multi_mx_call.out +++ b/src/test/regress/expected/multi_mx_call.out @@ -166,24 +166,32 @@ call mx_call_proc_custom_types('S', 'A'); -- Mark both procedures as distributed ... select create_distributed_function('mx_call_proc(int,int)'); +NOTICE: procedure multi_mx_call.mx_call_proc is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_proc_bigint(bigint,bigint)'); +NOTICE: procedure multi_mx_call.mx_call_proc_bigint is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_call_enum)'); +NOTICE: procedure multi_mx_call.mx_call_proc_custom_types is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_proc_copy(int)'); +NOTICE: procedure multi_mx_call.mx_call_proc_copy is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -559,8 +567,8 @@ CREATE FUNCTION mx_call_add(int, int) RETURNS int DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('mx_call_add(int,int)'); -DEBUG: switching to sequential query execution mode -DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +NOTICE: procedure multi_mx_call.mx_call_add is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_mx_call_0.out b/src/test/regress/expected/multi_mx_call_0.out index 474d3a637..37d538bf2 100644 --- a/src/test/regress/expected/multi_mx_call_0.out +++ b/src/test/regress/expected/multi_mx_call_0.out @@ -166,24 +166,32 @@ call mx_call_proc_custom_types('S', 'A'); -- Mark both procedures as distributed ... select create_distributed_function('mx_call_proc(int,int)'); +NOTICE: procedure multi_mx_call.mx_call_proc is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_proc_bigint(bigint,bigint)'); +NOTICE: procedure multi_mx_call.mx_call_proc_bigint is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_call_enum)'); +NOTICE: procedure multi_mx_call.mx_call_proc_custom_types is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_proc_copy(int)'); +NOTICE: procedure multi_mx_call.mx_call_proc_copy is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -559,8 +567,8 @@ CREATE FUNCTION mx_call_add(int, int) RETURNS int DEBUG: switching to sequential query execution mode DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands SELECT create_distributed_function('mx_call_add(int,int)'); -DEBUG: switching to sequential query execution mode -DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands +NOTICE: procedure multi_mx_call.mx_call_add is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_mx_function_call_delegation.out b/src/test/regress/expected/multi_mx_function_call_delegation.out index 552af7bf5..5ba1566dc 100644 --- a/src/test/regress/expected/multi_mx_function_call_delegation.out +++ b/src/test/regress/expected/multi_mx_function_call_delegation.out @@ -132,30 +132,40 @@ select mx_call_func(2, 0); -- Mark both functions as distributed ... select create_distributed_function('mx_call_func(int,int)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_func_bigint(bigint,bigint)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_bigint is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_call_enum)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_custom_types is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_func_copy(int)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_copy is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('squares(int)'); +NOTICE: procedure multi_mx_function_call_delegation.squares is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_mx_function_call_delegation_0.out b/src/test/regress/expected/multi_mx_function_call_delegation_0.out index cab2f6394..2d317b34e 100644 --- a/src/test/regress/expected/multi_mx_function_call_delegation_0.out +++ b/src/test/regress/expected/multi_mx_function_call_delegation_0.out @@ -132,30 +132,40 @@ select mx_call_func(2, 0); -- Mark both functions as distributed ... select create_distributed_function('mx_call_func(int,int)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_func_bigint(bigint,bigint)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_bigint is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_call_enum)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_custom_types is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('mx_call_func_copy(int)'); +NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_copy is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- (1 row) select create_distributed_function('squares(int)'); +NOTICE: procedure multi_mx_function_call_delegation.squares is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_mx_insert_select_repartition.out b/src/test/regress/expected/multi_mx_insert_select_repartition.out index 59dca03e3..47fd8d18f 100644 --- a/src/test/regress/expected/multi_mx_insert_select_repartition.out +++ b/src/test/regress/expected/multi_mx_insert_select_repartition.out @@ -24,6 +24,8 @@ CREATE FUNCTION square(int) RETURNS INT AS $$ SELECT $1 * $1 $$ LANGUAGE SQL; select create_distributed_function('square(int)'); +NOTICE: procedure multi_mx_insert_select_repartition.square is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_mx_node_metadata.out b/src/test/regress/expected/multi_mx_node_metadata.out index a4f49f320..933407024 100644 --- a/src/test/regress/expected/multi_mx_node_metadata.out +++ b/src/test/regress/expected/multi_mx_node_metadata.out @@ -860,6 +860,13 @@ NOTICE: drop cascades to default value for column a of table reference_table DROP TABLE ref_table; DROP TABLE reference_table; TRUNCATE pg_dist_colocation; +SELECT run_command_on_workers('TRUNCATE pg_dist_colocation'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"TRUNCATE TABLE") + (localhost,57638,t,"TRUNCATE TABLE") +(2 rows) + SELECT count(*) FROM (SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node) t; count --------------------------------------------------------------------- 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_mx_schema_support.out b/src/test/regress/expected/multi_mx_schema_support.out index 6c7173c1f..1228666c4 100644 --- a/src/test/regress/expected/multi_mx_schema_support.out +++ b/src/test/regress/expected/multi_mx_schema_support.out @@ -459,7 +459,7 @@ SELECT create_distributed_table('mx_old_schema.table_set_schema', 'id'); CREATE SCHEMA mx_new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema') ORDER BY "Distributed Schemas"; Distributed Schemas @@ -491,7 +491,7 @@ HINT: Connect to the coordinator and run it again. \c - - - :master_port ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema'); Distributed Schemas --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_prepare_sql.out b/src/test/regress/expected/multi_prepare_sql.out index 6681f4112..4ff7bba42 100644 --- a/src/test/regress/expected/multi_prepare_sql.out +++ b/src/test/regress/expected/multi_prepare_sql.out @@ -1049,6 +1049,8 @@ SELECT key, value FROM text_partition_column_table ORDER BY key; DROP TABLE text_partition_column_table; -- Domain type columns can give issues CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$'); +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. SELECT run_command_on_workers($$ CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$') $$); @@ -1062,12 +1064,16 @@ CREATE TABLE domain_partition_column_table ( key test_key NOT NULL, value int ); +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_table('domain_partition_column_table', 'key'); create_distributed_table --------------------------------------------------------------------- (1 row) +RESET citus.enable_metadata_sync; PREPARE prepared_coercion_to_domain_insert(text) AS INSERT INTO domain_partition_column_table VALUES ($1, 1); EXECUTE prepared_coercion_to_domain_insert('test-1'); 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/multi_row_router_insert.out b/src/test/regress/expected/multi_row_router_insert.out index 8f1b3366c..d5af7e467 100644 --- a/src/test/regress/expected/multi_row_router_insert.out +++ b/src/test/regress/expected/multi_row_router_insert.out @@ -71,6 +71,7 @@ BEGIN RETURN a*a; END; $$ LANGUAGE PLPGSQL STABLE; SELECT create_distributed_function('square(int)'); +NOTICE: procedure multi_row_router_insert.square is already distributed create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_schema_support.out b/src/test/regress/expected/multi_schema_support.out index eb1c47401..b5255d482 100644 --- a/src/test/regress/expected/multi_schema_support.out +++ b/src/test/regress/expected/multi_schema_support.out @@ -884,7 +884,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id'); CREATE SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); Distributed Schemas --------------------------------------------------------------------- @@ -906,7 +906,7 @@ SELECT table_schema AS "Shards' Schema" \c - - - :master_port ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); Distributed Schemas --------------------------------------------------------------------- @@ -944,7 +944,7 @@ SELECT create_distributed_table('table_set_schema', 'id'); CREATE SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid='new_schema'::regnamespace::oid; Distributed Schemas --------------------------------------------------------------------- @@ -965,7 +965,7 @@ SELECT table_schema AS "Shards' Schema" \c - - - :master_port ALTER TABLE table_set_schema SET SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid='new_schema'::regnamespace::oid; Distributed Schemas --------------------------------------------------------------------- @@ -1010,7 +1010,7 @@ SELECT create_distributed_table('table_set_schema', 'id'); CREATE SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); Distributed Schemas --------------------------------------------------------------------- @@ -1034,7 +1034,7 @@ SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts" SET search_path TO old_schema; ALTER TABLE table_set_schema SET SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); Distributed Schemas --------------------------------------------------------------------- @@ -1392,7 +1392,7 @@ BEGIN; ROLLBACK; -- Clean up the created schema SET client_min_messages TO WARNING; -SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema'); pg_identify_object_as_address --------------------------------------------------------------------- @@ -1401,7 +1401,7 @@ SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dis DROP SCHEMA run_test_schema, test_schema_support_join_1, test_schema_support_join_2, "Citus'Teen123", "CiTUS.TEEN2", bar, test_schema_support CASCADE; -- verify that the dropped schema is removed from worker's pg_dist_object -SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema'); pg_identify_object_as_address --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_sequence_default.out b/src/test/regress/expected/multi_sequence_default.out index 27f083e92..3303c5d00 100644 --- a/src/test/regress/expected/multi_sequence_default.out +++ b/src/test/regress/expected/multi_sequence_default.out @@ -852,13 +852,13 @@ SELECT create_distributed_table('test_seq_dist', 'a'); (1 row) -- Both sequence and dependency schema should be distributed -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass); pg_identify_object_as_address --------------------------------------------------------------------- (sequence,"{test_schema_for_sequence_default_propagation,seq_10}",{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace); pg_identify_object_as_address --------------------------------------------------------------------- (schema,{test_schema_for_sequence_default_propagation},{}) diff --git a/src/test/regress/expected/multi_test_helpers.out b/src/test/regress/expected/multi_test_helpers.out index 5a47bf409..2590b9aaf 100644 --- a/src/test/regress/expected/multi_test_helpers.out +++ b/src/test/regress/expected/multi_test_helpers.out @@ -83,7 +83,7 @@ END;$$; CREATE OR REPLACE FUNCTION colocate_proc_with_table(procname text, tablerelid regclass, argument_index int) RETURNS void LANGUAGE plpgsql AS $$ BEGIN - update citus.pg_dist_object + update pg_catalog.pg_dist_object set distribution_argument_index = argument_index, colocationid = pg_dist_partition.colocationid from pg_proc, pg_dist_partition where proname = procname and oid = objid and pg_dist_partition.logicalrelid = tablerelid; diff --git a/src/test/regress/expected/non_super_user_object_metadata.out b/src/test/regress/expected/non_super_user_object_metadata.out index 0923d873a..159e93a42 100644 --- a/src/test/regress/expected/non_super_user_object_metadata.out +++ b/src/test/regress/expected/non_super_user_object_metadata.out @@ -12,27 +12,27 @@ CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy'); CREATE SCHEMA test_sequence_schema; CREATE SEQUENCE test_sequence_schema.test_sequence; -- show that none of the objects above are marked as distributed -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (0 rows) @@ -97,105 +97,105 @@ SELECT create_distributed_function('plpgsql_dist_function(text)'); -- show that schema, types, function and sequence has marked as distributed -- on the coordinator node RESET ROLE; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid; pg_identify_object_as_address --------------------------------------------------------------------- (schema,{local_schema},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid; pg_identify_object_as_address --------------------------------------------------------------------- (schema,{test_sequence_schema},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (type,{local_schema.mood},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; pg_identify_object_as_address --------------------------------------------------------------------- (type,{local_schema.test_type},{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; pg_identify_object_as_address --------------------------------------------------------------------- (sequence,"{test_sequence_schema,test_sequence}",{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid; pg_identify_object_as_address --------------------------------------------------------------------- (sequence,"{local_schema,dist_table_e_seq}",{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{local_schema,test_function}",{integer}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid; pg_identify_object_as_address --------------------------------------------------------------------- (function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text}) (1 row) -- show those objects marked as distributed on metadata worker node as well -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (schema,{local_schema},{}) localhost | 57638 | t | (schema,{local_schema},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (schema,{test_sequence_schema},{}) localhost | 57638 | t | (schema,{test_sequence_schema},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (type,{local_schema.mood},{}) localhost | 57638 | t | (type,{local_schema.mood},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (type,{local_schema.test_type},{}) localhost | 57638 | t | (type,{local_schema.test_type},{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (sequence,"{test_sequence_schema,test_sequence}",{}) localhost | 57638 | t | (sequence,"{test_sequence_schema,test_sequence}",{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (sequence,"{local_schema,dist_table_e_seq}",{}) localhost | 57638 | t | (sequence,"{local_schema,dist_table_e_seq}",{}) (2 rows) -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{local_schema,test_function}",{integer}) localhost | 57638 | t | (function,"{local_schema,test_function}",{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 = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text}) @@ -203,13 +203,13 @@ SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(clas (2 rows) -- Show that extension plpgsql is also marked as distributed as a dependency of plpgsl_dist_function -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%'; obj_identifier --------------------------------------------------------------------- (extension,{plpgsql},{}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (extension,{plpgsql},{}) @@ -278,12 +278,12 @@ SET citus.enable_ddl_propagation TO ON; DROP TABLE dist_table CASCADE; RESET ROLE; SET search_path TO local_schema; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%'; obj_identifier --------------------------------------------------------------------- (0 rows) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | @@ -293,13 +293,13 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_ -- Show that altering the function's schema marks the schema distributed CREATE SCHEMA schema_to_prop_with_function; ALTER FUNCTION test_function SET SCHEMA schema_to_prop_with_function; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%'; obj_identifier --------------------------------------------------------------------- (schema,{schema_to_prop_with_function},{}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (schema,{schema_to_prop_with_function},{}) @@ -309,12 +309,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_ -- Show that dropping the function removes the metadata from pg_dist_object -- on both coordinator and metadata worker node DROP FUNCTION schema_to_prop_with_function.test_function; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%'; obj_identifier --------------------------------------------------------------------- (0 rows) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | @@ -324,13 +324,13 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_ -- Show that altering the type's schema marks the schema distributed CREATE SCHEMA schema_to_prop_with_type; ALTER TYPE test_type SET SCHEMA schema_to_prop_with_type; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%'; obj_identifier --------------------------------------------------------------------- (schema,{schema_to_prop_with_type},{}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (schema,{schema_to_prop_with_type},{}) @@ -341,12 +341,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_ -- on both coordinator and metadata worker node DROP TYPE mood CASCADE; DROP TYPE schema_to_prop_with_type.test_type CASCADE; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'; obj_identifier --------------------------------------------------------------------- (0 rows) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | @@ -379,13 +379,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)', (1 row) -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; distribution_argument_index --------------------------------------------------------------------- 0 (1 row) -SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | 0 @@ -399,13 +399,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)', (1 row) -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; distribution_argument_index --------------------------------------------------------------------- 1 (1 row) -SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | 1 @@ -418,12 +418,12 @@ NOTICE: drop cascades to 3 other objects DETAIL: drop cascades to function plpgsql_dist_function(text) drop cascades to table metadata_dist_test_table drop cascades to function metadata_dist_test_proc(integer,integer) -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%'; obj_identifier --------------------------------------------------------------------- (0 rows) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | @@ -433,26 +433,26 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_ -- Show that extension and dependent sequence also created and marked as distributed CREATE SCHEMA extension_schema; CREATE EXTENSION ltree WITH SCHEMA extension_schema; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%'; obj_identifier --------------------------------------------------------------------- (schema,{extension_schema},{}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (schema,{extension_schema},{}) localhost | 57638 | t | (schema,{extension_schema},{}) (2 rows) -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%'; obj_identifier --------------------------------------------------------------------- (extension,{ltree},{}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (extension,{ltree},{}) @@ -467,13 +467,13 @@ SELECT create_distributed_table('extension_schema.table_to_check_object', 'id'); (1 row) -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; obj_identifier --------------------------------------------------------------------- (table,"{extension_schema,table_to_check_object}",{}) (1 row) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | (table,"{extension_schema,table_to_check_object}",{}) @@ -481,12 +481,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_ (2 rows) DROP TABLE extension_schema.table_to_check_object; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; obj_identifier --------------------------------------------------------------------- (0 rows) -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; nodename | nodeport | success | result --------------------------------------------------------------------- localhost | 57637 | t | diff --git a/src/test/regress/expected/pg13.out b/src/test/regress/expected/pg13.out index 2ba782aaf..44664fb83 100644 --- a/src/test/regress/expected/pg13.out +++ b/src/test/regress/expected/pg13.out @@ -164,8 +164,7 @@ CREATE TABLE my_table (a int, b myvarchar); -- """Add ALTER TYPE options useful for extensions, -- like TOAST and I/O functions control (Tomas Vondra, Tom Lane)""" SELECT create_distributed_table('my_table', 'a'); -ERROR: type "test_pg13.myvarchar" does not exist -CONTEXT: while executing command on localhost:xxxxx +ERROR: Relation "my_table" has dependency on unsupported object "type myvarchar" CREATE TABLE test_table(a int, b tsvector); SELECT create_distributed_table('test_table', 'a'); create_distributed_table @@ -209,7 +208,7 @@ INSERT INTO test_wal VALUES(2,22); Tasks Shown: All -> Task Node: host=localhost port=xxxxx dbname=regression - -> Insert on test_wal_65013 (actual rows=0 loops=1) + -> Insert on test_wal_65011 (actual rows=0 loops=1) WAL: records=1 bytes=63 -> Result (actual rows=1 loops=1) (8 rows) @@ -227,7 +226,7 @@ INSERT INTO test_wal VALUES(3,33),(4,44),(5,55) RETURNING *; -> Task Tuple data received from node: 24 bytes Node: host=localhost port=xxxxx dbname=regression - -> Insert on test_wal_65012 citus_table_alias (actual rows=3 loops=1) + -> Insert on test_wal_65010 citus_table_alias (actual rows=3 loops=1) WAL: records=3 bytes=189 -> Values Scan on "*VALUES*" (actual rows=3 loops=1) (10 rows) diff --git a/src/test/regress/expected/pg14.out b/src/test/regress/expected/pg14.out index c49880109..aed913063 100644 --- a/src/test/regress/expected/pg14.out +++ b/src/test/regress/expected/pg14.out @@ -1232,6 +1232,8 @@ SELECT create_distributed_function('proc_with_out_param(date,int)'); ERROR: function "proc_with_out_param(date,int)" does not exist -- this should work SELECT create_distributed_function('proc_with_out_param(date)'); +NOTICE: procedure pg14.proc_with_out_param is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- 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/prepared_statements_4.out b/src/test/regress/expected/prepared_statements_4.out index 0dba296e8..2d66e04eb 100644 --- a/src/test/regress/expected/prepared_statements_4.out +++ b/src/test/regress/expected/prepared_statements_4.out @@ -20,15 +20,16 @@ SELECT key, value FROM text_partition_column_table ORDER BY key; test | 1 (7 rows) -PREPARE prepared_coercion_to_domain_insert(text) AS - INSERT INTO domain_partition_column_table VALUES ($1, 1); -EXECUTE prepared_coercion_to_domain_insert('test-1'); -EXECUTE prepared_coercion_to_domain_insert('test-2'); -EXECUTE prepared_coercion_to_domain_insert('test-3'); -EXECUTE prepared_coercion_to_domain_insert('test-4'); -EXECUTE prepared_coercion_to_domain_insert('test-5'); -EXECUTE prepared_coercion_to_domain_insert('test-6'); -EXECUTE prepared_coercion_to_domain_insert('test-7'); +-- TODO: Uncomment tests below once domains are supported +-- PREPARE prepared_coercion_to_domain_insert(text) AS +-- INSERT INTO domain_partition_column_table VALUES ($1, 1); +-- EXECUTE prepared_coercion_to_domain_insert('test-1'); +-- EXECUTE prepared_coercion_to_domain_insert('test-2'); +-- EXECUTE prepared_coercion_to_domain_insert('test-3'); +-- EXECUTE prepared_coercion_to_domain_insert('test-4'); +-- EXECUTE prepared_coercion_to_domain_insert('test-5'); +-- EXECUTE prepared_coercion_to_domain_insert('test-6'); +-- EXECUTE prepared_coercion_to_domain_insert('test-7'); PREPARE FOO AS INSERT INTO http_request ( site_id, ingest_time, url, request_country, ip_address, status_code, response_time_msec diff --git a/src/test/regress/expected/prepared_statements_create_load.out b/src/test/regress/expected/prepared_statements_create_load.out index 91f653d08..68d9baeb6 100644 --- a/src/test/regress/expected/prepared_statements_create_load.out +++ b/src/test/regress/expected/prepared_statements_create_load.out @@ -65,6 +65,9 @@ SELECT create_distributed_table('text_partition_column_table', 'key'); -- Domain type columns can give issues -- and we use offset to prevent output diverging CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$'); +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. Also uncomment related tests on +-- prepared_statements_4 test file. SELECT run_command_on_workers($$ CREATE DOMAIN "prepared statements".test_key AS text CHECK(VALUE ~ '^test-\d$') $$) OFFSET 10000; @@ -72,6 +75,9 @@ $$) OFFSET 10000; --------------------------------------------------------------------- (0 rows) +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; CREATE TABLE domain_partition_column_table ( key test_key NOT NULL, value int @@ -82,6 +88,7 @@ SELECT create_distributed_table('domain_partition_column_table', 'key'); (1 row) +RESET citus.enable_metadata_sync; -- verify we re-evaluate volatile functions every time CREATE TABLE http_request ( site_id INT, diff --git a/src/test/regress/expected/propagate_extension_commands.out b/src/test/regress/expected/propagate_extension_commands.out index 1c2fd19b4..55515a542 100644 --- a/src/test/regress/expected/propagate_extension_commands.out +++ b/src/test/regress/expected/propagate_extension_commands.out @@ -6,13 +6,13 @@ SET client_min_messages TO WARNING; -- the extension is on contrib, so should be avaliable for the regression tests CREATE EXTENSION seg; -- make sure that both the schema and the extension is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- 1 (1 row) -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'); count --------------------------------------------------------------------- 1 @@ -34,7 +34,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_ CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg); -- verify that the type that depends on the extension is also marked as distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test')); count --------------------------------------------------------------------- 1 @@ -67,7 +67,7 @@ BEGIN; COMMIT; -- make sure that the extension is distributed even if we run create extension in a transaction block -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); count --------------------------------------------------------------------- 1 @@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public; -- switch back to public schema as we set extension's schema to public SET search_path TO public; -- make sure that the extension is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); count --------------------------------------------------------------------- 1 @@ -282,7 +282,7 @@ BEGIN; ROLLBACK; -- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed -- make sure that the extension is not distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); count --------------------------------------------------------------------- 0 @@ -319,7 +319,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn'); -- drop extension should just work DROP EXTENSION seg CASCADE; -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- 0 @@ -333,7 +333,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname (2 rows) -- make sure that the extension is not avaliable anymore as a distributed object -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); count --------------------------------------------------------------------- 0 @@ -439,9 +439,9 @@ BEGIN; COMMIT; -- Check the pg_dist_object SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; distributedfunction --------------------------------------------------------------------- @@ -450,9 +450,9 @@ classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -468,7 +468,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port); (1 row) -- make sure that both extensions are created on both nodes -SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); count --------------------------------------------------------------------- 2 @@ -484,9 +484,9 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname -- Check the pg_dist_object on the both nodes SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -498,9 +498,9 @@ $$); DROP EXTENSION seg CASCADE; -- Recheck the pg_dist_object SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; distributedfunction --------------------------------------------------------------------- @@ -508,9 +508,9 @@ classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -524,7 +524,7 @@ SET citus.enable_ddl_propagation TO false; CREATE EXTENSION seg; SET citus.enable_ddl_propagation TO true; -- Check the extension in pg_dist_object -SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- @@ -533,9 +533,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -551,7 +551,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func (1 row) -- Recheck the extension in pg_dist_object -SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- @@ -559,9 +559,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); (1 row) SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; distributedfunction --------------------------------------------------------------------- @@ -570,9 +570,9 @@ classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -600,7 +600,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten (1 row) -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8'); distribution_argument_index --------------------------------------------------------------------- @@ -613,7 +613,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten (1 row) -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8'); distribution_argument_index --------------------------------------------------------------------- diff --git a/src/test/regress/expected/propagate_extension_commands_1.out b/src/test/regress/expected/propagate_extension_commands_1.out index e6c85bee4..5ee03fe29 100644 --- a/src/test/regress/expected/propagate_extension_commands_1.out +++ b/src/test/regress/expected/propagate_extension_commands_1.out @@ -6,13 +6,13 @@ SET client_min_messages TO WARNING; -- the extension is on contrib, so should be avaliable for the regression tests CREATE EXTENSION seg; -- make sure that both the schema and the extension is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- 1 (1 row) -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'); count --------------------------------------------------------------------- 1 @@ -34,7 +34,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_ CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg); -- verify that the type that depends on the extension is also marked as distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test')); count --------------------------------------------------------------------- 1 @@ -67,7 +67,7 @@ BEGIN; COMMIT; -- make sure that the extension is distributed even if we run create extension in a transaction block -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); count --------------------------------------------------------------------- 1 @@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public; -- switch back to public schema as we set extension's schema to public SET search_path TO public; -- make sure that the extension is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); count --------------------------------------------------------------------- 1 @@ -281,7 +281,7 @@ BEGIN; ROLLBACK; -- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed -- make sure that the extension is not distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); count --------------------------------------------------------------------- 0 @@ -318,7 +318,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn'); -- drop extension should just work DROP EXTENSION seg CASCADE; -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- 0 @@ -332,7 +332,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname (2 rows) -- make sure that the extension is not avaliable anymore as a distributed object -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); count --------------------------------------------------------------------- 0 @@ -438,9 +438,9 @@ BEGIN; COMMIT; -- Check the pg_dist_object SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; distributedfunction --------------------------------------------------------------------- @@ -449,9 +449,9 @@ classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -467,7 +467,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port); (1 row) -- make sure that both extensions are created on both nodes -SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); count --------------------------------------------------------------------- 2 @@ -483,9 +483,9 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname -- Check the pg_dist_object on the both nodes SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -497,9 +497,9 @@ $$); DROP EXTENSION seg CASCADE; -- Recheck the pg_dist_object SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; distributedfunction --------------------------------------------------------------------- @@ -507,9 +507,9 @@ classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -523,7 +523,7 @@ SET citus.enable_ddl_propagation TO false; CREATE EXTENSION seg; SET citus.enable_ddl_propagation TO true; -- Check the extension in pg_dist_object -SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- @@ -532,9 +532,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -550,7 +550,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func (1 row) -- Recheck the extension in pg_dist_object -SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); count --------------------------------------------------------------------- @@ -558,9 +558,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); (1 row) SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; distributedfunction --------------------------------------------------------------------- @@ -569,9 +569,9 @@ classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); run_command_on_workers @@ -599,7 +599,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten (1 row) -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8'); distribution_argument_index --------------------------------------------------------------------- @@ -612,7 +612,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten (1 row) -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8'); distribution_argument_index --------------------------------------------------------------------- diff --git a/src/test/regress/expected/propagate_foreign_servers.out b/src/test/regress/expected/propagate_foreign_servers.out index 2e9b164e7..c0dbfcdb9 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 pg_catalog.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/row_types.out b/src/test/regress/expected/row_types.out index c82b33b48..43a5ddf2d 100644 --- a/src/test/regress/expected/row_types.out +++ b/src/test/regress/expected/row_types.out @@ -15,6 +15,8 @@ BEGIN END; $$ language plpgsql; SELECT create_distributed_function('table_returner(int)'); +NOTICE: procedure row_types.table_returner is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -29,6 +31,8 @@ BEGIN END; $$ language plpgsql; SELECT create_distributed_function('record_returner(int)'); +NOTICE: procedure row_types.record_returner is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- @@ -42,6 +46,8 @@ BEGIN END; $$ language plpgsql; SELECT create_distributed_function('identity_returner(anyelement)'); +NOTICE: procedure row_types.identity_returner is already distributed +DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands create_distributed_function --------------------------------------------------------------------- diff --git a/src/test/regress/expected/run_command_on_all_nodes.out b/src/test/regress/expected/run_command_on_all_nodes.out new file mode 100644 index 000000000..9cf838c83 --- /dev/null +++ b/src/test/regress/expected/run_command_on_all_nodes.out @@ -0,0 +1,88 @@ +CREATE SCHEMA run_command_on_all_nodes; +SET search_path TO run_command_on_all_nodes; +-- check coordinator isn't in metadata +SELECT count(*) != 0 AS "Coordinator is in Metadata" +FROM pg_dist_node +WHERE groupid IN ( + SELECT groupid FROM pg_dist_local_group +); + Coordinator is in Metadata +--------------------------------------------------------------------- + f +(1 row) + +-- run a simple select query and check it also runs in coordinator +SELECT nodeid NOT IN (SELECT nodeid FROM pg_dist_node) AS "Is Coordinator", success, result FROM run_command_on_all_nodes('SELECT 1') ORDER BY 1; + Is Coordinator | success | result +--------------------------------------------------------------------- + f | t | 1 + f | t | 1 + t | t | 1 +(3 rows) + +-- check that when coordinator is not in metadata and run_command_on_all_nodes is called from +-- a worker node, command is not run on the coordinator +\c - - - :worker_1_port +SELECT nodeid NOT IN (SELECT nodeid FROM pg_dist_node) AS "Is Coordinator", success, result FROM run_command_on_all_nodes('SELECT 1') ORDER BY 1; + Is Coordinator | success | result +--------------------------------------------------------------------- + f | t | 1 + f | t | 1 +(2 rows) + +\c - - - :master_port +-- create a table +SELECT result FROM run_command_on_all_nodes('CREATE TABLE run_command_on_all_nodes.tbl (a INT)'); + result +--------------------------------------------------------------------- + CREATE TABLE + CREATE TABLE + CREATE TABLE +(3 rows) + +SELECT tablename FROM pg_tables WHERE schemaname = 'run_command_on_all_nodes'; + tablename +--------------------------------------------------------------------- + tbl +(1 row) + +\c - - - :worker_1_port +SELECT tablename FROM pg_tables WHERE schemaname = 'run_command_on_all_nodes'; + tablename +--------------------------------------------------------------------- + tbl +(1 row) + +\c - - - :master_port +SELECT result FROM run_command_on_all_nodes('SELECT tablename FROM pg_tables WHERE schemaname = ''run_command_on_all_nodes'';'); + result +--------------------------------------------------------------------- + tbl + tbl + tbl +(3 rows) + +-- break a node and check messages +SELECT nodeid AS worker_1_nodeid FROM pg_dist_node WHERE nodeport = :worker_1_port \gset +UPDATE pg_dist_node SET nodeport = 0 WHERE nodeid = :worker_1_nodeid; +SELECT nodeid = :worker_1_nodeid AS "Is Worker 1", success, result FROM run_command_on_all_nodes('SELECT 1') ORDER BY 1; + Is Worker 1 | success | result +--------------------------------------------------------------------- + f | t | 1 + f | t | 1 + t | f | failed to connect to localhost:xxxxx +(3 rows) + +SELECT nodeid = :worker_1_nodeid AS "Is Worker 1", success, result FROM run_command_on_all_nodes('SELECT 1', give_warning_for_connection_errors:=true) ORDER BY 1; +WARNING: Error on node with node id xxxxx: failed to connect to localhost:xxxxx +CONTEXT: PL/pgSQL function run_command_on_all_nodes(text,boolean,boolean) line XX at RAISE + Is Worker 1 | success | result +--------------------------------------------------------------------- + f | t | 1 + f | t | 1 + t | f | failed to connect to localhost:xxxxx +(3 rows) + +UPDATE pg_dist_node SET nodeport = :worker_1_port WHERE nodeid = :worker_1_nodeid; +DROP SCHEMA run_command_on_all_nodes CASCADE; +NOTICE: drop cascades to table run_command_on_all_nodes.tbl 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/single_node.out b/src/test/regress/expected/single_node.out index 3b9f9f2c7..c854ec48a 100644 --- a/src/test/regress/expected/single_node.out +++ b/src/test/regress/expected/single_node.out @@ -899,6 +899,9 @@ CREATE TABLE hpart0 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remaind CREATE TABLE hpart1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 1); CREATE TABLE hpart2 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 2); CREATE TABLE hpart3 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 3); +-- Disable metadata sync since citus doesn't support distributing +-- operator class for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_table('hash_parted ', 'a'); create_distributed_table --------------------------------------------------------------------- @@ -925,6 +928,7 @@ ALTER TABLE hash_parted DETACH PARTITION hpart0; ALTER TABLE hash_parted DETACH PARTITION hpart1; ALTER TABLE hash_parted DETACH PARTITION hpart2; ALTER TABLE hash_parted DETACH PARTITION hpart3; +RESET citus.enable_metadata_sync; -- test range partition without creating partitions and inserting with generate_series() -- should error out even in plain PG since no partition of relation "parent_tab" is found for row -- in Citus it errors out because it fails to evaluate partition key in insert @@ -1723,8 +1727,7 @@ SELECT pg_sleep(0.1); -- since max_cached_conns_per_worker == 0 at this point, the -- backend(s) that execute on the shards will be terminated --- so show that there is only a single client backend, --- which is actually the backend that executes here +-- so show that there no internal backends SET search_path TO single_node; SELECT count(*) from should_commit; count @@ -1732,7 +1735,13 @@ SELECT count(*) from should_commit; 0 (1 row) -SELECT pg_catalog.get_all_active_client_backend_count(); +SELECT count(*) FROM pg_stat_activity WHERE application_name LIKE 'citus_internal%'; + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT get_all_active_client_backend_count(); get_all_active_client_backend_count --------------------------------------------------------------------- 1 @@ -1757,10 +1766,17 @@ BEGIN; (1 row) -- now, we should have additional 32 connections - SELECT pg_catalog.get_all_active_client_backend_count(); + SELECT count(*) FROM pg_stat_activity WHERE application_name LIKE 'citus_internal%'; + count +--------------------------------------------------------------------- + 32 +(1 row) + + -- single external connection + SELECT get_all_active_client_backend_count(); get_all_active_client_backend_count --------------------------------------------------------------------- - 33 + 1 (1 row) ROLLBACK; diff --git a/src/test/regress/expected/sqlsmith_failures.out b/src/test/regress/expected/sqlsmith_failures.out index 46d71e8ef..d276f04a9 100644 --- a/src/test/regress/expected/sqlsmith_failures.out +++ b/src/test/regress/expected/sqlsmith_failures.out @@ -123,7 +123,7 @@ from (select pg_catalog.websearch_to_tsquery( cast(pg_catalog.regconfigin(cast(cast(null as cstring) as cstring)) as regconfig), - cast((select type from citus.pg_dist_object limit 1 offset 1) as text) + cast((select type from pg_catalog.pg_dist_object limit 1 offset 1) as text) ) as c0, sample_0.org_id as c1, sample_0.id as c2, diff --git a/src/test/regress/expected/tableam.out b/src/test/regress/expected/tableam.out index 7a11d7354..f82d0db90 100644 --- a/src/test/regress/expected/tableam.out +++ b/src/test/regress/expected/tableam.out @@ -23,6 +23,11 @@ $Q$); (1 row) +-- Since Citus assumes access methods are part of the extension, make fake_am +-- owned manually to be able to pass checks on Citus while distributing tables. +ALTER EXTENSION citus ADD ACCESS METHOD fake_am; +NOTICE: Citus does not propagate adding/dropping member objects +HINT: You can add/drop the member objects on the workers as well. -- -- Hash distributed table using a non-default table access method -- @@ -129,27 +134,6 @@ SELECT * FROM master_get_table_ddl_events('test_ref'); ALTER TABLE test_tableam.test_ref OWNER TO postgres (2 rows) --- replicate to coordinator -SET client_min_messages TO WARNING; -\set VERBOSIY terse -SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); - ?column? ---------------------------------------------------------------------- - 1 -(1 row) - -RESET client_min_messages; -delete from test_ref; -WARNING: fake_scan_getnextslot -DETAIL: from localhost:xxxxx -ERROR: fake_tuple_delete not implemented -CONTEXT: while executing command on localhost:xxxxx -SELECT master_remove_node('localhost', :master_port); - master_remove_node ---------------------------------------------------------------------- - -(1 row) - -- -- Range partitioned table using a non-default table access method -- @@ -323,5 +307,7 @@ CREATE TABLE test_partitioned(id int, p int, val int) PARTITION BY RANGE (p) USING fake_am; ERROR: specifying a table access method is not supported on a partitioned table \set VERBOSITY terse +ALTER EXTENSION citus DROP ACCESS METHOD fake_am; +NOTICE: Citus does not propagate adding/dropping member objects drop schema test_tableam cascade; -NOTICE: drop cascades to 6 other objects +NOTICE: drop cascades to 5 other objects diff --git a/src/test/regress/expected/text_search.out b/src/test/regress/expected/text_search.out index 1b4f652c1..d1dbf6511 100644 --- a/src/test/regress/expected/text_search.out +++ b/src/test/regress/expected/text_search.out @@ -419,7 +419,7 @@ SELECT * FROM run_command_on_workers($$ SELECT 'text_search.concurrent_index_con -- verify the objid is correctly committed locally due to the somewhat convoluted commit and new transaction starting when creating an index concurrently SELECT pg_catalog.pg_identify_object_as_address(classid, objid, objsubid) - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE classid = 3602 AND objid = 'text_search.concurrent_index_config'::regconfig::oid; pg_identify_object_as_address --------------------------------------------------------------------- @@ -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_citus_stat_activity.out b/src/test/regress/expected/upgrade_citus_stat_activity.out new file mode 100644 index 000000000..6564e9b40 --- /dev/null +++ b/src/test/regress/expected/upgrade_citus_stat_activity.out @@ -0,0 +1,17 @@ +SELECT column_name FROM information_schema.columns WHERE table_name = 'citus_stat_activity' AND column_name NOT IN ('leader_pid', 'query_id') +EXCEPT SELECT column_name FROM information_schema.columns WHERE table_name = 'pg_stat_activity' +ORDER BY 1; + column_name +--------------------------------------------------------------------- + global_pid + is_worker_query + nodeid +(3 rows) + +SELECT column_name FROM information_schema.columns WHERE table_name = 'pg_stat_activity' +EXCEPT SELECT column_name FROM information_schema.columns WHERE table_name = 'citus_stat_activity' +ORDER BY 1; + column_name +--------------------------------------------------------------------- + (0 rows) + diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index 185bf19c5..afd30523c 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 @@ -37,7 +37,9 @@ ORDER BY 1; function citus_add_node(text,integer,integer,noderole,name) function citus_add_rebalance_strategy(name,regproc,regproc,regproc,real,real,real) function citus_add_secondary_node(text,integer,text,integer,name) + function citus_backend_gpid() function citus_blocking_pids(integer) + function citus_calculate_gpid(integer,integer) function citus_check_cluster_node_health() function citus_check_connection_to_node(text,integer) function citus_cleanup_orphaned_shards() @@ -51,12 +53,12 @@ ORDER BY 1; function citus_dist_partition_cache_invalidate() function citus_dist_placement_cache_invalidate() function citus_dist_shard_cache_invalidate() - function citus_dist_stat_activity() function citus_drain_node(text,integer,citus.shard_transfer_mode,name) function citus_drop_all_shards(regclass,text,text,boolean) 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() @@ -69,10 +71,12 @@ ORDER BY 1; function citus_internal.replace_isolation_tester_func() function citus_internal.restore_isolation_tester_func() function citus_internal.upgrade_columnar_storage(regclass) + function citus_internal_add_colocation_metadata(integer,integer,integer,regtype,oid) function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) function citus_internal_add_partition_metadata(regclass,"char",text,integer,"char") function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint) function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text) + function citus_internal_delete_colocation_metadata(integer) function citus_internal_delete_shard_metadata(bigint) function citus_internal_global_blocked_processes() function citus_internal_local_blocked_processes() @@ -86,6 +90,10 @@ ORDER BY 1; function citus_local_disk_space_stats() function citus_move_shard_placement(bigint,text,integer,text,integer,citus.shard_transfer_mode) function citus_node_capacity_1(integer) + function citus_nodeid_for_gpid(bigint) + function citus_nodename_for_nodeid(integer) + function citus_nodeport_for_nodeid(integer) + function citus_pid_for_gpid(bigint) function citus_prepare_pg_upgrade() function citus_query_stats() function citus_relation_size(regclass) @@ -102,6 +110,7 @@ ORDER BY 1; function citus_shard_indexes_on_worker() function citus_shard_sizes() function citus_shards_on_worker() + function citus_stat_activity() function citus_stat_statements() function citus_stat_statements_reset() function citus_table_is_visible(oid) @@ -115,7 +124,6 @@ ORDER BY 1; function citus_update_table_statistics(regclass) function citus_validate_rebalance_strategy_functions(regproc,regproc,regproc) function citus_version() - function citus_worker_stat_activity() function column_name_to_column(regclass,text) function column_to_column_name(regclass,text) function columnar.columnar_handler(internal) @@ -142,6 +150,7 @@ ORDER BY 1; function get_current_transaction_id() function get_global_active_transactions() function get_missing_time_partition_ranges(regclass,interval,timestamp with time zone,timestamp with time zone) + function get_nodeid_for_groupid(integer) function get_rebalance_progress() function get_rebalance_table_shards_plan(regclass,real,integer,bigint[],boolean,name,real) function get_shard_id_for_distribution_column(regclass,"any") @@ -175,7 +184,7 @@ ORDER BY 1; function master_update_table_statistics(regclass) function notify_constraint_dropped() function pg_cancel_backend(bigint) - function pg_terminate_backend(bigint, bigint) + function pg_terminate_backend(bigint,bigint) function poolinfo_valid(text) function read_intermediate_result(text,citus_copy_format) function read_intermediate_results(text[],citus_copy_format) @@ -186,6 +195,7 @@ ORDER BY 1; function replicate_reference_tables() function replicate_table_shards(regclass,integer,integer,bigint[],citus.shard_transfer_mode) function role_exists(name) + function run_command_on_all_nodes(text,boolean,boolean) function run_command_on_colocated_placements(regclass,regclass,text,boolean) function run_command_on_placements(regclass,text,boolean) function run_command_on_shards(regclass,text,boolean) @@ -225,7 +235,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) @@ -242,7 +252,6 @@ ORDER BY 1; sequence pg_dist_node_nodeid_seq sequence pg_dist_placement_placementid_seq sequence pg_dist_shardid_seq - table citus.pg_dist_object table columnar.chunk table columnar.chunk_group table columnar.options @@ -252,6 +261,7 @@ ORDER BY 1; table pg_dist_local_group table pg_dist_node table pg_dist_node_metadata + table pg_dist_object table pg_dist_partition table pg_dist_placement table pg_dist_poolinfo @@ -268,9 +278,9 @@ ORDER BY 1; view citus_shard_indexes_on_worker view citus_shards view citus_shards_on_worker + view citus_stat_activity view citus_stat_statements - view citus_worker_stat_activity view pg_dist_shard_placement view time_partitions -(256 rows) +(266 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..bcbb497ef 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 @@ -36,15 +36,41 @@ drop cascades to table upgrade_basic.t_range -- "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; +SELECT i.* FROM pg_catalog.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..183db7e1c --- /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 pg_catalog.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..b38be95c7 --- /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 pg_catalog.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 pg_catalog.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/input/multi_alter_table_statements.source b/src/test/regress/input/multi_alter_table_statements.source index fa828bd7b..a93e45f81 100644 --- a/src/test/regress/input/multi_alter_table_statements.source +++ b/src/test/regress/input/multi_alter_table_statements.source @@ -651,8 +651,8 @@ CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10; ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10'); -- Should be distributed along with the sequence -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass); -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace); DROP SCHEMA test_schema_for_sequence_propagation CASCADE; DROP TABLE table_without_sequence; diff --git a/src/test/regress/input/multi_copy.source b/src/test/regress/input/multi_copy.source index 0b384c047..a017fe342 100644 --- a/src/test/regress/input/multi_copy.source +++ b/src/test/regress/input/multi_copy.source @@ -500,13 +500,13 @@ SELECT shardid, nodename, nodeport -- before adding the node, add pg_dist_object entry for tables created with -- master_create_distributed_table as we don't have the entry for them. -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0); +INSERT INTO pg_catalog.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); RESET client_min_messages; diff --git a/src/test/regress/isolation_schedule b/src/test/regress/isolation_schedule index e5602618f..8f849d5dd 100644 --- a/src/test/regress/isolation_schedule +++ b/src/test/regress/isolation_schedule @@ -62,6 +62,7 @@ test: isolation_validate_vs_insert test: isolation_insert_select_conflict test: shared_connection_waits test: isolation_cancellation +test: isolation_max_client_connections test: isolation_undistribute_table test: isolation_fix_partition_shard_index_names test: isolation_global_pid diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index b80d2f722..49d54592c 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -94,6 +94,7 @@ test: alter_distributed_table test: issue_5248 issue_5099 test: object_propagation_debug test: undistribute_table +test: run_command_on_all_nodes # --------- diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index 2e01b0518..3707ab016 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -1275,13 +1275,13 @@ CREATE SCHEMA test_schema_for_sequence_propagation; CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10; ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10'); -- Should be distributed along with the sequence -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass); pg_identify_object_as_address --------------------------------------------------------------- (sequence,"{test_schema_for_sequence_propagation,seq_10}",{}) (1 row) -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace); pg_identify_object_as_address ---------------------------------------------------- (schema,{test_schema_for_sequence_propagation},{}) diff --git a/src/test/regress/output/multi_copy.source b/src/test/regress/output/multi_copy.source index d6d72d963..f709263fd 100644 --- a/src/test/regress/output/multi_copy.source +++ b/src/test/regress/output/multi_copy.source @@ -627,16 +627,16 @@ SELECT shardid, nodename, nodeport -- add the node back -- before adding the node, add pg_dist_object entry for tables created with -- master_create_distributed_table as we don't have the entry for them. -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0); +INSERT INTO pg_catalog.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..b2e977dda 100644 --- a/src/test/regress/spec/isolation_citus_dist_activity.spec +++ b/src/test/regress/spec/isolation_citus_dist_activity.spec @@ -1,5 +1,9 @@ setup { + CREATE OR REPLACE FUNCTION test_assign_global_pid() + RETURNS void + LANGUAGE C STRICT + AS 'citus', $$test_assign_global_pid$$; SET citus.shard_replication_factor TO 1; SET citus.shard_count TO 4; select setval('pg_dist_shardid_seq', GREATEST(1300000, nextval('pg_dist_shardid_seq'))); @@ -71,8 +75,7 @@ step "s2-sleep" 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 '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; - + SELECT query, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%'), ('%citus_add_node%')) AND backend_type = 'client backend' ORDER BY query DESC; } session "s3" @@ -89,12 +92,28 @@ 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, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true AND backend_type = 'client backend' ORDER BY query DESC; } +session "s4" +step "add-coordinator-to-metadata" +{ + SELECT 1 FROM citus_add_node('localhost', 57636, groupid:=0); + SELECT test_assign_global_pid(); +} + +step "remove-coordinator-from-metadata" +{ + SELECT citus_remove_node('localhost', 57636); +} + +permutation "add-coordinator-to-metadata" + // we prefer to sleep before "s2-view-dist" so that we can ensure // the "wait_event" in the output doesn't change randomly (e.g., NULL to CliendRead etc.) permutation "s1-cache-connections" "s1-begin" "s2-begin" "s3-begin" "s1-alter-table" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" permutation "s1-cache-connections" "s1-begin" "s2-begin" "s3-begin" "s1-insert" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" permutation "s1-cache-connections" "s1-begin" "s2-begin" "s3-begin" "s1-select" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" permutation "s1-cache-connections" "s1-begin" "s2-begin" "s3-begin" "s1-select-router" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" + +permutation "remove-coordinator-from-metadata" 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_ensure_dependency_activate_node.spec b/src/test/regress/spec/isolation_ensure_dependency_activate_node.spec index f46f80753..30a4e6d21 100644 --- a/src/test/regress/spec/isolation_ensure_dependency_activate_node.spec +++ b/src/test/regress/spec/isolation_ensure_dependency_activate_node.spec @@ -51,7 +51,7 @@ step "s1-print-distributed-objects" SELECT 1 FROM master_add_node('localhost', 57638); -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; @@ -123,7 +123,7 @@ step "s2-commit" step "s2-print-distributed-objects" { -- print an overview of all distributed objects - SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1; + SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1; -- print if the schema has been created SELECT count(*) FROM pg_namespace where nspname = 'myschema'; diff --git a/src/test/regress/spec/isolation_extension_commands.spec b/src/test/regress/spec/isolation_extension_commands.spec index 270a60330..98a8eb6dc 100644 --- a/src/test/regress/spec/isolation_extension_commands.spec +++ b/src/test/regress/spec/isolation_extension_commands.spec @@ -41,7 +41,7 @@ step "s1-create-extension-with-schema2" step "s1-print" { - select count(*) from citus.pg_dist_object ; + select count(*) from pg_catalog.pg_dist_object ; select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg'; SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); 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_get_distributed_wait_queries_mx.spec b/src/test/regress/spec/isolation_get_distributed_wait_queries_mx.spec index 84daaf792..dfe1587be 100644 --- a/src/test/regress/spec/isolation_get_distributed_wait_queries_mx.spec +++ b/src/test/regress/spec/isolation_get_distributed_wait_queries_mx.spec @@ -1,6 +1,7 @@ #include "isolation_mx_common.include.spec" setup { + SELECT citus_add_node('localhost', 57636, groupid:=0); CREATE TABLE ref_table(user_id int, value_1 int); SELECT create_reference_table('ref_table'); INSERT INTO ref_table VALUES (1, 11), (2, 21), (3, 31), (4, 41), (5, 51), (6, 61), (7, 71); @@ -17,6 +18,7 @@ teardown DROP TABLE ref_table; DROP TABLE tt1; SELECT citus_internal.restore_isolation_tester_func(); + SELECT citus_remove_node('localhost', 57636); } session "s1" @@ -159,7 +161,13 @@ session "s3" step "s3-select-distributed-waiting-queries" { - SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; +} + +// only works for the coordinator +step "s3-show-actual-gpids" +{ + SELECT global_pid > 0 as gpid_exists, query FROM citus_stat_activity WHERE state = 'active' AND query IN (SELECT blocked_statement FROM citus_lock_waits UNION SELECT current_statement_in_blocking_process FROM citus_lock_waits) ORDER BY 1 DESC; } // session s1 and s4 executes the commands on the same worker node @@ -189,6 +197,59 @@ step "s4-commit-worker" SELECT run_commands_on_session_level_connection_to_node('COMMIT'); } + + +// on the coordinator, show that even if a backend is blocked on a DDL as the first command +// (e.g., as of today global pid has not been assigned), we can still show the blocking activity +// we use the following 4 sessions 5,6,7,8 for this purpose +session "s5" + +step "s5-begin" +{ + BEGIN; +} + +step "s5-alter" +{ + ALTER TABLE tt1 ADD COLUMN new_column INT; +} + +step "s5-rollback" +{ + ROLLBACK; +} + +session "s6" + +step "s6-select" +{ + SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1; +} + +session "s7" + +step "s7-alter" +{ + ALTER TABLE tt1 ADD COLUMN new_column INT; +} + +session "s8" + +step "s8-begin" +{ + BEGIN; +} + +step "s8-select" +{ + SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1; +} + +step "s8-rollback" +{ + ROLLBACK; +} + permutation "s1-begin" "s1-update-ref-table-from-coordinator" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit" "s2-commit-worker" "s2-stop-connection" permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-ref-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-dist-table" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" @@ -212,3 +273,10 @@ permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update- // we can find the blocking relationship permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table-id-1" "s2-start-session-level-connection" "s2-update-dist-table-id-1" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s1-stop-connection" "s2-stop-connection" permutation "s1-begin" "s1-update-ref-table-from-coordinator" "s2-start-session-level-connection" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit" "s2-stop-connection" + +// show that we can see blocking activity even if these are the first commands in the sessions +// such that global_pids have not been assigned +// in the second permutation, s3-show-actual-gpids shows the gpid for ALTER TABLE +// because ALTER TABLE is not blocked on the parser but during the execution (hence gpid already asssigned) +"s5-begin" "s5-alter" "s6-select" "s3-select-distributed-waiting-queries" "s3-show-actual-gpids" "s5-rollback" +"s8-begin" "s8-select" "s7-alter" "s3-select-distributed-waiting-queries" "s3-show-actual-gpids" "s8-rollback" diff --git a/src/test/regress/spec/isolation_global_pid.spec b/src/test/regress/spec/isolation_global_pid.spec index 62f45d1e9..690d1768a 100644 --- a/src/test/regress/spec/isolation_global_pid.spec +++ b/src/test/regress/spec/isolation_global_pid.spec @@ -2,6 +2,7 @@ setup { + SELECT citus_add_node('localhost', 57636, groupid:=0); SET citus.next_shard_id TO 12345000; CREATE TABLE dist_table (a INT, b INT); SELECT create_distributed_table('dist_table', 'a', shard_count:=4); @@ -11,6 +12,7 @@ teardown { DROP TABLE dist_table; SELECT citus_internal.restore_isolation_tester_func(); + SELECT citus_remove_node('localhost', 57636); } session "s1" @@ -60,37 +62,48 @@ step "s1-stop-session-level-connection" session "s2" -step "s2-coordinator-citus_dist_stat_activity" +step "s2-coordinator-citus_stat_activity" { - SELECT global_pid != 0 FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' and query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; + SELECT global_pid != 0 FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; } -step "s2-coordinator-citus_worker_stat_activity" +step "s2-coordinator-citus_dist_stat_activity" { - SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN ( - SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' + SELECT query FROM citus_dist_stat_activity WHERE global_pid IN ( + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' ) + AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%' + ORDER BY 1; +} + +step "s2-coordinator-citus_stat_activity-in-workers" +{ + SELECT query FROM citus_stat_activity WHERE global_pid IN ( + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' + ) + AND is_worker_query = true + AND backend_type = 'client backend' ORDER BY 1; } step "s2-coordinator-get_all_active_transactions" { SELECT count(*) FROM get_all_active_transactions() WHERE global_pid IN ( - SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' ); } step "s2-coordinator-get_global_active_transactions" { SELECT count(*) FROM get_global_active_transactions() WHERE global_pid IN ( - SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' + SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' ) AND transaction_number != 0; } // worker - coordinator -permutation "s1-start-session-level-connection" "s1-worker-begin" "s1-worker-select" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_worker_stat_activity" "s1-worker-commit" "s1-stop-session-level-connection" +permutation "s1-start-session-level-connection" "s1-worker-begin" "s1-worker-select" "s2-coordinator-citus_stat_activity" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_stat_activity-in-workers" "s1-worker-commit" "s1-stop-session-level-connection" // coordinator - coordinator -permutation "s1-coordinator-begin" "s1-coordinator-select" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_worker_stat_activity" "s2-coordinator-get_all_active_transactions" "s2-coordinator-get_global_active_transactions" "s1-coordinator-commit" +permutation "s1-coordinator-begin" "s1-coordinator-select" "s2-coordinator-citus_stat_activity" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_stat_activity-in-workers" "s2-coordinator-get_all_active_transactions" "s2-coordinator-get_global_active_transactions" "s1-coordinator-commit" diff --git a/src/test/regress/spec/isolation_max_client_connections.spec b/src/test/regress/spec/isolation_max_client_connections.spec new file mode 100644 index 000000000..d7d80ed76 --- /dev/null +++ b/src/test/regress/spec/isolation_max_client_connections.spec @@ -0,0 +1,66 @@ +setup +{ + SET citus.shard_replication_factor TO 1; + + CREATE USER my_user; + SELECT run_command_on_workers('CREATE USER my_user'); + + CREATE TABLE my_table (test_id integer NOT NULL, data text); + SELECT create_distributed_table('my_table', 'test_id'); + + GRANT USAGE ON SCHEMA public TO my_user; + GRANT SELECT ON TABLE my_table TO my_user; + + CREATE FUNCTION make_external_connection_to_node(text,int,text,text) + RETURNS void + AS 'citus' + LANGUAGE C STRICT; + + SELECT run_command_on_workers('ALTER SYSTEM SET citus.max_client_connections TO 1'); + SELECT run_command_on_workers('SELECT pg_reload_conf()'); +} + +teardown +{ + SELECT run_command_on_workers('ALTER SYSTEM RESET citus.max_client_connections'); + SELECT run_command_on_workers('SELECT pg_reload_conf()'); +} + +session "s1" + +// Setup runs as a transaction, so run_command_on_placements must be separate +step "s1-grant" +{ + SELECT result FROM run_command_on_placements('my_table', 'GRANT SELECT ON TABLE %s TO my_user'); +} + +// Open one external connection as non-superuser, is allowed +step "s1-connect" +{ + SELECT make_external_connection_to_node('localhost', 57637, 'my_user', current_database()); +} + +session "s2" + +// Open another external connection as non-superuser, not allowed +step "s2-connect" +{ + SELECT make_external_connection_to_node('localhost', 57637, 'my_user', current_database()); +} + +// Open another external connection as superuser, allowed +step "s2-connect-superuser" +{ + SELECT make_external_connection_to_node('localhost', 57637, 'postgres', current_database()); +} + +session "s3" + +// Open internal connections as non-superuser, allowed +step "s3-select" +{ + SET ROLE my_user; + SELECT count(*) FROM my_table; +} + +permutation "s1-grant" "s1-connect" "s2-connect" "s2-connect-superuser" "s3-select" 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..f55f869dd 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,19 +81,24 @@ 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, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE backend_type = 'client backend' AND query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%pg_isolation_test_session_is_blocked%'), ('%BEGIN%'), ('%add_node%')) ORDER BY query DESC; } step "s2-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%' AND - query NOT ILIKE '%dump_local_%' AND - query NOT ILIKE '%citus_internal_local_blocked_processes%' - ORDER BY query, query_hostport DESC; + SELECT query, state, wait_event_type, wait_event, usename, datname + FROM citus_stat_activity + WHERE query NOT ILIKE ALL(VALUES + ('%pg_prepared_xacts%'), + ('%COMMIT%'), + ('%dump_local_%'), + ('%citus_internal_local_blocked_processes%'), + ('%add_node%'), + ('%csa_from_one_node%')) + AND is_worker_query = true + AND backend_type = 'client backend' + AND query != '' + ORDER BY query DESC; } @@ -123,14 +126,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..9c62ee074 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 pg_catalog.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 pg_catalog.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 pg_catalog.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/alter_table_set_access_method.sql b/src/test/regress/sql/alter_table_set_access_method.sql index 3d3ff2599..7ddadc531 100644 --- a/src/test/regress/sql/alter_table_set_access_method.sql +++ b/src/test/regress/sql/alter_table_set_access_method.sql @@ -173,10 +173,10 @@ CREATE TABLE table_type_pg_local (a INT); SELECT table_name, citus_table_type, distribution_column, shard_count, access_method FROM public.citus_tables WHERE table_name::text LIKE 'table\_type%' ORDER BY 1; SELECT c.relname, a.amname FROM pg_class c, pg_am a where c.relname SIMILAR TO 'table_type\D*' AND c.relnamespace = 'alter_table_set_access_method'::regnamespace AND c.relam = a.oid; -SELECT alter_table_set_access_method('table_type_dist', 'fake_am'); -SELECT alter_table_set_access_method('table_type_ref', 'fake_am'); -SELECT alter_table_set_access_method('table_type_pg_local', 'fake_am'); -SELECT alter_table_set_access_method('table_type_citus_local', 'fake_am'); +SELECT alter_table_set_access_method('table_type_dist', 'columnar'); +SELECT alter_table_set_access_method('table_type_ref', 'columnar'); +SELECT alter_table_set_access_method('table_type_pg_local', 'columnar'); +SELECT alter_table_set_access_method('table_type_citus_local', 'columnar'); SELECT table_name, citus_table_type, distribution_column, shard_count, access_method FROM public.citus_tables WHERE table_name::text LIKE 'table\_type%' ORDER BY 1; SELECT c.relname, a.amname FROM pg_class c, pg_am a where c.relname SIMILAR TO 'table_type\D*' AND c.relnamespace = 'alter_table_set_access_method'::regnamespace AND c.relam = a.oid; diff --git a/src/test/regress/sql/citus_local_tables.sql b/src/test/regress/sql/citus_local_tables.sql index 3272bdfbd..ae3db0060 100644 --- a/src/test/regress/sql/citus_local_tables.sql +++ b/src/test/regress/sql/citus_local_tables.sql @@ -181,7 +181,11 @@ CREATE FOREIGN TABLE foreign_table ( -- observe that we do not create fdw server for shell table, both shard relation -- & shell relation points to the same same server object +-- Disable metadata sync since citus doesn't support distributing +-- foreign data wrappers for now. +SET citus.enable_metadata_sync TO OFF; SELECT citus_add_local_table_to_metadata('foreign_table'); +RESET citus.enable_metadata_sync; DROP FOREIGN TABLE foreign_table; diff --git a/src/test/regress/sql/coordinator_evaluation.sql b/src/test/regress/sql/coordinator_evaluation.sql index 1f313a76a..047d19c3f 100644 --- a/src/test/regress/sql/coordinator_evaluation.sql +++ b/src/test/regress/sql/coordinator_evaluation.sql @@ -215,13 +215,20 @@ SELECT count(*) FROM coordinator_evaluation_table_2 WHERE key = 101; CREATE TYPE comptype_int as (int_a int); CREATE DOMAIN domain_comptype_int AS comptype_int CHECK ((VALUE).int_a > 0); -- citus does not propagate domain types +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. SELECT run_command_on_workers( $$ CREATE DOMAIN coordinator_evaluation.domain_comptype_int AS coordinator_evaluation.comptype_int CHECK ((VALUE).int_a > 0) $$); CREATE TABLE reference_table(column_a coordinator_evaluation.domain_comptype_int); + +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_reference_table('reference_table'); +RESET citus.enable_metadata_sync; INSERT INTO reference_table (column_a) VALUES ('(1)'); INSERT INTO reference_table (column_a) VALUES ('(2)'), ('(3)'); 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/disable_object_propagation.sql b/src/test/regress/sql/disable_object_propagation.sql index 431d56d61..d7573b777 100644 --- a/src/test/regress/sql/disable_object_propagation.sql +++ b/src/test/regress/sql/disable_object_propagation.sql @@ -45,7 +45,7 @@ DROP TABLE t4; -- as long as the table is using the type some operations are har COMMIT; -- verify the type is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid; +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid; ALTER TYPE tt3 ADD ATTRIBUTE c int, DROP ATTRIBUTE b, ALTER ATTRIBUTE a SET DATA TYPE text COLLATE "POSIX"; ALTER TYPE tt3 OWNER TO typeowner_for_disabled_object_propagation_guc; diff --git a/src/test/regress/sql/distributed_functions.sql b/src/test/regress/sql/distributed_functions.sql index 97a97fd9d..9d31dbc1e 100644 --- a/src/test/regress/sql/distributed_functions.sql +++ b/src/test/regress/sql/distributed_functions.sql @@ -180,7 +180,7 @@ SELECT create_distributed_table('streaming_table','id'); -- if not paremeters are supplied, we'd see that function doesn't have -- distribution_argument_index and colocationid SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)'); -SELECT distribution_argument_index is NULL, colocationid is NULL from citus.pg_dist_object +SELECT distribution_argument_index is NULL, colocationid is NULL from pg_catalog.pg_dist_object WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure; -- also show that we can use the function @@ -380,7 +380,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', '$1' -- show that the colocationIds are the same SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; @@ -388,7 +388,7 @@ WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass -- group preserved, because we're using the default shard creation settings SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val1'); SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; @@ -405,13 +405,13 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', colo -- to coerce the values SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_with:='replicated_table_func_test_4'); SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure; SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:='replicated_table_func_test_4'); SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated -FROM pg_dist_partition, citus.pg_dist_object as objects +FROM pg_dist_partition, pg_catalog.pg_dist_object as objects WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND objects.objid = 'add_text(text, text)'::regprocedure; 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..e22533101 100644 --- a/src/test/regress/sql/distributed_types.sql +++ b/src/test/regress/sql/distributed_types.sql @@ -262,12 +262,19 @@ UPDATE field_indirection_test_2 SET (ct2_col.text_1, ct1_col.int_2) = ('text2', CREATE TYPE two_ints as (if1 int, if2 int); CREATE DOMAIN domain AS two_ints CHECK ((VALUE).if1 > 0); -- citus does not propagate domain objects +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. SELECT run_command_on_workers( $$ CREATE DOMAIN type_tests.domain AS type_tests.two_ints CHECK ((VALUE).if1 > 0); $$); CREATE TABLE domain_indirection_test (f1 int, f3 domain, domain_array domain[]); + +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_table('domain_indirection_test', 'f1'); +RESET citus.enable_metadata_sync; -- not supported (field indirection to underlying composite type) INSERT INTO domain_indirection_test (f1,f3.if1, f3.if2) VALUES (0, 1, 2); @@ -281,6 +288,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/drop_partitioned_table.sql b/src/test/regress/sql/drop_partitioned_table.sql index fc6e3ac4d..a9842b10a 100644 --- a/src/test/regress/sql/drop_partitioned_table.sql +++ b/src/test/regress/sql/drop_partitioned_table.sql @@ -240,3 +240,75 @@ ROLLBACK; DROP SCHEMA drop_partitioned_table CASCADE; SET search_path TO public; + +-- dropping the schema should drop the metadata on the workers +CREATE SCHEMA partitioning_schema; +SET search_path TO partitioning_schema; + +CREATE TABLE part_table ( + col timestamp + ) PARTITION BY RANGE (col); + +CREATE TABLE part_table_1 + PARTITION OF part_table + FOR VALUES FROM ('2010-01-01') TO ('2015-01-01'); + +SELECT create_distributed_table('part_table', 'col'); + +-- show we have pg_dist_partition entries on the workers +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); +-- show we have pg_dist_object entries on the workers +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + +DROP SCHEMA partitioning_schema CASCADE; + +-- show we don't have pg_dist_partition entries on the workers after dropping the schema +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); + +-- show we don't have pg_dist_object entries on the workers after dropping the schema +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + +-- dropping the parent should drop the metadata on the workers +CREATE SCHEMA partitioning_schema; +SET search_path TO partitioning_schema; + +CREATE TABLE part_table ( + col timestamp + ) PARTITION BY RANGE (col); + +CREATE TABLE part_table_1 + PARTITION OF part_table + FOR VALUES FROM ('2010-01-01') TO ('2015-01-01'); + +SELECT create_distributed_table('part_table', 'col'); + +DROP TABLE part_table; + +-- show we don't have pg_dist_partition entries on the workers after dropping the parent +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); + +-- show we don't have pg_dist_object entries on the workers after dropping the parent +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + +SET search_path TO partitioning_schema; + +CREATE TABLE part_table ( + col timestamp + ) PARTITION BY RANGE (col); + +CREATE TABLE part_table_1 + PARTITION OF part_table + FOR VALUES FROM ('2010-01-01') TO ('2015-01-01'); + +SELECT create_distributed_table('part_table', 'col'); + +DROP TABLE part_table_1; + +-- show we have pg_dist_partition entries for the parent on the workers after dropping the partition +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$); + +-- show we have pg_dist_object entries for the parent on the workers after dropping the partition +SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$); + +-- clean-up +DROP SCHEMA partitioning_schema CASCADE; diff --git a/src/test/regress/sql/function_propagation.sql b/src/test/regress/sql/function_propagation.sql index 79168497b..18137a67d 100644 --- a/src/test/regress/sql/function_propagation.sql +++ b/src/test/regress/sql/function_propagation.sql @@ -18,12 +18,12 @@ END; $$; -- Check all dependent objects and function depends on all nodes -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2; SET citus.enable_metadata_sync TO OFF; CREATE TYPE function_prop_type_2 AS (a int, b int); @@ -38,10 +38,10 @@ BEGIN END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2; -- Have a separate check for type created in transaction BEGIN; @@ -60,9 +60,9 @@ BEGIN END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2; -- Check table CREATE TABLE function_prop_table(a int, b int); @@ -97,8 +97,8 @@ BEGIN END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2; -- Views are not supported CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table; @@ -133,8 +133,8 @@ BEGIN; $$; -- Within transaction functions are not distributed - SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; - SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; COMMIT; -- Show that recreating it outside transaction distributes the function and dependencies @@ -147,10 +147,10 @@ BEGIN END; $$; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2; -- Test for SQL function with unsupported object in function body CREATE TABLE table_in_sql_body(id int); @@ -163,9 +163,9 @@ $$ $$; -- Show that only function has propagated, since the table is not resolved as dependency -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2; -- Check extension owned table CREATE TABLE extension_owned_table(a int); @@ -188,6 +188,574 @@ 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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2; + + +-- Adding a column with default function depending on non-distributable table should fail +BEGIN; + CREATE TABLE non_dist_table_for_function(id int); + + CREATE OR REPLACE FUNCTION non_dist_func(col_1 non_dist_table_for_function) + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + CREATE TABLE table_to_dist(id int); + SELECT create_distributed_table('table_to_dist', 'id'); + + ALTER TABLE table_to_dist ADD COLUMN col_1 int default function_propagation_schema.non_dist_func(NULL::non_dist_table_for_function); + +ROLLBACK; + + +-- 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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2; + + +-- Show that having a dependency on another dist table work out tx +CREATE TABLE loc_for_func_dist ( + product_no integer, + name text, + price numeric CONSTRAINT positive_price CHECK (price > 0)); + +SELECT create_distributed_table('loc_for_func_dist', 'product_no'); + +CREATE OR REPLACE FUNCTION non_sense_func_for_default_val(loc_for_func_dist) +RETURNS int +LANGUAGE plpgsql IMMUTABLE AS +$$ +BEGIN +return 1; +END; +$$; + +CREATE TABLE table_non_for_func_dist ( + a int, + b int DEFAULT non_sense_func_for_default_val(NULL::loc_for_func_dist)); + +SELECT create_distributed_table('table_non_for_func_dist', 'a'); +SET citus.shard_replication_factor = 1; +-- test creating a colocated function +CREATE TABLE tbl_to_colocate (a int); +SELECT create_distributed_table('tbl_to_colocate', 'a'); +-- first test colocating function with a ref table +CREATE TABLE tbl_to_colocate_ref (a int); +SELECT create_reference_table('tbl_to_colocate_ref'); + +CREATE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql; +-- see the empty pg_dist_object entries +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + +-- colocate the function with ref table +SELECT create_distributed_function('func_to_colocate(int)', colocate_with:='tbl_to_colocate_ref'); +-- see the pg_dist_object entry +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + +-- convert to non-delegated +SELECT create_distributed_function('func_to_colocate(int)'); +-- show that the pg_dist_object fields are gone +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + +-- colocate the function with distributed table +SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate'); +-- see the pg_dist_object entry +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + +-- try create or replace the same func +CREATE OR REPLACE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql; +-- verify the pg_dist_object entry is the same +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + +-- convert to non-delegated +SELECT create_distributed_function('func_to_colocate(int)'); +-- show that the pg_dist_object fields are gone +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + +-- force delegate +SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate', true); +-- show pg_dist_object fields +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + +-- convert to non-delegated +SELECT create_distributed_function('func_to_colocate(int)'); +-- show that the pg_dist_object fields are gone +SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc; + RESET search_path; SET client_min_messages TO WARNING; DROP SCHEMA function_propagation_schema CASCADE; diff --git a/src/test/regress/sql/global_cancel.sql b/src/test/regress/sql/global_cancel.sql index edf380771..6c4341877 100644 --- a/src/test/regress/sql/global_cancel.sql +++ b/src/test/regress/sql/global_cancel.sql @@ -29,14 +29,17 @@ CREATE USER global_cancel_user; SELECT 1 FROM run_command_on_workers('CREATE USER global_cancel_user'); RESET client_min_messages; -SET ROLE global_cancel_user; +\c - global_cancel_user - :master_port SELECT pg_typeof(:maintenance_daemon_gpid); SELECT pg_cancel_backend(:maintenance_daemon_gpid); SELECT pg_terminate_backend(:maintenance_daemon_gpid); -RESET ROLE; +-- we can cancel our own backend +SELECT pg_cancel_backend(citus_backend_gpid()); + +\c - postgres - :master_port SELECT nodeid AS coordinator_node_id FROM pg_dist_node WHERE nodeport = :master_port \gset @@ -48,4 +51,13 @@ SELECT pg_terminate_backend(10000000000 * :coordinator_node_id + 0); RESET client_min_messages; +SELECT citus_backend_gpid() = citus_calculate_gpid(:coordinator_node_id, pg_backend_pid()); + +SELECT nodename = citus_nodename_for_nodeid(nodeid) AND nodeport = citus_nodeport_for_nodeid(nodeid) +FROM pg_dist_node +WHERE isactive = true AND noderole = 'primary'; + +SELECT citus_nodeid_for_gpid(10000000000 * 2 + 3); +SELECT citus_pid_for_gpid(10000000000 * 2 + 3); + DROP SCHEMA global_cancel 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/master_copy_shard_placement.sql b/src/test/regress/sql/master_copy_shard_placement.sql index 6448cb63b..8d5ed13b3 100644 --- a/src/test/regress/sql/master_copy_shard_placement.sql +++ b/src/test/regress/sql/master_copy_shard_placement.sql @@ -93,8 +93,8 @@ SELECT count(*) FROM history; SET citus.shard_replication_factor TO 1; -- metadata sync will succeed even if we have rep > 1 tables -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0); SELECT start_metadata_sync_to_node('localhost', :worker_1_port); CREATE TABLE mx_table(a int); diff --git a/src/test/regress/sql/multi_cluster_management.sql b/src/test/regress/sql/multi_cluster_management.sql index f5a67e148..b24c79232 100644 --- a/src/test/regress/sql/multi_cluster_management.sql +++ b/src/test/regress/sql/multi_cluster_management.sql @@ -46,6 +46,7 @@ SELECT * FROM rebalance_table_shards(); -- TODO: Figure out why this is necessary, rebalance_table_shards shouldn't -- insert stuff into pg_dist_colocation TRUNCATE pg_dist_colocation; +SELECT run_command_on_workers('TRUNCATE pg_dist_colocation'); ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000; SELECT 1 FROM citus_activate_node('localhost', :worker_2_port); @@ -107,8 +108,8 @@ SELECT run_command_on_workers('GRANT ALL ON SCHEMA citus TO node_metadata_user') SELECT master_remove_node('localhost', :worker_2_port); -- Removing public schema from pg_dist_object because it breaks the next tests -DELETE FROM citus.pg_dist_object WHERE objid = 'public'::regnamespace::oid; -DELETE FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql'); +DELETE FROM pg_catalog.pg_dist_object WHERE objid = 'public'::regnamespace::oid; +DELETE FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql'); -- try to manipulate node metadata via non-super user SET ROLE non_super_user; diff --git a/src/test/regress/sql/multi_colocation_utils.sql b/src/test/regress/sql/multi_colocation_utils.sql index 6729b1cbf..245c2ce6d 100644 --- a/src/test/regress/sql/multi_colocation_utils.sql +++ b/src/test/regress/sql/multi_colocation_utils.sql @@ -4,6 +4,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 4; -- Delete orphaned entries from pg_dist_colocation DELETE FROM pg_dist_colocation where colocationid = 5 or colocationid = 6; +SELECT 1 FROM run_command_on_workers('DELETE FROM pg_dist_colocation where colocationid = 5 or colocationid = 6'); -- =================================================================== -- create test utility function @@ -65,11 +66,6 @@ CREATE FUNCTION find_shard_interval_index(bigint) AS 'citus' LANGUAGE C STRICT; --- remove tables from pg_dist_partition, if they don't exist i.e not found in pg_class -delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid); -select 1 from run_command_on_workers($$ - delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid);$$); - -- =================================================================== -- test co-location util functions -- =================================================================== @@ -161,6 +157,7 @@ SELECT find_shard_interval_index(1300016); SELECT count(*) FROM pg_dist_partition WHERE colocationid IN (4, 5); DELETE FROM pg_dist_colocation WHERE colocationid IN (4, 5); +SELECT 1 FROM run_command_on_workers('DELETE FROM pg_dist_colocation WHERE colocationid IN (4, 5)'); SET citus.shard_count = 2; @@ -213,6 +210,12 @@ SELECT * FROM pg_dist_colocation WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY colocationid; +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c) ORDER BY colocationid) FROM pg_dist_colocation c + WHERE colocationid >= 1 AND colocationid < 1000 +$$); + SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY logicalrelid; @@ -221,10 +224,21 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition DROP TABLE table1_groupA; SELECT * FROM pg_dist_colocation WHERE colocationid = 4; +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c)) FROM pg_dist_colocation c WHERE colocationid = 4 +$$); + + -- dropping all tables in a colocation group also deletes the colocation group DROP TABLE table2_groupA; SELECT * FROM pg_dist_colocation WHERE colocationid = 4; +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c)) FROM pg_dist_colocation c WHERE colocationid = 4 +$$); + -- create dropped colocation group again SET citus.shard_count = 2; @@ -350,6 +364,7 @@ ORDER BY ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1; DELETE FROM pg_dist_colocation WHERE colocationid >= 1 AND colocationid < 1000; +SELECT 1 FROM run_command_on_workers('DELETE FROM pg_dist_colocation WHERE colocationid >= 1 AND colocationid < 1000'); UPDATE pg_dist_partition SET colocationid = 0 WHERE colocationid >= 1 AND colocationid < 1000; @@ -401,6 +416,12 @@ SELECT * FROM pg_dist_colocation WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY colocationid; +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c) ORDER BY colocationid) FROM pg_dist_colocation c + WHERE colocationid >= 1 AND colocationid < 1000 +$$); + SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY colocationid, logicalrelid; @@ -411,11 +432,11 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with => SELECT update_distributed_table_colocation('table1_group_none', colocate_with => 'table3_groupE'); -- activate nodes to get rid of inconsistencies in pg_dist tables -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0); -INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0); +INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0); SELECT 1 FROM citus_activate_node('localhost', :worker_1_port); SELECT 1 FROM citus_activate_node('localhost', :worker_2_port); @@ -427,6 +448,12 @@ SELECT * FROM pg_dist_colocation WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY colocationid; +-- check to see whether metadata is synced +SELECT nodeport, unnest(result::jsonb[]) FROM run_command_on_workers($$ +SELECT array_agg(row_to_json(c) ORDER BY colocationid) FROM pg_dist_colocation c + WHERE colocationid >= 1 AND colocationid < 1000 +$$); + SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE colocationid >= 1 AND colocationid < 1000 ORDER BY colocationid, logicalrelid; diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index aaed45803..7647d61b2 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -213,16 +213,17 @@ SELECT * FROM multi_extension.print_extension_changes(); -- Test downgrade to 9.4-1 from 9.5-1 ALTER EXTENSION citus UPDATE TO '9.5-1'; -BEGIN; - SET citus.enable_metadata_sync TO on; - SELECT master_add_node('localhost', :master_port, groupId=>0); - CREATE TABLE citus_local_table (a int); - SELECT create_citus_local_table('citus_local_table'); - RESET citus.enable_metadata_sync; - - -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table - ALTER EXTENSION citus UPDATE TO '9.4-1'; -ROLLBACK; +-- TODO: This test should be moved to a valid downgrade testing suite where the downgrade is done, both on the schema and the binaries. Later changes in Citus made a C vs Schema discrepancy error here +-- BEGIN; +-- SET citus.enable_metadata_sync TO on; +-- SELECT master_add_node('localhost', :master_port, groupId=>0); +-- CREATE TABLE citus_local_table (a int); +-- SELECT create_citus_local_table('citus_local_table'); +-- RESET citus.enable_metadata_sync; +-- +-- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table +-- ALTER EXTENSION citus UPDATE TO '9.4-1'; +-- ROLLBACK; -- now we can downgrade as there is no citus local table ALTER EXTENSION citus UPDATE TO '9.4-1'; @@ -430,7 +431,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 +527,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 +536,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_metadata_attributes.sql b/src/test/regress/sql/multi_metadata_attributes.sql index 5cb1a89a9..58351310c 100644 --- a/src/test/regress/sql/multi_metadata_attributes.sql +++ b/src/test/regress/sql/multi_metadata_attributes.sql @@ -10,5 +10,5 @@ FROM pg_attribute WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, 'pg_dist_rebalance_strategy'::regclass, 'pg_dist_partition'::regclass, - 'citus.pg_dist_object'::regclass) + 'pg_dist_object'::regclass) ORDER BY attrelid, attname; diff --git a/src/test/regress/sql/multi_metadata_sync.sql b/src/test/regress/sql/multi_metadata_sync.sql index 13c984396..b2f457e5a 100644 --- a/src/test/regress/sql/multi_metadata_sync.sql +++ b/src/test/regress/sql/multi_metadata_sync.sql @@ -129,7 +129,7 @@ SELECT "Column", "Type", "Definition" FROM index_attrs WHERE SELECT "Column", "Type", "Definition" FROM index_attrs WHERE relid = 'mx_testing_schema.mx_index'::regclass; --- Check that pg_dist_colocation is not synced +-- Check that pg_dist_colocation is synced SELECT * FROM pg_dist_colocation ORDER BY colocationid; -- Make sure that truncate trigger has been set for the MX table on worker @@ -544,10 +544,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence; -- owner CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement; CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition; -CREATE TABLE pg_dist_object_temp AS SELECT * FROM citus.pg_dist_object; +CREATE TABLE pg_dist_object_temp AS SELECT * FROM pg_catalog.pg_dist_object; DELETE FROM pg_dist_placement; DELETE FROM pg_dist_partition; -DELETE FROM citus.pg_dist_object; +DELETE FROM pg_catalog.pg_dist_object; SELECT groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset SELECT master_remove_node('localhost', :worker_2_port); @@ -586,7 +586,7 @@ DROP TABLE mx_table; \c - postgres - :master_port INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp; INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_temp; -INSERT INTO citus.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING; +INSERT INTO pg_catalog.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING; DROP TABLE pg_dist_placement_temp; DROP TABLE pg_dist_partition_temp; DROP TABLE pg_dist_object_temp; @@ -637,6 +637,9 @@ ORDER BY SELECT shardid AS ref_table_shardid FROM pg_dist_shard WHERE logicalrelid='mx_ref'::regclass \gset +-- make sure we have the pg_dist_colocation record on the worker +SELECT count(*) FROM pg_dist_colocation WHERE distributioncolumntype = 0; + -- Check that DDL commands are propagated to reference tables on workers \c - - - :master_port ALTER TABLE mx_ref ADD COLUMN col_3 NUMERIC DEFAULT 0; diff --git a/src/test/regress/sql/multi_mx_alter_distributed_table.sql b/src/test/regress/sql/multi_mx_alter_distributed_table.sql index 7134527f5..3bd2f2433 100644 --- a/src/test/regress/sql/multi_mx_alter_distributed_table.sql +++ b/src/test/regress/sql/multi_mx_alter_distributed_table.sql @@ -67,7 +67,7 @@ END;$$; SELECT create_distributed_function('proc_0(float8)', 'dist_key', 'test_proc_colocation_0' ); SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); SET client_min_messages TO DEBUG1; CALL proc_0(1.0); @@ -81,7 +81,7 @@ CALL proc_0(1.0); RESET client_min_messages; SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); -- colocatewith is not null && list_length(colocatedTableList) = 1 SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4); @@ -94,7 +94,7 @@ CALL proc_0(1.0); RESET client_min_messages; SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); -- shardCount is not null && cascade_to_colocated is true SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 8, cascade_to_colocated := true); @@ -104,7 +104,7 @@ CALL proc_0(1.0); RESET client_min_messages; SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); -- colocatewith is not null && cascade_to_colocated is true SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4, cascade_to_colocated := true); @@ -117,7 +117,7 @@ CALL proc_0(1.0); RESET client_min_messages; SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0'); +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0'); -- try a case with more than one procedure CREATE OR REPLACE procedure proc_1(dist_key float8) @@ -134,7 +134,7 @@ END;$$; SELECT create_distributed_function('proc_1(float8)', 'dist_key', 'test_proc_colocation_0' ); SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; SET client_min_messages TO DEBUG1; CALL proc_0(1.0); @@ -149,14 +149,14 @@ CALL proc_1(2.0); RESET client_min_messages; SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; -- case which shouldn't preserve colocation for now -- shardCount is not null && cascade_to_colocated is false SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 18, cascade_to_colocated := false); SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0'); -SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; +SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname; SET client_min_messages TO WARNING; DROP SCHEMA mx_alter_distributed_table CASCADE; diff --git a/src/test/regress/sql/multi_mx_node_metadata.sql b/src/test/regress/sql/multi_mx_node_metadata.sql index a54917f87..e9b375337 100644 --- a/src/test/regress/sql/multi_mx_node_metadata.sql +++ b/src/test/regress/sql/multi_mx_node_metadata.sql @@ -388,6 +388,7 @@ DROP SEQUENCE sequence CASCADE; DROP TABLE ref_table; DROP TABLE reference_table; TRUNCATE pg_dist_colocation; +SELECT run_command_on_workers('TRUNCATE pg_dist_colocation'); SELECT count(*) FROM (SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node) t; ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART :last_group_id; ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART :last_node_id; 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_mx_schema_support.sql b/src/test/regress/sql/multi_mx_schema_support.sql index 80cc61311..52bdb4588 100644 --- a/src/test/regress/sql/multi_mx_schema_support.sql +++ b/src/test/regress/sql/multi_mx_schema_support.sql @@ -308,7 +308,7 @@ SELECT create_distributed_table('mx_old_schema.table_set_schema', 'id'); CREATE SCHEMA mx_new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema') ORDER BY "Distributed Schemas"; \c - - - :worker_1_port @@ -325,7 +325,7 @@ ALTER SCHEMA mx_old_schema RENAME TO temp_mx_old_schema; ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema'); \c - - - :worker_1_port SELECT table_schema AS "Table's Schema" FROM information_schema.tables WHERE table_name='table_set_schema'; diff --git a/src/test/regress/sql/multi_prepare_sql.sql b/src/test/regress/sql/multi_prepare_sql.sql index 54893d025..37a5f0690 100644 --- a/src/test/regress/sql/multi_prepare_sql.sql +++ b/src/test/regress/sql/multi_prepare_sql.sql @@ -546,6 +546,9 @@ DROP TABLE text_partition_column_table; -- Domain type columns can give issues CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$'); + +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. SELECT run_command_on_workers($$ CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$') $$); @@ -554,7 +557,12 @@ CREATE TABLE domain_partition_column_table ( key test_key NOT NULL, value int ); + +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_table('domain_partition_column_table', 'key'); +RESET citus.enable_metadata_sync; PREPARE prepared_coercion_to_domain_insert(text) AS INSERT INTO domain_partition_column_table VALUES ($1, 1); 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/multi_schema_support.sql b/src/test/regress/sql/multi_schema_support.sql index 02ca41d01..2f7e040e0 100644 --- a/src/test/regress/sql/multi_schema_support.sql +++ b/src/test/regress/sql/multi_schema_support.sql @@ -672,7 +672,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id'); CREATE SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); \c - - - :worker_1_port SELECT table_schema AS "Shards' Schema" @@ -685,7 +685,7 @@ SELECT table_schema AS "Shards' Schema" ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); \c - - - :worker_1_port SELECT table_schema AS "Shards' Schema" @@ -706,7 +706,7 @@ SELECT create_distributed_table('table_set_schema', 'id'); CREATE SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid='new_schema'::regnamespace::oid; \c - - - :worker_1_port SELECT table_schema AS "Shards' Schema" @@ -719,7 +719,7 @@ SELECT table_schema AS "Shards' Schema" ALTER TABLE table_set_schema SET SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid='new_schema'::regnamespace::oid; \c - - - :worker_1_port SELECT table_schema AS "Shards' Schema" @@ -743,7 +743,7 @@ SELECT create_distributed_table('table_set_schema', 'id'); CREATE SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); \c - - - :worker_1_port SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts" @@ -757,7 +757,7 @@ SET search_path TO old_schema; ALTER TABLE table_set_schema SET SCHEMA new_schema; SELECT objid::oid::regnamespace as "Distributed Schemas" - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema'); \c - - - :worker_1_port SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts" @@ -969,9 +969,9 @@ ROLLBACK; -- Clean up the created schema SET client_min_messages TO WARNING; -SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema'); DROP SCHEMA run_test_schema, test_schema_support_join_1, test_schema_support_join_2, "Citus'Teen123", "CiTUS.TEEN2", bar, test_schema_support CASCADE; -- verify that the dropped schema is removed from worker's pg_dist_object -SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema'); diff --git a/src/test/regress/sql/multi_sequence_default.sql b/src/test/regress/sql/multi_sequence_default.sql index 8c664f393..a185d5cff 100644 --- a/src/test/regress/sql/multi_sequence_default.sql +++ b/src/test/regress/sql/multi_sequence_default.sql @@ -418,8 +418,8 @@ CREATE TABLE test_seq_dist(a int, x BIGINT DEFAULT nextval('test_schema_for_sequ SELECT create_distributed_table('test_seq_dist', 'a'); -- Both sequence and dependency schema should be distributed -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass); -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass); +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace); -- Show that sequence can stay on the worker node if the transaction is -- rollbacked after distributing the table diff --git a/src/test/regress/sql/multi_test_helpers.sql b/src/test/regress/sql/multi_test_helpers.sql index e89eefdc4..8c55fe964 100644 --- a/src/test/regress/sql/multi_test_helpers.sql +++ b/src/test/regress/sql/multi_test_helpers.sql @@ -91,7 +91,7 @@ END;$$; CREATE OR REPLACE FUNCTION colocate_proc_with_table(procname text, tablerelid regclass, argument_index int) RETURNS void LANGUAGE plpgsql AS $$ BEGIN - update citus.pg_dist_object + update pg_catalog.pg_dist_object set distribution_argument_index = argument_index, colocationid = pg_dist_partition.colocationid from pg_proc, pg_dist_partition where proname = procname and oid = objid and pg_dist_partition.logicalrelid = tablerelid; diff --git a/src/test/regress/sql/non_super_user_object_metadata.sql b/src/test/regress/sql/non_super_user_object_metadata.sql index 63d16800b..e94d177f0 100644 --- a/src/test/regress/sql/non_super_user_object_metadata.sql +++ b/src/test/regress/sql/non_super_user_object_metadata.sql @@ -17,11 +17,11 @@ CREATE SCHEMA test_sequence_schema; CREATE SEQUENCE test_sequence_schema.test_sequence; -- show that none of the objects above are marked as distributed -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; SET client_min_messages TO ERROR; CREATE USER non_super_user_test_user; @@ -70,28 +70,28 @@ SELECT create_distributed_function('plpgsql_dist_function(text)'); -- show that schema, types, function and sequence has marked as distributed -- on the coordinator node RESET ROLE; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; -SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid; +SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid; -- show those objects marked as distributed on metadata worker node as well -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2; -SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::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 = 'local_schema.dist_table_e_seq'::regclass::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 = 'local_schema.test_function'::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 = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2; -- Show that extension plpgsql is also marked as distributed as a dependency of plpgsl_dist_function -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2; -- show that schema is owned by the superuser SELECT rolname FROM pg_roles JOIN pg_namespace ON(pg_namespace.nspowner = pg_roles.oid) WHERE nspname = 'local_schema'; @@ -122,36 +122,36 @@ DROP TABLE dist_table CASCADE; RESET ROLE; SET search_path TO local_schema; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2; -- Show that altering the function's schema marks the schema distributed CREATE SCHEMA schema_to_prop_with_function; ALTER FUNCTION test_function SET SCHEMA schema_to_prop_with_function; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2; -- Show that dropping the function removes the metadata from pg_dist_object -- on both coordinator and metadata worker node DROP FUNCTION schema_to_prop_with_function.test_function; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2; -- Show that altering the type's schema marks the schema distributed CREATE SCHEMA schema_to_prop_with_type; ALTER TYPE test_type SET SCHEMA schema_to_prop_with_type; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2; -- Show that dropping type removes the metadata from pg_dist_object -- on both coordinator and metadata worker node DROP TYPE mood CASCADE; DROP TYPE schema_to_prop_with_type.test_type CASCADE; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2; -- Show that distributed function related metadata are also propagated set citus.shard_replication_factor to 1; @@ -171,41 +171,41 @@ END;$$; -- create a distributed function and show its distribution_argument_index SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)', 'dist_key', 'metadata_dist_test_table'); -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; -SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; +SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; -- re-distribute and show that now the distribution_argument_index is updated on both the coordinator and workers SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)', 'dist_key_2', 'metadata_dist_test_table'); -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; -SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc; +SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2; -- Show that the schema is dropped on worker node as well DROP SCHEMA local_schema CASCADE; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2; -- Show that extension and dependent sequence also created and marked as distributed CREATE SCHEMA extension_schema; CREATE EXTENSION ltree WITH SCHEMA extension_schema; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2; -- Show that dropping a distributed table drops the pg_dist_object entry on worker CREATE TABLE extension_schema.table_to_check_object(id int); SELECT create_distributed_table('extension_schema.table_to_check_object', 'id'); -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; DROP TABLE extension_schema.table_to_check_object; -SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; -SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; +SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%'; +SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2; -- Revert the settings for following tests RESET citus.enable_ddl_propagation; 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/prepared_statements_4.sql b/src/test/regress/sql/prepared_statements_4.sql index e921a83cf..4aa79387c 100644 --- a/src/test/regress/sql/prepared_statements_4.sql +++ b/src/test/regress/sql/prepared_statements_4.sql @@ -16,17 +16,17 @@ SELECT key, value FROM text_partition_column_table ORDER BY key; +-- TODO: Uncomment tests below once domains are supported +-- PREPARE prepared_coercion_to_domain_insert(text) AS +-- INSERT INTO domain_partition_column_table VALUES ($1, 1); -PREPARE prepared_coercion_to_domain_insert(text) AS - INSERT INTO domain_partition_column_table VALUES ($1, 1); - -EXECUTE prepared_coercion_to_domain_insert('test-1'); -EXECUTE prepared_coercion_to_domain_insert('test-2'); -EXECUTE prepared_coercion_to_domain_insert('test-3'); -EXECUTE prepared_coercion_to_domain_insert('test-4'); -EXECUTE prepared_coercion_to_domain_insert('test-5'); -EXECUTE prepared_coercion_to_domain_insert('test-6'); -EXECUTE prepared_coercion_to_domain_insert('test-7'); +-- EXECUTE prepared_coercion_to_domain_insert('test-1'); +-- EXECUTE prepared_coercion_to_domain_insert('test-2'); +-- EXECUTE prepared_coercion_to_domain_insert('test-3'); +-- EXECUTE prepared_coercion_to_domain_insert('test-4'); +-- EXECUTE prepared_coercion_to_domain_insert('test-5'); +-- EXECUTE prepared_coercion_to_domain_insert('test-6'); +-- EXECUTE prepared_coercion_to_domain_insert('test-7'); diff --git a/src/test/regress/sql/prepared_statements_create_load.sql b/src/test/regress/sql/prepared_statements_create_load.sql index b2e5684c0..af7baa026 100644 --- a/src/test/regress/sql/prepared_statements_create_load.sql +++ b/src/test/regress/sql/prepared_statements_create_load.sql @@ -54,16 +54,24 @@ SELECT create_distributed_table('text_partition_column_table', 'key'); -- and we use offset to prevent output diverging CREATE DOMAIN test_key AS text CHECK(VALUE ~ '^test-\d$'); + +-- TODO: Once domains are supported, remove enable_metadata_sync off/on change +-- on dependent table distribution below. Also uncomment related tests on +-- prepared_statements_4 test file. SELECT run_command_on_workers($$ CREATE DOMAIN "prepared statements".test_key AS text CHECK(VALUE ~ '^test-\d$') $$) OFFSET 10000; +-- Disable metadata sync since citus doesn't support distributing +-- domains for now. +SET citus.enable_metadata_sync TO OFF; CREATE TABLE domain_partition_column_table ( key test_key NOT NULL, value int ); -SELECT create_distributed_table('domain_partition_column_table', 'key'); +SELECT create_distributed_table('domain_partition_column_table', 'key'); +RESET citus.enable_metadata_sync; -- verify we re-evaluate volatile functions every time CREATE TABLE http_request ( diff --git a/src/test/regress/sql/propagate_extension_commands.sql b/src/test/regress/sql/propagate_extension_commands.sql index 3640e2f17..ba589cb1c 100644 --- a/src/test/regress/sql/propagate_extension_commands.sql +++ b/src/test/regress/sql/propagate_extension_commands.sql @@ -10,8 +10,8 @@ SET client_min_messages TO WARNING; CREATE EXTENSION seg; -- make sure that both the schema and the extension is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'); CREATE TABLE test_table (key int, value seg); SELECT create_distributed_table('test_table', 'key'); @@ -22,7 +22,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_ CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg); -- verify that the type that depends on the extension is also marked as distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test')); -- now try to run CREATE EXTENSION within a transction block, all should work fine BEGIN; @@ -40,7 +40,7 @@ BEGIN; COMMIT; -- make sure that the extension is distributed even if we run create extension in a transaction block -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'isn'$$); @@ -72,7 +72,7 @@ ALTER EXTENSION isn SET SCHEMA public; SET search_path TO public; -- make sure that the extension is distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); -- show that the ALTER EXTENSION command is propagated SELECT run_command_on_workers($$SELECT nspname from pg_namespace where oid=(SELECT extnamespace FROM pg_extension WHERE extname = 'isn')$$); @@ -156,7 +156,7 @@ ROLLBACK; -- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed -- make sure that the extension is not distributed -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn'); -- and the extension does not exist on workers SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'isn'$$); @@ -184,11 +184,11 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn'); -- drop extension should just work DROP EXTENSION seg CASCADE; -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'seg'$$); -- make sure that the extension is not avaliable anymore as a distributed object -SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); CREATE SCHEMA "extension'test"; SET search_path TO "extension'test"; @@ -257,16 +257,16 @@ COMMIT; -- Check the pg_dist_object SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); @@ -274,15 +274,15 @@ $$); SELECT 1 from master_add_node('localhost', :worker_2_port); -- make sure that both extensions are created on both nodes -SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn')); SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname IN ('seg', 'isn')$$); -- Check the pg_dist_object on the both nodes SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); @@ -290,16 +290,16 @@ DROP EXTENSION seg CASCADE; -- Recheck the pg_dist_object SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); @@ -309,33 +309,33 @@ CREATE EXTENSION seg; SET citus.enable_ddl_propagation TO true; -- Check the extension in pg_dist_object -SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function'); -- Recheck the extension in pg_dist_object -SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND +SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_extension WHERE extname = 'seg'); SELECT pg_proc.proname as DistributedFunction -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; SELECT run_command_on_workers($$ SELECT count(*) -FROM citus.pg_dist_object, pg_proc +FROM pg_catalog.pg_dist_object, pg_proc WHERE pg_proc.proname = 'seg_in' and -pg_proc.oid = citus.pg_dist_object.objid and +pg_proc.oid = pg_catalog.pg_dist_object.objid and classid = 'pg_proc'::regclass; $$); DROP EXTENSION seg; @@ -350,11 +350,11 @@ SELECT create_distributed_table('test_extension_function', 'col1', colocate_with CREATE EXTENSION cube; SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_extension_function'); -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8'); SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_extension_function'); -SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND +SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8'); ROLLBACK; diff --git a/src/test/regress/sql/propagate_foreign_servers.sql b/src/test/regress/sql/propagate_foreign_servers.sql index f27a10e5d..eea09b9ab 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 pg_catalog.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/run_command_on_all_nodes.sql b/src/test/regress/sql/run_command_on_all_nodes.sql new file mode 100644 index 000000000..46f838eb5 --- /dev/null +++ b/src/test/regress/sql/run_command_on_all_nodes.sql @@ -0,0 +1,41 @@ +CREATE SCHEMA run_command_on_all_nodes; +SET search_path TO run_command_on_all_nodes; + +-- check coordinator isn't in metadata +SELECT count(*) != 0 AS "Coordinator is in Metadata" +FROM pg_dist_node +WHERE groupid IN ( + SELECT groupid FROM pg_dist_local_group +); + +-- run a simple select query and check it also runs in coordinator +SELECT nodeid NOT IN (SELECT nodeid FROM pg_dist_node) AS "Is Coordinator", success, result FROM run_command_on_all_nodes('SELECT 1') ORDER BY 1; + +-- check that when coordinator is not in metadata and run_command_on_all_nodes is called from +-- a worker node, command is not run on the coordinator +\c - - - :worker_1_port +SELECT nodeid NOT IN (SELECT nodeid FROM pg_dist_node) AS "Is Coordinator", success, result FROM run_command_on_all_nodes('SELECT 1') ORDER BY 1; + +\c - - - :master_port + +-- create a table +SELECT result FROM run_command_on_all_nodes('CREATE TABLE run_command_on_all_nodes.tbl (a INT)'); + +SELECT tablename FROM pg_tables WHERE schemaname = 'run_command_on_all_nodes'; + +\c - - - :worker_1_port +SELECT tablename FROM pg_tables WHERE schemaname = 'run_command_on_all_nodes'; + +\c - - - :master_port +SELECT result FROM run_command_on_all_nodes('SELECT tablename FROM pg_tables WHERE schemaname = ''run_command_on_all_nodes'';'); + +-- break a node and check messages +SELECT nodeid AS worker_1_nodeid FROM pg_dist_node WHERE nodeport = :worker_1_port \gset +UPDATE pg_dist_node SET nodeport = 0 WHERE nodeid = :worker_1_nodeid; + +SELECT nodeid = :worker_1_nodeid AS "Is Worker 1", success, result FROM run_command_on_all_nodes('SELECT 1') ORDER BY 1; +SELECT nodeid = :worker_1_nodeid AS "Is Worker 1", success, result FROM run_command_on_all_nodes('SELECT 1', give_warning_for_connection_errors:=true) ORDER BY 1; + +UPDATE pg_dist_node SET nodeport = :worker_1_port WHERE nodeid = :worker_1_nodeid; + +DROP SCHEMA run_command_on_all_nodes CASCADE; 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/single_node.sql b/src/test/regress/sql/single_node.sql index 3ca456108..74c857d4e 100644 --- a/src/test/regress/sql/single_node.sql +++ b/src/test/regress/sql/single_node.sql @@ -538,6 +538,9 @@ CREATE TABLE hpart1 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remaind CREATE TABLE hpart2 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 2); CREATE TABLE hpart3 PARTITION OF hash_parted FOR VALUES WITH (modulus 4, remainder 3); +-- Disable metadata sync since citus doesn't support distributing +-- operator class for now. +SET citus.enable_metadata_sync TO OFF; SELECT create_distributed_table('hash_parted ', 'a'); INSERT INTO hash_parted VALUES (1, generate_series(1, 10)); @@ -548,6 +551,7 @@ ALTER TABLE hash_parted DETACH PARTITION hpart0; ALTER TABLE hash_parted DETACH PARTITION hpart1; ALTER TABLE hash_parted DETACH PARTITION hpart2; ALTER TABLE hash_parted DETACH PARTITION hpart3; +RESET citus.enable_metadata_sync; -- test range partition without creating partitions and inserting with generate_series() -- should error out even in plain PG since no partition of relation "parent_tab" is found for row @@ -915,11 +919,11 @@ SELECT table_name, citus_table_type, distribution_column, shard_count FROM publi SELECT pg_sleep(0.1); -- since max_cached_conns_per_worker == 0 at this point, the -- backend(s) that execute on the shards will be terminated --- so show that there is only a single client backend, --- which is actually the backend that executes here +-- so show that there no internal backends SET search_path TO single_node; SELECT count(*) from should_commit; -SELECT pg_catalog.get_all_active_client_backend_count(); +SELECT count(*) FROM pg_stat_activity WHERE application_name LIKE 'citus_internal%'; +SELECT get_all_active_client_backend_count(); BEGIN; SET LOCAL citus.shard_count TO 32; SET LOCAL citus.force_max_query_parallelization TO ON; @@ -931,7 +935,10 @@ BEGIN; SELECT count(*) FROM test; -- now, we should have additional 32 connections - SELECT pg_catalog.get_all_active_client_backend_count(); + SELECT count(*) FROM pg_stat_activity WHERE application_name LIKE 'citus_internal%'; + + -- single external connection + SELECT get_all_active_client_backend_count(); ROLLBACK; diff --git a/src/test/regress/sql/sqlsmith_failures.sql b/src/test/regress/sql/sqlsmith_failures.sql index cacbf1e98..8190fdbb8 100644 --- a/src/test/regress/sql/sqlsmith_failures.sql +++ b/src/test/regress/sql/sqlsmith_failures.sql @@ -108,7 +108,7 @@ from (select pg_catalog.websearch_to_tsquery( cast(pg_catalog.regconfigin(cast(cast(null as cstring) as cstring)) as regconfig), - cast((select type from citus.pg_dist_object limit 1 offset 1) as text) + cast((select type from pg_catalog.pg_dist_object limit 1 offset 1) as text) ) as c0, sample_0.org_id as c1, sample_0.id as c2, diff --git a/src/test/regress/sql/tableam.sql b/src/test/regress/sql/tableam.sql index ab2b640a6..3c7cb69f0 100644 --- a/src/test/regress/sql/tableam.sql +++ b/src/test/regress/sql/tableam.sql @@ -22,12 +22,17 @@ SELECT public.run_command_on_coordinator_and_workers($Q$ CREATE ACCESS METHOD fake_am TYPE TABLE HANDLER fake_am_handler; $Q$); +-- Since Citus assumes access methods are part of the extension, make fake_am +-- owned manually to be able to pass checks on Citus while distributing tables. +ALTER EXTENSION citus ADD ACCESS METHOD fake_am; + -- -- Hash distributed table using a non-default table access method -- create table test_hash_dist(id int, val int) using fake_am; insert into test_hash_dist values (1, 1); + select create_distributed_table('test_hash_dist','id'); select * from test_hash_dist; @@ -48,6 +53,7 @@ SELECT * FROM master_get_table_ddl_events('test_hash_dist'); create table test_ref(a int) using fake_am; insert into test_ref values (1); + select create_reference_table('test_ref'); select * from test_ref; @@ -62,20 +68,14 @@ RESET client_min_messages; -- ddl events should include "USING fake_am" SELECT * FROM master_get_table_ddl_events('test_ref'); --- replicate to coordinator -SET client_min_messages TO WARNING; -\set VERBOSIY terse -SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); -RESET client_min_messages; -delete from test_ref; -SELECT master_remove_node('localhost', :master_port); - -- -- Range partitioned table using a non-default table access method -- CREATE TABLE test_range_dist(id int, val int) using fake_am; + SELECT create_distributed_table('test_range_dist', 'id', 'range'); + CALL public.create_range_partitioned_shards('test_range_dist', '{"0","25"}','{"24","49"}'); select * from test_range_dist; @@ -148,4 +148,5 @@ CREATE TABLE test_partitioned(id int, p int, val int) PARTITION BY RANGE (p) USING fake_am; \set VERBOSITY terse +ALTER EXTENSION citus DROP ACCESS METHOD fake_am; drop schema test_tableam cascade; diff --git a/src/test/regress/sql/text_search.sql b/src/test/regress/sql/text_search.sql index 916644fd6..c0f433599 100644 --- a/src/test/regress/sql/text_search.sql +++ b/src/test/regress/sql/text_search.sql @@ -221,7 +221,7 @@ SELECT * FROM run_command_on_workers($$ SELECT 'text_search.concurrent_index_con -- verify the objid is correctly committed locally due to the somewhat convoluted commit and new transaction starting when creating an index concurrently SELECT pg_catalog.pg_identify_object_as_address(classid, objid, objsubid) - FROM citus.pg_dist_object + FROM pg_catalog.pg_dist_object WHERE classid = 3602 AND objid = 'text_search.concurrent_index_config'::regconfig::oid; -- verify old text search configurations get renamed if they are not the same as the newly propagated configuration. @@ -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_citus_stat_activity.sql b/src/test/regress/sql/upgrade_citus_stat_activity.sql new file mode 100644 index 000000000..1d97b59b6 --- /dev/null +++ b/src/test/regress/sql/upgrade_citus_stat_activity.sql @@ -0,0 +1,7 @@ +SELECT column_name FROM information_schema.columns WHERE table_name = 'citus_stat_activity' AND column_name NOT IN ('leader_pid', 'query_id') +EXCEPT SELECT column_name FROM information_schema.columns WHERE table_name = 'pg_stat_activity' +ORDER BY 1; + +SELECT column_name FROM information_schema.columns WHERE table_name = 'pg_stat_activity' +EXCEPT SELECT column_name FROM information_schema.columns WHERE table_name = 'citus_stat_activity' +ORDER BY 1; diff --git a/src/test/regress/sql/upgrade_pg_dist_object_test_after.sql b/src/test/regress/sql/upgrade_pg_dist_object_test_after.sql index 6427d93c7..823d0214c 100644 --- a/src/test/regress/sql/upgrade_pg_dist_object_test_after.sql +++ b/src/test/regress/sql/upgrade_pg_dist_object_test_after.sql @@ -26,4 +26,4 @@ DROP SCHEMA upgrade_basic CASCADE; -- 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; +SELECT i.* FROM pg_catalog.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3; 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..a106b9fcf --- /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 pg_catalog.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 pg_catalog.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');