From e1afd302636340711f38fac5d390e17f333fd20e Mon Sep 17 00:00:00 2001 From: Jelte Fennema Date: Wed, 23 Feb 2022 13:03:29 +0100 Subject: [PATCH 01/24] Speed up test runs on WSL2 a lot (#5736) It turns out `whereis` is incredibly slow on WSL2 (at least on my machine): ``` $ time whereis diff diff: /usr/bin/diff /usr/share/man/man1/diff.1.gz real 0m0.408s user 0m0.010s sys 0m0.101s ``` This command is run by our custom `diff` script, which is run for every test file that is run. So this adds lots of unnecessary runtime time to tests. This changes our custom `diff` script to only call `whereis` in the strange case that `/usr/bin/diff` does not exist. The impact of this small change on the total runtime of the tests on WSL is huge. As an example the following command takes 18 seconds without this change and 7 seconds with it: ``` make -C src/test/regress/ check-arbitrary-configs CONFIGS=PostgresConfig ``` --- src/test/regress/bin/diff | 17 ++++++++++++----- 1 file changed, 12 insertions(+), 5 deletions(-) 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:-}" From 1fb970224eb6ec0f0e067f54c75c287b61b7349a Mon Sep 17 00:00:00 2001 From: Nils Dijk Date: Wed, 23 Feb 2022 15:53:26 +0100 Subject: [PATCH 02/24] Fix: partitioned index dependencies (#5741) #5685 introduced the resolution of dependencies for indices. This missed support for indices on partitioned tables. This change adds support for partitioned indices to the dependency resolution code. --- .../distributed/commands/dependencies.c | 3 ++- src/backend/distributed/metadata/dependency.c | 3 ++- src/test/regress/expected/text_search.out | 20 +++++++++++++++++++ src/test/regress/sql/text_search.sql | 17 ++++++++++++++++ 4 files changed, 41 insertions(+), 2 deletions(-) diff --git a/src/backend/distributed/commands/dependencies.c b/src/backend/distributed/commands/dependencies.c index 84e875602..c1bd4c340 100644 --- a/src/backend/distributed/commands/dependencies.c +++ b/src/backend/distributed/commands/dependencies.c @@ -247,7 +247,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; } diff --git a/src/backend/distributed/metadata/dependency.c b/src/backend/distributed/metadata/dependency.c index 2d1f67ad3..bb25e2b6b 100644 --- a/src/backend/distributed/metadata/dependency.c +++ b/src/backend/distributed/metadata/dependency.c @@ -722,7 +722,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; } diff --git a/src/test/regress/expected/text_search.out b/src/test/regress/expected/text_search.out index 1b4f652c1..a4af57051 100644 --- a/src/test/regress/expected/text_search.out +++ b/src/test/regress/expected/text_search.out @@ -484,6 +484,26 @@ SELECT create_distributed_table('t5', 'name'); (1 row) +-- make sure partial indices propagate their dependencies +-- first have a TEXT SEARCH CONFIGURATION that is not distributed +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION partial_index_test_config ( parser = default ); +RESET citus.enable_ddl_propagation; +CREATE TABLE sensors( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_a_partition PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_search_name ON sensors USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + SET client_min_messages TO 'warning'; DROP SCHEMA text_search, text_search2, "Text Search Requiring Quote's" CASCADE; DROP ROLE text_search_owner; diff --git a/src/test/regress/sql/text_search.sql b/src/test/regress/sql/text_search.sql index 916644fd6..bb8254f0c 100644 --- a/src/test/regress/sql/text_search.sql +++ b/src/test/regress/sql/text_search.sql @@ -258,6 +258,23 @@ CREATE TABLE t5(id int, name text); CREATE INDEX t5_search_name ON t5 USING gin (to_tsvector('"Text Search Requiring Quote''s"."Quoted Config Name"'::regconfig, (COALESCE(name, ''::character varying))::text)); SELECT create_distributed_table('t5', 'name'); +-- make sure partial indices propagate their dependencies +-- first have a TEXT SEARCH CONFIGURATION that is not distributed +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION partial_index_test_config ( parser = default ); +RESET citus.enable_ddl_propagation; + +CREATE TABLE sensors( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_a_partition PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_search_name ON sensors USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors', 'measureid'); + SET client_min_messages TO 'warning'; DROP SCHEMA text_search, text_search2, "Text Search Requiring Quote's" CASCADE; DROP ROLE text_search_owner; From 72d8fde28ba55711ef5896eff8f806637d7077bf Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Mon, 25 Oct 2021 11:11:46 +0200 Subject: [PATCH 03/24] Use intermediate results for re-partition joins --- .../distributed/executor/adaptive_executor.c | 39 +- .../directed_acyclic_graph_execution.c | 33 +- .../distributed_intermediate_results.c | 33 +- .../executor/intermediate_results.c | 19 +- .../partitioned_intermediate_results.c | 63 ++- .../executor/repartition_join_execution.c | 87 +--- .../planner/multi_physical_planner.c | 393 +++++++++--------- .../distributed/sql/citus--10.2-4--11.0-1.sql | 3 +- .../sql/downgrades/citus--11.0-1--10.2-4.sql | 4 + .../worker_partition_query_result/11.0-1.sql | 20 + .../worker_partition_query_result/latest.sql | 8 +- src/include/distributed/adaptive_executor.h | 1 + .../distributed/intermediate_results.h | 27 ++ .../distributed/multi_physical_planner.h | 13 +- .../distributed/repartition_join_execution.h | 1 - .../adaptive_executor_repartition.out | 7 +- .../expected/coordinator_shouldhaveshards.out | 24 +- .../regress/expected/follower_single_node.out | 63 ++- .../regress/expected/intermediate_results.out | 10 +- src/test/regress/expected/multi_extension.out | 154 +++---- .../multi_follower_select_statements.out | 14 +- .../multi_mx_repartition_udt_prepare.out | 14 +- .../expected/multi_repartition_udt.out | 14 +- .../expected/upgrade_list_citus_objects.out | 4 +- src/test/regress/output/multi_copy.source | 2 +- .../sql/adaptive_executor_repartition.sql | 1 - .../sql/coordinator_shouldhaveshards.sql | 2 +- src/test/regress/sql/intermediate_results.sql | 7 +- .../sql/multi_mx_repartition_udt_prepare.sql | 14 +- .../regress/sql/multi_repartition_udt.sql | 14 +- 30 files changed, 603 insertions(+), 485 deletions(-) create mode 100644 src/backend/distributed/sql/udfs/worker_partition_query_result/11.0-1.sql 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/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/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..e1018191e 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,11 @@ 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); @@ -93,21 +87,6 @@ EnsureCompatibleLocalExecutionState(List *taskList) } -/* - * 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 +118,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/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 28b750e8f..6035e5d5b 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); @@ -219,10 +219,12 @@ static uint32 TaskListHighestTaskId(List *taskList); static List * MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList); static StringInfo CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, uint32 partitionColumnIndex); +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); + /* TODO: determine binary safety automatically */ + bool useBinaryFormat = BinaryWorkerCopyFormat; - 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 @@ -4432,17 +4402,15 @@ CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, { 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 +4418,104 @@ 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"; + + /* TODO: determine binary safety automatically */ + bool useBinaryFormat = BinaryWorkerCopyFormat; + + /* + * 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. @@ -4521,6 +4555,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 +4660,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 +4722,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 +4736,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 +4775,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 +4802,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 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..78156d634 100644 --- a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql +++ b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql @@ -42,8 +42,9 @@ GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; #include "udfs/pg_cancel_backend/11.0-1.sql" #include "udfs/pg_terminate_backend/11.0-1.sql" +#include "udfs/worker_partition_query_result/11.0-1.sql" -DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text); +DROP FUNCTION pg_catalog.master_apply_delete_command(text); DROP FUNCTION pg_catalog.master_get_table_metadata(text); DROP FUNCTION pg_catalog.master_append_table_to_shard(bigint, text, text, integer); 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..5da5752ca 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,9 @@ -- citus--11.0-1--10.2-4 DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool); + +#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 +24,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) 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/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/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/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..3f6be8068 100644 --- a/src/include/distributed/repartition_join_execution.h +++ b/src/include/distributed/repartition_join_execution.h @@ -14,7 +14,6 @@ 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/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/coordinator_shouldhaveshards.out b/src/test/regress/expected/coordinator_shouldhaveshards.out index 5d5fa982c..fd44793c2 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,2147483643}'::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,2147483643}'::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,2147483643}'::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,2147483643}'::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 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/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/multi_extension.out b/src/test/regress/expected/multi_extension.out index e5cbcd994..dc3f36d5e 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -428,20 +428,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 +469,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) @@ -578,20 +578,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 +619,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) @@ -994,36 +994,38 @@ DROP TABLE e_transactions; ALTER EXTENSION citus UPDATE TO '10.2-4'; -- Should be empty result since upgrade+downgrade should be a no-op SELECT * FROM multi_extension.print_extension_changes(); - previous_object | current_object + previous_object | current_object --------------------------------------------------------------------- -(0 rows) + | function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) SETOF record +(1 row) -- Snapshot of state at 11.0-1 ALTER EXTENSION citus UPDATE TO '11.0-1'; SELECT * FROM multi_extension.print_extension_changes(); - previous_object | current_object + previous_object | current_object --------------------------------------------------------------------- - function citus_disable_node(text,integer) void | - function create_distributed_function(regprocedure,text,text) void | - function master_append_table_to_shard(bigint,text,text,integer) real | - function master_apply_delete_command(text) integer | - function master_get_table_metadata(text) record | - | function citus_check_cluster_node_health() SETOF record - | function citus_check_connection_to_node(text,integer) boolean - | function citus_disable_node(text,integer,boolean) void - | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void - | function citus_internal_global_blocked_processes() SETOF record - | function citus_internal_local_blocked_processes() SETOF record - | function citus_run_local_command(text) void - | function citus_shard_indexes_on_worker() SETOF record - | function citus_shards_on_worker() SETOF record - | function create_distributed_function(regprocedure,text,text,boolean) void - | function pg_cancel_backend(bigint) boolean - | function pg_terminate_backend(bigint,bigint) boolean - | function worker_create_or_replace_object(text[]) boolean - | function worker_drop_sequence_dependency(text) void - | function worker_drop_shell_table(text) void -(20 rows) + function citus_disable_node(text,integer) void | + function create_distributed_function(regprocedure,text,text) void | + function master_append_table_to_shard(bigint,text,text,integer) real | + function master_apply_delete_command(text) integer | + function master_get_table_metadata(text) record | + function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) SETOF record | + | function citus_check_cluster_node_health() SETOF record + | function citus_check_connection_to_node(text,integer) boolean + | function citus_disable_node(text,integer,boolean) void + | function citus_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 +(21 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version diff --git a/src/test/regress/expected/multi_follower_select_statements.out b/src/test/regress/expected/multi_follower_select_statements.out index 9f92db197..42c3058ee 100644 --- a/src/test/regress/expected/multi_follower_select_statements.out +++ b/src/test/regress/expected/multi_follower_select_statements.out @@ -117,12 +117,18 @@ order by s_i_id; SET citus.enable_repartition_joins TO ON; SELECT count(*) FROM the_table t1 JOIN the_table t2 USING(b); -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + count +--------------------------------------------------------------------- + 2 +(1 row) + SET citus.enable_single_hash_repartition_joins TO ON; SELECT count(*) FROM the_table t1 , the_table t2 WHERE t1.a = t2.b; -ERROR: writing to worker nodes is not currently allowed -DETAIL: the database is read-only + count +--------------------------------------------------------------------- + 2 +(1 row) + SELECT node_name, node_port FROM diff --git a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out index c9357e954..8e5242607 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -27,11 +27,17 @@ AS 'SELECT hashtext( ($1.i + $1.i2)::text);' LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +LANGUAGE SQL +IMMUTABLE +RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -66,7 +72,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -88,7 +95,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), diff --git a/src/test/regress/expected/multi_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index ee6ac8800..437e188ee 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -28,11 +28,17 @@ AS 'SELECT hashtext( ($1.i + $1.i2)::text);' LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +LANGUAGE SQL +IMMUTABLE +RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -67,7 +73,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), @@ -89,7 +96,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS OPERATOR 1 = (test_udt, test_udt), diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index 185bf19c5..e5e7b9215 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 @@ -225,7 +225,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) diff --git a/src/test/regress/output/multi_copy.source b/src/test/regress/output/multi_copy.source index d6d72d963..4b35d4465 100644 --- a/src/test/regress/output/multi_copy.source +++ b/src/test/regress/output/multi_copy.source @@ -636,7 +636,7 @@ INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::re INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0); SELECT 1 FROM master_activate_node('localhost', :worker_1_port); NOTICE: Replicating postgres objects to node localhost:57637 -DETAIL: There are 114 objects to replicate, depending on your environment this might take a while +DETAIL: There are 115 objects to replicate, depending on your environment this might take a while ?column? --------------------------------------------------------------------- 1 diff --git a/src/test/regress/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/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/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/multi_mx_repartition_udt_prepare.sql b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql index 779f4c24e..962f77318 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql @@ -35,12 +35,18 @@ LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +LANGUAGE SQL +IMMUTABLE +RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -84,7 +90,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -112,7 +119,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS diff --git a/src/test/regress/sql/multi_repartition_udt.sql b/src/test/regress/sql/multi_repartition_udt.sql index 2e5c1de24..2b71ec116 100644 --- a/src/test/regress/sql/multi_repartition_udt.sql +++ b/src/test/regress/sql/multi_repartition_udt.sql @@ -36,12 +36,18 @@ LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT; +CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int +AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END' +LANGUAGE SQL +IMMUTABLE +RETURNS NULL ON NULL INPUT; -- We need to define two different operator classes for the composite types -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -85,7 +91,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS @@ -113,7 +120,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash; -- One uses BTREE the other uses HASH CREATE OPERATOR CLASS tudt_op_fam_clas3 DEFAULT FOR TYPE test_udt USING BTREE AS -OPERATOR 3 = (test_udt, test_udt); +OPERATOR 3 = (test_udt, test_udt), +FUNCTION 1 test_udt_cmp(test_udt, test_udt); CREATE OPERATOR CLASS tudt_op_fam_class DEFAULT FOR TYPE test_udt USING HASH AS From 5ac0d31e8b4103a8ab72733d09e6c4ccd1e65a86 Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Sun, 7 Nov 2021 23:54:44 +0100 Subject: [PATCH 04/24] Fix re-partition hash range generation --- src/backend/distributed/planner/multi_physical_planner.c | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 6035e5d5b..9232447d8 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -4538,6 +4538,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); From 3cd9aa655ade143f715c75744f680952f186b6df Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Mon, 25 Oct 2021 12:18:43 +0200 Subject: [PATCH 05/24] Stop using citus.binary_worker_copy_format --- .../planner/multi_physical_planner.c | 17 +++++++++-------- src/backend/distributed/shared_library_init.c | 2 +- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 9232447d8..d95040049 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -218,7 +218,7 @@ 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, @@ -4177,8 +4177,8 @@ FragmentAlias(RangeTblEntry *rangeTableEntry, RangeTableFragment *fragment) List *resultNameList = FetchTaskResultNameList(mapOutputFetchTaskList); List *mapJobTargetList = mergeTask->mapJobTargetList; - /* TODO: determine binary safety automatically */ - bool useBinaryFormat = BinaryWorkerCopyFormat; + /* determine whether all types have binary input/output functions */ + bool useBinaryFormat = CanUseBinaryCopyFormatForTargetList(mapJobTargetList); /* generate the query on the intermediate result */ Query *fragmentSetQuery = BuildReadIntermediateResultsArrayQuery(mapJobTargetList, @@ -4347,11 +4347,15 @@ 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; @@ -4398,7 +4402,7 @@ 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; @@ -4449,9 +4453,6 @@ CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, char *partitionMethodString = partitionType == RANGE_PARTITION_TYPE ? "range" : "hash"; - /* TODO: determine binary safety automatically */ - bool useBinaryFormat = BinaryWorkerCopyFormat; - /* * 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 diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index a3a6cb3c7..e6b98a843 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -603,7 +603,7 @@ RegisterCitusConfigVariables(void) false, #endif PGC_SIGHUP, - GUC_STANDARD, + GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomBoolVariable( From 490765a7542aa4243b12981aa79bc3121220e009 Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Wed, 10 Nov 2021 09:42:59 +0100 Subject: [PATCH 06/24] Enable re-partition joins after local execution --- .../distributed/executor/local_executor.c | 19 +++++++ .../executor/repartition_join_execution.c | 22 -------- src/include/distributed/local_executor.h | 1 + .../distributed/repartition_join_execution.h | 1 - .../expected/coordinator_shouldhaveshards.out | 38 ++++++++++--- .../expected/local_shard_execution.out | 49 +++++++++++++++-- .../local_shard_execution_replicated.out | 53 +++++++++++++++++-- .../regress/sql/local_shard_execution.sql | 1 + .../sql/local_shard_execution_replicated.sql | 1 + 9 files changed, 149 insertions(+), 36 deletions(-) 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/repartition_join_execution.c b/src/backend/distributed/executor/repartition_join_execution.c index e1018191e..29d994e59 100644 --- a/src/backend/distributed/executor/repartition_join_execution.c +++ b/src/backend/distributed/executor/repartition_join_execution.c @@ -57,9 +57,6 @@ List * ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob) { List *allTasks = CreateTaskListForJobTree(topLevelTasks); - - EnsureCompatibleLocalExecutionState(allTasks); - List *jobIds = ExtractJobsInJobTree(topLevelJob); ExecuteTasksInDependencyOrder(allTasks, topLevelTasks, jobIds); @@ -68,25 +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(); - } -} - - /* * ExtractJobsInJobTree returns all job ids in the job tree * where the given job is root. 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/repartition_join_execution.h b/src/include/distributed/repartition_join_execution.h index 3f6be8068..7f2b648cf 100644 --- a/src/include/distributed/repartition_join_execution.h +++ b/src/include/distributed/repartition_join_execution.h @@ -13,7 +13,6 @@ #include "nodes/pg_list.h" extern List * ExecuteDependentTasks(List *taskList, Job *topLevelJob); -extern void EnsureCompatibleLocalExecutionState(List *taskList); #endif /* REPARTITION_JOIN_EXECUTION_H */ diff --git a/src/test/regress/expected/coordinator_shouldhaveshards.out b/src/test/regress/expected/coordinator_shouldhaveshards.out index fd44793c2..485e7f11b 100644 --- a/src/test/regress/expected/coordinator_shouldhaveshards.out +++ b/src/test/regress/expected/coordinator_shouldhaveshards.out @@ -251,10 +251,10 @@ SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; BEGIN; 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,2147483643}'::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,2147483643}'::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,2147483643}'::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,2147483643}'::text[],true,true,true) WHERE rows_written > 0 +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 @@ -290,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/local_shard_execution.out b/src/test/regress/expected/local_shard_execution.out index 781a9c86c..5cb61b2d5 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -91,6 +91,7 @@ ALTER TABLE abcd DROP COLUMN a; -- connection worker and get ready for the tests \c - - - :worker_1_port SET search_path TO local_shard_execution; +SET citus.enable_unique_job_ids TO off; -- returns true of the distribution key filter -- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard -- placement which is local to this not @@ -733,9 +734,51 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar (1 row) SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); -ERROR: cannot execute command because a local execution has accessed a placement in the transaction -DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally -HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" +NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_1','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470001 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_3','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470003 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_66_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_66_1','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470001 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartition_66_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_66_3','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470003 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_0,repartition_65_2_0,repartition_65_3_0,repartition_65_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_0,repartition_66_2_0,repartition_66_3_0,repartition_66_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_1,repartition_65_2_1,repartition_65_3_1,repartition_65_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_1,repartition_66_2_1,repartition_66_3_1,repartition_66_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_2,repartition_65_2_2,repartition_65_3_2,repartition_65_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_2,repartition_66_2_2,repartition_66_3_2,repartition_66_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_3,repartition_65_2_3,repartition_65_3_3,repartition_65_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_3,repartition_66_2_3,repartition_66_3_3,repartition_66_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + ROLLBACK; -- a local query is followed by an INSERT..SELECT with re-partitioning BEGIN; diff --git a/src/test/regress/expected/local_shard_execution_replicated.out b/src/test/regress/expected/local_shard_execution_replicated.out index 731c825c3..1c380c654 100644 --- a/src/test/regress/expected/local_shard_execution_replicated.out +++ b/src/test/regress/expected/local_shard_execution_replicated.out @@ -682,6 +682,7 @@ NOTICE: executing the copy locally for shard xxxxx ROLLBACK; BEGIN; SET citus.enable_repartition_joins TO ON; +SET citus.enable_unique_job_ids TO off; SELECT count(*) FROM distributed_table; NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true @@ -693,9 +694,55 @@ 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_2' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_2','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500002 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_64_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_4','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500004 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_2' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_2','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500002 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 partition_index, 'repartition_65_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_4','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500004 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_0']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_0']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_1']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_1']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_2']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_2']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_3']::text[],'localhost',57637) bytes +NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_3']::text[],'localhost',57638) bytes +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_0,repartition_64_2_0,repartition_64_3_0,repartition_64_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_0,repartition_65_2_0,repartition_65_3_0,repartition_65_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_1,repartition_64_2_1,repartition_64_3_1,repartition_64_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_1,repartition_65_2_1,repartition_65_3_1,repartition_65_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_2,repartition_64_2_2,repartition_64_3_2,repartition_64_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_2,repartition_65_2_2,repartition_65_3_2,repartition_65_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_3,repartition_64_2_3,repartition_64_3_3,repartition_64_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_3,repartition_65_2_3,repartition_65_3_3,repartition_65_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + ROLLBACK; -- a local query is followed by an INSERT..SELECT with re-partitioning BEGIN; diff --git a/src/test/regress/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; From 8de802eec5abec74232fee460a5fa15e10e69a96 Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Wed, 10 Nov 2021 10:50:48 +0100 Subject: [PATCH 07/24] Enable local_shared_pool_size 5 in arbitrary configs test --- src/test/regress/citus_tests/config.py | 3 +-- .../expected/prepared_statements_1.out | 22 +++++++++++++++++++ .../regress/sql/prepared_statements_1.sql | 15 +++++++++++++ 3 files changed, 38 insertions(+), 2 deletions(-) diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index 52b0714c8..beccf8b26 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -227,8 +227,7 @@ class CitusSmallSharedPoolSizeConfig(CitusMXBaseClusterConfig): def __init__(self, arguments): super().__init__(arguments) self.new_settings = { - # can be uncommented after https://github.com/citusdata/citus/issues/5342 - # "citus.local_shared_pool_size": 5, + "citus.local_shared_pool_size": 5, "citus.max_shared_pool_size": 5, } diff --git a/src/test/regress/expected/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/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; From ef1ceb395348a4c6cd232ac8244481deb078aeb2 Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Sat, 27 Nov 2021 11:52:29 +0100 Subject: [PATCH 08/24] Only use a single placement for map tasks --- src/backend/distributed/planner/multi_physical_planner.c | 8 ++++++++ .../regress/expected/local_shard_execution_replicated.out | 4 ---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index d95040049..9f578eac9 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -4362,6 +4362,13 @@ MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList) 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); } @@ -5398,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/test/regress/expected/local_shard_execution_replicated.out b/src/test/regress/expected/local_shard_execution_replicated.out index 1c380c654..285327095 100644 --- a/src/test/regress/expected/local_shard_execution_replicated.out +++ b/src/test/regress/expected/local_shard_execution_replicated.out @@ -695,13 +695,9 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); 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_2' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_2','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500002 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_64_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_4','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500004 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_2' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_2','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500002 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 partition_index, 'repartition_65_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_4','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500004 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 From b70949ae8cabb50111667c43da42867688aea848 Mon Sep 17 00:00:00 2001 From: Hanefi Onaldi Date: Mon, 21 Feb 2022 22:27:20 +0300 Subject: [PATCH 09/24] Lock nodes when building ddl task lists --- src/backend/distributed/commands/utility_hook.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index cf9012dd5..eb4b4644b 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -1563,7 +1563,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) { From f4e8af2c224613694999021cfb22001160a86a5d Mon Sep 17 00:00:00 2001 From: Hanefi Onaldi Date: Mon, 21 Feb 2022 22:29:00 +0300 Subject: [PATCH 10/24] Do not acquire locks on node metadata explicitly --- src/backend/distributed/commands/extension.c | 39 ------------------- src/backend/distributed/commands/role.c | 7 ---- src/backend/distributed/commands/type.c | 10 ----- .../distributed/metadata/node_metadata.c | 9 ----- 4 files changed, 65 deletions(-) diff --git a/src/backend/distributed/commands/extension.c b/src/backend/distributed/commands/extension.c index dc1363a65..fac8a783a 100644 --- a/src/backend/distributed/commands/extension.c +++ b/src/backend/distributed/commands/extension.c @@ -148,16 +148,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 +249,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 +375,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 +434,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 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/type.c b/src/backend/distributed/commands/type.c index eb59e8522..47dd0c307 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); diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 706f000cb..967d6bef3 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -188,9 +188,6 @@ citus_set_coordinator_host(PG_FUNCTION_ARGS) Name nodeClusterName = PG_GETARG_NAME(3); nodeMetadata.nodeCluster = NameStr(*nodeClusterName); - /* prevent concurrent modification */ - LockRelationOid(DistNodeRelationId(), RowShareLock); - bool isCoordinatorInMetadata = false; WorkerNode *coordinatorNode = PrimaryNodeForGroup(COORDINATOR_GROUP_ID, &isCoordinatorInMetadata); @@ -1780,12 +1777,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) { From 7bd6c2c9acfbfd075db5127cc5a83348ec4a9b50 Mon Sep 17 00:00:00 2001 From: Hanefi Onaldi Date: Mon, 21 Feb 2022 22:29:53 +0300 Subject: [PATCH 11/24] Isolation tests for various ddl operations and metadata sync --- .../expected/isolation_drop_vs_all.out | 126 +++-- .../isolation_metadata_sync_vs_all.out | 529 ++++++++++++++++++ .../regress/spec/isolation_drop_vs_all.spec | 54 +- .../spec/isolation_metadata_sync_vs_all.spec | 45 ++ 4 files changed, 701 insertions(+), 53 deletions(-) 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_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/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_metadata_sync_vs_all.spec b/src/test/regress/spec/isolation_metadata_sync_vs_all.spec index 7ff246365..d11565e2f 100644 --- a/src/test/regress/spec/isolation_metadata_sync_vs_all.spec +++ b/src/test/regress/spec/isolation_metadata_sync_vs_all.spec @@ -110,6 +110,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 +149,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 +168,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 +196,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 +219,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 +228,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" From 0c4e3cb69c97b5673d0d78c1b311b83a4fa276a1 Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Wed, 23 Feb 2022 22:09:15 +0100 Subject: [PATCH 12/24] Drop worker_partition_query_result on downgrade --- .../sql/downgrades/citus--11.0-1--10.2-4.sql | 1 + src/test/regress/expected/multi_extension.out | 10 +++++----- 2 files changed, 6 insertions(+), 5 deletions(-) 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 5da5752ca..21f9107da 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 @@ -2,6 +2,7 @@ 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) diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index dc3f36d5e..8117e64e2 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -994,15 +994,14 @@ DROP TABLE e_transactions; ALTER EXTENSION citus UPDATE TO '10.2-4'; -- Should be empty result since upgrade+downgrade should be a no-op SELECT * FROM multi_extension.print_extension_changes(); - previous_object | current_object + previous_object | current_object --------------------------------------------------------------------- - | function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) SETOF record -(1 row) +(0 rows) -- 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 | @@ -1025,7 +1024,8 @@ SELECT * FROM multi_extension.print_extension_changes(); | function worker_create_or_replace_object(text[]) boolean | function worker_drop_sequence_dependency(text) void | function worker_drop_shell_table(text) void -(21 rows) + | function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) SETOF record +(22 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version From df95d59e33196ce11ad8c2a217e98e55956b5a0b Mon Sep 17 00:00:00 2001 From: Onder Kalaci Date: Wed, 23 Feb 2022 11:06:47 +0100 Subject: [PATCH 13/24] Drop support for CitusInitiatedBackend CitusInitiatedBackend was a pre-mature implemenation of the whole GlobalPID infrastructure. We used it to track whether any individual query is triggered by Citus or not. As of now, after GlobalPID is already in place, we don't need CitusInitiatedBackend, in fact it could even be wrong. --- .../commands/citus_global_signal.c | 5 ++- .../distributed/executor/citus_custom_scan.c | 2 - .../distributed/metadata/node_metadata.c | 7 ++- .../distributed/transaction/backend_data.c | 41 +++-------------- .../transaction/citus_dist_stat_activity.c | 40 +++++++++-------- src/include/distributed/backend_data.h | 14 +----- src/include/distributed/worker_manager.h | 2 +- .../isolation_distributed_transaction_id.out | 32 ++++++------- ...lation_get_distributed_wait_queries_mx.out | 22 ++++----- ...licate_reference_tables_to_coordinator.out | 45 ++++++++++++++----- .../isolation_distributed_transaction_id.spec | 4 +- .../spec/isolation_metadata_sync_vs_all.spec | 2 - ...icate_reference_tables_to_coordinator.spec | 24 +++++++--- 13 files changed, 119 insertions(+), 121 deletions(-) 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/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/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 967d6bef3..498497e64 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -1541,7 +1541,7 @@ FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort) * If the node cannot be found this functions errors. */ WorkerNode * -FindNodeWithNodeId(int nodeId) +FindNodeWithNodeId(int nodeId, bool missingOk) { List *workerList = ActiveReadableNodeList(); WorkerNode *workerNode = NULL; @@ -1555,7 +1555,10 @@ FindNodeWithNodeId(int nodeId) } /* there isn't any node with nodeId in pg_dist_node */ - elog(ERROR, "worker node with node id %d could not be found", nodeId); + if (!missingOk) + { + elog(ERROR, "worker node with node id %d could not be found", nodeId); + } return NULL; } diff --git a/src/backend/distributed/transaction/backend_data.c b/src/backend/distributed/transaction/backend_data.c index 5ca517199..12ff8d78b 100644 --- a/src/backend/distributed/transaction/backend_data.c +++ b/src/backend/distributed/transaction/backend_data.c @@ -152,9 +152,6 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS) MyBackendData->transactionId.timestamp = timestamp; MyBackendData->transactionId.transactionOriginator = false; - MyBackendData->citusBackend.initiatorNodeIdentifier = - MyBackendData->transactionId.initiatorNodeIdentifier; - SpinLockRelease(&MyBackendData->mutex); PG_RETURN_VOID(); @@ -385,7 +382,6 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto &backendManagementShmemData->backends[backendIndex]; /* to work on data after releasing g spinlock to protect against errors */ - int initiatorNodeIdentifier = -1; uint64 transactionNumber = 0; SpinLockAcquire(¤tBackend->mutex); @@ -408,7 +404,6 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto 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,9 +418,12 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto SpinLockRelease(¤tBackend->mutex); + bool missingOk = true; + int nodeId = ExtractNodeIdFromGlobalPID(currentBackend->globalPID, missingOk); + values[0] = ObjectIdGetDatum(databaseId); values[1] = Int32GetDatum(backendPid); - values[2] = Int32GetDatum(initiatorNodeIdentifier); + values[2] = Int32GetDatum(nodeId); values[3] = !distributedCommandOriginator; values[4] = UInt64GetDatum(transactionNumber); values[5] = TimestampTzGetDatum(transactionIdTimestamp); @@ -520,7 +518,6 @@ BackendManagementShmemInit(void) { BackendData *backendData = &backendManagementShmemData->backends[backendIndex]; - backendData->citusBackend.initiatorNodeIdentifier = -1; SpinLockInit(&backendData->mutex); } } @@ -660,8 +657,6 @@ UnSetDistributedTransactionId(void) MyBackendData->transactionId.transactionNumber = 0; MyBackendData->transactionId.timestamp = 0; - MyBackendData->citusBackend.initiatorNodeIdentifier = -1; - SpinLockRelease(&MyBackendData->mutex); } } @@ -772,29 +767,6 @@ AssignDistributedTransactionId(void) 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); } @@ -926,11 +898,12 @@ ExtractGlobalPID(char *applicationName) * gives us the node id. */ int -ExtractNodeIdFromGlobalPID(uint64 globalPID) +ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk) { int nodeId = (int) (globalPID / GLOBAL_PID_NODE_ID_MULTIPLIER); - if (nodeId == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) + if (!missingOk && + nodeId == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) { ereport(ERROR, (errmsg("originator node of the query with the global pid " "%lu is not in Citus' metadata", globalPID), diff --git a/src/backend/distributed/transaction/citus_dist_stat_activity.c b/src/backend/distributed/transaction/citus_dist_stat_activity.c index ec4f6e8a3..0ee3925fb 100644 --- a/src/backend/distributed/transaction/citus_dist_stat_activity.c +++ b/src/backend/distributed/transaction/citus_dist_stat_activity.c @@ -538,33 +538,35 @@ ReplaceInitiatorNodeIdentifier(int initiator_node_identifier, * transaction. However, we cannot know which node has initiated * the worker query. */ - if (initiator_node_identifier > 0) - { - bool nodeExists = false; - - initiatorWorkerNode = PrimaryNodeForGroup(initiator_node_identifier, &nodeExists); - - /* a query should run on an existing node */ - Assert(nodeExists); - if (initiatorWorkerNode == NULL) - { - ereport(ERROR, (errmsg("no primary node found for group %d", - initiator_node_identifier))); - } - citusDistStat->master_query_host_name = - cstring_to_text(initiatorWorkerNode->workerName); - citusDistStat->master_query_host_port = initiatorWorkerNode->workerPort; - } - else if (initiator_node_identifier == 0 && IsCoordinator()) + if (initiator_node_identifier == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA && + IsCoordinator()) { citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); citusDistStat->master_query_host_port = PostPortNumber; } - else if (initiator_node_identifier == 0) + else if (initiator_node_identifier == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) { citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); citusDistStat->master_query_host_port = 0; } + else if (initiator_node_identifier > 0) + { + /* a query should run on an existing node, but lets be defensive */ + bool missingOk = true; + initiatorWorkerNode = FindNodeWithNodeId(initiator_node_identifier, missingOk); + + if (initiatorWorkerNode) + { + citusDistStat->master_query_host_name = + cstring_to_text(initiatorWorkerNode->workerName); + citusDistStat->master_query_host_port = initiatorWorkerNode->workerPort; + } + else + { + citusDistStat->master_query_host_name = NULL; + citusDistStat->master_query_host_port = 0; + } + } else { citusDistStat->master_query_host_name = NULL; diff --git a/src/include/distributed/backend_data.h b/src/include/distributed/backend_data.h index b463b89f5..c34d94670 100644 --- a/src/include/distributed/backend_data.h +++ b/src/include/distributed/backend_data.h @@ -22,16 +22,6 @@ #include "storage/s_lock.h" -/* - * CitusInitiatedBackend keeps some information about the backends that are - * initiated by Citus. - */ -typedef struct CitusInitiatedBackend -{ - int initiatorNodeIdentifier; -} CitusInitiatedBackend; - - /* * Each backend's active distributed transaction information is tracked via * BackendData in shared memory. @@ -51,7 +41,6 @@ typedef struct BackendData bool cancelledDueToDeadlock; uint64 globalPID; bool distributedCommandOriginator; - CitusInitiatedBackend citusBackend; DistributedTransactionId transactionId; } BackendData; @@ -64,13 +53,12 @@ extern void UnlockBackendSharedMemory(void); extern void UnSetDistributedTransactionId(void); extern void UnSetGlobalPID(void); extern void AssignDistributedTransactionId(void); -extern void MarkCitusInitiatedCoordinatorBackend(void); extern void AssignGlobalPID(void); extern uint64 GetGlobalPID(void); extern void OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator); extern uint64 ExtractGlobalPID(char *applicationName); -extern int ExtractNodeIdFromGlobalPID(uint64 globalPID); +extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk); extern int ExtractProcessIdFromGlobalPID(uint64 globalPID); extern void GetBackendDataForProc(PGPROC *proc, BackendData *result); extern void CancelTransactionDueToDeadlock(PGPROC *proc); diff --git a/src/include/distributed/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/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_get_distributed_wait_queries_mx.out b/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out index cae2222ed..109c61186 100644 --- a/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out +++ b/src/test/regress/expected/isolation_get_distributed_wait_queries_mx.out @@ -33,7 +33,7 @@ blocked_statement |current_statement_in_blockin --------------------------------------------------------------------- UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 15; -|localhost |coordinator_host | 57638| 57636 +|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit: @@ -116,7 +116,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -212,7 +212,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |localhost |localhost | 57638| 57637 +UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -308,7 +308,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -404,7 +404,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -594,7 +594,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV|localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -878,7 +878,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE |localhost |localhost | 57638| 57637 +UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE |coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -963,7 +963,7 @@ blocked_statement |current_s --------------------------------------------------------------------- ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); -|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |localhost | 57636| 57638 +|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s2-commit-worker: @@ -1073,7 +1073,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |localhost |localhost | 57637| 57637 +UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -1161,7 +1161,7 @@ step s3-select-distributed-waiting-queries: blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port --------------------------------------------------------------------- -UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|localhost |localhost | 57638| 57637 +UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit-worker: @@ -1225,7 +1225,7 @@ blocked_statement |current_statement_in_blockin --------------------------------------------------------------------- UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 15; -|localhost |coordinator_host | 57638| 57636 +|coordinator_host |coordinator_host | 57636| 57636 (1 row) step s1-commit: diff --git a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out index 2d49f8586..dd7ddefad 100644 --- a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out +++ b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out @@ -1,11 +1,19 @@ -Parsed test spec with 3 sessions +Parsed test spec with 4 sessions -starting permutation: s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end +starting permutation: add-node s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end create_distributed_table --------------------------------------------------------------------- (1 row) +step add-node: + SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0); + +?column? +--------------------------------------------------------------------- + 1 +(1 row) + step s1-begin: BEGIN; @@ -59,12 +67,20 @@ master_remove_node (1 row) -starting permutation: s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end +starting permutation: add-node s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end create_distributed_table --------------------------------------------------------------------- (1 row) +step add-node: + SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0); + +?column? +--------------------------------------------------------------------- + 1 +(1 row) + step s1-begin: BEGIN; @@ -83,7 +99,7 @@ pg_sleep (1 row) step s2-view-dist: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC; query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -92,7 +108,7 @@ query |query_hostname |query_hostport|distri |coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression update ref_table set a = a + 1; - |coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression + |coordinator_host| 57636| | 0|idle in transaction|Client |ClientRead|postgres|regression (2 rows) step s2-view-worker: @@ -102,13 +118,14 @@ step s2-view-worker: WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%dump_local_%' AND - query NOT ILIKE '%citus_internal_local_blocked_processes%' + query NOT ILIKE '%citus_internal_local_blocked_processes%' AND + query NOT ILIKE '%add_node%' ORDER BY query, query_hostport DESC; -query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname +query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638| | 0|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637| | 0|idle in transaction|Client |ClientRead|postgres|regression (2 rows) step s2-end: @@ -123,12 +140,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/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_metadata_sync_vs_all.spec b/src/test/regress/spec/isolation_metadata_sync_vs_all.spec index d11565e2f..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; 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..c4d6c8fc1 100644 --- a/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec +++ b/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec @@ -3,8 +3,6 @@ setup SELECT citus_internal.replace_isolation_tester_func(); SELECT citus_internal.refresh_isolation_tester_prepared_statement(); - SELECT master_add_node('localhost', 57636, groupid => 0); - CREATE TABLE ref_table(a int primary key); SELECT create_reference_table('ref_table'); INSERT INTO ref_table VALUES (1), (3), (5), (7); @@ -83,7 +81,7 @@ step "s2-lock-ref-table-placement-on-coordinator" step "s2-view-dist" { - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC; } step "s2-view-worker" @@ -94,7 +92,8 @@ step "s2-view-worker" WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%dump_local_%' AND - query NOT ILIKE '%citus_internal_local_blocked_processes%' + query NOT ILIKE '%citus_internal_local_blocked_processes%' AND + query NOT ILIKE '%add_node%' ORDER BY query, query_hostport DESC; } @@ -123,14 +122,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" From 6c25eea62ffa1cabdb409df5fbaeac16a1af0213 Mon Sep 17 00:00:00 2001 From: Hanefi Onaldi Date: Thu, 24 Feb 2022 17:12:46 +0300 Subject: [PATCH 14/24] Fix some typos in comments --- src/backend/distributed/commands/utility_hook.c | 2 +- .../sql/udfs/worker_create_or_replace_object/11.0-1.sql | 2 +- .../sql/udfs/worker_create_or_replace_object/latest.sql | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index eb4b4644b..1611da462 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -258,7 +258,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; 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'; From b825232ecbcc67afa7a17a876d1b42abd4b25625 Mon Sep 17 00:00:00 2001 From: Gledis Zeneli <43916939+gledis69@users.noreply.github.com> Date: Fri, 25 Feb 2022 19:54:30 +0300 Subject: [PATCH 15/24] Handle rebalance / replication when a node is disabled (Fix #5664) (#5729) The issue in question is caused when rebalance / replication call `FullShardPlacementList` which returns all shard placements (including those in disabled nodes with `citus_disable_node`). Eventually, `FindFillStateForPlacement` looks for the state across active workers and fails to find a state for the placements which are in the disabled workers causing a seg fault shortly after. Approach: * `ActivePlacementHash` was not using the status of the shard placement's node to determine if the node it is active. Initially, I just fixed that. * Additionally, I refactored the code which handles active shards in replication / rebalance to: * use a single function to determine if a shard placement is active. * do the shard active shard filtering before calling `RebalancePlacementUpdates` and `ReplicationPlacementUpdates`, so test methods like `shard_placement_rebalance_array` and `shard_placement_replication_array` which have different shard placement active requirements can do their own filtering while using the same rebalance / replicate logic that `rebalance_table_shards` and `replicate_table_shards` use. Fix #5664 --- .../distributed/metadata/metadata_utility.c | 92 +++++++++++++------ .../distributed/operations/shard_rebalancer.c | 53 ++++++----- .../distributed/test/shard_rebalancer.c | 28 +++++- src/include/distributed/metadata_utility.h | 3 + src/include/distributed/shard_rebalancer.h | 3 +- .../regress/expected/shard_rebalancer.out | 61 ++++++++++++ src/test/regress/sql/shard_rebalancer.sql | 31 +++++++ 7 files changed, 210 insertions(+), 61 deletions(-) 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/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/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/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 3e7a3b6f3..75e76ec8d 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); 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/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/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; From f17872aed4cdfdae9fdddaf50faf76ff3500767c Mon Sep 17 00:00:00 2001 From: Burak Velioglu Date: Thu, 24 Feb 2022 11:27:03 +0300 Subject: [PATCH 16/24] Expand functions while resolving dependencies --- src/backend/distributed/metadata/dependency.c | 24 +- .../distributed/metadata/metadata_sync.c | 116 +++ src/include/distributed/metadata_sync.h | 1 + .../regress/expected/function_propagation.out | 704 ++++++++++++++++++ src/test/regress/sql/function_propagation.sql | 477 ++++++++++++ 5 files changed, 1321 insertions(+), 1 deletion(-) diff --git a/src/backend/distributed/metadata/dependency.c b/src/backend/distributed/metadata/dependency.c index bb25e2b6b..4b3595575 100644 --- a/src/backend/distributed/metadata/dependency.c +++ b/src/backend/distributed/metadata/dependency.c @@ -122,6 +122,7 @@ typedef struct ViewDependencyNode static List * GetRelationSequenceDependencyList(Oid relationId); +static List * GetRelationFunctionDependencyList(Oid relationId); static List * GetRelationTriggerFunctionDependencyList(Oid relationId); static List * GetRelationStatsSchemaDependencyList(Oid relationId); static List * GetRelationIndicesDependencyList(Oid relationId); @@ -1091,9 +1092,15 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe * with them. */ List *sequenceDependencyList = GetRelationSequenceDependencyList(relationId); - result = list_concat(result, sequenceDependencyList); + /* + * Get the dependent functions for tables as columns has default values + * and contraints, then expand dependency list with them. + */ + List *functionDependencyList = GetRelationFunctionDependencyList(relationId); + result = list_concat(result, functionDependencyList); + /* * Tables could have indexes. Indexes themself could have dependencies that * need to be propagated. eg. TEXT SEARCH CONFIGRUATIONS. Here we add the @@ -1134,6 +1141,21 @@ GetRelationSequenceDependencyList(Oid relationId) } +/* + * GetRelationFunctionDependencyList returns the function dependency definition + * list for the given relation. + */ +static List * +GetRelationFunctionDependencyList(Oid relationId) +{ + List *dependentFunctionOids = GetDependentFunctionsWithRelation(relationId); + List *functionDependencyDefList = + CreateObjectAddressDependencyDefList(ProcedureRelationId, dependentFunctionOids); + + return functionDependencyDefList; +} + + /* * GetRelationStatsSchemaDependencyList returns a list of DependencyDefinition * objects for the schemas that statistics' of the relation with relationId depends. diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 490eeb4ca..651c70d35 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -28,9 +28,11 @@ #include "catalog/indexing.h" #include "catalog/pg_am.h" #include "catalog/pg_attrdef.h" +#include "catalog/pg_constraint.h" #include "catalog/pg_depend.h" #include "catalog/pg_foreign_server.h" #include "catalog/pg_namespace.h" +#include "catalog/pg_proc.h" #include "catalog/pg_type.h" #include "commands/async.h" #include "distributed/argutils.h" @@ -85,6 +87,7 @@ char *EnableManualMetadataChangesForUser = ""; static void EnsureObjectMetadataIsSane(int distributionArgumentIndex, int colocationId); +static List * GetFunctionDependenciesForObjects(ObjectAddress *objectAddress); static char * SchemaOwnerName(Oid objectId); static bool HasMetadataWorkers(void); static void CreateShellTableOnWorkers(Oid relationId); @@ -1544,6 +1547,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 diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index 69d500da4..31154a203 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -73,6 +73,7 @@ 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); #define DELETE_ALL_NODES "DELETE FROM pg_dist_node" diff --git a/src/test/regress/expected/function_propagation.out b/src/test/regress/expected/function_propagation.out index a40f1f9a2..ac645c263 100644 --- a/src/test/regress/expected/function_propagation.out +++ b/src/test/regress/expected/function_propagation.out @@ -316,6 +316,710 @@ BEGIN return 1; END; $$; +-- Show that functions are propagated (or not) as a dependency +-- Function as a default column +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Function shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func(id int, col_1 int default func_in_transaction_def()); +SELECT create_distributed_table('table_to_prop_func','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Function should be marked as distributed after distributing the table that depends on it +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_def}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_def}",{}) +(2 rows) + +-- Multiple functions as a default column +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_1() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +CREATE OR REPLACE FUNCTION func_in_transaction_2() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2()); +SELECT create_distributed_table('table_to_prop_func_2','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Functions should be marked as distribued after distributing the table that depends on it +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_1}",{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_2}",{}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{}) +(2 rows) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_2}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_2}",{}) +(2 rows) + +-- If function has dependency on non-distributed table it should error out +BEGIN; +CREATE TABLE non_dist_table(id int); +CREATE OR REPLACE FUNCTION func_in_transaction_3(param_1 non_dist_table) +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +CREATE TABLE table_to_prop_func_3(id int, col_1 int default func_in_transaction_3(NULL::non_dist_table)); +-- It should error out as there is a non-distributed table dependency +SELECT create_distributed_table('table_to_prop_func_3','id'); +ERROR: type function_propagation_schema.non_dist_table does not exist +CONTEXT: while executing command on localhost:xxxxx +COMMIT; +-- Adding a column with default value should propagate the function +BEGIN; +CREATE TABLE table_to_prop_func_4(id int); +SELECT create_distributed_table('table_to_prop_func_4', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION func_in_transaction_4() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Function shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4(); +-- Function should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_4}",{}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_4}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_4}",{}) +(2 rows) + +-- Adding multiple columns with default values should propagate the function +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_5() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +CREATE OR REPLACE FUNCTION func_in_transaction_6() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_5(id int, col_1 int default func_in_transaction_5(), col_2 int default func_in_transaction_6()); +SELECT create_distributed_table('table_to_prop_func_5', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Functions should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_5}",{}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_6}",{}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{}) +(2 rows) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_6}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_6}",{}) +(2 rows) + +-- Adding a constraint with function check should propagate the function +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_7(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_6(id int, col_1 int check (function_propagation_schema.func_in_transaction_7(col_1))); +SELECT create_distributed_table('table_to_prop_func_6', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Function should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_7}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_7}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_7}",{integer}) +(2 rows) + +-- Adding a constraint with multiple functions check should propagate the function +BEGIN; +CREATE OR REPLACE FUNCTION func_in_transaction_8(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +CREATE OR REPLACE FUNCTION func_in_transaction_9(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_to_prop_func_7(id int, col_1 int check (function_propagation_schema.func_in_transaction_8(col_1) and function_propagation_schema.func_in_transaction_9(col_1))); +SELECT create_distributed_table('table_to_prop_func_7', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Function should be marked as distributed after adding the column +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_8}",{integer}) +(1 row) + +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_9}",{integer}) +(1 row) + +COMMIT; +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer}) +(2 rows) + +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_9}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_9}",{integer}) +(2 rows) + +-- Adding a column with constraint should propagate the function +BEGIN; +CREATE TABLE table_to_prop_func_8(id int, col_1 int); +SELECT create_distributed_table('table_to_prop_func_8', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION func_in_transaction_10(param_1 int) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +-- Functions shouldn't be propagated within transaction +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1)); +-- Function should be marked as distributed after adding the constraint +SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_10}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_10}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_10}",{integer}) +(2 rows) + +-- If constraint depends on a non-distributed table it should error out +BEGIN; +CREATE TABLE local_table_for_const(id int); +CREATE OR REPLACE FUNCTION func_in_transaction_11(param_1 int, param_2 local_table_for_const) +RETURNS boolean +LANGUAGE plpgsql AS +$$ +BEGIN + return param_1 > 5; +END; +$$; +CREATE TABLE table_to_prop_func_9(id int, col_1 int check (func_in_transaction_11(col_1, NULL::local_table_for_const))); +-- It should error out since there is non-distributed table dependency exists +SELECT create_distributed_table('table_to_prop_func_9', 'id'); +ERROR: type function_propagation_schema.local_table_for_const does not exist +CONTEXT: while executing command on localhost:xxxxx +COMMIT; +-- Show that function as a part of generated always is supporte +BEGIN; + CREATE OR REPLACE FUNCTION non_sense_func_for_generated_always() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE people ( + id int, + height_cm numeric, + height_in numeric GENERATED ALWAYS AS (height_cm / non_sense_func_for_generated_always()) STORED); + SELECT create_distributed_table('people', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,non_sense_func_for_generated_always}",{}) +(1 row) + +COMMIT; +-- Show that functions depending table via rule are also distributed +BEGIN; +CREATE OR REPLACE FUNCTION func_for_rule() +RETURNS int +LANGUAGE plpgsql STABLE AS +$$ +BEGIN + return 4; +END; +$$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + +CREATE TABLE table_1_for_rule(id int, col_1 int); +CREATE TABLE table_2_for_rule(id int, col_1 int); +CREATE RULE rule_1 AS ON UPDATE TO table_1_for_rule DO ALSO UPDATE table_2_for_rule SET col_1 = col_1 * func_for_rule(); +SELECT create_distributed_table('table_1_for_rule','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Functions should be distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_for_rule}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_for_rule}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_for_rule}",{}) +(2 rows) + +-- Show that functions as partitioning functions are supported +BEGIN; + CREATE OR REPLACE FUNCTION non_sense_func_for_partitioning(int) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE partitioned_table_to_test_func_prop(id INT, a INT) PARTITION BY RANGE (non_sense_func_for_partitioning(id)); + SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer}) +(2 rows) + +-- Test function dependency on citus local table +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_for_local_table() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE citus_local_table_to_test_func(l1 int DEFAULT func_in_transaction_for_local_table()); + SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + + SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_for_local_table}",{}) +(1 row) + +ROLLBACK; +-- Show that having a function dependency on exlude also works +BEGIN; + CREATE OR REPLACE FUNCTION exclude_bool_func() + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return true; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE exclusion_func_prop_table (id int, EXCLUDE USING btree (id WITH =) WHERE (exclude_bool_func())); + SELECT create_distributed_table('exclusion_func_prop_table', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,exclude_bool_func}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,exclude_bool_func}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,exclude_bool_func}",{}) +(2 rows) + +-- Show that having a function dependency for index also works +BEGIN; + CREATE OR REPLACE FUNCTION func_for_index_predicate(col_1 int) + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return col_1 > 5; + END; + $$; + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE TABLE table_to_check_func_index_dep (id int, col_2 int); + CREATE INDEX on table_to_check_func_index_dep(col_2) WHERE (func_for_index_predicate(col_2)); + SELECT create_distributed_table('table_to_check_func_index_dep', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_for_index_predicate}",{integer}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_for_index_predicate}",{integer}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_for_index_predicate}",{integer}) +(2 rows) + +-- Test function to function dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_for_func_dep_1() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + CREATE TABLE func_dep_table(a int, b int default func_for_func_dep_1()); + CREATE OR REPLACE FUNCTION func_for_func_dep_2(col_1 func_dep_table) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + SELECT create_distributed_table('func_dep_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_for_func_dep_1}",{}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_for_func_dep_1}",{}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_for_func_dep_1}",{}) +(2 rows) + +-- Test function with SQL language and sequence dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_seq(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT 2 * val; + $$; + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_func(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT func_in_transaction_def_with_seq(val); + $$; + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- +(0 rows) + + CREATE SEQUENCE myseq; + CREATE TABLE table_to_prop_seq_func(id int, col_1 bigint default func_in_transaction_def_with_func(func_in_transaction_def_with_seq(nextval('myseq')))); + SELECT create_distributed_table('table_to_prop_seq_func','id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint}) +(1 row) + +COMMIT; +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2; + nodename | nodeport | success | result +--------------------------------------------------------------------- + localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint}) + localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint}) +(2 rows) + RESET search_path; SET client_min_messages TO WARNING; DROP SCHEMA function_propagation_schema CASCADE; diff --git a/src/test/regress/sql/function_propagation.sql b/src/test/regress/sql/function_propagation.sql index 79168497b..a9a6d04d8 100644 --- a/src/test/regress/sql/function_propagation.sql +++ b/src/test/regress/sql/function_propagation.sql @@ -188,6 +188,483 @@ BEGIN END; $$; +-- Show that functions are propagated (or not) as a dependency + +-- Function as a default column +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_def() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; + + CREATE TABLE table_to_prop_func(id int, col_1 int default func_in_transaction_def()); + SELECT create_distributed_table('table_to_prop_func','id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2; + + +-- Multiple functions as a default column +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_1() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_2() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid; + + CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2()); + SELECT create_distributed_table('table_to_prop_func_2','id'); + + -- Functions should be marked as distribued after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2; + + +-- If function has dependency on non-distributed table it should error out +BEGIN; + CREATE TABLE non_dist_table(id int); + + CREATE OR REPLACE FUNCTION func_in_transaction_3(param_1 non_dist_table) + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + CREATE TABLE table_to_prop_func_3(id int, col_1 int default func_in_transaction_3(NULL::non_dist_table)); + + -- It should error out as there is a non-distributed table dependency + SELECT create_distributed_table('table_to_prop_func_3','id'); +COMMIT; + + +-- Adding a column with default value should propagate the function +BEGIN; + CREATE TABLE table_to_prop_func_4(id int); + SELECT create_distributed_table('table_to_prop_func_4', 'id'); + + CREATE OR REPLACE FUNCTION func_in_transaction_4() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; + + ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4(); + + -- Function should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2; + + +-- Adding multiple columns with default values should propagate the function +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_5() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_6() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid; + + CREATE TABLE table_to_prop_func_5(id int, col_1 int default func_in_transaction_5(), col_2 int default func_in_transaction_6()); + SELECT create_distributed_table('table_to_prop_func_5', 'id'); + + -- Functions should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2; + +-- Adding a constraint with function check should propagate the function +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_7(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; + + CREATE TABLE table_to_prop_func_6(id int, col_1 int check (function_propagation_schema.func_in_transaction_7(col_1))); + SELECT create_distributed_table('table_to_prop_func_6', 'id'); + + -- Function should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2; + + +-- Adding a constraint with multiple functions check should propagate the function +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_8(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_9(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid; + + CREATE TABLE table_to_prop_func_7(id int, col_1 int check (function_propagation_schema.func_in_transaction_8(col_1) and function_propagation_schema.func_in_transaction_9(col_1))); + SELECT create_distributed_table('table_to_prop_func_7', 'id'); + + -- Function should be marked as distributed after adding the column + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid; + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid; +COMMIT; + +-- Functions should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2; +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2; + + +-- Adding a column with constraint should propagate the function +BEGIN; + CREATE TABLE table_to_prop_func_8(id int, col_1 int); + SELECT create_distributed_table('table_to_prop_func_8', 'id'); + + CREATE OR REPLACE FUNCTION func_in_transaction_10(param_1 int) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; + + ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1)); + + -- Function should be marked as distributed after adding the constraint + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2; + + +-- If constraint depends on a non-distributed table it should error out +BEGIN; + CREATE TABLE local_table_for_const(id int); + + CREATE OR REPLACE FUNCTION func_in_transaction_11(param_1 int, param_2 local_table_for_const) + RETURNS boolean + LANGUAGE plpgsql AS + $$ + BEGIN + return param_1 > 5; + END; + $$; + + CREATE TABLE table_to_prop_func_9(id int, col_1 int check (func_in_transaction_11(col_1, NULL::local_table_for_const))); + + -- It should error out since there is non-distributed table dependency exists + SELECT create_distributed_table('table_to_prop_func_9', 'id'); +COMMIT; + + +-- Show that function as a part of generated always is supporte +BEGIN; + + CREATE OR REPLACE FUNCTION non_sense_func_for_generated_always() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; + + CREATE TABLE people ( + id int, + height_cm numeric, + height_in numeric GENERATED ALWAYS AS (height_cm / non_sense_func_for_generated_always()) STORED); + + SELECT create_distributed_table('people', 'id'); + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid; +COMMIT; + + +-- Show that functions depending table via rule are also distributed +BEGIN; + CREATE OR REPLACE FUNCTION func_for_rule() + RETURNS int + LANGUAGE plpgsql STABLE AS + $$ + BEGIN + return 4; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; + + CREATE TABLE table_1_for_rule(id int, col_1 int); + CREATE TABLE table_2_for_rule(id int, col_1 int); + + CREATE RULE rule_1 AS ON UPDATE TO table_1_for_rule DO ALSO UPDATE table_2_for_rule SET col_1 = col_1 * func_for_rule(); + + SELECT create_distributed_table('table_1_for_rule','id'); + + -- Functions should be distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2; + + +-- Show that functions as partitioning functions are supported +BEGIN; + + CREATE OR REPLACE FUNCTION non_sense_func_for_partitioning(int) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 1; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; + + CREATE TABLE partitioned_table_to_test_func_prop(id INT, a INT) PARTITION BY RANGE (non_sense_func_for_partitioning(id)); + + SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id'); + + -- Show that function is distributed after distributing the table + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2; + + +-- Test function dependency on citus local table +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_for_local_table() + RETURNS int + LANGUAGE plpgsql AS + $$ + BEGIN + return 1; + END; + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; + + CREATE TABLE citus_local_table_to_test_func(l1 int DEFAULT func_in_transaction_for_local_table()); + SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid; +ROLLBACK; + +-- Show that having a function dependency on exlude also works +BEGIN; + CREATE OR REPLACE FUNCTION exclude_bool_func() + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return true; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; + + CREATE TABLE exclusion_func_prop_table (id int, EXCLUDE USING btree (id WITH =) WHERE (exclude_bool_func())); + SELECT create_distributed_table('exclusion_func_prop_table', 'id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2; + + +-- Show that having a function dependency for index also works +BEGIN; + CREATE OR REPLACE FUNCTION func_for_index_predicate(col_1 int) + RETURNS boolean + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return col_1 > 5; + END; + $$; + + -- Functions shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; + + CREATE TABLE table_to_check_func_index_dep (id int, col_2 int); + CREATE INDEX on table_to_check_func_index_dep(col_2) WHERE (func_for_index_predicate(col_2)); + + SELECT create_distributed_table('table_to_check_func_index_dep', 'id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2; + + +-- Test function to function dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_for_func_dep_1() + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + + CREATE TABLE func_dep_table(a int, b int default func_for_func_dep_1()); + + CREATE OR REPLACE FUNCTION func_for_func_dep_2(col_1 func_dep_table) + RETURNS int + LANGUAGE plpgsql IMMUTABLE AS + $$ + BEGIN + return 5; + END; + $$; + + SELECT create_distributed_table('func_dep_table', 'a'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2; + + +-- Test function with SQL language and sequence dependency +BEGIN; + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_seq(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT 2 * val; + $$; + + CREATE OR REPLACE FUNCTION func_in_transaction_def_with_func(val bigint) + RETURNS bigint + LANGUAGE SQL AS + $$ + SELECT func_in_transaction_def_with_seq(val); + $$; + + -- Function shouldn't be propagated within transaction + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; + + CREATE SEQUENCE myseq; + CREATE TABLE table_to_prop_seq_func(id int, col_1 bigint default func_in_transaction_def_with_func(func_in_transaction_def_with_seq(nextval('myseq')))); + + SELECT create_distributed_table('table_to_prop_seq_func','id'); + + -- Function should be marked as distributed after distributing the table that depends on it + SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid; +COMMIT; + +-- Function should be marked as distributed on the worker after committing changes +SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2; + RESET search_path; SET client_min_messages TO WARNING; DROP SCHEMA function_propagation_schema CASCADE; From 65bd540943cfa36ca886ef3025c7e4920a3e8ad9 Mon Sep 17 00:00:00 2001 From: Nils Dijk Date: Tue, 1 Mar 2022 15:29:31 +0100 Subject: [PATCH 17/24] Feature: configure object propagation behaviour in transactions (#5724) DESCRIPTION: Add GUC to control ddl creation behaviour in transactions Historically we would _not_ propagate objects when we are in a transaction block. Creation of distributed tables would not always work in sequential mode, hence objects created in the same transaction as distributing a table that would use the just created object wouldn't work. The benefit was that the user could still benefit from parallelism. Now that the creation of distributed tables is supported in sequential mode it would make sense for users to force transactional consistency of ddl commands for distributed tables. A transaction could switch more aggressively to sequential mode when creating new objects in a transaction. We don't change the default behaviour just yet. Also, many objects would not even propagate their creation when the transaction was already set to sequential, leaving the probability of a self deadlock. The new policy checks solve this discrepancy between objects as well. --- src/backend/distributed/commands/collation.c | 13 +-- .../distributed/commands/dependencies.c | 84 +++++++++++++++ src/backend/distributed/commands/extension.c | 7 +- .../distributed/commands/foreign_server.c | 29 ++++- src/backend/distributed/commands/schema.c | 4 +- .../distributed/commands/text_search.c | 12 +-- src/backend/distributed/commands/type.c | 2 +- .../distributed/commands/utility_hook.c | 1 + .../distributed/executor/multi_executor.c | 5 + src/backend/distributed/shared_library_init.c | 25 +++++ .../distributed/commands/utility_hook.h | 8 ++ src/include/distributed/metadata_utility.h | 1 + .../regress/expected/distributed_types.out | 101 ++++++++++++++++++ src/test/regress/sql/distributed_types.sql | 41 +++++++ 14 files changed, 308 insertions(+), 25 deletions(-) diff --git a/src/backend/distributed/commands/collation.c b/src/backend/distributed/commands/collation.c index 7f047ec1d..12bf1404a 100644 --- a/src/backend/distributed/commands/collation.c +++ b/src/backend/distributed/commands/collation.c @@ -530,11 +530,14 @@ PreprocessDefineCollationStmt(Node *node, const char *queryString, { Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION); - if (ShouldPropagateDefineCollationStmt()) + if (!ShouldPropagateDefineCollationStmt()) { - EnsureCoordinator(); + return NIL; } + EnsureCoordinator(); + EnsureSequentialMode(OBJECT_COLLATION); + return NIL; } @@ -575,8 +578,7 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString) * ShouldPropagateDefineCollationStmt checks if collation define * statement should be propagated. Don't propagate if: * - metadata syncing if off - * - statement is part of a multi stmt transaction and the multi shard connection - * type is not sequential + * - create statement should be propagated according the the ddl propagation policy */ static bool ShouldPropagateDefineCollationStmt() @@ -586,8 +588,7 @@ ShouldPropagateDefineCollationStmt() return false; } - if (IsMultiStatementTransaction() && - MultiShardConnectionType != SEQUENTIAL_CONNECTION) + if (!ShouldPropagateCreateInCoordinatedTransction()) { return false; } diff --git a/src/backend/distributed/commands/dependencies.c b/src/backend/distributed/commands/dependencies.c index c1bd4c340..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" @@ -459,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/extension.c b/src/backend/distributed/commands/extension.c index fac8a783a..f585b6a67 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; } 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/schema.c b/src/backend/distributed/commands/schema.c index 77f3fcc32..635e7adde 100644 --- a/src/backend/distributed/commands/schema.c +++ b/src/backend/distributed/commands/schema.c @@ -367,8 +367,8 @@ ShouldPropagateCreateSchemaStmt() return false; } - if (IsMultiStatementTransaction() && - MultiShardConnectionType != SEQUENTIAL_CONNECTION) + /* check creation against multi-statement transaction policy */ + if (!ShouldPropagateCreateInCoordinatedTransction()) { return false; } diff --git a/src/backend/distributed/commands/text_search.c b/src/backend/distributed/commands/text_search.c index be78057f7..53080c42b 100644 --- a/src/backend/distributed/commands/text_search.c +++ b/src/backend/distributed/commands/text_search.c @@ -73,16 +73,10 @@ PostprocessCreateTextSearchConfigurationStmt(Node *node, const char *queryString return NIL; } - /* - * If the create command is a part of a multi-statement transaction that is not in - * sequential mode, don't propagate. Instead we will rely on back filling. - */ - if (IsMultiStatementTransaction()) + /* check creation against multi-statement transaction policy */ + if (!ShouldPropagateCreateInCoordinatedTransction()) { - if (MultiShardConnectionType != SEQUENTIAL_CONNECTION) - { - return NIL; - } + return NIL; } EnsureCoordinator(); diff --git a/src/backend/distributed/commands/type.c b/src/backend/distributed/commands/type.c index 47dd0c307..c124388d4 100644 --- a/src/backend/distributed/commands/type.c +++ b/src/backend/distributed/commands/type.c @@ -1151,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 1611da462..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; diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 28498e0f2..f43235aac 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -763,6 +763,11 @@ GetObjectTypeString(ObjectType objType) return "extension"; } + case OBJECT_FOREIGN_SERVER: + { + return "foreign server"; + } + case OBJECT_FUNCTION: { return "function"; diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index e6b98a843..7897cbefe 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -218,6 +218,13 @@ static const struct config_enum_entry explain_analyze_sort_method_options[] = { { NULL, 0, false } }; +static const struct config_enum_entry create_object_propagation_options[] = { + {"deferred", CREATE_OBJECT_PROPAGATION_DEFERRED, false}, + {"automatic", CREATE_OBJECT_PROPAGATION_AUTOMATIC, false}, + {"immediate", CREATE_OBJECT_PROPAGATION_IMMEDIATE, false}, + {NULL, 0, false} +}; + /* *INDENT-ON* */ @@ -669,6 +676,24 @@ RegisterCitusConfigVariables(void) GUC_STANDARD, NULL, NULL, NULL); + DefineCustomEnumVariable( + "citus.create_object_propagation", + gettext_noop("Controls the behavior of CREATE statements in transactions for " + "supported objects"), + gettext_noop("When creating new objects in transactions this setting is used to " + "determine the behavior for propagating. When objects are created " + "in a multi-statement transaction block Citus needs to switch to " + "sequential mode (if not already) to make sure the objects are " + "visible to later statements on shards. The switch to sequential is " + "not always desired. By changing this behavior the user can trade " + "off performance for full transactional consistency on the creation " + "of new objects."), + &CreateObjectPropagationMode, + CREATE_OBJECT_PROPAGATION_DEFERRED, create_object_propagation_options, + PGC_USERSET, + GUC_NO_SHOW_ALL, + NULL, NULL, NULL); + DefineCustomBoolVariable( "citus.defer_drop_after_shard_move", gettext_noop("When enabled a shard move will mark the original shards " diff --git a/src/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/metadata_utility.h b/src/include/distributed/metadata_utility.h index 75e76ec8d..c03b3abe7 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -251,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/test/regress/expected/distributed_types.out b/src/test/regress/expected/distributed_types.out index dcf8dd8b2..2e2ef9c1f 100644 --- a/src/test/regress/expected/distributed_types.out +++ b/src/test/regress/expected/distributed_types.out @@ -453,6 +453,107 @@ SELECT * FROM field_indirection_test_2 ORDER BY 1,2,3; 8 | (10," text10",20) | (40,50) (2 rows) +-- test different ddl propagation modes +SET citus.create_object_propagation TO deferred; +BEGIN; +CREATE TYPE deferred_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +CREATE TABLE deferred_table(a int,b deferred_type); +SELECT create_distributed_table('deferred_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +COMMIT; +SET citus.create_object_propagation TO automatic; +BEGIN; +CREATE TYPE automatic_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +CREATE TABLE automatic_table(a int,b automatic_type); +SELECT create_distributed_table('automatic_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +COMMIT; +SET citus.create_object_propagation TO automatic; +BEGIN; +-- force parallel execution by preceding with a analytical query +SET LOCAL citus.force_max_query_parallelization TO on; +SELECT count(*) FROM automatic_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +CREATE TYPE automatic2_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +CREATE TABLE automatic2_table(a int,b automatic2_type); +SELECT create_distributed_table('automatic2_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + parallel +(1 row) + +COMMIT; +SET citus.create_object_propagation TO immediate; +BEGIN; +CREATE TYPE immediate_type AS (a int); +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +CREATE TABLE immediate_table(a int,b immediate_type); +SELECT create_distributed_table('immediate_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SHOW citus.multi_shard_modify_mode; + citus.multi_shard_modify_mode +--------------------------------------------------------------------- + sequential +(1 row) + +COMMIT; -- clear objects SET client_min_messages TO error; -- suppress cascading objects dropping DROP SCHEMA type_tests CASCADE; diff --git a/src/test/regress/sql/distributed_types.sql b/src/test/regress/sql/distributed_types.sql index a136e2fa8..23d84d26b 100644 --- a/src/test/regress/sql/distributed_types.sql +++ b/src/test/regress/sql/distributed_types.sql @@ -281,6 +281,47 @@ UPDATE field_indirection_test_2 SET (ct2_col, ct1_col) = ('(10, "text10", 20)', SELECT * FROM field_indirection_test_2 ORDER BY 1,2,3; +-- test different ddl propagation modes +SET citus.create_object_propagation TO deferred; +BEGIN; +CREATE TYPE deferred_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE deferred_table(a int,b deferred_type); +SELECT create_distributed_table('deferred_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + +SET citus.create_object_propagation TO automatic; +BEGIN; +CREATE TYPE automatic_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE automatic_table(a int,b automatic_type); +SELECT create_distributed_table('automatic_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + +SET citus.create_object_propagation TO automatic; +BEGIN; +-- force parallel execution by preceding with a analytical query +SET LOCAL citus.force_max_query_parallelization TO on; +SELECT count(*) FROM automatic_table; + +CREATE TYPE automatic2_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE automatic2_table(a int,b automatic2_type); +SELECT create_distributed_table('automatic2_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + +SET citus.create_object_propagation TO immediate; +BEGIN; +CREATE TYPE immediate_type AS (a int); +SHOW citus.multi_shard_modify_mode; +CREATE TABLE immediate_table(a int,b immediate_type); +SELECT create_distributed_table('immediate_table', 'a'); +SHOW citus.multi_shard_modify_mode; +COMMIT; + -- clear objects SET client_min_messages TO error; -- suppress cascading objects dropping DROP SCHEMA type_tests CASCADE; From a4133c69e8dd4f2eb272a960134b014ddad3a298 Mon Sep 17 00:00:00 2001 From: ywj Date: Tue, 1 Mar 2022 12:31:14 -0800 Subject: [PATCH 18/24] Build Columnar.so and make Citus depends on it (#5661) * [Columnar] Build columnar.so and let citus depends on it Co-authored-by: Yanwen Jin Co-authored-by: Ying Xu <32597660+yxu2162@users.noreply.github.com> Co-authored-by: jeff-davis --- Makefile | 14 ++++++++++---- src/backend/columnar/Makefile | 17 +++++++++++++++++ src/backend/columnar/columnar_debug.c | 1 - src/backend/columnar/mod.c | 7 ++++++- src/backend/columnar/safeclib | 1 + src/backend/distributed/Makefile | 10 ++++++---- src/backend/distributed/shared_library_init.c | 8 +++++++- src/include/columnar/mod.h | 1 + 8 files changed, 48 insertions(+), 11 deletions(-) create mode 100644 src/backend/columnar/Makefile create mode 120000 src/backend/columnar/safeclib diff --git a/Makefile b/Makefile index 7569fd6ce..22a363fbb 100644 --- a/Makefile +++ b/Makefile @@ -10,13 +10,18 @@ ifeq (,$(wildcard Makefile.global)) endif include Makefile.global - all: extension + +#build columnar only +columnar: + $(MAKE) -C src/backend/columnar all # build extension -extension: $(citus_top_builddir)/src/include/citus_version.h +extension: $(citus_top_builddir)/src/include/citus_version.h columnar $(MAKE) -C src/backend/distributed/ all -install-extension: extension +install-columnar: + $(MAKE) -C src/backend/columnar install +install-extension: extension install-columnar $(MAKE) -C src/backend/distributed/ install install-headers: extension $(MKDIR_P) '$(DESTDIR)$(includedir_server)/distributed/' @@ -27,6 +32,7 @@ install-headers: extension clean-extension: $(MAKE) -C src/backend/distributed/ clean + $(MAKE) -C src/backend/columnar/ clean clean-full: $(MAKE) -C src/backend/distributed/ clean-full .PHONY: extension install-extension clean-extension clean-full @@ -35,8 +41,8 @@ install: install-extension install-headers install-downgrades: $(MAKE) -C src/backend/distributed/ install-downgrades install-all: install-headers + $(MAKE) -C src/backend/columnar/ install-all $(MAKE) -C src/backend/distributed/ install-all - clean: clean-extension # apply or check style diff --git a/src/backend/columnar/Makefile b/src/backend/columnar/Makefile new file mode 100644 index 000000000..abda9c90d --- /dev/null +++ b/src/backend/columnar/Makefile @@ -0,0 +1,17 @@ +citus_subdir = src/backend/columnar +citus_top_builddir = ../../.. +safestringlib_srcdir = $(citus_abs_top_srcdir)/vendor/safestringlib +SUBDIRS = . safeclib +SUBDIRS += +ENSURE_SUBDIRS_EXIST := $(shell mkdir -p $(SUBDIRS)) +OBJS += \ + $(patsubst $(citus_abs_srcdir)/%.c,%.o,$(foreach dir,$(SUBDIRS), $(sort $(wildcard $(citus_abs_srcdir)/$(dir)/*.c)))) + +MODULE_big = citus_columnar + +PG_CPPFLAGS += -I$(libpq_srcdir) -I$(safestringlib_srcdir)/include + +include $(citus_top_builddir)/Makefile.global + +.PHONY: install-all +install-all: install diff --git a/src/backend/columnar/columnar_debug.c b/src/backend/columnar/columnar_debug.c index f72ec5f8f..220d259fe 100644 --- a/src/backend/columnar/columnar_debug.c +++ b/src/backend/columnar/columnar_debug.c @@ -11,7 +11,6 @@ #include "postgres.h" #include "funcapi.h" -#include "pg_config.h" #include "access/nbtree.h" #include "access/table.h" #include "catalog/pg_am.h" diff --git a/src/backend/columnar/mod.c b/src/backend/columnar/mod.c index f2679f326..c064b54c6 100644 --- a/src/backend/columnar/mod.c +++ b/src/backend/columnar/mod.c @@ -22,8 +22,13 @@ #include "columnar/columnar_tableam.h" + +PG_MODULE_MAGIC; + +void _PG_init(void); + void -columnar_init(void) +_PG_init(void) { columnar_init_gucs(); columnar_tableam_init(); diff --git a/src/backend/columnar/safeclib b/src/backend/columnar/safeclib new file mode 120000 index 000000000..c4da5b7ad --- /dev/null +++ b/src/backend/columnar/safeclib @@ -0,0 +1 @@ +../../../vendor/safestringlib/safeclib/ \ No newline at end of file diff --git a/src/backend/distributed/Makefile b/src/backend/distributed/Makefile index a2ffcc142..b88d6b77f 100644 --- a/src/backend/distributed/Makefile +++ b/src/backend/distributed/Makefile @@ -19,8 +19,6 @@ DATA_built = $(generated_sql_files) # directories with source files SUBDIRS = . commands connection ddl deparser executor metadata operations planner progress relay safeclib test transaction utils worker -# columnar modules -SUBDIRS += ../columnar # enterprise modules SUBDIRS += @@ -39,9 +37,12 @@ all: NO_PGXS = 1 SHLIB_LINK = $(libpq) - +SHLIB_LINK_INTERNAL += -L../columnar -l:citus_columnar.so +override enable_rpath = yes include $(citus_top_builddir)/Makefile.global +rpathdir := $(shell $(PG_CONFIG) --libdir):$(shell $(PG_CONFIG) --pkglibdir) + # make sure citus_version.o is recompiled whenever any change is made to the binary or any # other artifact being installed to reflect the correct gitref for every build CITUS_VERSION_INVALIDATE := $(filter-out utils/citus_version.o,$(OBJS)) @@ -84,7 +85,8 @@ endif .PHONY: clean-full install install-downgrades install-all cleanup-before-install: - rm -f $(DESTDIR)$(datadir)/$(datamoduledir)/citus* + rm -f $(DESTDIR)$(datadir)/$(datamoduledir)/citus.control + rm -f $(DESTDIR)$(datadir)/$(datamoduledir)/citus--* install: cleanup-before-install diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 7897cbefe..5e9d01e26 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -353,7 +353,13 @@ _PG_init(void) { DoInitialCleanup(); } - columnar_init(); + + /* + * For convenience and backwards compatibility, we avoid users having to add both + * citus and columnar to shared_preload_libraries by loading citus_columnar.so as part of + * loading citus.so. + */ + load_file(COLUMNAR_LIB_NAME, false); } diff --git a/src/include/columnar/mod.h b/src/include/columnar/mod.h index 7440a9fd7..7e24b3b6e 100644 --- a/src/include/columnar/mod.h +++ b/src/include/columnar/mod.h @@ -11,6 +11,7 @@ #ifndef MOD_H #define MOD_H +#define COLUMNAR_LIB_NAME "citus_columnar" /* Function declarations for extension loading and unloading */ extern void columnar_init(void); From b79a0052a471382e925ce599b982333b1a440a31 Mon Sep 17 00:00:00 2001 From: Onder Kalaci Date: Tue, 1 Mar 2022 18:10:27 +0100 Subject: [PATCH 19/24] Drop function in the tests on a never version As dropping the function now relies on pg_dist_object, which exists with 9.0+ --- src/test/regress/expected/multi_extension.out | 111 +++++++++--------- src/test/regress/sql/multi_extension.sql | 4 +- 2 files changed, 57 insertions(+), 58 deletions(-) diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 8117e64e2..c7fcc73e5 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -428,20 +428,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 +469,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) @@ -578,20 +578,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 +619,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) @@ -1115,8 +1115,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'; @@ -1125,8 +1124,8 @@ ERROR: cannot drop function relation_is_a_known_shard(regclass) because extensi HINT: You can drop extension citus instead. SET citus.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false'; -ALTER EXTENSION citus UPDATE TO '8.1-1'; -NOTICE: version "8.1-1" of extension "citus" is already installed +ALTER EXTENSION citus UPDATE TO '9.1-1'; +NOTICE: version "9.1-1" of extension "citus" is already installed -- Test updating to the latest version without specifying the version number ALTER EXTENSION citus UPDATE; -- re-create in newest version diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index aaed45803..bd97fc961 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -526,7 +526,7 @@ $function$; SET citus.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false'; -- This will fail because of previous function declaration -ALTER EXTENSION citus UPDATE TO '8.1-1'; +ALTER EXTENSION citus UPDATE TO '9.1-1'; -- We can DROP problematic function and continue ALTER EXTENSION even when version checks are on SET citus.enable_version_checks TO 'true'; @@ -535,7 +535,7 @@ DROP FUNCTION pg_catalog.relation_is_a_known_shard(regclass); SET citus.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false'; -ALTER EXTENSION citus UPDATE TO '8.1-1'; +ALTER EXTENSION citus UPDATE TO '9.1-1'; -- Test updating to the latest version without specifying the version number ALTER EXTENSION citus UPDATE; From e1809af376176d6bad55c78701221890881d4c2c Mon Sep 17 00:00:00 2001 From: Ahmet Gedemenli Date: Wed, 23 Feb 2022 13:17:52 +0300 Subject: [PATCH 20/24] Propagate CREATE AGGREGATE commands --- src/backend/distributed/commands/aggregate.c | 107 +++++++++++++ .../commands/distribute_object_ops.c | 6 +- src/backend/distributed/commands/function.c | 25 +-- .../deparser/qualify_aggregate_stmts.c | 34 ++++ .../distributed/executor/multi_executor.c | 5 + .../udfs/citus_finish_pg_upgrade/10.2-4.sql | 7 + .../udfs/citus_finish_pg_upgrade/latest.sql | 7 + src/include/distributed/commands.h | 4 + src/include/distributed/deparser.h | 3 + .../regress/expected/aggregate_support.out | 150 +++++++++++++++++- .../expected/propagate_foreign_servers.out | 27 +++- src/test/regress/sql/aggregate_support.sql | 83 +++++++++- .../regress/sql/propagate_foreign_servers.sql | 17 ++ 13 files changed, 456 insertions(+), 19 deletions(-) create mode 100644 src/backend/distributed/commands/aggregate.c create mode 100644 src/backend/distributed/deparser/qualify_aggregate_stmts.c 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/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/function.c b/src/backend/distributed/commands/function.c index 80b870b62..46eb6a2a9 100644 --- a/src/backend/distributed/commands/function.c +++ b/src/backend/distributed/commands/function.c @@ -82,7 +82,6 @@ static void EnsureFunctionCanBeColocatedWithTable(Oid functionOid, Oid static bool ShouldPropagateCreateFunction(CreateFunctionStmt *stmt); static bool ShouldPropagateAlterFunction(const ObjectAddress *address); static bool ShouldAddFunctionSignature(FunctionParameterMode mode); -static ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress); static ObjectAddress FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs, bool missing_ok); @@ -1352,7 +1351,7 @@ PostprocessCreateFunctionStmt(Node *node, const char *queryString) * GetUndistributableDependency checks whether object has any non-distributable * dependency. If any one found, it will be returned. */ -static ObjectAddress * +ObjectAddress * GetUndistributableDependency(ObjectAddress *objectAddress) { List *dependencies = GetAllDependenciesForObject(objectAddress); @@ -1443,10 +1442,18 @@ DefineAggregateStmtObjectAddress(Node *node, bool missing_ok) ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs); objectWithArgs->objname = stmt->defnames; - FunctionParameter *funcParam = NULL; - foreach_ptr(funcParam, linitial(stmt->args)) + if (stmt->args != NIL) { - objectWithArgs->objargs = lappend(objectWithArgs->objargs, funcParam->argType); + FunctionParameter *funcParam = NULL; + foreach_ptr(funcParam, linitial(stmt->args)) + { + objectWithArgs->objargs = lappend(objectWithArgs->objargs, + funcParam->argType); + } + } + else + { + objectWithArgs->objargs = list_make1(makeTypeName("anyelement")); } return FunctionToObjectAddress(OBJECT_AGGREGATE, objectWithArgs, missing_ok); @@ -2019,10 +2026,10 @@ ShouldAddFunctionSignature(FunctionParameterMode mode) /* - * FunctionToObjectAddress returns the ObjectAddress of a Function or Procedure based on - * its type and ObjectWithArgs describing the Function/Procedure. If missing_ok is set to - * false an error will be raised by postgres explaining the Function/Procedure could not - * be found. + * FunctionToObjectAddress returns the ObjectAddress of a Function, Procedure or + * Aggregate based on its type and ObjectWithArgs describing the + * Function/Procedure/Aggregate. If missing_ok is set to false an error will be + * raised by postgres explaining the Function/Procedure could not be found. */ static ObjectAddress FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs, diff --git a/src/backend/distributed/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/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index f43235aac..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"; diff --git a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql index 2921de962..fa13dc7bd 100644 --- a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql +++ b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/10.2-4.sql @@ -12,15 +12,22 @@ BEGIN IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN EXECUTE $cmd$ + -- disable propagation to prevent EnsureCoordinator errors + -- the aggregate created here does not depend on Citus extension (yet) + -- since we add the dependency with the next command + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray); COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; ELSE EXECUTE $cmd$ + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray); COMMENT ON AGGREGATE array_cat_agg(anyarray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; END IF; diff --git a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql index 2921de962..fa13dc7bd 100644 --- a/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_finish_pg_upgrade/latest.sql @@ -12,15 +12,22 @@ BEGIN IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN EXECUTE $cmd$ + -- disable propagation to prevent EnsureCoordinator errors + -- the aggregate created here does not depend on Citus extension (yet) + -- since we add the dependency with the next command + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray); COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; ELSE EXECUTE $cmd$ + SET citus.enable_ddl_propagation TO OFF; CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray); COMMENT ON AGGREGATE array_cat_agg(anyarray) IS 'concatenate input arrays into a single array'; + RESET citus.enable_ddl_propagation; $cmd$; END IF; diff --git a/src/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/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/test/regress/expected/aggregate_support.out b/src/test/regress/expected/aggregate_support.out index bf35d7f4b..57bb03060 100644 --- a/src/test/regress/expected/aggregate_support.out +++ b/src/test/regress/expected/aggregate_support.out @@ -282,6 +282,21 @@ create aggregate binstragg(text, text)( combinefunc=binstragg_combinefunc, stype=text ); +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + select create_distributed_function('binstragg(text,text)'); create_distributed_function --------------------------------------------------------------------- @@ -549,7 +564,6 @@ SELECT id%5, first(val ORDER BY key), last(val ORDER BY key) FROM aggdata GROUP BY id%5 ORDER BY id%5; ERROR: unsupported aggregate function first -- test aggregate with stype which is not a by-value datum --- also test our handling of the aggregate not existing on workers create function sumstring_sfunc(state text, x text) returns text immutable language plpgsql as $$ begin return (state::float8 + x::float8)::text; @@ -561,9 +575,20 @@ create aggregate sumstring(text) ( combinefunc = sumstring_sfunc, initcond = '0' ); -select sumstring(valf::text) from aggdata where valf is not null; -ERROR: function "aggregate_support.sumstring(text)" does not exist -CONTEXT: while executing command on localhost:xxxxx +-- verify that the aggregate is propagated +select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%'; + aggfnoid +--------------------------------------------------------------------- + sumstring +(1 row) + +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.sumstring) + (localhost,57638,t,aggregate_support.sumstring) +(2 rows) + select create_distributed_function('sumstring(text)'); create_distributed_function --------------------------------------------------------------------- @@ -1042,5 +1067,122 @@ LEFT JOIN ref_table ON TRUE; 109 (1 row) +-- try createing aggregate having non-distributable dependency type +create table dummy_tbl (a int); +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; +WARNING: Citus can't distribute function "dummy_fnc" having dependency on non-distributed relation "dummy_tbl" +DETAIL: Function will be created only locally +HINT: To distribute function, distribute dependent relations first. Then, re-create the function +-- should give warning and create aggregate local only +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +WARNING: Citus can't distribute aggregate "dependent_agg" having dependency on non-distributed relation "dummy_tbl" +DETAIL: Aggregate will be created only locally +HINT: To distribute aggregate, distribute dependent relations first. Then, re-create the aggregate +-- clear and try again with distributed table +DROP TABLE dummy_tbl CASCADE; +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to function dummy_fnc(dummy_tbl,double precision) +drop cascades to function dependent_agg(double precision) +create table dummy_tbl (a int); +SELECT create_distributed_table('dummy_tbl','a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; +-- test in tx block +-- shouldn't distribute, as citus.create_object_propagation is set to deferred +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify not distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,"") + (localhost,57638,t,"") +(2 rows) + +drop aggregate dependent_agg ( double precision); +-- now try with create_object_propagation = immediate +SET citus.create_object_propagation TO immediate; +-- should distribute, as citus.create_object_propagation is set to immediate +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + +drop aggregate dependent_agg ( double precision); +-- now try with create_object_propagation = automatic +SET citus.create_object_propagation TO automatic; +-- should distribute, as citus.create_object_propagation is set to automatic +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,1) + (localhost,57638,t,1) +(2 rows) + +RESET citus.create_object_propagation; +-- drop and test outside of tx block +drop aggregate dependent_agg (float8); +-- verify that the aggregate is removed from pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist") + (localhost,57638,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist") +(2 rows) + +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +--verify +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + +DROP TABLE dummy_tbl CASCADE; +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to function dummy_fnc(dummy_tbl,double precision) +drop cascades to function dependent_agg(double precision) +SET citus.create_object_propagation TO automatic; +begin; + create type typ1 as (a int); + create or replace function fnagg(a typ1, d double precision) RETURNS typ1 AS $$SELECT 1;$$LANGUAGE sql; + create aggregate dependent_agg (float8) (stype=typ1, sfunc=fnagg); +commit; +RESET citus.create_object_propagation; +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57637,t,aggregate_support.dependent_agg) + (localhost,57638,t,aggregate_support.dependent_agg) +(2 rows) + set client_min_messages to error; drop schema aggregate_support cascade; diff --git a/src/test/regress/expected/propagate_foreign_servers.out b/src/test/regress/expected/propagate_foreign_servers.out index 2e9b164e7..bebb5a4f6 100644 --- a/src/test/regress/expected/propagate_foreign_servers.out +++ b/src/test/regress/expected/propagate_foreign_servers.out @@ -7,6 +7,15 @@ SELECT citus_remove_node('localhost', :worker_1_port); (1 row) +-- not related, but added here to test propagation of aggregates +-- to newly added nodes +CREATE AGGREGATE array_agg (anynonarray) +( + sfunc = array_agg_transfn, + stype = internal, + finalfunc = array_agg_finalfn, + finalfunc_extra +); -- create schema, extension and foreign server while the worker is removed SET citus.enable_ddl_propagation TO OFF; CREATE SCHEMA test_dependent_schema; @@ -28,6 +37,20 @@ NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipp 1 (1 row) +-- verify that the aggregate is propagated to the new node +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%propagate_foreign_server.array_agg%';$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57638,t,propagate_foreign_server.array_agg) +(1 row) + +-- verify that the aggregate is added top pg_dist_object on the new node +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57638,t,1) +(1 row) + SELECT citus_add_local_table_to_metadata('foreign_table'); citus_add_local_table_to_metadata --------------------------------------------------------------------- @@ -163,4 +186,6 @@ ERROR: operation is not allowed on this node HINT: Connect to the coordinator and run it again. \c - - - :master_port DROP SCHEMA propagate_foreign_server CASCADE; -NOTICE: drop cascades to extension postgres_fdw +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to function propagate_foreign_server.array_agg(anynonarray) +drop cascades to extension postgres_fdw diff --git a/src/test/regress/sql/aggregate_support.sql b/src/test/regress/sql/aggregate_support.sql index 7c82418a7..a83688d82 100644 --- a/src/test/regress/sql/aggregate_support.sql +++ b/src/test/regress/sql/aggregate_support.sql @@ -160,6 +160,10 @@ create aggregate binstragg(text, text)( combinefunc=binstragg_combinefunc, stype=text ); +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$); + +SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$); select create_distributed_function('binstragg(text,text)'); @@ -313,7 +317,6 @@ SELECT id%5, first(val ORDER BY key), last(val ORDER BY key) FROM aggdata GROUP BY id%5 ORDER BY id%5; -- test aggregate with stype which is not a by-value datum --- also test our handling of the aggregate not existing on workers create function sumstring_sfunc(state text, x text) returns text immutable language plpgsql as $$ begin return (state::float8 + x::float8)::text; @@ -326,8 +329,10 @@ create aggregate sumstring(text) ( combinefunc = sumstring_sfunc, initcond = '0' ); +-- verify that the aggregate is propagated +select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%'; +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';$$); -select sumstring(valf::text) from aggdata where valf is not null; select create_distributed_function('sumstring(text)'); select sumstring(valf::text) from aggdata where valf is not null; @@ -533,5 +538,79 @@ SELECT floor(AVG(COALESCE(agg_col, 10))) FROM dist_table LEFT JOIN ref_table ON TRUE; +-- try createing aggregate having non-distributable dependency type +create table dummy_tbl (a int); +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; +-- should give warning and create aggregate local only +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); + +-- clear and try again with distributed table +DROP TABLE dummy_tbl CASCADE; + +create table dummy_tbl (a int); +SELECT create_distributed_table('dummy_tbl','a'); +create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl + AS $$SELECT 1;$$ LANGUAGE sql; + +-- test in tx block +-- shouldn't distribute, as citus.create_object_propagation is set to deferred +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; +-- verify not distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +drop aggregate dependent_agg ( double precision); + +-- now try with create_object_propagation = immediate +SET citus.create_object_propagation TO immediate; +-- should distribute, as citus.create_object_propagation is set to immediate +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; + +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +drop aggregate dependent_agg ( double precision); + +-- now try with create_object_propagation = automatic +SET citus.create_object_propagation TO automatic; +-- should distribute, as citus.create_object_propagation is set to automatic +-- will switch to sequential mode +BEGIN; +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +COMMIT; + +-- verify distributed +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +-- verify that the aggregate is added into pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); + +RESET citus.create_object_propagation; + +-- drop and test outside of tx block +drop aggregate dependent_agg (float8); +-- verify that the aggregate is removed from pg_dist_object, on each worker +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$); +create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc); +--verify +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + +DROP TABLE dummy_tbl CASCADE; + +SET citus.create_object_propagation TO automatic; +begin; + create type typ1 as (a int); + create or replace function fnagg(a typ1, d double precision) RETURNS typ1 AS $$SELECT 1;$$LANGUAGE sql; + create aggregate dependent_agg (float8) (stype=typ1, sfunc=fnagg); +commit; +RESET citus.create_object_propagation; + +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); + set client_min_messages to error; drop schema aggregate_support cascade; diff --git a/src/test/regress/sql/propagate_foreign_servers.sql b/src/test/regress/sql/propagate_foreign_servers.sql index f27a10e5d..cd64f7c9b 100644 --- a/src/test/regress/sql/propagate_foreign_servers.sql +++ b/src/test/regress/sql/propagate_foreign_servers.sql @@ -4,6 +4,16 @@ SET search_path TO propagate_foreign_server; -- remove node to add later SELECT citus_remove_node('localhost', :worker_1_port); +-- not related, but added here to test propagation of aggregates +-- to newly added nodes +CREATE AGGREGATE array_agg (anynonarray) +( + sfunc = array_agg_transfn, + stype = internal, + finalfunc = array_agg_finalfn, + finalfunc_extra +); + -- create schema, extension and foreign server while the worker is removed SET citus.enable_ddl_propagation TO OFF; CREATE SCHEMA test_dependent_schema; @@ -20,6 +30,13 @@ CREATE FOREIGN TABLE foreign_table ( OPTIONS (schema_name 'test_dependent_schema', table_name 'foreign_table_test'); SELECT 1 FROM citus_add_node('localhost', :master_port, groupId=>0); + +-- verify that the aggregate is propagated to the new node +SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%propagate_foreign_server.array_agg%';$$); + +-- verify that the aggregate is added top pg_dist_object on the new node +SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$); + SELECT citus_add_local_table_to_metadata('foreign_table'); ALTER TABLE foreign_table OWNER TO pg_monitor; From dcfbb51b6b92b2136287265ba1b345220b583431 Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Wed, 2 Mar 2022 11:33:15 +0100 Subject: [PATCH 21/24] Revert "Build Columnar.so and make Citus depends on it (#5661)" This reverts commit a4133c69e8dd4f2eb272a960134b014ddad3a298. --- Makefile | 14 ++++---------- src/backend/columnar/Makefile | 17 ----------------- src/backend/columnar/columnar_debug.c | 1 + src/backend/columnar/mod.c | 7 +------ src/backend/columnar/safeclib | 1 - src/backend/distributed/Makefile | 10 ++++------ src/backend/distributed/shared_library_init.c | 8 +------- src/include/columnar/mod.h | 1 - 8 files changed, 11 insertions(+), 48 deletions(-) delete mode 100644 src/backend/columnar/Makefile delete mode 120000 src/backend/columnar/safeclib diff --git a/Makefile b/Makefile index 22a363fbb..7569fd6ce 100644 --- a/Makefile +++ b/Makefile @@ -10,18 +10,13 @@ ifeq (,$(wildcard Makefile.global)) endif include Makefile.global + all: extension - -#build columnar only -columnar: - $(MAKE) -C src/backend/columnar all # build extension -extension: $(citus_top_builddir)/src/include/citus_version.h columnar +extension: $(citus_top_builddir)/src/include/citus_version.h $(MAKE) -C src/backend/distributed/ all -install-columnar: - $(MAKE) -C src/backend/columnar install -install-extension: extension install-columnar +install-extension: extension $(MAKE) -C src/backend/distributed/ install install-headers: extension $(MKDIR_P) '$(DESTDIR)$(includedir_server)/distributed/' @@ -32,7 +27,6 @@ install-headers: extension clean-extension: $(MAKE) -C src/backend/distributed/ clean - $(MAKE) -C src/backend/columnar/ clean clean-full: $(MAKE) -C src/backend/distributed/ clean-full .PHONY: extension install-extension clean-extension clean-full @@ -41,8 +35,8 @@ install: install-extension install-headers install-downgrades: $(MAKE) -C src/backend/distributed/ install-downgrades install-all: install-headers - $(MAKE) -C src/backend/columnar/ install-all $(MAKE) -C src/backend/distributed/ install-all + clean: clean-extension # apply or check style diff --git a/src/backend/columnar/Makefile b/src/backend/columnar/Makefile deleted file mode 100644 index abda9c90d..000000000 --- a/src/backend/columnar/Makefile +++ /dev/null @@ -1,17 +0,0 @@ -citus_subdir = src/backend/columnar -citus_top_builddir = ../../.. -safestringlib_srcdir = $(citus_abs_top_srcdir)/vendor/safestringlib -SUBDIRS = . safeclib -SUBDIRS += -ENSURE_SUBDIRS_EXIST := $(shell mkdir -p $(SUBDIRS)) -OBJS += \ - $(patsubst $(citus_abs_srcdir)/%.c,%.o,$(foreach dir,$(SUBDIRS), $(sort $(wildcard $(citus_abs_srcdir)/$(dir)/*.c)))) - -MODULE_big = citus_columnar - -PG_CPPFLAGS += -I$(libpq_srcdir) -I$(safestringlib_srcdir)/include - -include $(citus_top_builddir)/Makefile.global - -.PHONY: install-all -install-all: install diff --git a/src/backend/columnar/columnar_debug.c b/src/backend/columnar/columnar_debug.c index 220d259fe..f72ec5f8f 100644 --- a/src/backend/columnar/columnar_debug.c +++ b/src/backend/columnar/columnar_debug.c @@ -11,6 +11,7 @@ #include "postgres.h" #include "funcapi.h" +#include "pg_config.h" #include "access/nbtree.h" #include "access/table.h" #include "catalog/pg_am.h" diff --git a/src/backend/columnar/mod.c b/src/backend/columnar/mod.c index c064b54c6..f2679f326 100644 --- a/src/backend/columnar/mod.c +++ b/src/backend/columnar/mod.c @@ -22,13 +22,8 @@ #include "columnar/columnar_tableam.h" - -PG_MODULE_MAGIC; - -void _PG_init(void); - void -_PG_init(void) +columnar_init(void) { columnar_init_gucs(); columnar_tableam_init(); diff --git a/src/backend/columnar/safeclib b/src/backend/columnar/safeclib deleted file mode 120000 index c4da5b7ad..000000000 --- a/src/backend/columnar/safeclib +++ /dev/null @@ -1 +0,0 @@ -../../../vendor/safestringlib/safeclib/ \ No newline at end of file diff --git a/src/backend/distributed/Makefile b/src/backend/distributed/Makefile index b88d6b77f..a2ffcc142 100644 --- a/src/backend/distributed/Makefile +++ b/src/backend/distributed/Makefile @@ -19,6 +19,8 @@ DATA_built = $(generated_sql_files) # directories with source files SUBDIRS = . commands connection ddl deparser executor metadata operations planner progress relay safeclib test transaction utils worker +# columnar modules +SUBDIRS += ../columnar # enterprise modules SUBDIRS += @@ -37,11 +39,8 @@ all: NO_PGXS = 1 SHLIB_LINK = $(libpq) -SHLIB_LINK_INTERNAL += -L../columnar -l:citus_columnar.so -override enable_rpath = yes -include $(citus_top_builddir)/Makefile.global -rpathdir := $(shell $(PG_CONFIG) --libdir):$(shell $(PG_CONFIG) --pkglibdir) +include $(citus_top_builddir)/Makefile.global # make sure citus_version.o is recompiled whenever any change is made to the binary or any # other artifact being installed to reflect the correct gitref for every build @@ -85,8 +84,7 @@ endif .PHONY: clean-full install install-downgrades install-all cleanup-before-install: - rm -f $(DESTDIR)$(datadir)/$(datamoduledir)/citus.control - rm -f $(DESTDIR)$(datadir)/$(datamoduledir)/citus--* + rm -f $(DESTDIR)$(datadir)/$(datamoduledir)/citus* install: cleanup-before-install diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 5e9d01e26..7897cbefe 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -353,13 +353,7 @@ _PG_init(void) { DoInitialCleanup(); } - - /* - * For convenience and backwards compatibility, we avoid users having to add both - * citus and columnar to shared_preload_libraries by loading citus_columnar.so as part of - * loading citus.so. - */ - load_file(COLUMNAR_LIB_NAME, false); + columnar_init(); } diff --git a/src/include/columnar/mod.h b/src/include/columnar/mod.h index 7e24b3b6e..7440a9fd7 100644 --- a/src/include/columnar/mod.h +++ b/src/include/columnar/mod.h @@ -11,7 +11,6 @@ #ifndef MOD_H #define MOD_H -#define COLUMNAR_LIB_NAME "citus_columnar" /* Function declarations for extension loading and unloading */ extern void columnar_init(void); From 98751058a9e050ad9df82ff3ac6d680e926b3545 Mon Sep 17 00:00:00 2001 From: Onder Kalaci Date: Wed, 2 Mar 2022 10:01:08 +0100 Subject: [PATCH 22/24] Add Primary key to the table Otherwise enterprise tests fail --- .../regress/expected/distributed_triggers.out | 14 ++++++++------ src/test/regress/sql/distributed_triggers.sql | 15 +++++++++------ 2 files changed, 17 insertions(+), 12 deletions(-) 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/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 From 35ec9721b42097d1632d3da8dd87e8c243467cd6 Mon Sep 17 00:00:00 2001 From: Onder Kalaci Date: Tue, 22 Feb 2022 16:13:53 +0100 Subject: [PATCH 23/24] Add a new API for enabling Citus MX for clusters upgrading from earlier versions Clusters created pre-Citus 11 mostly didn't have metadata sync enabled. For those clusters, we add a utility UDF which fixes some minor issues and sync the necessary objects to the workers. --- src/backend/distributed/commands/extension.c | 21 ++ .../distributed/metadata/metadata_sync.c | 3 +- .../distributed/sql/citus--10.2-4--11.0-1.sql | 3 + .../sql/downgrades/citus--11.0-1--10.2-4.sql | 3 +- .../11.0-1.sql | 224 ++++++++++++++++++ .../latest.sql | 224 ++++++++++++++++++ src/include/distributed/metadata_sync.h | 1 + .../after_citus_upgrade_coord_schedule | 1 + .../before_citus_upgrade_coord_schedule | 1 + src/test/regress/expected/multi_extension.out | 5 +- .../expected/upgrade_list_citus_objects.out | 3 +- .../upgrade_pg_dist_object_test_after.out | 42 +++- .../upgrade_pg_dist_object_test_after_1.out | 54 +++++ .../expected/upgrade_post_11_after.out | 76 ++++++ .../expected/upgrade_post_11_before.out | 184 ++++++++++++++ src/test/regress/sql/multi_extension.sql | 2 +- .../regress/sql/upgrade_post_11_after.sql | 42 ++++ .../regress/sql/upgrade_post_11_before.sql | 155 ++++++++++++ 18 files changed, 1029 insertions(+), 15 deletions(-) create mode 100644 src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql create mode 100644 src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/latest.sql create mode 100644 src/test/regress/expected/upgrade_pg_dist_object_test_after_1.out create mode 100644 src/test/regress/expected/upgrade_post_11_after.out create mode 100644 src/test/regress/expected/upgrade_post_11_before.out create mode 100644 src/test/regress/sql/upgrade_post_11_after.sql create mode 100644 src/test/regress/sql/upgrade_post_11_before.sql diff --git a/src/backend/distributed/commands/extension.c b/src/backend/distributed/commands/extension.c index f585b6a67..3aa782c06 100644 --- a/src/backend/distributed/commands/extension.c +++ b/src/backend/distributed/commands/extension.c @@ -513,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); @@ -536,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/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 651c70d35..ef7e64ec1 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -107,7 +107,6 @@ static List * GetObjectsForGrantStmt(ObjectType objectType, Oid objectId); static AccessPriv * GetAccessPrivObjectForGrantStmt(char *permission); static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid, AclItem *aclItem); -static void SetLocalEnableMetadataSync(bool state); static void SetLocalReplicateReferenceTablesOnActivate(bool state); static char * GenerateSetRoleQuery(Oid roleOid); static void MetadataSyncSigTermHandler(SIGNAL_ARGS); @@ -1948,7 +1947,7 @@ GetAccessPrivObjectForGrantStmt(char *permission) /* * SetLocalEnableMetadataSync sets the enable_metadata_sync locally */ -static void +void SetLocalEnableMetadataSync(bool state) { set_config_option("citus.enable_metadata_sync", state == true ? "on" : "off", diff --git a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql index 78156d634..d836d4b72 100644 --- a/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql +++ b/src/backend/distributed/sql/citus--10.2-4--11.0-1.sql @@ -80,3 +80,6 @@ BEGIN UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb(partitioned_table_exists), true); END; $$; + +#include "udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql" + diff --git a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql index 21f9107da..f11a9d450 100644 --- a/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql +++ b/src/backend/distributed/sql/downgrades/citus--11.0-1--10.2-4.sql @@ -90,7 +90,6 @@ DROP FUNCTION pg_catalog.citus_shard_indexes_on_worker(); #include "../udfs/create_distributed_function/9.0-1.sql" ALTER TABLE citus.pg_dist_object DROP COLUMN force_delegation; - SET search_path = 'pg_catalog'; @@ -347,4 +346,6 @@ JOIN ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; +DROP FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool); + RESET search_path; diff --git a/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql new file mode 100644 index 000000000..d93dd0f93 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql @@ -0,0 +1,224 @@ +-- citus_finalize_upgrade_to_citus11() is a helper UDF ensures +-- the upgrade to Citus 11 is finished successfully. Upgrade to +-- Citus 11 requires all active primary worker nodes to get the +-- metadata. And, this function's job is to sync the metadata to +-- the nodes that does not already have +-- once the function finishes without any errors and returns true +-- the cluster is ready for running distributed queries from +-- the worker nodes. When debug is enabled, the function provides +-- more information to the user. +CREATE OR REPLACE FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(enforce_version_check bool default true) + RETURNS bool + LANGUAGE plpgsql + AS $$ +BEGIN + + --------------------------------------------- + -- This script consists of N stages + -- Each step is documented, and if log level + -- is reduced to DEBUG1, each step is logged + -- as well + --------------------------------------------- + +------------------------------------------------------------------------------------------ + -- STAGE 0: Ensure no concurrent node metadata changing operation happens while this + -- script is running via acquiring a strong lock on the pg_dist_node +------------------------------------------------------------------------------------------ +BEGIN + LOCK TABLE pg_dist_node IN EXCLUSIVE MODE NOWAIT; + + EXCEPTION WHEN OTHERS THEN + RAISE 'Another node metadata changing operation is in progress, try again.'; +END; + +------------------------------------------------------------------------------------------ + -- STAGE 1: We want all the commands to run in the same transaction block. Without + -- sequential mode, metadata syncing cannot be done in a transaction block along with + -- other commands +------------------------------------------------------------------------------------------ + SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; + +------------------------------------------------------------------------------------------ + -- STAGE 2: Ensure we have the prerequisites + -- (a) only superuser can run this script + -- (b) cannot be executed when enable_ddl_propagation is False + -- (c) can only be executed from the coordinator +------------------------------------------------------------------------------------------ +DECLARE + is_superuser_running boolean := False; + enable_ddl_prop boolean:= False; + local_group_id int := 0; +BEGIN + SELECT rolsuper INTO is_superuser_running FROM pg_roles WHERE rolname = current_user; + IF is_superuser_running IS NOT True THEN + RAISE EXCEPTION 'This operation can only be initiated by superuser'; + END IF; + + SELECT current_setting('citus.enable_ddl_propagation') INTO enable_ddl_prop; + IF enable_ddl_prop IS NOT True THEN + RAISE EXCEPTION 'This operation cannot be completed when citus.enable_ddl_propagation is False.'; + END IF; + + SELECT groupid INTO local_group_id FROM pg_dist_local_group; + + IF local_group_id != 0 THEN + RAISE EXCEPTION 'Operation is not allowed on this node. Connect to the coordinator and run it again.'; + ELSE + RAISE DEBUG 'We are on the coordinator, continue to sync metadata'; + END IF; +END; + + + ------------------------------------------------------------------------------------------ + -- STAGE 3: Ensure all primary nodes are active + ------------------------------------------------------------------------------------------ + DECLARE + primary_disabled_worker_node_count int := 0; + BEGIN + SELECT count(*) INTO primary_disabled_worker_node_count FROM pg_dist_node + WHERE groupid != 0 AND noderole = 'primary' AND NOT isactive; + + IF primary_disabled_worker_node_count != 0 THEN + RAISE EXCEPTION 'There are inactive primary worker nodes, you need to activate the nodes first.' + 'Use SELECT citus_activate_node() to activate the disabled nodes'; + ELSE + RAISE DEBUG 'There are no disabled worker nodes, continue to sync metadata'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 4: Ensure there is no connectivity issues in the cluster + ------------------------------------------------------------------------------------------ + DECLARE + all_nodes_can_connect_to_each_other boolean := False; + BEGIN + SELECT bool_and(coalesce(result, false)) INTO all_nodes_can_connect_to_each_other FROM citus_check_cluster_node_health(); + + IF all_nodes_can_connect_to_each_other != True THEN + RAISE EXCEPTION 'There are unhealth primary nodes, you need to ensure all ' + 'nodes are up and runnnig. Also, make sure that all nodes can connect ' + 'to each other. Use SELECT * FROM citus_check_cluster_node_health(); ' + 'to check the cluster health'; + ELSE + RAISE DEBUG 'Cluster is healthy, all nodes can connect to each other'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 5: Ensure all nodes are on the same version + ------------------------------------------------------------------------------------------ + DECLARE + coordinator_version text := ''; + worker_node_version text := ''; + worker_node_version_count int := 0; + + BEGIN + SELECT extversion INTO coordinator_version from pg_extension WHERE extname = 'citus'; + + -- first, check if all nodes have the same versions + SELECT + count(*) INTO worker_node_version_count + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + IF enforce_version_check AND worker_node_version_count != 1 THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the some of the workers has version different versions'; + ELSE + RAISE DEBUG 'All worker nodes have the same Citus version'; + END IF; + + -- second, check if all nodes have the same versions + SELECT + result INTO worker_node_version + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + + IF enforce_version_check AND coordinator_version != worker_node_version THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the coordinator has version % and the worker(s) has %', + coordinator_version, worker_node_version; + ELSE + RAISE DEBUG 'All nodes have the same Citus version'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 6: Ensure all the partitioned tables have the proper naming structure + -- As described on https://github.com/citusdata/citus/issues/4962 + -- existing indexes on partitioned distributed tables can collide + -- with the index names exists on the shards + -- luckily, we know how to fix it. + -- And, note that we should do this even if the cluster is a basic plan + -- (e.g., single node Citus) such that when cluster scaled out, everything + -- works as intended + -- And, this should be done only ONCE for a cluster as it can be a pretty + -- time consuming operation. Thus, even if the function is called multiple time, + -- we keep track of it and do not re-execute this part if not needed. + ------------------------------------------------------------------------------------------ + DECLARE + partitioned_table_exists_pre_11 boolean:=False; + BEGIN + + -- we recorded if partitioned tables exists during upgrade to Citus 11 + SELECT metadata->>'partitioned_citus_table_exists_pre_11' INTO partitioned_table_exists_pre_11 + FROM pg_dist_node_metadata; + + IF partitioned_table_exists_pre_11 IS NOT NULL AND partitioned_table_exists_pre_11 THEN + + -- this might take long depending on the number of partitions and shards... + RAISE NOTICE 'Preparing all the existing partitioned table indexes'; + PERFORM pg_catalog.fix_all_partition_shard_index_names(); + + -- great, we are done with fixing the existing wrong index names + -- so, lets remove this + UPDATE pg_dist_node_metadata + SET metadata=jsonb_delete(metadata, 'partitioned_citus_table_exists_pre_11'); + ELSE + RAISE DEBUG 'There are no partitioned tables that should be fixed'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 7: Return early if there are no primary worker nodes + -- We don't strictly need this step, but it gives a nicer notice message + ------------------------------------------------------------------------------------------ + DECLARE + primary_worker_node_count bigint :=0; + BEGIN + SELECT count(*) INTO primary_worker_node_count FROM pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + + IF primary_worker_node_count = 0 THEN + RAISE NOTICE 'There are no primary worker nodes, no need to sync metadata to any node'; + RETURN true; + ELSE + RAISE DEBUG 'There are % primary worker nodes, continue to sync metadata', primary_worker_node_count; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 8: Do the actual metadata & object syncing to the worker nodes + -- For the "already synced" metadata nodes, we do not strictly need to + -- sync the objects & metadata, but there is no harm to do it anyway + -- it'll only cost some execution time but makes sure that we have a + -- a consistent metadata & objects across all the nodes + ------------------------------------------------------------------------------------------ + DECLARE + BEGIN + + -- this might take long depending on the number of tables & objects ... + RAISE NOTICE 'Preparing to sync the metadata to all nodes'; + + PERFORM start_metadata_sync_to_node(nodename,nodeport) + FROM + pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + END; + + RETURN true; +END; +$$; +COMMENT ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) + IS 'finalizes upgrade to Citus'; + +REVOKE ALL ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) FROM PUBLIC; diff --git a/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/latest.sql b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/latest.sql new file mode 100644 index 000000000..d93dd0f93 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/latest.sql @@ -0,0 +1,224 @@ +-- citus_finalize_upgrade_to_citus11() is a helper UDF ensures +-- the upgrade to Citus 11 is finished successfully. Upgrade to +-- Citus 11 requires all active primary worker nodes to get the +-- metadata. And, this function's job is to sync the metadata to +-- the nodes that does not already have +-- once the function finishes without any errors and returns true +-- the cluster is ready for running distributed queries from +-- the worker nodes. When debug is enabled, the function provides +-- more information to the user. +CREATE OR REPLACE FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(enforce_version_check bool default true) + RETURNS bool + LANGUAGE plpgsql + AS $$ +BEGIN + + --------------------------------------------- + -- This script consists of N stages + -- Each step is documented, and if log level + -- is reduced to DEBUG1, each step is logged + -- as well + --------------------------------------------- + +------------------------------------------------------------------------------------------ + -- STAGE 0: Ensure no concurrent node metadata changing operation happens while this + -- script is running via acquiring a strong lock on the pg_dist_node +------------------------------------------------------------------------------------------ +BEGIN + LOCK TABLE pg_dist_node IN EXCLUSIVE MODE NOWAIT; + + EXCEPTION WHEN OTHERS THEN + RAISE 'Another node metadata changing operation is in progress, try again.'; +END; + +------------------------------------------------------------------------------------------ + -- STAGE 1: We want all the commands to run in the same transaction block. Without + -- sequential mode, metadata syncing cannot be done in a transaction block along with + -- other commands +------------------------------------------------------------------------------------------ + SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; + +------------------------------------------------------------------------------------------ + -- STAGE 2: Ensure we have the prerequisites + -- (a) only superuser can run this script + -- (b) cannot be executed when enable_ddl_propagation is False + -- (c) can only be executed from the coordinator +------------------------------------------------------------------------------------------ +DECLARE + is_superuser_running boolean := False; + enable_ddl_prop boolean:= False; + local_group_id int := 0; +BEGIN + SELECT rolsuper INTO is_superuser_running FROM pg_roles WHERE rolname = current_user; + IF is_superuser_running IS NOT True THEN + RAISE EXCEPTION 'This operation can only be initiated by superuser'; + END IF; + + SELECT current_setting('citus.enable_ddl_propagation') INTO enable_ddl_prop; + IF enable_ddl_prop IS NOT True THEN + RAISE EXCEPTION 'This operation cannot be completed when citus.enable_ddl_propagation is False.'; + END IF; + + SELECT groupid INTO local_group_id FROM pg_dist_local_group; + + IF local_group_id != 0 THEN + RAISE EXCEPTION 'Operation is not allowed on this node. Connect to the coordinator and run it again.'; + ELSE + RAISE DEBUG 'We are on the coordinator, continue to sync metadata'; + END IF; +END; + + + ------------------------------------------------------------------------------------------ + -- STAGE 3: Ensure all primary nodes are active + ------------------------------------------------------------------------------------------ + DECLARE + primary_disabled_worker_node_count int := 0; + BEGIN + SELECT count(*) INTO primary_disabled_worker_node_count FROM pg_dist_node + WHERE groupid != 0 AND noderole = 'primary' AND NOT isactive; + + IF primary_disabled_worker_node_count != 0 THEN + RAISE EXCEPTION 'There are inactive primary worker nodes, you need to activate the nodes first.' + 'Use SELECT citus_activate_node() to activate the disabled nodes'; + ELSE + RAISE DEBUG 'There are no disabled worker nodes, continue to sync metadata'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 4: Ensure there is no connectivity issues in the cluster + ------------------------------------------------------------------------------------------ + DECLARE + all_nodes_can_connect_to_each_other boolean := False; + BEGIN + SELECT bool_and(coalesce(result, false)) INTO all_nodes_can_connect_to_each_other FROM citus_check_cluster_node_health(); + + IF all_nodes_can_connect_to_each_other != True THEN + RAISE EXCEPTION 'There are unhealth primary nodes, you need to ensure all ' + 'nodes are up and runnnig. Also, make sure that all nodes can connect ' + 'to each other. Use SELECT * FROM citus_check_cluster_node_health(); ' + 'to check the cluster health'; + ELSE + RAISE DEBUG 'Cluster is healthy, all nodes can connect to each other'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 5: Ensure all nodes are on the same version + ------------------------------------------------------------------------------------------ + DECLARE + coordinator_version text := ''; + worker_node_version text := ''; + worker_node_version_count int := 0; + + BEGIN + SELECT extversion INTO coordinator_version from pg_extension WHERE extname = 'citus'; + + -- first, check if all nodes have the same versions + SELECT + count(*) INTO worker_node_version_count + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + IF enforce_version_check AND worker_node_version_count != 1 THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the some of the workers has version different versions'; + ELSE + RAISE DEBUG 'All worker nodes have the same Citus version'; + END IF; + + -- second, check if all nodes have the same versions + SELECT + result INTO worker_node_version + FROM + run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';') + GROUP BY result; + + IF enforce_version_check AND coordinator_version != worker_node_version THEN + RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently ' + 'the coordinator has version % and the worker(s) has %', + coordinator_version, worker_node_version; + ELSE + RAISE DEBUG 'All nodes have the same Citus version'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 6: Ensure all the partitioned tables have the proper naming structure + -- As described on https://github.com/citusdata/citus/issues/4962 + -- existing indexes on partitioned distributed tables can collide + -- with the index names exists on the shards + -- luckily, we know how to fix it. + -- And, note that we should do this even if the cluster is a basic plan + -- (e.g., single node Citus) such that when cluster scaled out, everything + -- works as intended + -- And, this should be done only ONCE for a cluster as it can be a pretty + -- time consuming operation. Thus, even if the function is called multiple time, + -- we keep track of it and do not re-execute this part if not needed. + ------------------------------------------------------------------------------------------ + DECLARE + partitioned_table_exists_pre_11 boolean:=False; + BEGIN + + -- we recorded if partitioned tables exists during upgrade to Citus 11 + SELECT metadata->>'partitioned_citus_table_exists_pre_11' INTO partitioned_table_exists_pre_11 + FROM pg_dist_node_metadata; + + IF partitioned_table_exists_pre_11 IS NOT NULL AND partitioned_table_exists_pre_11 THEN + + -- this might take long depending on the number of partitions and shards... + RAISE NOTICE 'Preparing all the existing partitioned table indexes'; + PERFORM pg_catalog.fix_all_partition_shard_index_names(); + + -- great, we are done with fixing the existing wrong index names + -- so, lets remove this + UPDATE pg_dist_node_metadata + SET metadata=jsonb_delete(metadata, 'partitioned_citus_table_exists_pre_11'); + ELSE + RAISE DEBUG 'There are no partitioned tables that should be fixed'; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 7: Return early if there are no primary worker nodes + -- We don't strictly need this step, but it gives a nicer notice message + ------------------------------------------------------------------------------------------ + DECLARE + primary_worker_node_count bigint :=0; + BEGIN + SELECT count(*) INTO primary_worker_node_count FROM pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + + IF primary_worker_node_count = 0 THEN + RAISE NOTICE 'There are no primary worker nodes, no need to sync metadata to any node'; + RETURN true; + ELSE + RAISE DEBUG 'There are % primary worker nodes, continue to sync metadata', primary_worker_node_count; + END IF; + END; + + ------------------------------------------------------------------------------------------ + -- STAGE 8: Do the actual metadata & object syncing to the worker nodes + -- For the "already synced" metadata nodes, we do not strictly need to + -- sync the objects & metadata, but there is no harm to do it anyway + -- it'll only cost some execution time but makes sure that we have a + -- a consistent metadata & objects across all the nodes + ------------------------------------------------------------------------------------------ + DECLARE + BEGIN + + -- this might take long depending on the number of tables & objects ... + RAISE NOTICE 'Preparing to sync the metadata to all nodes'; + + PERFORM start_metadata_sync_to_node(nodename,nodeport) + FROM + pg_dist_node WHERE groupid != 0 AND noderole = 'primary'; + END; + + RETURN true; +END; +$$; +COMMENT ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) + IS 'finalizes upgrade to Citus'; + +REVOKE ALL ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) FROM PUBLIC; diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index 31154a203..905d2cab5 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -75,6 +75,7 @@ extern void GetDependentSequencesWithRelation(Oid relationId, List **attnumList, attnum); extern List * GetDependentFunctionsWithRelation(Oid relationId); extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum); +extern void SetLocalEnableMetadataSync(bool state); #define DELETE_ALL_NODES "DELETE FROM pg_dist_node" #define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement" diff --git a/src/test/regress/after_citus_upgrade_coord_schedule b/src/test/regress/after_citus_upgrade_coord_schedule index 49b4e73d9..e73837c59 100644 --- a/src/test/regress/after_citus_upgrade_coord_schedule +++ b/src/test/regress/after_citus_upgrade_coord_schedule @@ -4,3 +4,4 @@ test: upgrade_basic_after test: upgrade_partition_constraints_after test: upgrade_pg_dist_object_test_after test: upgrade_columnar_metapage_after +test: upgrade_post_11_after diff --git a/src/test/regress/before_citus_upgrade_coord_schedule b/src/test/regress/before_citus_upgrade_coord_schedule index 006217e8a..169a7f418 100644 --- a/src/test/regress/before_citus_upgrade_coord_schedule +++ b/src/test/regress/before_citus_upgrade_coord_schedule @@ -4,3 +4,4 @@ test: upgrade_basic_before test: upgrade_partition_constraints_before test: upgrade_pg_dist_object_test_before test: upgrade_columnar_metapage_before +test: upgrade_post_11_before diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index c7fcc73e5..79d21dd3e 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -966,7 +966,7 @@ DELETE FROM pg_dist_shard WHERE shardid = 1; CREATE TABLE e_transactions(order_id varchar(255) NULL, transaction_id int) PARTITION BY LIST(transaction_id); CREATE TABLE orders_2020_07_01 PARTITION OF e_transactions FOR VALUES IN (1,2,3); -INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', NULL, 7, 's'); +INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', '{VAR :varno 1 :varattno 1 :vartype 1043 :vartypmod 259 :varcollid 100 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1}', 7, 's'); SELECT (metadata->>'partitioned_citus_table_exists_pre_11')::boolean as partitioned_citus_table_exists_pre_11, (metadata->>'partitioned_citus_table_exists_pre_11') IS NULL as is_null @@ -1012,6 +1012,7 @@ SELECT * FROM multi_extension.print_extension_changes(); | function citus_check_cluster_node_health() SETOF record | function citus_check_connection_to_node(text,integer) boolean | function citus_disable_node(text,integer,boolean) void + | function citus_finalize_upgrade_to_citus11(boolean) boolean | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void | function citus_internal_global_blocked_processes() SETOF record | function citus_internal_local_blocked_processes() SETOF record @@ -1025,7 +1026,7 @@ SELECT * FROM multi_extension.print_extension_changes(); | 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 -(22 rows) +(23 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index e5e7b9215..5f5ea7b1e 100644 --- a/src/test/regress/expected/upgrade_list_citus_objects.out +++ b/src/test/regress/expected/upgrade_list_citus_objects.out @@ -57,6 +57,7 @@ ORDER BY 1; function citus_drop_trigger() function citus_executor_name(integer) function citus_extradata_container(internal) + function citus_finalize_upgrade_to_citus11(boolean) function citus_finish_pg_upgrade() function citus_get_active_worker_nodes() function citus_internal.columnar_ensure_am_depends_catalog() @@ -272,5 +273,5 @@ ORDER BY 1; view citus_worker_stat_activity view pg_dist_shard_placement view time_partitions -(256 rows) +(257 rows) diff --git a/src/test/regress/expected/upgrade_pg_dist_object_test_after.out b/src/test/regress/expected/upgrade_pg_dist_object_test_after.out index 616b4fc32..443cb6205 100644 --- a/src/test/regress/expected/upgrade_pg_dist_object_test_after.out +++ b/src/test/regress/expected/upgrade_pg_dist_object_test_after.out @@ -39,12 +39,38 @@ drop cascades to table upgrade_basic.t_range SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3; type | object_names | object_args --------------------------------------------------------------------- - database | {postgres} | {} - extension | {isn} | {} - role | {postgres} | {} - schema | {fooschema} | {} - schema | {new_schema} | {} - schema | {public} | {} - type | {fooschema.footype} | {} - (7 rows) + collation | {post_11_upgrade,german_phonebook_unpropagated} | {} + database | {postgres} | {} + extension | {isn} | {} + extension | {plpgsql} | {} + function | {post_11_upgrade,func_in_transaction_def} | {} + role | {postgres} | {} + schema | {fooschema} | {} + schema | {new_schema} | {} + schema | {post_11_upgrade} | {} + schema | {public} | {} + table | {fooschema,footable} | {} + table | {new_schema,another_dist_table} | {} + table | {post_11_upgrade,colocated_dist_table} | {} + table | {post_11_upgrade,colocated_partitioned_table} | {} + table | {post_11_upgrade,colocated_partitioned_table_2020_01_01} | {} + table | {post_11_upgrade,dist} | {} + table | {post_11_upgrade,index_backed_rep_identity} | {} + table | {post_11_upgrade,part_table} | {} + table | {post_11_upgrade,part_table_p202008} | {} + table | {post_11_upgrade,part_table_p202009} | {} + table | {post_11_upgrade,reference_table} | {} + table | {post_11_upgrade,sensors} | {} + table | {post_11_upgrade,sensors_2020_01_01} | {} + table | {post_11_upgrade,sensors_news} | {} + table | {post_11_upgrade,sensors_old} | {} + table | {post_11_upgrade,sensors_parser} | {} + table | {post_11_upgrade,sensors_parser_a_partition} | {} + table | {post_11_upgrade,test_propagate_collate} | {} + table | {public,dist_table} | {} + table | {public,isn_dist_table} | {} + text search configuration | {post_11_upgrade,partial_index_test_config} | {} + type | {fooschema.footype} | {} + type | {post_11_upgrade.my_type} | {} + (33 rows) diff --git a/src/test/regress/expected/upgrade_pg_dist_object_test_after_1.out b/src/test/regress/expected/upgrade_pg_dist_object_test_after_1.out new file mode 100644 index 000000000..743164bfc --- /dev/null +++ b/src/test/regress/expected/upgrade_pg_dist_object_test_after_1.out @@ -0,0 +1,54 @@ +-- run this test only when old citus version is 9.0 +\set upgrade_test_old_citus_version `echo "$CITUS_OLD_VERSION"` +SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int = 9 AND + substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int = 0 +AS upgrade_test_old_citus_version_e_9_0; + upgrade_test_old_citus_version_e_9_0 +--------------------------------------------------------------------- + t +(1 row) + +\gset +\if :upgrade_test_old_citus_version_e_9_0 +\else +\q +\endif +-- drop objects from previous test (uprade_basic_after.sql) for a clean test +-- drop upgrade_basic schema and switch back to public schema +SET search_path to public; +DROP SCHEMA upgrade_basic CASCADE; +NOTICE: drop cascades to 7 other objects +DETAIL: drop cascades to table upgrade_basic.t +drop cascades to table upgrade_basic.tp +drop cascades to table upgrade_basic.t_ab +drop cascades to table upgrade_basic.t2 +drop cascades to table upgrade_basic.r +drop cascades to table upgrade_basic.tr +drop cascades to table upgrade_basic.t_range +-- as we updated citus to available version, +-- "isn" extension +-- "new_schema" schema +-- "public" schema +-- "fooschema" schema +-- "footype" type (under schema 'fooschema') + -- will now be marked as distributed + -- but, + -- "seg" extension + -- will not be marked as distributed +-- see underlying objects +SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3; + type | object_names | object_args +--------------------------------------------------------------------- + database | {postgres} | {} + extension | {isn} | {} + role | {postgres} | {} + schema | {fooschema} | {} + schema | {new_schema} | {} + schema | {public} | {} + table | {fooschema,footable} | {} + table | {new_schema,another_dist_table} | {} + table | {public,dist_table} | {} + table | {public,isn_dist_table} | {} + type | {fooschema.footype} | {} + (11 rows) + diff --git a/src/test/regress/expected/upgrade_post_11_after.out b/src/test/regress/expected/upgrade_post_11_after.out new file mode 100644 index 000000000..2da475c08 --- /dev/null +++ b/src/test/regress/expected/upgrade_post_11_after.out @@ -0,0 +1,76 @@ +SET search_path = post_11_upgrade; +-- make sure that we always (re)sync the metadata +UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb('true'::bool), true); +SELECT citus_finalize_upgrade_to_citus11(enforce_version_check:=false); +NOTICE: Preparing all the existing partitioned table indexes +NOTICE: Preparing to sync the metadata to all nodes + citus_finalize_upgrade_to_citus11 +--------------------------------------------------------------------- + t +(1 row) + +-- tables are objects with Citus 11+ +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1; + pg_identify_object_as_address +--------------------------------------------------------------------- + (function,"{post_11_upgrade,func_in_transaction_def}",{}) + (schema,{post_11_upgrade},{}) + (table,"{post_11_upgrade,part_table}",{}) + (table,"{post_11_upgrade,sensors}",{}) + ("text search configuration","{post_11_upgrade,partial_index_test_config}",{}) + (type,{post_11_upgrade.my_type},{}) +(6 rows) + +-- on all nodes +SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1; + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57636,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}") + (localhost,57637,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}") +(2 rows) + +-- Create the necessary test utility function +CREATE OR REPLACE FUNCTION activate_node_snapshot() + RETURNS text[] + LANGUAGE C STRICT + AS 'citus'; +-- make sure that workers and the coordinator has the same datesyle +SET datestyle = "ISO, YMD"; +SELECT 1 FROM run_command_on_workers($$ALTER SYSTEM SET datestyle = "ISO, YMD";$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +SELECT 1 FROM run_command_on_workers($$SELECT pg_reload_conf()$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +-- make sure that the metadata is consistent across all nodes +-- we exclude the distributed_object_data as they are +-- not sorted in the same order (as OIDs differ on the nodes) +SELECT count(*) = 0 AS same_metadata_in_workers FROM +( + ( + SELECT unnest(activate_node_snapshot()) as command + EXCEPT + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + ) +UNION + ( + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) as command + ) +) AS foo WHERE command NOT ILIKE '%distributed_object_data%'; + same_metadata_in_workers +--------------------------------------------------------------------- + t +(1 row) + diff --git a/src/test/regress/expected/upgrade_post_11_before.out b/src/test/regress/expected/upgrade_post_11_before.out new file mode 100644 index 000000000..37bbab11b --- /dev/null +++ b/src/test/regress/expected/upgrade_post_11_before.out @@ -0,0 +1,184 @@ +-- test cases for #3970 +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +CREATE SCHEMA post_11_upgrade; +SET search_path = post_11_upgrade; +--1. create a partitioned table, and a vanilla table that will be colocated with this table +CREATE TABLE part_table ( + work_ymdt timestamp without time zone NOT NULL, + seq bigint NOT NULL, + my_seq bigint NOT NULL, + work_memo character varying(150), + CONSTRAINT work_memo_check CHECK ((octet_length((work_memo)::text) <= 150)), + PRIMARY KEY(seq, work_ymdt) +) +PARTITION BY RANGE (work_ymdt); +CREATE TABLE dist(seq bigint UNIQUE); +--2. perform create_distributed_table +SELECT create_distributed_table('part_table', 'seq'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist','seq'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +--3. add a partitions +CREATE TABLE part_table_p202008 PARTITION OF part_table FOR VALUES FROM ('2020-08-01 00:00:00') TO ('2020-09-01 00:00:00'); +CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00'); +--3. create indexes +CREATE INDEX i_part_1 ON part_table(seq); +CREATE INDEX i_part_2 ON part_table(my_seq, seq); +CREATE INDEX i_part_3 ON part_table(work_memo, seq); +CREATE TABLE sensors( +measureid integer, +eventdatetime date, +measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_old PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE TABLE sensors_2020_01_01 PARTITION OF sensors FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +CREATE TABLE sensors_news PARTITION OF sensors FOR VALUES FROM ('2020-05-01') TO ('2025-01-01'); +CREATE INDEX index_on_parent ON sensors(lower(measureid::text)); +CREATE INDEX index_on_child ON sensors_2020_01_01(lower(measure_data::text)); +CREATE INDEX hash_index ON sensors USING HASH((measure_data->'IsFailed')); +CREATE INDEX index_with_include ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime); +CREATE STATISTICS s1 (dependencies) ON measureid, eventdatetime FROM sensors; +CREATE STATISTICS s2 (dependencies) ON measureid, eventdatetime FROM sensors_2020_01_01; +ALTER INDEX index_on_parent ALTER COLUMN 1 SET STATISTICS 1000; +ALTER INDEX index_on_child ALTER COLUMN 1 SET STATISTICS 1000; +CLUSTER sensors_2020_01_01 USING index_on_child; +SELECT create_distributed_table('sensors', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- create a colocated distributed tables and create foreign keys FROM/TO +-- the partitions +CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY); +SELECT create_distributed_table('colocated_dist_table', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CLUSTER colocated_dist_table USING colocated_dist_table_pkey; +WARNING: not propagating CLUSTER command to worker nodes +CREATE TABLE colocated_partitioned_table( + measureid integer, + eventdatetime date, + PRIMARY KEY (measureid, eventdatetime)) +PARTITION BY RANGE(eventdatetime); +CREATE TABLE colocated_partitioned_table_2020_01_01 PARTITION OF colocated_partitioned_table FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +SELECT create_distributed_table('colocated_partitioned_table', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CLUSTER colocated_partitioned_table_2020_01_01 USING colocated_partitioned_table_2020_01_01_pkey; +WARNING: not propagating CLUSTER command to worker nodes +CREATE TABLE reference_table (measureid integer PRIMARY KEY); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +-- this table is used to make sure that index backed +-- replica identites can have clustered indexes +-- and no index statistics +CREATE TABLE index_backed_rep_identity(key int NOT NULL); +CREATE UNIQUE INDEX uqx ON index_backed_rep_identity(key); +ALTER TABLE index_backed_rep_identity REPLICA IDENTITY USING INDEX uqx; +CLUSTER index_backed_rep_identity USING uqx; +SELECT create_distributed_table('index_backed_rep_identity', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- from parent to regular dist +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid); +-- from parent to parent +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_parent FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table(measureid, eventdatetime); +-- from parent to child +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_child FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid, eventdatetime); +-- load some data +INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_partitioned_table SELECT i, '2020-01-05' FROM generate_series(0,1000)i; +INSERT INTO sensors SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i; +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default ); +SELECT 1 FROM run_command_on_workers($$CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default );$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; +SELECT run_command_on_workers('SET citus.enable_ddl_propagation TO off; +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$;'); + run_command_on_workers +--------------------------------------------------------------------- + (localhost,57636,t,SET) + (localhost,57637,t,SET) +(2 rows) + +CREATE TYPE post_11_upgrade.my_type AS (a int); +RESET citus.enable_ddl_propagation; +CREATE TABLE sensors_parser( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + col_with_def int DEFAULT post_11_upgrade.func_in_transaction_def(), + col_with_type post_11_upgrade.my_type, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_parser_a_partition PARTITION OF sensors_parser FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_parser_search_name ON sensors_parser USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors_parser', 'measureid'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET citus.enable_ddl_propagation TO off; +CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk'); +SELECT 1 FROM run_command_on_workers($$CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk');$$); + ?column? +--------------------------------------------------------------------- + 1 + 1 +(2 rows) + +SET citus.enable_ddl_propagation TO on; +CREATE TABLE test_propagate_collate(id int, t2 text COLLATE german_phonebook_unpropagated); +SELECT create_distributed_table('test_propagate_collate', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index bd97fc961..0bac6d067 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -430,7 +430,7 @@ DELETE FROM pg_dist_shard WHERE shardid = 1; CREATE TABLE e_transactions(order_id varchar(255) NULL, transaction_id int) PARTITION BY LIST(transaction_id); CREATE TABLE orders_2020_07_01 PARTITION OF e_transactions FOR VALUES IN (1,2,3); -INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', NULL, 7, 's'); +INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', '{VAR :varno 1 :varattno 1 :vartype 1043 :vartypmod 259 :varcollid 100 :varlevelsup 0 :varnosyn 1 :varattnosyn 1 :location -1}', 7, 's'); SELECT (metadata->>'partitioned_citus_table_exists_pre_11')::boolean as partitioned_citus_table_exists_pre_11, diff --git a/src/test/regress/sql/upgrade_post_11_after.sql b/src/test/regress/sql/upgrade_post_11_after.sql new file mode 100644 index 000000000..5e08ca8b1 --- /dev/null +++ b/src/test/regress/sql/upgrade_post_11_after.sql @@ -0,0 +1,42 @@ +SET search_path = post_11_upgrade; + +-- make sure that we always (re)sync the metadata +UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb('true'::bool), true); +SELECT citus_finalize_upgrade_to_citus11(enforce_version_check:=false); + +-- tables are objects with Citus 11+ +SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1; + +-- on all nodes +SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1; + +-- Create the necessary test utility function +CREATE OR REPLACE FUNCTION activate_node_snapshot() + RETURNS text[] + LANGUAGE C STRICT + AS 'citus'; + +-- make sure that workers and the coordinator has the same datesyle +SET datestyle = "ISO, YMD"; +SELECT 1 FROM run_command_on_workers($$ALTER SYSTEM SET datestyle = "ISO, YMD";$$); +SELECT 1 FROM run_command_on_workers($$SELECT pg_reload_conf()$$); + +-- make sure that the metadata is consistent across all nodes +-- we exclude the distributed_object_data as they are +-- not sorted in the same order (as OIDs differ on the nodes) +SELECT count(*) = 0 AS same_metadata_in_workers FROM +( + ( + SELECT unnest(activate_node_snapshot()) as command + EXCEPT + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + ) +UNION + ( + SELECT unnest(result::text[]) AS command + FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$) + EXCEPT + SELECT unnest(activate_node_snapshot()) as command + ) +) AS foo WHERE command NOT ILIKE '%distributed_object_data%'; diff --git a/src/test/regress/sql/upgrade_post_11_before.sql b/src/test/regress/sql/upgrade_post_11_before.sql new file mode 100644 index 000000000..959b026f8 --- /dev/null +++ b/src/test/regress/sql/upgrade_post_11_before.sql @@ -0,0 +1,155 @@ + +-- test cases for #3970 +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; + +CREATE SCHEMA post_11_upgrade; +SET search_path = post_11_upgrade; + +--1. create a partitioned table, and a vanilla table that will be colocated with this table +CREATE TABLE part_table ( + work_ymdt timestamp without time zone NOT NULL, + seq bigint NOT NULL, + my_seq bigint NOT NULL, + work_memo character varying(150), + CONSTRAINT work_memo_check CHECK ((octet_length((work_memo)::text) <= 150)), + PRIMARY KEY(seq, work_ymdt) +) +PARTITION BY RANGE (work_ymdt); + +CREATE TABLE dist(seq bigint UNIQUE); + +--2. perform create_distributed_table +SELECT create_distributed_table('part_table', 'seq'); +SELECT create_distributed_table('dist','seq'); + +--3. add a partitions +CREATE TABLE part_table_p202008 PARTITION OF part_table FOR VALUES FROM ('2020-08-01 00:00:00') TO ('2020-09-01 00:00:00'); +CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00'); + +--3. create indexes +CREATE INDEX i_part_1 ON part_table(seq); +CREATE INDEX i_part_2 ON part_table(my_seq, seq); +CREATE INDEX i_part_3 ON part_table(work_memo, seq); + + +CREATE TABLE sensors( +measureid integer, +eventdatetime date, +measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); + +CREATE TABLE sensors_old PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE TABLE sensors_2020_01_01 PARTITION OF sensors FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +CREATE TABLE sensors_news PARTITION OF sensors FOR VALUES FROM ('2020-05-01') TO ('2025-01-01'); + +CREATE INDEX index_on_parent ON sensors(lower(measureid::text)); +CREATE INDEX index_on_child ON sensors_2020_01_01(lower(measure_data::text)); +CREATE INDEX hash_index ON sensors USING HASH((measure_data->'IsFailed')); +CREATE INDEX index_with_include ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime); + +CREATE STATISTICS s1 (dependencies) ON measureid, eventdatetime FROM sensors; +CREATE STATISTICS s2 (dependencies) ON measureid, eventdatetime FROM sensors_2020_01_01; + +ALTER INDEX index_on_parent ALTER COLUMN 1 SET STATISTICS 1000; +ALTER INDEX index_on_child ALTER COLUMN 1 SET STATISTICS 1000; + +CLUSTER sensors_2020_01_01 USING index_on_child; +SELECT create_distributed_table('sensors', 'measureid'); + + +-- create a colocated distributed tables and create foreign keys FROM/TO +-- the partitions +CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY); +SELECT create_distributed_table('colocated_dist_table', 'measureid'); + +CLUSTER colocated_dist_table USING colocated_dist_table_pkey; + +CREATE TABLE colocated_partitioned_table( + measureid integer, + eventdatetime date, + PRIMARY KEY (measureid, eventdatetime)) +PARTITION BY RANGE(eventdatetime); + +CREATE TABLE colocated_partitioned_table_2020_01_01 PARTITION OF colocated_partitioned_table FOR VALUES FROM ('2020-01-01') TO ('2020-02-01'); +SELECT create_distributed_table('colocated_partitioned_table', 'measureid'); + +CLUSTER colocated_partitioned_table_2020_01_01 USING colocated_partitioned_table_2020_01_01_pkey; + +CREATE TABLE reference_table (measureid integer PRIMARY KEY); +SELECT create_reference_table('reference_table'); + +-- this table is used to make sure that index backed +-- replica identites can have clustered indexes +-- and no index statistics +CREATE TABLE index_backed_rep_identity(key int NOT NULL); +CREATE UNIQUE INDEX uqx ON index_backed_rep_identity(key); +ALTER TABLE index_backed_rep_identity REPLICA IDENTITY USING INDEX uqx; +CLUSTER index_backed_rep_identity USING uqx; +SELECT create_distributed_table('index_backed_rep_identity', 'key'); + +-- from parent to regular dist +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid); + +-- from parent to parent +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_parent FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table(measureid, eventdatetime); + +-- from parent to child +ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_child FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid, eventdatetime); + +-- load some data +INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i; +INSERT INTO colocated_partitioned_table SELECT i, '2020-01-05' FROM generate_series(0,1000)i; +INSERT INTO sensors SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i; + + +SET citus.enable_ddl_propagation TO off; +CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default ); +SELECT 1 FROM run_command_on_workers($$CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default );$$); + +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$; + +SELECT run_command_on_workers('SET citus.enable_ddl_propagation TO off; +CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def() +RETURNS int +LANGUAGE plpgsql AS +$$ +BEGIN + return 1; +END; +$$;'); + +CREATE TYPE post_11_upgrade.my_type AS (a int); + +RESET citus.enable_ddl_propagation; + +CREATE TABLE sensors_parser( + measureid integer, + eventdatetime date, + measure_data jsonb, + name text, + col_with_def int DEFAULT post_11_upgrade.func_in_transaction_def(), + col_with_type post_11_upgrade.my_type, + PRIMARY KEY (measureid, eventdatetime, measure_data) +) PARTITION BY RANGE(eventdatetime); +CREATE TABLE sensors_parser_a_partition PARTITION OF sensors_parser FOR VALUES FROM ('2000-01-01') TO ('2020-01-01'); +CREATE INDEX sensors_parser_search_name ON sensors_parser USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text)); +SELECT create_distributed_table('sensors_parser', 'measureid'); + + +SET citus.enable_ddl_propagation TO off; +CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk'); +SELECT 1 FROM run_command_on_workers($$CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk');$$); +SET citus.enable_ddl_propagation TO on; + +CREATE TABLE test_propagate_collate(id int, t2 text COLLATE german_phonebook_unpropagated); +SELECT create_distributed_table('test_propagate_collate', 'id'); From e80a36c4b6545fc414a29c456f36fe31f4b45298 Mon Sep 17 00:00:00 2001 From: Onder Kalaci Date: Mon, 28 Feb 2022 10:26:40 +0100 Subject: [PATCH 24/24] Improve visibility rules for non-priviledge roles It seems like our approach is way too restrictive and some places are wrong. Now, we follow very similar approach to pg_stat_activity. Some of the changes are pre-requsite for implementing citus_dist_stat_activity via citus_stat_activity. --- .../distributed/transaction/backend_data.c | 122 ++++++++++++------ .../isolation_citus_dist_activity.out | 8 +- .../isolation_get_all_active_transactions.out | 73 ++++++----- ...licate_reference_tables_to_coordinator.out | 5 +- .../spec/isolation_citus_dist_activity.spec | 2 +- ...isolation_get_all_active_transactions.spec | 13 +- ...icate_reference_tables_to_coordinator.spec | 3 +- 7 files changed, 147 insertions(+), 79 deletions(-) diff --git a/src/backend/distributed/transaction/backend_data.c b/src/backend/distributed/transaction/backend_data.c index 12ff8d78b..3af20631a 100644 --- a/src/backend/distributed/transaction/backend_data.c +++ b/src/backend/distributed/transaction/backend_data.c @@ -83,6 +83,7 @@ typedef struct BackendManagementShmemData static void StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescriptor); +static bool UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId); static uint64 GenerateGlobalPID(void); static shmem_startup_hook_type prev_shmem_startup_hook = NULL; @@ -114,8 +115,6 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS) { CheckCitusVersion(ERROR); - Oid userId = GetUserId(); - /* prepare data before acquiring spinlock to protect against errors */ int32 initiatorNodeIdentifier = PG_GETARG_INT32(0); uint64 transactionNumber = PG_GETARG_INT64(1); @@ -144,9 +143,6 @@ 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; @@ -357,49 +353,44 @@ 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 */ 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; @@ -418,16 +409,42 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto SpinLockRelease(¤tBackend->mutex); - bool missingOk = true; - int nodeId = ExtractNodeIdFromGlobalPID(currentBackend->globalPID, missingOk); + memset(values, 0, sizeof(values)); + memset(isNulls, false, sizeof(isNulls)); - values[0] = ObjectIdGetDatum(databaseId); - values[1] = Int32GetDatum(backendPid); - values[2] = Int32GetDatum(nodeId); - values[3] = !distributedCommandOriginator; - values[4] = UInt64GetDatum(transactionNumber); - values[5] = TimestampTzGetDatum(transactionIdTimestamp); - values[6] = UInt64GetDatum(currentBackend->globalPID); + /* + * 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); @@ -444,6 +461,35 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto } +/* + * UserHasPermissionToViewStatsOf returns true if currentUserId can + * see backends of backendOwnedId. + * + * We follow the same approach with pg_stat_activity. + */ +static +bool +UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId) +{ + if (has_privs_of_role(currentUserId, backendOwnedId)) + { + return true; + } + + if (is_member_of_role(currentUserId, +#if PG_VERSION_NUM >= PG_VERSION_14 + ROLE_PG_READ_ALL_STATS)) +#else + DEFAULT_ROLE_READ_ALL_STATS)) +#endif + { + return true; + } + + return false; +} + + /* * InitializeBackendManagement requests the necessary shared memory * from Postgres and sets up the shared memory startup hook. @@ -649,8 +695,6 @@ UnSetDistributedTransactionId(void) { SpinLockAcquire(&MyBackendData->mutex); - MyBackendData->databaseId = 0; - MyBackendData->userId = 0; MyBackendData->cancelledDueToDeadlock = false; MyBackendData->transactionId.initiatorNodeIdentifier = 0; MyBackendData->transactionId.transactionOriginator = false; @@ -674,6 +718,8 @@ UnSetGlobalPID(void) SpinLockAcquire(&MyBackendData->mutex); MyBackendData->globalPID = 0; + MyBackendData->databaseId = 0; + MyBackendData->userId = 0; SpinLockRelease(&MyBackendData->mutex); } @@ -755,13 +801,9 @@ 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; @@ -793,9 +835,15 @@ AssignGlobalPID(void) globalPID = ExtractGlobalPID(application_name); } + Oid userId = GetUserId(); + SpinLockAcquire(&MyBackendData->mutex); + MyBackendData->globalPID = globalPID; MyBackendData->distributedCommandOriginator = distributedCommandOriginator; + MyBackendData->databaseId = MyDatabaseId; + MyBackendData->userId = userId; + SpinLockRelease(&MyBackendData->mutex); } diff --git a/src/test/regress/expected/isolation_citus_dist_activity.out b/src/test/regress/expected/isolation_citus_dist_activity.out index f6cd7da9b..aa536e17f 100644 --- a/src/test/regress/expected/isolation_citus_dist_activity.out +++ b/src/test/regress/expected/isolation_citus_dist_activity.out @@ -42,7 +42,7 @@ query |query_hostname |query_hostport|d (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -112,7 +112,7 @@ query |query_hostname |query_hostport|di (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -176,7 +176,7 @@ query |query_hostname |query_hostport|distribute (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -243,7 +243,7 @@ query |query_hostname |query_ (1 row) step s3-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- diff --git a/src/test/regress/expected/isolation_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_replicate_reference_tables_to_coordinator.out b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out index dd7ddefad..8fef72010 100644 --- a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out +++ b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out @@ -119,10 +119,11 @@ step s2-view-worker: query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%dump_local_%' AND query NOT ILIKE '%citus_internal_local_blocked_processes%' AND - query NOT ILIKE '%add_node%' + query NOT ILIKE '%add_node%' AND + backend_type = 'client backend' ORDER BY query, query_hostport DESC; -query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname +query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638| | 0|idle in transaction|Client |ClientRead|postgres|regression UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637| | 0|idle in transaction|Client |ClientRead|postgres|regression diff --git a/src/test/regress/spec/isolation_citus_dist_activity.spec b/src/test/regress/spec/isolation_citus_dist_activity.spec index c41c671f0..5047a656c 100644 --- a/src/test/regress/spec/isolation_citus_dist_activity.spec +++ b/src/test/regress/spec/isolation_citus_dist_activity.spec @@ -89,7 +89,7 @@ step "s3-rollback" step "s3-view-worker" { - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; } // we prefer to sleep before "s2-view-dist" so that we can ensure diff --git a/src/test/regress/spec/isolation_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_replicate_reference_tables_to_coordinator.spec b/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec index c4d6c8fc1..09da5970d 100644 --- a/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec +++ b/src/test/regress/spec/isolation_replicate_reference_tables_to_coordinator.spec @@ -93,7 +93,8 @@ step "s2-view-worker" query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%dump_local_%' AND query NOT ILIKE '%citus_internal_local_blocked_processes%' AND - query NOT ILIKE '%add_node%' + query NOT ILIKE '%add_node%' AND + backend_type = 'client backend' ORDER BY query, query_hostport DESC; }