From 245a62df3e96f3eef0496d574f58e6f6c37701f5 Mon Sep 17 00:00:00 2001 From: Colm Date: Tue, 22 Jul 2025 17:16:53 +0100 Subject: [PATCH 01/13] Avoid query deparse and planning of shard query in local execution. (#8035) DESCRIPTION: Avoid query deparse and planning of shard query in local execution. Adds citus.enable_local_execution_local_plan GUC to allow avoiding unnecessary query deparsing to improve performance of fast-path queries targeting local shards. If a fast path query resolves to a shard that is local to the node planning the query, a shortcut can be taken so that the OID of the shard is plugged into the parse tree, which is then planned by Postgres. In `local_executor.c` the task uses that plan instead of parsing and planning a shard query. How this is done: The fast path planner identifies if the shortcut is possible, and then the distributed planner checks, using `CheckAndBuildDelayedFastPathPlan()`, if a local plan can be generated or if the shard query should be generated. This optimization is controlled by a GUC `citus.enable_local_execution_local_plan` which is on by default. A new regress test `local_execution_local_plan` tests both row-sharding and schema sharding. Negative tests are added to `local_shard_execution_dropped_column` to verify that the optimization is not taken when the shard is local but there is a difference between the shard and distributed table because of a dropped column. --- .../distributed/executor/citus_custom_scan.c | 4 +- .../distributed/executor/local_executor.c | 40 +- .../distributed/planner/deparse_shard_query.c | 41 ++ .../distributed/planner/distributed_planner.c | 72 ++-- .../planner/fast_path_router_planner.c | 188 ++++++--- .../planner/multi_router_planner.c | 293 +++++++++++++- src/backend/distributed/shared_library_init.c | 32 ++ .../distributed/utils/citus_copyfuncs.c | 9 + src/include/distributed/deparse_shard_query.h | 2 + src/include/distributed/distributed_planner.h | 6 + .../distributed/multi_physical_planner.h | 18 +- .../distributed/multi_router_planner.h | 10 +- .../insert_select_single_shard_table.out | 12 + .../expected/local_execution_local_plan.out | 377 ++++++++++++++++++ .../expected/local_shard_execution.out | 8 + .../local_shard_execution_dropped_column.out | 204 +++++++++- .../expected/merge_schema_sharding.out | 2 + .../expected/query_single_shard_table.out | 20 + src/test/regress/multi_mx_schedule | 1 + .../sql/local_execution_local_plan.sql | 214 ++++++++++ .../local_shard_execution_dropped_column.sql | 45 ++- 21 files changed, 1487 insertions(+), 111 deletions(-) create mode 100644 src/test/regress/expected/local_execution_local_plan.out create mode 100644 src/test/regress/sql/local_execution_local_plan.sql diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 5ba60b5ad..53b0ccb0f 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -682,11 +682,13 @@ RegenerateTaskForFasthPathQuery(Job *workerJob) } bool isLocalTableModification = false; + bool delayedFastPath = false; GenerateSingleShardRouterTaskList(workerJob, relationShardList, placementList, shardId, - isLocalTableModification); + isLocalTableModification, + delayedFastPath); } diff --git a/src/backend/distributed/executor/local_executor.c b/src/backend/distributed/executor/local_executor.c index 2ced0a43f..07bd89116 100644 --- a/src/backend/distributed/executor/local_executor.c +++ b/src/backend/distributed/executor/local_executor.c @@ -313,6 +313,7 @@ ExecuteLocalTaskListExtended(List *taskList, { int taskNumParams = numParams; Oid *taskParameterTypes = parameterTypes; + int taskType = GetTaskQueryType(task); if (task->parametersInQueryStringResolved) { @@ -330,7 +331,7 @@ ExecuteLocalTaskListExtended(List *taskList, * for concatenated strings, we set queryStringList so that we can access * each query string. */ - if (GetTaskQueryType(task) == TASK_QUERY_TEXT_LIST) + if (taskType == TASK_QUERY_TEXT_LIST) { List *queryStringList = task->taskQuery.data.queryStringList; totalRowsProcessed += @@ -342,22 +343,31 @@ ExecuteLocalTaskListExtended(List *taskList, continue; } - Query *shardQuery = ParseQueryString(TaskQueryString(task), - taskParameterTypes, - taskNumParams); + if (taskType != TASK_QUERY_LOCAL_PLAN) + { + Query *shardQuery = ParseQueryString(TaskQueryString(task), + taskParameterTypes, + taskNumParams); + int cursorOptions = CURSOR_OPT_PARALLEL_OK; - int cursorOptions = CURSOR_OPT_PARALLEL_OK; - - /* - * Altough the shardQuery is local to this node, we prefer planner() - * over standard_planner(). The primary reason for that is Citus itself - * is not very tolarent standard_planner() calls that doesn't go through - * distributed_planner() because of the way that restriction hooks are - * implemented. So, let planner to call distributed_planner() which - * eventually calls standard_planner(). - */ - localPlan = planner(shardQuery, NULL, cursorOptions, paramListInfo); + /* + * Altough the shardQuery is local to this node, we prefer planner() + * over standard_planner(). The primary reason for that is Citus itself + * is not very tolarent standard_planner() calls that doesn't go through + * distributed_planner() because of the way that restriction hooks are + * implemented. So, let planner to call distributed_planner() which + * eventually calls standard_planner(). + */ + localPlan = planner(shardQuery, NULL, cursorOptions, paramListInfo); + } + else + { + ereport(DEBUG2, (errmsg( + "Local executor: Using task's cached local plan for task %u", + task->taskId))); + localPlan = TaskQueryLocalPlan(task); + } } char *shardQueryString = NULL; diff --git a/src/backend/distributed/planner/deparse_shard_query.c b/src/backend/distributed/planner/deparse_shard_query.c index 2542d931a..1aedbac17 100644 --- a/src/backend/distributed/planner/deparse_shard_query.c +++ b/src/backend/distributed/planner/deparse_shard_query.c @@ -439,6 +439,27 @@ SetTaskQueryStringList(Task *task, List *queryStringList) } +void +SetTaskQueryPlan(Task *task, Query *query, PlannedStmt *localPlan) +{ + Assert(localPlan != NULL); + task->taskQuery.queryType = TASK_QUERY_LOCAL_PLAN; + task->taskQuery.data.localCompiled = (LocalCompilation *) palloc0( + sizeof(LocalCompilation)); + task->taskQuery.data.localCompiled->query = query; + task->taskQuery.data.localCompiled->plan = localPlan; + task->queryCount = 1; +} + + +PlannedStmt * +TaskQueryLocalPlan(Task *task) +{ + Assert(task->taskQuery.queryType == TASK_QUERY_LOCAL_PLAN); + return task->taskQuery.data.localCompiled->plan; +} + + /* * DeparseTaskQuery is a general way of deparsing a query based on a task. */ @@ -524,6 +545,26 @@ TaskQueryString(Task *task) { return task->taskQuery.data.queryStringLazy; } + else if (taskQueryType == TASK_QUERY_LOCAL_PLAN) + { + Query *query = task->taskQuery.data.localCompiled->query; + Assert(query != NULL); + + /* + * Use the query of the local compilation to generate the + * query string. For local compiled tasks, the query is retained + * for this purpose, which may be EXPLAIN ANALYZing the task, or + * command logging. Generating the query string on the fly is + * acceptable because the plan of the local compilation is used + * for query execution. + */ + MemoryContext previousContext = MemoryContextSwitchTo(GetMemoryChunkContext( + query)); + UpdateRelationToShardNames((Node *) query, task->relationShardList); + MemoryContextSwitchTo(previousContext); + return AnnotateQuery(DeparseTaskQuery(task, query), + task->partitionKeyValue, task->colocationId); + } Query *jobQueryReferenceForLazyDeparsing = task->taskQuery.data.jobQueryReferenceForLazyDeparsing; diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 193e2f250..e22296ec7 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -135,13 +135,13 @@ static void AdjustReadIntermediateResultsCostInternal(RelOptInfo *relOptInfo, Const *resultFormatConst); static List * OuterPlanParamsList(PlannerInfo *root); static List * CopyPlanParamList(List *originalPlanParamList); -static PlannerRestrictionContext * CreateAndPushPlannerRestrictionContext(void); +static PlannerRestrictionContext * CreateAndPushPlannerRestrictionContext( + FastPathRestrictionContext *fastPathContext); static PlannerRestrictionContext * CurrentPlannerRestrictionContext(void); static void PopPlannerRestrictionContext(void); static void ResetPlannerRestrictionContext( PlannerRestrictionContext *plannerRestrictionContext); -static PlannedStmt * PlanFastPathDistributedStmt(DistributedPlanningContext *planContext, - Node *distributionKeyValue); +static PlannedStmt * PlanFastPathDistributedStmt(DistributedPlanningContext *planContext); static PlannedStmt * PlanDistributedStmt(DistributedPlanningContext *planContext, int rteIdCounter); static RTEListProperties * GetRTEListProperties(List *rangeTableList); @@ -166,7 +166,7 @@ distributed_planner(Query *parse, { bool needsDistributedPlanning = false; bool fastPathRouterQuery = false; - Node *distributionKeyValue = NULL; + FastPathRestrictionContext fastPathContext = { 0 }; List *rangeTableList = ExtractRangeTableEntryList(parse); @@ -191,8 +191,7 @@ distributed_planner(Query *parse, &maybeHasForeignDistributedTable); if (needsDistributedPlanning) { - fastPathRouterQuery = FastPathRouterQuery(parse, &distributionKeyValue); - + fastPathRouterQuery = FastPathRouterQuery(parse, &fastPathContext); if (maybeHasForeignDistributedTable) { WarnIfListHasForeignDistributedTable(rangeTableList); @@ -247,8 +246,9 @@ distributed_planner(Query *parse, */ HideCitusDependentObjectsOnQueriesOfPgMetaTables((Node *) parse, NULL); - /* create a restriction context and put it at the end if context list */ - planContext.plannerRestrictionContext = CreateAndPushPlannerRestrictionContext(); + /* create a restriction context and put it at the end of context list */ + planContext.plannerRestrictionContext = CreateAndPushPlannerRestrictionContext( + &fastPathContext); /* * We keep track of how many times we've recursed into the planner, primarily @@ -264,7 +264,7 @@ distributed_planner(Query *parse, { if (fastPathRouterQuery) { - result = PlanFastPathDistributedStmt(&planContext, distributionKeyValue); + result = PlanFastPathDistributedStmt(&planContext); } else { @@ -649,30 +649,21 @@ IsMultiTaskPlan(DistributedPlan *distributedPlan) * the FastPathPlanner. */ static PlannedStmt * -PlanFastPathDistributedStmt(DistributedPlanningContext *planContext, - Node *distributionKeyValue) +PlanFastPathDistributedStmt(DistributedPlanningContext *planContext) { FastPathRestrictionContext *fastPathContext = planContext->plannerRestrictionContext->fastPathRestrictionContext; + Assert(fastPathContext != NULL); + Assert(fastPathContext->fastPathRouterQuery); - planContext->plannerRestrictionContext->fastPathRestrictionContext-> - fastPathRouterQuery = true; + FastPathPreprocessParseTree(planContext->query); - if (distributionKeyValue == NULL) + if (!fastPathContext->delayFastPathPlanning) { - /* nothing to record */ + planContext->plan = FastPathPlanner(planContext->originalQuery, + planContext->query, + planContext->boundParams); } - else if (IsA(distributionKeyValue, Const)) - { - fastPathContext->distributionKeyValue = (Const *) distributionKeyValue; - } - else if (IsA(distributionKeyValue, Param)) - { - fastPathContext->distributionKeyHasParam = true; - } - - planContext->plan = FastPathPlanner(planContext->originalQuery, planContext->query, - planContext->boundParams); return CreateDistributedPlannedStmt(planContext); } @@ -803,6 +794,8 @@ CreateDistributedPlannedStmt(DistributedPlanningContext *planContext) RaiseDeferredError(distributedPlan->planningError, ERROR); } + CheckAndBuildDelayedFastPathPlan(planContext, distributedPlan); + /* remember the plan's identifier for identifying subplans */ distributedPlan->planId = planId; @@ -2407,13 +2400,15 @@ CopyPlanParamList(List *originalPlanParamList) /* - * CreateAndPushPlannerRestrictionContext creates a new relation restriction context - * and a new join context, inserts it to the beginning of the - * plannerRestrictionContextList. Finally, the planner restriction context is - * inserted to the beginning of the plannerRestrictionContextList and it is returned. + * CreateAndPushPlannerRestrictionContext creates a new planner restriction + * context with an empty relation restriction context and an empty join and + * a copy of the given fast path restriction context (if present). Finally, + * the planner restriction context is inserted to the beginning of the + * global plannerRestrictionContextList and it is returned. */ static PlannerRestrictionContext * -CreateAndPushPlannerRestrictionContext(void) +CreateAndPushPlannerRestrictionContext( + FastPathRestrictionContext *fastPathRestrictionContext) { PlannerRestrictionContext *plannerRestrictionContext = palloc0(sizeof(PlannerRestrictionContext)); @@ -2427,6 +2422,21 @@ CreateAndPushPlannerRestrictionContext(void) plannerRestrictionContext->fastPathRestrictionContext = palloc0(sizeof(FastPathRestrictionContext)); + if (fastPathRestrictionContext != NULL) + { + /* copy the given fast path restriction context */ + FastPathRestrictionContext *plannersFastPathCtx = + plannerRestrictionContext->fastPathRestrictionContext; + plannersFastPathCtx->fastPathRouterQuery = + fastPathRestrictionContext->fastPathRouterQuery; + plannersFastPathCtx->distributionKeyValue = + fastPathRestrictionContext->distributionKeyValue; + plannersFastPathCtx->distributionKeyHasParam = + fastPathRestrictionContext->distributionKeyHasParam; + plannersFastPathCtx->delayFastPathPlanning = + fastPathRestrictionContext->delayFastPathPlanning; + } + plannerRestrictionContext->memoryContext = CurrentMemoryContext; /* we'll apply logical AND as we add tables */ diff --git a/src/backend/distributed/planner/fast_path_router_planner.c b/src/backend/distributed/planner/fast_path_router_planner.c index 59f80bb40..0d4840652 100644 --- a/src/backend/distributed/planner/fast_path_router_planner.c +++ b/src/backend/distributed/planner/fast_path_router_planner.c @@ -43,8 +43,10 @@ #include "pg_version_constants.h" +#include "distributed/citus_clauses.h" #include "distributed/distributed_planner.h" #include "distributed/insert_select_planner.h" +#include "distributed/local_executor.h" #include "distributed/metadata_cache.h" #include "distributed/multi_physical_planner.h" /* only to use some utility functions */ #include "distributed/multi_router_planner.h" @@ -53,6 +55,7 @@ #include "distributed/shardinterval_utils.h" bool EnableFastPathRouterPlanner = true; +bool EnableLocalFastPathQueryOptimization = true; static bool ColumnAppearsMultipleTimes(Node *quals, Var *distributionKey); static bool DistKeyInSimpleOpExpression(Expr *clause, Var *distColumn, @@ -61,6 +64,24 @@ static bool ConjunctionContainsColumnFilter(Node *node, Var *column, Node **distributionKeyValue); +/* + * FastPathPreprocessParseTree is used to apply transformations on the parse tree + * that are expected by the Postgres planner. This is called on both delayed FastPath + * and non-delayed FastPath queries. + */ +void +FastPathPreprocessParseTree(Query *parse) +{ + /* + * Citus planner relies on some of the transformations on constant + * evaluation on the parse tree. + */ + parse->targetList = + (List *) eval_const_expressions(NULL, (Node *) parse->targetList); + parse->jointree->quals = + (Node *) eval_const_expressions(NULL, (Node *) parse->jointree->quals); +} + /* * FastPathPlanner is intended to be used instead of standard_planner() for trivial @@ -73,15 +94,6 @@ static bool ConjunctionContainsColumnFilter(Node *node, PlannedStmt * FastPathPlanner(Query *originalQuery, Query *parse, ParamListInfo boundParams) { - /* - * Citus planner relies on some of the transformations on constant - * evaluation on the parse tree. - */ - parse->targetList = - (List *) eval_const_expressions(NULL, (Node *) parse->targetList); - parse->jointree->quals = - (Node *) eval_const_expressions(NULL, (Node *) parse->jointree->quals); - PlannedStmt *result = GeneratePlaceHolderPlannedStmt(originalQuery); return result; @@ -112,9 +124,9 @@ GeneratePlaceHolderPlannedStmt(Query *parse) Plan *plan = &scanNode->plan; #endif - Node *distKey PG_USED_FOR_ASSERTS_ONLY = NULL; + FastPathRestrictionContext fprCtxt PG_USED_FOR_ASSERTS_ONLY = { 0 }; - Assert(FastPathRouterQuery(parse, &distKey)); + Assert(FastPathRouterQuery(parse, &fprCtxt)); /* there is only a single relation rte */ #if PG_VERSION_NUM >= PG_VERSION_16 @@ -150,27 +162,83 @@ GeneratePlaceHolderPlannedStmt(Query *parse) } +/* + * InitializeFastPathContext - helper function to initialize a FastPath + * restriction context with the details that the FastPath code path needs. + */ +static void +InitializeFastPathContext(FastPathRestrictionContext *fastPathContext, + Node *distributionKeyValue, + bool canAvoidDeparse, + Query *query) +{ + Assert(fastPathContext != NULL); + Assert(!fastPathContext->fastPathRouterQuery); + Assert(!fastPathContext->delayFastPathPlanning); + + /* + * We're looking at a fast path query, so we can fill the + * fastPathContext with relevant details. + */ + fastPathContext->fastPathRouterQuery = true; + if (distributionKeyValue == NULL) + { + /* nothing to record */ + } + else if (IsA(distributionKeyValue, Const)) + { + fastPathContext->distributionKeyValue = (Const *) distributionKeyValue; + } + else if (IsA(distributionKeyValue, Param)) + { + fastPathContext->distributionKeyHasParam = true; + } + + /* + * If local execution and the fast path optimization to + * avoid deparse are enabled, and it is safe to do local + * execution.. + */ + if (EnableLocalFastPathQueryOptimization && + EnableLocalExecution && + GetCurrentLocalExecutionStatus() != LOCAL_EXECUTION_DISABLED) + { + /* + * .. we can delay fast path planning until we know whether + * or not the shard is local. Make a final check for volatile + * functions in the query tree to determine if we should delay + * the fast path planning. + */ + fastPathContext->delayFastPathPlanning = canAvoidDeparse && + !FindNodeMatchingCheckFunction( + (Node *) query, + CitusIsVolatileFunction); + } +} + + /* * FastPathRouterQuery gets a query and returns true if the query is eligible for - * being a fast path router query. + * being a fast path router query. It also fills the given fastPathContext with + * details about the query such as the distribution key value (if available), + * whether the distribution key is a parameter, and the range table entry for the + * table being queried. * The requirements for the fast path query can be listed below: * * - SELECT/UPDATE/DELETE query without CTES, sublinks-subqueries, set operations * - The query should touch only a single hash distributed or reference table * - The distribution with equality operator should be in the WHERE clause * and it should be ANDed with any other filters. Also, the distribution - * key should only exists once in the WHERE clause. So basically, + * key should only exist once in the WHERE clause. So basically, * SELECT ... FROM dist_table WHERE dist_key = X * If the filter is a const, distributionKeyValue is set * - All INSERT statements (including multi-row INSERTs) as long as the commands * don't have any sublinks/CTEs etc + * - */ bool -FastPathRouterQuery(Query *query, Node **distributionKeyValue) +FastPathRouterQuery(Query *query, FastPathRestrictionContext *fastPathContext) { - FromExpr *joinTree = query->jointree; - Node *quals = NULL; - if (!EnableFastPathRouterPlanner) { return false; @@ -201,6 +269,7 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue) else if (query->commandType == CMD_INSERT) { /* we don't need to do any further checks, all INSERTs are fast-path */ + InitializeFastPathContext(fastPathContext, NULL, true, query); return true; } @@ -225,6 +294,10 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue) return false; } + bool isFastPath = false; + bool canAvoidDeparse = false; + Node *distributionKeyValue = NULL; + /* * If the table doesn't have a distribution column, we don't need to * check anything further. @@ -232,45 +305,62 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue) Var *distributionKey = PartitionColumn(distributedTableId, 1); if (!distributionKey) { - return true; + /* + * Local execution may avoid a deparse on single shard distributed tables or + * citus local tables. We don't yet support reference tables in this code-path + * because modifications on reference tables are complicated to support here. + */ + canAvoidDeparse = IsCitusTableTypeCacheEntry(cacheEntry, + SINGLE_SHARD_DISTRIBUTED) || + IsCitusTableTypeCacheEntry(cacheEntry, CITUS_LOCAL_TABLE); + isFastPath = true; } - - /* WHERE clause should not be empty for distributed tables */ - if (joinTree == NULL || - (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && joinTree->quals == - NULL)) + else { - return false; + FromExpr *joinTree = query->jointree; + Node *quals = NULL; + + canAvoidDeparse = IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE); + + if (joinTree == NULL || + (joinTree->quals == NULL && canAvoidDeparse)) + { + /* no quals, not a fast path query */ + return false; + } + + quals = joinTree->quals; + if (quals != NULL && IsA(quals, List)) + { + quals = (Node *) make_ands_explicit((List *) quals); + } + + /* + * Distribution column must be used in a simple equality match check and it must be + * place at top level conjunction operator. In simple words, we should have + * WHERE dist_key = VALUE [AND ....]; + * + * We're also not allowing any other appearances of the distribution key in the quals. + * + * Overall the logic might sound fuzzy since it involves two individual checks: + * (a) Check for top level AND operator with one side being "dist_key = const" + * (b) Only allow single appearance of "dist_key" in the quals + * + * This is to simplify both of the individual checks and omit various edge cases + * that might arise with multiple distribution keys in the quals. + */ + isFastPath = (ConjunctionContainsColumnFilter(quals, distributionKey, + &distributionKeyValue) && + !ColumnAppearsMultipleTimes(quals, distributionKey)); } - /* convert list of expressions into expression tree for further processing */ - quals = joinTree->quals; - if (quals != NULL && IsA(quals, List)) + if (isFastPath) { - quals = (Node *) make_ands_explicit((List *) quals); + InitializeFastPathContext(fastPathContext, distributionKeyValue, canAvoidDeparse, + query); } - /* - * Distribution column must be used in a simple equality match check and it must be - * place at top level conjunction operator. In simple words, we should have - * WHERE dist_key = VALUE [AND ....]; - * - * We're also not allowing any other appearances of the distribution key in the quals. - * - * Overall the logic might sound fuzzy since it involves two individual checks: - * (a) Check for top level AND operator with one side being "dist_key = const" - * (b) Only allow single appearance of "dist_key" in the quals - * - * This is to simplify both of the individual checks and omit various edge cases - * that might arise with multiple distribution keys in the quals. - */ - if (ConjunctionContainsColumnFilter(quals, distributionKey, distributionKeyValue) && - !ColumnAppearsMultipleTimes(quals, distributionKey)) - { - return true; - } - - return false; + return isFastPath; } diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 19d386343..59a1d7bb4 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -16,6 +16,8 @@ #include "postgres.h" #include "access/stratnum.h" +#include "access/tupdesc.h" +#include "access/tupdesc_details.h" #include "access/xact.h" #include "catalog/pg_opfamily.h" #include "catalog/pg_proc.h" @@ -34,6 +36,7 @@ #include "optimizer/pathnode.h" #include "optimizer/paths.h" #include "optimizer/planmain.h" +#include "optimizer/planner.h" #include "optimizer/restrictinfo.h" #include "parser/parse_oper.h" #include "parser/parsetree.h" @@ -81,6 +84,7 @@ #include "distributed/relay_utility.h" #include "distributed/resource_lock.h" #include "distributed/shard_pruning.h" +#include "distributed/shard_utils.h" #include "distributed/shardinterval_utils.h" /* intermediate value for INSERT processing */ @@ -164,7 +168,7 @@ static List * SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, List *placementList, uint64 shardId, bool parametersInQueryResolved, bool isLocalTableModification, Const *partitionKeyValue, - int colocationId); + int colocationId, bool delayedFastPath); static bool RowLocksOnRelations(Node *node, List **rtiLockList); static void ReorderTaskPlacementsByTaskAssignmentPolicy(Job *job, TaskAssignmentPolicyType @@ -173,7 +177,7 @@ static void ReorderTaskPlacementsByTaskAssignmentPolicy(Job *job, static bool ModifiesLocalTableWithRemoteCitusLocalTable(List *rangeTableList); static DeferredErrorMessage * DeferErrorIfUnsupportedLocalTableJoin(List *rangeTableList); static bool IsLocallyAccessibleCitusLocalTable(Oid relationId); - +static bool ConvertToQueryOnShard(Query *query, Oid relationID, Oid shardRelationId); /* * CreateRouterPlan attempts to create a router executor plan for the given @@ -1940,7 +1944,9 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon { GenerateSingleShardRouterTaskList(job, relationShardList, placementList, shardId, - isLocalTableModification); + isLocalTableModification, + fastPathRestrictionContext-> + delayFastPathPlanning); } job->requiresCoordinatorEvaluation = requiresCoordinatorEvaluation; @@ -1948,6 +1954,263 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon } +/* + * CheckAttributesMatch checks if the attributes of the Citus table and the shard + * table match. + * + * It is used to ensure that the shard table has the same schema as the Citus + * table before replacing the Citus table OID with the shard table OID in the + * parse tree we (Citus planner) recieved from Postgres. + */ +static +bool +CheckAttributesMatch(Oid citusTableId, Oid shardTableId) +{ + bool same_schema = false; + Relation citusRelation = RelationIdGetRelation(citusTableId); + Relation shardRelation = RelationIdGetRelation(shardTableId); + + if (RelationIsValid(citusRelation) && RelationIsValid(shardRelation)) + { + TupleDesc citusTupDesc = citusRelation->rd_att; + TupleDesc shardTupDesc = shardRelation->rd_att; + + if (citusTupDesc->natts == shardTupDesc->natts) + { + /* + * Do an attribute-by-attribute comparison. This is borrowed from + * the Postgres function equalTupleDescs(), which we cannot use + * because the citus table and shard table have different composite + * types. + */ + same_schema = true; + for (int i = 0; i < citusTupDesc->natts && same_schema; i++) + { + Form_pg_attribute attr1 = TupleDescAttr(citusTupDesc, i); + Form_pg_attribute attr2 = TupleDescAttr(shardTupDesc, i); + + if (strcmp(NameStr(attr1->attname), NameStr(attr2->attname)) != 0) + { + same_schema = false; + } + if (attr1->atttypid != attr2->atttypid) + { + same_schema = false; + } + if (attr1->atttypmod != attr2->atttypmod) + { + same_schema = false; + } + if (attr1->attcollation != attr2->attcollation) + { + same_schema = false; + } + + /* Record types derived from tables could have dropped fields. */ + if (attr1->attisdropped != attr2->attisdropped) + { + same_schema = false; + } + } + } + } + + RelationClose(citusRelation); + RelationClose(shardRelation); + return same_schema; +} + + +/* + * CheckAndBuildDelayedFastPathPlan() - if the query being planned is a fast + * path query, not marked for deferred pruning and the placement for the task + * is not a dummy placement then if the placement is local to this node we can + * take a shortcut of replacing the OID of the citus table with the OID of the + * shard in the query tree and plan that directly, instead of deparsing the + * parse tree to a SQL query on the shard and parsing and planning that in + * the local executor. Instead, the local executor can use the plan created + * here. + */ +void +CheckAndBuildDelayedFastPathPlan(DistributedPlanningContext *planContext, + DistributedPlan *plan) +{ + FastPathRestrictionContext *fastPathContext = + planContext->plannerRestrictionContext->fastPathRestrictionContext; + + if (!fastPathContext->delayFastPathPlanning) + { + return; + } + + Job *job = plan->workerJob; + Assert(job != NULL); + + if (job->deferredPruning) + { + /* Execution time pruning => don't know which shard at this point */ + planContext->plan = FastPathPlanner(planContext->originalQuery, + planContext->query, + planContext->boundParams); + return; + } + + List *tasks = job->taskList; + Assert(list_length(tasks) == 1); + Task *task = (Task *) linitial(tasks); + List *placements = task->taskPlacementList; + int32 localGroupId = GetLocalGroupId(); + + /* + * Today FastPathRouterQuery() doesn't set delayFastPathPlanning to true for + * reference tables. We should be looking at 1 placement, or ShardReplicationFactor + * of them. + */ + Assert(list_length(placements) == 1 || list_length(placements) == + ShardReplicationFactor); + ShardPlacement *primaryPlacement = (ShardPlacement *) linitial(placements); + + bool isLocalExecution = !IsDummyPlacement(primaryPlacement) && + (primaryPlacement->groupId == localGroupId); + bool canBuildLocalPlan = true; + + if (isLocalExecution) + { + List *relationShards = task->relationShardList; + Assert(list_length(relationShards) == 1); + RelationShard *relationShard = (RelationShard *) linitial(relationShards); + Assert(relationShard->shardId == primaryPlacement->shardId); + + canBuildLocalPlan = ConvertToQueryOnShard(planContext->query, + relationShard->relationId, + relationShard->shardId); + if (canBuildLocalPlan) + { + /* Plan the query with the new shard relation id */ + planContext->plan = standard_planner(planContext->query, NULL, + planContext->cursorOptions, + planContext->boundParams); + SetTaskQueryPlan(task, job->jobQuery, planContext->plan); + + ereport(DEBUG2, (errmsg( + "Fast-path router query: created local execution plan " + "to avoid deparse and compile of shard query"))); + return; + } + } + + /* + * Either the shard is not local to this node, or it was not safe to replace + * the OIDs in the parse tree; in any case we fall back to generating the shard + * query and compiling that. + */ + Assert(!isLocalExecution || (isLocalExecution && !canBuildLocalPlan)); + + /* Fall back to fast path planner and generating SQL query on the shard */ + planContext->plan = FastPathPlanner(planContext->originalQuery, + planContext->query, + planContext->boundParams); + UpdateRelationToShardNames((Node *) job->jobQuery, task->relationShardList); + SetTaskQueryIfShouldLazyDeparse(task, job->jobQuery); +} + + +/* + * ConvertToQueryOnShard() converts the given query on a citus table (identified by + * citusTableOid) to a query on a shard (identified by shardId). + * + * The function assumes that the query is a "fast path" query - it has only one + * RangeTblEntry and one RTEPermissionInfo. + * + * It acquires the same lock on the shard that was acquired on the citus table + * by the Postgres parser. It checks that the attribute numbers and metadata of + * the shard table and citus table are identical - otherwise it is not safe + * to proceed with this shortcut. Assuming the attributes do match, the actual + * conversion involves changing the target list entries that reference the + * citus table's oid to reference the shard's relation id instead. Finally, + * it changes the RangeTblEntry's relid to the shard's relation id and (PG16+) + * changes the RTEPermissionInfo's relid to the shard's relation id also. + * At this point the Query is ready for the postgres planner. + */ +static bool +ConvertToQueryOnShard(Query *query, Oid citusTableOid, Oid shardId) +{ + Assert(list_length(query->rtable) == 1); + RangeTblEntry *citusTableRte = (RangeTblEntry *) linitial(query->rtable); + Assert(citusTableRte->relid == citusTableOid); + Assert(list_length(query->rteperminfos) == 1); + + const char *citusTableName = get_rel_name(citusTableOid); + Assert(citusTableName != NULL); + + /* construct shard relation name */ + char *shardRelationName = pstrdup(citusTableName); + AppendShardIdToName(&shardRelationName, shardId); + + /* construct the schema name */ + char *schemaName = get_namespace_name(get_rel_namespace(citusTableOid)); + + /* now construct a range variable for the shard */ + RangeVar shardRangeVar = { + .relname = shardRelationName, + .schemaname = schemaName, + .inh = citusTableRte->inh, + .relpersistence = RELPERSISTENCE_PERMANENT, + }; + + /* Must apply the same lock to the shard that was applied to the citus table */ + Oid shardRelationId = RangeVarGetRelidExtended(&shardRangeVar, + citusTableRte->rellockmode, + 0, NULL, NULL); + + /* Verify that the attributes of citus table and shard table match */ + if (!CheckAttributesMatch(citusTableOid, shardRelationId)) + { + /* There is a difference between the attributes of the citus + * table and the shard table. This can happen if there is a DROP + * COLUMN on the citus table. In this case, we cannot + * convert the query to a shard query, so clean up and return. + */ + UnlockRelationOid(shardRelationId, citusTableRte->rellockmode); + ereport(DEBUG2, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg( + "Router planner fast path cannot modify parse tree for local execution: shard table \"%s.%s\" does not match the " + "distributed table \"%s.%s\"", + schemaName, shardRelationName, schemaName, + citusTableName))); + pfree(shardRelationName); + pfree(schemaName); + + return false; + } + + /* Change the target list entries that reference the original citus table's relation id */ + ListCell *lc = NULL; + foreach(lc, query->targetList) + { + TargetEntry *targetEntry = (TargetEntry *) lfirst(lc); + if (targetEntry->resorigtbl == citusTableOid) + { + targetEntry->resorigtbl = shardRelationId; + } + } + + /* Change the range table entry's oid to that of the shard's */ + Assert(shardRelationId != InvalidOid); + citusTableRte->relid = shardRelationId; + +#if PG_VERSION_NUM >= PG_VERSION_16 + + /* Change the range table permission oid to that of the shard's (PG16+) */ + Assert(list_length(query->rteperminfos) == 1); + RTEPermissionInfo *rtePermInfo = (RTEPermissionInfo *) linitial(query->rteperminfos); + rtePermInfo->relid = shardRelationId; +#endif + + return true; +} + + /* * GenerateSingleShardRouterTaskList is a wrapper around other corresponding task * list generation functions specific to single shard selects and modifications. @@ -1957,7 +2220,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon void GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, List *placementList, uint64 shardId, bool - isLocalTableModification) + isLocalTableModification, bool delayedFastPath) { Query *originalQuery = job->jobQuery; @@ -1970,7 +2233,8 @@ GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, shardId, job->parametersInJobQueryResolved, isLocalTableModification, - job->partitionKeyValue, job->colocationId); + job->partitionKeyValue, job->colocationId, + delayedFastPath); /* * Queries to reference tables, or distributed tables with multiple replica's have @@ -2001,7 +2265,8 @@ GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, shardId, job->parametersInJobQueryResolved, isLocalTableModification, - job->partitionKeyValue, job->colocationId); + job->partitionKeyValue, job->colocationId, + delayedFastPath); } } @@ -2096,7 +2361,7 @@ SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, List *placementList, uint64 shardId, bool parametersInQueryResolved, bool isLocalTableModification, Const *partitionKeyValue, - int colocationId) + int colocationId, bool delayedFastPath) { TaskType taskType = READ_TASK; char replicationModel = 0; @@ -2168,7 +2433,10 @@ SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, task->taskPlacementList = placementList; task->partitionKeyValue = partitionKeyValue; task->colocationId = colocationId; - SetTaskQueryIfShouldLazyDeparse(task, query); + if (!delayedFastPath) + { + SetTaskQueryIfShouldLazyDeparse(task, query); + } task->anchorShardId = shardId; task->jobId = jobId; task->relationShardList = relationShardList; @@ -2449,10 +2717,15 @@ PlanRouterQuery(Query *originalQuery, /* * If this is an UPDATE or DELETE query which requires coordinator evaluation, - * don't try update shard names, and postpone that to execution phase. + * don't try update shard names, and postpone that to execution phase. Also, if + * this is a delayed fast path query, we don't update the shard names + * either, as the shard names will be updated in the fast path query planner. */ bool isUpdateOrDelete = UpdateOrDeleteOrMergeQuery(originalQuery); - if (!(isUpdateOrDelete && RequiresCoordinatorEvaluation(originalQuery))) + bool delayedFastPath = + plannerRestrictionContext->fastPathRestrictionContext->delayFastPathPlanning; + if (!(isUpdateOrDelete && RequiresCoordinatorEvaluation(originalQuery)) && + !delayedFastPath) { UpdateRelationToShardNames((Node *) originalQuery, *relationShardList); } diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index b4c64e070..61b16610b 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -212,6 +212,7 @@ static const char * MaxSharedPoolSizeGucShowHook(void); static const char * LocalPoolSizeGucShowHook(void); static bool StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource source); +static bool WarnIfLocalExecutionDisabled(bool *newval, void **extra, GucSource source); static void CitusAuthHook(Port *port, int status); static bool IsSuperuser(char *userName); static void AdjustDynamicLibraryPathForCdcDecoders(void); @@ -1426,6 +1427,17 @@ RegisterCitusConfigVariables(void) GUC_STANDARD, NULL, NULL, NULL); + DefineCustomBoolVariable( + "citus.enable_local_fast_path_query_optimization", + gettext_noop("Enables the planner to avoid a query deparse and planning if " + "the shard is local to the current node."), + NULL, + &EnableLocalFastPathQueryOptimization, + true, + PGC_USERSET, + GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE, + WarnIfLocalExecutionDisabled, NULL, NULL); + DefineCustomBoolVariable( "citus.enable_local_reference_table_foreign_keys", gettext_noop("Enables foreign keys from/to local tables"), @@ -2851,6 +2863,26 @@ WarnIfDeprecatedExecutorUsed(int *newval, void **extra, GucSource source) } +/* + * WarnIfLocalExecutionDisabled is used to emit a warning message when + * enabling citus.enable_local_fast_path_query_optimization if + * citus.enable_local_execution was disabled. + */ +static bool +WarnIfLocalExecutionDisabled(bool *newval, void **extra, GucSource source) +{ + if (*newval == true && EnableLocalExecution == false) + { + ereport(WARNING, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg( + "citus.enable_local_execution must be set in order for " + "citus.enable_local_fast_path_query_optimization to be effective."))); + } + + return true; +} + + /* * NoticeIfSubqueryPushdownEnabled prints a notice when a user sets * citus.subquery_pushdown to ON. It doesn't print the notice if the diff --git a/src/backend/distributed/utils/citus_copyfuncs.c b/src/backend/distributed/utils/citus_copyfuncs.c index 4b4a334c8..51716cff3 100644 --- a/src/backend/distributed/utils/citus_copyfuncs.c +++ b/src/backend/distributed/utils/citus_copyfuncs.c @@ -287,6 +287,15 @@ CopyTaskQuery(Task *newnode, Task *from) break; } + case TASK_QUERY_LOCAL_PLAN: + { + newnode->taskQuery.data.localCompiled = + (LocalCompilation *) palloc0(sizeof(LocalCompilation)); + COPY_NODE_FIELD(taskQuery.data.localCompiled->plan); + COPY_NODE_FIELD(taskQuery.data.localCompiled->query); + break; + } + default: { break; diff --git a/src/include/distributed/deparse_shard_query.h b/src/include/distributed/deparse_shard_query.h index 8fb012588..efcdb3032 100644 --- a/src/include/distributed/deparse_shard_query.h +++ b/src/include/distributed/deparse_shard_query.h @@ -27,7 +27,9 @@ extern bool UpdateRelationToShardNames(Node *node, List *relationShardList); extern void SetTaskQueryIfShouldLazyDeparse(Task *task, Query *query); extern void SetTaskQueryString(Task *task, char *queryString); extern void SetTaskQueryStringList(Task *task, List *queryStringList); +extern void SetTaskQueryPlan(Task *task, Query *query, PlannedStmt *localPlan); extern char * TaskQueryString(Task *task); +extern PlannedStmt * TaskQueryLocalPlan(Task *task); extern char * TaskQueryStringAtIndex(Task *task, int index); extern int GetTaskQueryType(Task *task); extern void AddInsertAliasIfNeeded(Query *query); diff --git a/src/include/distributed/distributed_planner.h b/src/include/distributed/distributed_planner.h index 33a9c2fa8..f416aa911 100644 --- a/src/include/distributed/distributed_planner.h +++ b/src/include/distributed/distributed_planner.h @@ -99,6 +99,12 @@ typedef struct FastPathRestrictionContext * Set to true when distKey = Param; in the queryTree */ bool distributionKeyHasParam; + + /* + * Indicates to hold off calling the fast path planner until its + * known if the shard is local or not. + */ + bool delayFastPathPlanning; } FastPathRestrictionContext; typedef struct PlannerRestrictionContext diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 475a41b37..e5ec2205d 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -174,9 +174,16 @@ typedef enum TaskQueryType TASK_QUERY_NULL, TASK_QUERY_TEXT, TASK_QUERY_OBJECT, - TASK_QUERY_TEXT_LIST + TASK_QUERY_TEXT_LIST, + TASK_QUERY_LOCAL_PLAN, } TaskQueryType; +typedef struct LocalCompilation +{ + PlannedStmt *plan; /* the local plan for this task */ + Query *query; /* query to deparse for EXPLAIN ANALYZE or local command logging */ +} LocalCompilation; + typedef struct TaskQuery { TaskQueryType queryType; @@ -219,6 +226,15 @@ typedef struct TaskQuery * when we want to access each query string. */ List *queryStringList; + + /* + * For tasks that can be executed locally, this field contains the + * local plan for the task. This is only set when the shard that the + * task is assigned to is local to the node that executes the task. + * The query field is used to deparse the query for EXPLAIN ANALYZE + * or local command logging. + */ + LocalCompilation *localCompiled; /* only applies to local tasks */ }data; }TaskQuery; diff --git a/src/include/distributed/multi_router_planner.h b/src/include/distributed/multi_router_planner.h index 44be2736e..af3421e96 100644 --- a/src/include/distributed/multi_router_planner.h +++ b/src/include/distributed/multi_router_planner.h @@ -28,6 +28,7 @@ extern bool EnableRouterExecution; extern bool EnableFastPathRouterPlanner; +extern bool EnableLocalFastPathQueryOptimization; extern bool EnableNonColocatedRouterQueryPushdown; @@ -91,16 +92,19 @@ extern void GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, List *placementList, uint64 shardId, - bool isLocalTableModification); + bool isLocalTableModification, + bool delayedFastPath); /* * FastPathPlanner is a subset of router planner, that's why we prefer to * keep the external function here. */extern PlannedStmt * GeneratePlaceHolderPlannedStmt(Query *parse); +extern void FastPathPreprocessParseTree(Query *parse); extern PlannedStmt * FastPathPlanner(Query *originalQuery, Query *parse, ParamListInfo boundParams); -extern bool FastPathRouterQuery(Query *query, Node **distributionKeyValue); +extern bool FastPathRouterQuery(Query *query, + FastPathRestrictionContext *fastPathContext); extern bool JoinConditionIsOnFalse(List *relOptInfo); extern Oid ResultRelationOidForQuery(Query *query); extern DeferredErrorMessage * TargetlistAndFunctionsSupported(Oid resultRelationId, @@ -120,5 +124,7 @@ extern Job * RouterJob(Query *originalQuery, DeferredErrorMessage **planningError); extern bool ContainsOnlyLocalOrReferenceTables(RTEListProperties *rteProperties); extern RangeTblEntry * ExtractSourceResultRangeTableEntry(Query *query); +extern void CheckAndBuildDelayedFastPathPlan(DistributedPlanningContext *planContext, + DistributedPlan *plan); #endif /* MULTI_ROUTER_PLANNER_H */ diff --git a/src/test/regress/expected/insert_select_single_shard_table.out b/src/test/regress/expected/insert_select_single_shard_table.out index f282ca28e..4a66dd64a 100644 --- a/src/test/regress/expected/insert_select_single_shard_table.out +++ b/src/test/regress/expected/insert_select_single_shard_table.out @@ -226,10 +226,12 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM insert_select_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM (insert_select_single_shard_table.nullkey_c1_t1 JOIN (SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table USING (a)) DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Local executor: Using task's cached local plan for task 0 -- use a postgres local table INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time @@ -338,10 +340,12 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM insert_select_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM (insert_select_single_shard_table.nullkey_c1_t1 JOIN (SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table USING (a)) DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Local executor: Using task's cached local plan for task 0 -- use a postgres local table INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time @@ -400,10 +404,12 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM insert_select_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM (insert_select_single_shard_table.nullkey_c1_t1 JOIN (SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table USING (a)) DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Local executor: Using task's cached local plan for task 0 -- use a postgres local table INSERT INTO citus_local_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time @@ -416,6 +422,8 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator SELECT avg(a), avg(b) FROM citus_local_table ORDER BY 1, 2; DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query +DEBUG: Local executor: Using task's cached local plan for task 0 avg | avg --------------------------------------------------------------------- 4.5270270270270270 | 4.5270270270270270 @@ -458,7 +466,9 @@ DEBUG: Creating router plan DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Local executor: Using task's cached local plan for task 0 INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN reference_table USING (a) JOIN postgres_local_table USING (a) ORDER BY 1,2 OFFSET 7; DEBUG: Creating router plan DEBUG: distributed INSERT ... SELECT cannot select from a local table @@ -470,10 +480,12 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM insert_select_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT citus_local_table.a, citus_local_table.b FROM ((SELECT citus_local_table_1.a, citus_local_table_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) citus_local_table_1) citus_local_table JOIN insert_select_single_shard_table.nullkey_c1_t1 USING (a)) DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Local executor: Using task's cached local plan for task 0 -- use a distributed table INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2; DEBUG: INSERT target relation and all source relations of the SELECT must be colocated in distributed INSERT ... SELECT diff --git a/src/test/regress/expected/local_execution_local_plan.out b/src/test/regress/expected/local_execution_local_plan.out new file mode 100644 index 000000000..3bfaea4e0 --- /dev/null +++ b/src/test/regress/expected/local_execution_local_plan.out @@ -0,0 +1,377 @@ +-- Test local execution with local plan in a sharded environment. +-- This is an enhancement to local execution where instead of deparsing +-- and compiling the shard query, the planner replaces the OID of the +-- distributed table with the OID of the local shard in the parse tree +-- and plans that. +-- +-- https://github.com/citusdata/citus/pull/8035 +CREATE SCHEMA local_shard_execution_local_plan; +SET search_path TO local_shard_execution_local_plan; +SET citus.next_shard_id TO 86000000; +-- Test row-based sharding +SET citus.shard_replication_factor TO 1; +CREATE TABLE test_tbl (a int, b int, data_f double precision); +SELECT create_distributed_table('test_tbl', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT setseed(0.42); -- make the random data inserted deterministic + setseed +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO test_tbl +SELECT (random()*20)::int AS a, + (random()*20)::int AS b, + random()*10000.0 AS data_f +FROM generate_series(1, 10000); +-- Put the shard on worker 1 to ensure consistent test output across different schedules +SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 1 +SELECT citus_move_shard_placement(86000000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes'); + citus_move_shard_placement +--------------------------------------------------------------------- + +(1 row) + +SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests + wait_for_resource_cleanup +--------------------------------------------------------------------- + +(1 row) + +RESET client_min_messages; +\c - - - :worker_1_port +SET search_path TO local_shard_execution_local_plan; +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; +-- This query resolves to a single shard (aka fast path) +-- which is located on worker_1; with client_min_messages +-- at DEBUG2 we see a message that the planner is avoiding +-- query deparse and plan +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b +DEBUG: Local executor: Using task's cached local plan for task 0 + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +BEGIN; + -- force accessing local placements via remote connections first + SET citus.enable_local_execution TO false; + RESET client_min_messages; + RESET citus.log_remote_commands; + RESET citus.log_local_commands; + SELECT count(*), b FROM test_tbl GROUP BY b ORDER BY b; + count | b +--------------------------------------------------------------------- + 246 | 0 + 530 | 1 + 517 | 2 + 530 | 3 + 513 | 4 + 487 | 5 + 501 | 6 + 490 | 7 + 501 | 8 + 480 | 9 + 510 | 10 + 472 | 11 + 484 | 12 + 505 | 13 + 463 | 14 + 484 | 15 + 516 | 16 + 523 | 17 + 507 | 18 + 481 | 19 + 260 | 20 +(21 rows) + + -- Now, even if we enable local execution back before the query that + -- could normally benefit from fast path local query optimizations, + -- this time it won't be the case because local execution was implicitly + -- disabled by Citus as we accessed local shard placements via remote + -- connections. + SET citus.enable_local_execution TO true; + SET client_min_messages TO DEBUG2; + SET citus.log_remote_commands TO ON; + SET citus.log_local_commands TO ON; + SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) + FROM test_tbl + WHERE a = 8 AND b IN (1,3,5,8,13,21) + GROUP BY b + ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 8 +NOTICE: issuing SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +SET client_min_messages TO ERROR; -- keep COMMIT output quiet +COMMIT; +SET client_min_messages TO DEBUG2; +SET citus.enable_local_fast_path_query_optimization TO OFF; +-- With local execution local plan disabled, the same query +-- does query deparse and planning of the shard query and +-- provides the same results +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +\c - - - :worker_2_port +SET search_path TO local_shard_execution_local_plan; +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; +-- Run the same query on the other worker - the local +-- execution path is not taken because the shard is not +-- local to this worker +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 8 +NOTICE: issuing SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +\c - - - :master_port +SET search_path TO local_shard_execution_local_plan; +SET citus.next_shard_id TO 86001000; +-- Test citus local and reference tables +CREATE TABLE ref_tbl (a int, b int, data_f double precision); +SELECT create_reference_table('ref_tbl'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +SELECT setseed(0.42); -- make the random data inserted deterministic + setseed +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO ref_tbl +SELECT (random()*20)::int AS a, + (random()*20)::int AS b, + random()*10000.0 AS data_f +FROM generate_series(1, 10000); +SET citus.next_shard_id TO 86002000; +CREATE TABLE citus_tbl (a int, b int, data_f double precision); +SELECT citus_set_coordinator_host('localhost', :master_port); + citus_set_coordinator_host +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('citus_tbl'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO citus_tbl SELECT a, b, data_f FROM ref_tbl; +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; +-- citus local table: can use the fast path optimization +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM citus_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query +NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.citus_tbl_86002000 citus_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b +DEBUG: Local executor: Using task's cached local plan for task 0 + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +-- reference table: does not use the fast path optimization. +-- It may be enabled in a future enhancement. +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM ref_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.ref_tbl_86001000 ref_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +\c - - - :master_port +-- Now test local execution with local plan for a schema sharded table. +SET citus.enable_schema_based_sharding to on; +CREATE SCHEMA schema_sharding_test; +SET search_path TO schema_sharding_test; +SET citus.next_shard_id TO 87000000; +SET citus.shard_replication_factor TO 1; +CREATE TABLE test_tbl (a int, b int, data_f double precision); +SELECT setseed(0.42); -- make the random data inserted deterministic + setseed +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO test_tbl +SELECT (random()*20)::int AS a, + (random()*20)::int AS b, + random()*10000.0 AS data_f +FROM generate_series(1, 10000); +-- Put the shard on worker 2 to ensure consistent test output across different schedules +SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 2 +SELECT citus_move_shard_placement(87000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes'); + citus_move_shard_placement +--------------------------------------------------------------------- + +(1 row) + +SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests + wait_for_resource_cleanup +--------------------------------------------------------------------- + +(1 row) + +RESET client_min_messages; +\c - - - :worker_1_port +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; +-- Run the test query on worker_1; with schema based sharding +-- the data is not local to this worker so local execution +-- path is not taken. +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM schema_sharding_test.test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +NOTICE: issuing SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM schema_sharding_test.test_tbl_87000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +\c - - - :worker_2_port +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; +-- Run the test query on worker_2; with schema based sharding +-- the data is local to this worker so local execution +-- path is taken, and the planner avoids query deparse and +-- planning of the shard query. +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM schema_sharding_test.test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query +NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM schema_sharding_test.test_tbl_87000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b +DEBUG: Local executor: Using task's cached local plan for task 0 + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +SET citus.enable_local_fast_path_query_optimization TO OFF; +-- Run the test query on worker_2 but with local execution +-- local plan disabled; now the planner does query deparse +-- and planning of the shard query. +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM schema_sharding_test.test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM schema_sharding_test.test_tbl_87000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b + b | avg | min | max | count +--------------------------------------------------------------------- + 1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21 + 3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23 + 5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25 + 8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27 + 13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18 +(5 rows) + +\c - - - :master_port +SET client_min_messages to ERROR; +DROP SCHEMA local_shard_execution_local_plan CASCADE; +DROP SCHEMA schema_sharding_test CASCADE; +RESET ALL; diff --git a/src/test/regress/expected/local_shard_execution.out b/src/test/regress/expected/local_shard_execution.out index 2b1fa3c0b..ddfc18baf 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -2410,8 +2410,10 @@ NOTICE: executing the command locally: UPDATE local_shard_execution.event_respo SELECT count(*) FROM event_responses WHERE event_id = 16; DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: query has a single distribution column value: 16 NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.event_responses_1480001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16) +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 1 @@ -2420,8 +2422,10 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar SELECT count(*) FROM event_responses WHERE event_id = 16; DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: query has a single distribution column value: 16 NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.event_responses_1480001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16) +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 1 @@ -2430,8 +2434,10 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar UPDATE event_responses SET response = 'no' WHERE event_id = 16; DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: query has a single distribution column value: 16 NOTICE: executing the command locally: UPDATE local_shard_execution.event_responses_1480001 event_responses SET response = 'no'::local_shard_execution.invite_resp WHERE (event_id OPERATOR(pg_catalog.=) 16) +DEBUG: Local executor: Using task's cached local plan for task 0 INSERT INTO event_responses VALUES (16, 666, 'maybe') ON CONFLICT (event_id, user_id) DO UPDATE SET response = EXCLUDED.response RETURNING *; @@ -2529,8 +2535,10 @@ SET citus.log_remote_commands TO ON; SELECT * FROM event_responses_no_pkey WHERE event_id = 2; DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: query has a single distribution column value: 2 NOTICE: executing the command locally: SELECT event_id, user_id, response FROM local_shard_execution.event_responses_no_pkey_1480007 event_responses_no_pkey WHERE (event_id OPERATOR(pg_catalog.=) 2) +DEBUG: Local executor: Using task's cached local plan for task 0 event_id | user_id | response --------------------------------------------------------------------- (0 rows) diff --git a/src/test/regress/expected/local_shard_execution_dropped_column.out b/src/test/regress/expected/local_shard_execution_dropped_column.out index 6a2fe1b0b..271d03455 100644 --- a/src/test/regress/expected/local_shard_execution_dropped_column.out +++ b/src/test/regress/expected/local_shard_execution_dropped_column.out @@ -193,8 +193,102 @@ execute p4(8); NOTICE: executing the command locally: UPDATE local_shard_execution_dropped_column.t1_2460000 t1 SET a = (a OPERATOR(pg_catalog.+) 1) WHERE (c OPERATOR(pg_catalog.=) 8) execute p4(8); NOTICE: executing the command locally: UPDATE local_shard_execution_dropped_column.t1_2460000 t1 SET a = (a OPERATOR(pg_catalog.+) 1) WHERE (c OPERATOR(pg_catalog.=) 8) +-- Test that "Avoid deparse and planning of shard query for local execution" (*) +-- does not take the fast path of modifying the parse tree with the shard OID, as +-- the dropped column means the attribute check between the distributed table and +-- shard fails. With client_min_messages at DEBUG2, we see "cannot modify parse tree +-- for local execution", indicating that router planning has detected the difference. +-- +-- (*) https://github.com/citusdata/citus/pull/8035 +SET client_min_messages to DEBUG2; +prepare p5(int) as SELECT count(*) FROM t1 WHERE c = 8 and a = $1 GROUP BY c; +execute p5(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +execute p5(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- +(0 rows) + +RESET client_min_messages; \c - - - :master_port --- one another combination is that the shell table +-- one other combination is that the shell table -- has a dropped column but not the shard, via rebalance operation SET search_path TO local_shard_execution_dropped_column; ALTER TABLE t1 DROP COLUMN a; @@ -204,6 +298,12 @@ SELECT citus_move_shard_placement(2460000, 'localhost', :worker_1_port, 'localho (1 row) +SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests + wait_for_resource_cleanup +--------------------------------------------------------------------- + +(1 row) + \c - - - :worker_2_port SET search_path TO local_shard_execution_dropped_column; -- show the dropped columns @@ -331,6 +431,108 @@ execute p3(8); NOTICE: executing the command locally: INSERT INTO local_shard_execution_dropped_column.t1_2460000 AS citus_table_alias (c) VALUES (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8) ON CONFLICT DO NOTHING execute p3(8); NOTICE: executing the command locally: INSERT INTO local_shard_execution_dropped_column.t1_2460000 AS citus_table_alias (c) VALUES (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8) ON CONFLICT DO NOTHING +-- Test that "Avoid deparse and planning of shard query for local execution" +-- does not take the fast path of modifying the parse tree with the shard OID +-- for this scenario (rebalance) also. +-- +-- (*) https://github.com/citusdata/citus/pull/8035 +SET client_min_messages to DEBUG2; +prepare p4(int) as SELECT count(*) FROM t1 WHERE c = 8 and 5 = $1 GROUP BY c; +execute p4(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1" +DEBUG: query has a single distribution column value: 8 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +execute p4(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c + count +--------------------------------------------------------------------- + 1 +(1 row) + +RESET client_min_messages; \c - - - :master_port DROP SCHEMA local_shard_execution_dropped_column CASCADE; NOTICE: drop cascades to table local_shard_execution_dropped_column.t1 diff --git a/src/test/regress/expected/merge_schema_sharding.out b/src/test/regress/expected/merge_schema_sharding.out index a6fb11998..3fac41aaf 100644 --- a/src/test/regress/expected/merge_schema_sharding.out +++ b/src/test/regress/expected/merge_schema_sharding.out @@ -192,7 +192,9 @@ DEBUG: A mix of distributed and citus-local table, routable query is not possib DEBUG: Creating MERGE repartition plan DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: Collect source query results on coordinator +DEBUG: Local executor: Using task's cached local plan for task 0 DEBUG: Create a MERGE task list that needs to be routed DEBUG: DEBUG: distributed statement: MERGE INTO schema_shard_table1.nullkey_c1_t1_4005006 citus_table_alias USING (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('merge_into_XXX_4005006'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) citus_local_table ON (citus_table_alias.a OPERATOR(pg_catalog.=) citus_local_table.a) WHEN MATCHED THEN UPDATE SET b = citus_local_table.b diff --git a/src/test/regress/expected/query_single_shard_table.out b/src/test/regress/expected/query_single_shard_table.out index 0945bc1d7..fa1641a2b 100644 --- a/src/test/regress/expected/query_single_shard_table.out +++ b/src/test/regress/expected/query_single_shard_table.out @@ -308,9 +308,11 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" "d1" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS "dummy-1" FROM query_single_shard_table.citus_local_table d1 WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT NULL::integer AS a, NULL::integer AS b FROM (SELECT intermediate_result."dummy-1" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result("dummy-1" integer)) d1_1) d1, query_single_shard_table.nullkey_c1_t1 DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 121 @@ -599,9 +601,11 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM query_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b, citus_local_table.b FROM (query_single_shard_table.nullkey_c1_t1 JOIN (SELECT citus_local_table_1.a, citus_local_table_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) citus_local_table_1) citus_local_table USING (a)) ORDER BY nullkey_c1_t1.a, nullkey_c1_t1.b, citus_local_table.b DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 a | b | b --------------------------------------------------------------------- 1 | 1 | 1 @@ -850,9 +854,11 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (query_single_shard_table.nullkey_c1_t1 LEFT JOIN (SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table USING (a)) DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 11 @@ -863,6 +869,7 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table WHERE true DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "nullkey_c1_t1" since it is part of a distributed join node that is outer joined with a recurring rel @@ -872,6 +879,7 @@ DEBUG: Creating router plan DEBUG: generating subplan XXX_2 for subquery SELECT a FROM query_single_shard_table.nullkey_c1_t1 WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table LEFT JOIN (SELECT nullkey_c1_t1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) nullkey_c1_t1_1) nullkey_c1_t1 USING (a)) DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 14 @@ -910,6 +918,7 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table WHERE true DEBUG: recursively planning left side of the full join since the other side is a recurring rel DEBUG: recursively planning distributed relation "nullkey_c1_t1" since it is part of a distributed join node that is outer joined with a recurring rel @@ -919,6 +928,7 @@ DEBUG: Creating router plan DEBUG: generating subplan XXX_2 for subquery SELECT a FROM query_single_shard_table.nullkey_c1_t1 WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT nullkey_c1_t1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) nullkey_c1_t1_1) nullkey_c1_t1 FULL JOIN (SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table USING (a)) DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 14 @@ -1222,12 +1232,14 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" "t1" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table t1 WHERE true DEBUG: recursively planning right side of the left join since the outer side is a recurring rel DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: skipping recursive planning for the subquery since it contains references to outer queries DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT t1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t1_1) t1 LEFT JOIN LATERAL (SELECT t2.a, t2.b FROM query_single_shard_table.nullkey_c1_t1 t2 WHERE (t2.b OPERATOR(pg_catalog.>) t1.a)) q USING (a)) DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 11 @@ -1279,9 +1291,11 @@ DEBUG: Local tables cannot be used in distributed queries. DEBUG: Wrapping relation "citus_local_table" "t1" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table t1 WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT t1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t1_1) t1 JOIN LATERAL (SELECT t2.a, t2.b FROM query_single_shard_table.nullkey_c1_t1 t2 WHERE (t2.b OPERATOR(pg_catalog.>) t1.a)) q USING (a)) DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 count --------------------------------------------------------------------- 2 @@ -1428,7 +1442,9 @@ DEBUG: Creating router plan DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Local executor: Using task's cached local plan for task 0 INSERT INTO nullkey_c1_t1 SELECT * FROM postgres_local_table; DEBUG: Creating router plan DEBUG: distributed INSERT ... SELECT can only select from distributed tables @@ -2112,9 +2128,11 @@ DEBUG: local table citus_local_table cannot be joined with these distributed ta DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT b FROM query_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE query_single_shard_table.nullkey_c1_t1 SET b = 5 FROM (SELECT NULL::integer AS a, citus_local_table_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) citus_local_table_1) citus_local_table WHERE (nullkey_c1_t1.b OPERATOR(pg_catalog.=) citus_local_table.b) DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 UPDATE nullkey_c1_t1 SET b = 5 FROM postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b; DEBUG: relation postgres_local_table is not distributed DEBUG: Wrapping relation "postgres_local_table" to a subquery @@ -2164,9 +2182,11 @@ DEBUG: local table citus_local_table cannot be joined with these distributed ta DEBUG: Wrapping relation "citus_local_table" to a subquery DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan +DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query DEBUG: generating subplan XXX_1 for subquery SELECT b FROM query_single_shard_table.citus_local_table WHERE true DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM query_single_shard_table.nullkey_c1_t1 USING (SELECT NULL::integer AS a, citus_local_table_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) citus_local_table_1) citus_local_table WHERE (nullkey_c1_t1.b OPERATOR(pg_catalog.=) citus_local_table.b) DEBUG: Creating router plan +DEBUG: Local executor: Using task's cached local plan for task 0 DELETE FROM nullkey_c1_t1 USING postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b; DEBUG: relation postgres_local_table is not distributed DEBUG: Wrapping relation "postgres_local_table" to a subquery diff --git a/src/test/regress/multi_mx_schedule b/src/test/regress/multi_mx_schedule index 6654b4ab0..487552663 100644 --- a/src/test/regress/multi_mx_schedule +++ b/src/test/regress/multi_mx_schedule @@ -70,6 +70,7 @@ test: metadata_sync_helpers test: issue_6592 test: executor_local_failure +test: local_execution_local_plan # test that no tests leaked intermediate results. This should always be last test: ensure_no_intermediate_data_leak diff --git a/src/test/regress/sql/local_execution_local_plan.sql b/src/test/regress/sql/local_execution_local_plan.sql new file mode 100644 index 000000000..091330865 --- /dev/null +++ b/src/test/regress/sql/local_execution_local_plan.sql @@ -0,0 +1,214 @@ +-- Test local execution with local plan in a sharded environment. +-- This is an enhancement to local execution where instead of deparsing +-- and compiling the shard query, the planner replaces the OID of the +-- distributed table with the OID of the local shard in the parse tree +-- and plans that. +-- +-- https://github.com/citusdata/citus/pull/8035 + +CREATE SCHEMA local_shard_execution_local_plan; +SET search_path TO local_shard_execution_local_plan; + +SET citus.next_shard_id TO 86000000; + +-- Test row-based sharding + +SET citus.shard_replication_factor TO 1; +CREATE TABLE test_tbl (a int, b int, data_f double precision); +SELECT create_distributed_table('test_tbl', 'a'); +SELECT setseed(0.42); -- make the random data inserted deterministic +INSERT INTO test_tbl +SELECT (random()*20)::int AS a, + (random()*20)::int AS b, + random()*10000.0 AS data_f +FROM generate_series(1, 10000); + +-- Put the shard on worker 1 to ensure consistent test output across different schedules +SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 1 +SELECT citus_move_shard_placement(86000000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes'); +SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests +RESET client_min_messages; + +\c - - - :worker_1_port +SET search_path TO local_shard_execution_local_plan; + +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; + +-- This query resolves to a single shard (aka fast path) +-- which is located on worker_1; with client_min_messages +-- at DEBUG2 we see a message that the planner is avoiding +-- query deparse and plan +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +BEGIN; + -- force accessing local placements via remote connections first + SET citus.enable_local_execution TO false; + RESET client_min_messages; + RESET citus.log_remote_commands; + RESET citus.log_local_commands; + SELECT count(*), b FROM test_tbl GROUP BY b ORDER BY b; + -- Now, even if we enable local execution back before the query that + -- could normally benefit from fast path local query optimizations, + -- this time it won't be the case because local execution was implicitly + -- disabled by Citus as we accessed local shard placements via remote + -- connections. + SET citus.enable_local_execution TO true; + SET client_min_messages TO DEBUG2; + SET citus.log_remote_commands TO ON; + SET citus.log_local_commands TO ON; + SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) + FROM test_tbl + WHERE a = 8 AND b IN (1,3,5,8,13,21) + GROUP BY b + ORDER BY b; +SET client_min_messages TO ERROR; -- keep COMMIT output quiet +COMMIT; +SET client_min_messages TO DEBUG2; + +SET citus.enable_local_fast_path_query_optimization TO OFF; + +-- With local execution local plan disabled, the same query +-- does query deparse and planning of the shard query and +-- provides the same results +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +\c - - - :worker_2_port +SET search_path TO local_shard_execution_local_plan; + +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; + +-- Run the same query on the other worker - the local +-- execution path is not taken because the shard is not +-- local to this worker +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +\c - - - :master_port + +SET search_path TO local_shard_execution_local_plan; +SET citus.next_shard_id TO 86001000; + +-- Test citus local and reference tables +CREATE TABLE ref_tbl (a int, b int, data_f double precision); +SELECT create_reference_table('ref_tbl'); +SELECT setseed(0.42); -- make the random data inserted deterministic +INSERT INTO ref_tbl +SELECT (random()*20)::int AS a, + (random()*20)::int AS b, + random()*10000.0 AS data_f +FROM generate_series(1, 10000); + +SET citus.next_shard_id TO 86002000; +CREATE TABLE citus_tbl (a int, b int, data_f double precision); +SELECT citus_set_coordinator_host('localhost', :master_port); +SELECT citus_add_local_table_to_metadata('citus_tbl'); +INSERT INTO citus_tbl SELECT a, b, data_f FROM ref_tbl; + +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; + +-- citus local table: can use the fast path optimization +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM citus_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +-- reference table: does not use the fast path optimization. +-- It may be enabled in a future enhancement. +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM ref_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +\c - - - :master_port + +-- Now test local execution with local plan for a schema sharded table. + +SET citus.enable_schema_based_sharding to on; +CREATE SCHEMA schema_sharding_test; +SET search_path TO schema_sharding_test; + +SET citus.next_shard_id TO 87000000; + +SET citus.shard_replication_factor TO 1; +CREATE TABLE test_tbl (a int, b int, data_f double precision); +SELECT setseed(0.42); -- make the random data inserted deterministic +INSERT INTO test_tbl +SELECT (random()*20)::int AS a, + (random()*20)::int AS b, + random()*10000.0 AS data_f +FROM generate_series(1, 10000); + +-- Put the shard on worker 2 to ensure consistent test output across different schedules +SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 2 +SELECT citus_move_shard_placement(87000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes'); +SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests +RESET client_min_messages; + +\c - - - :worker_1_port + +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; + +-- Run the test query on worker_1; with schema based sharding +-- the data is not local to this worker so local execution +-- path is not taken. +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM schema_sharding_test.test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +\c - - - :worker_2_port + +SET client_min_messages TO DEBUG2; +SET citus.log_remote_commands TO ON; +SET citus.log_local_commands TO ON; + +-- Run the test query on worker_2; with schema based sharding +-- the data is local to this worker so local execution +-- path is taken, and the planner avoids query deparse and +-- planning of the shard query. + +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM schema_sharding_test.test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +SET citus.enable_local_fast_path_query_optimization TO OFF; + +-- Run the test query on worker_2 but with local execution +-- local plan disabled; now the planner does query deparse +-- and planning of the shard query. +SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1) +FROM schema_sharding_test.test_tbl +WHERE a = 8 AND b IN (1,3,5,8,13,21) +GROUP BY b +ORDER BY b; + +\c - - - :master_port + +SET client_min_messages to ERROR; +DROP SCHEMA local_shard_execution_local_plan CASCADE; +DROP SCHEMA schema_sharding_test CASCADE; +RESET ALL; diff --git a/src/test/regress/sql/local_shard_execution_dropped_column.sql b/src/test/regress/sql/local_shard_execution_dropped_column.sql index 1d7dac0b7..921fbf18c 100644 --- a/src/test/regress/sql/local_shard_execution_dropped_column.sql +++ b/src/test/regress/sql/local_shard_execution_dropped_column.sql @@ -78,14 +78,37 @@ execute p4(8); execute p4(8); execute p4(8); +-- Test that "Avoid deparse and planning of shard query for local execution" (*) +-- does not take the fast path of modifying the parse tree with the shard OID, as +-- the dropped column means the attribute check between the distributed table and +-- shard fails. With client_min_messages at DEBUG2, we see "cannot modify parse tree +-- for local execution", indicating that router planning has detected the difference. +-- +-- (*) https://github.com/citusdata/citus/pull/8035 + +SET client_min_messages to DEBUG2; +prepare p5(int) as SELECT count(*) FROM t1 WHERE c = 8 and a = $1 GROUP BY c; +execute p5(5); +execute p5(5); +execute p5(5); +execute p5(5); +execute p5(5); +execute p5(5); +execute p5(5); +execute p5(5); +execute p5(5); +execute p5(5); +RESET client_min_messages; + \c - - - :master_port --- one another combination is that the shell table +-- one other combination is that the shell table -- has a dropped column but not the shard, via rebalance operation SET search_path TO local_shard_execution_dropped_column; ALTER TABLE t1 DROP COLUMN a; SELECT citus_move_shard_placement(2460000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes'); +SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests \c - - - :worker_2_port SET search_path TO local_shard_execution_dropped_column; @@ -132,5 +155,25 @@ execute p3(8); execute p3(8); execute p3(8); +-- Test that "Avoid deparse and planning of shard query for local execution" +-- does not take the fast path of modifying the parse tree with the shard OID +-- for this scenario (rebalance) also. +-- +-- (*) https://github.com/citusdata/citus/pull/8035 + +SET client_min_messages to DEBUG2; +prepare p4(int) as SELECT count(*) FROM t1 WHERE c = 8 and 5 = $1 GROUP BY c; +execute p4(5); +execute p4(5); +execute p4(5); +execute p4(5); +execute p4(5); +execute p4(5); +execute p4(5); +execute p4(5); +execute p4(5); +execute p4(5); +RESET client_min_messages; + \c - - - :master_port DROP SCHEMA local_shard_execution_dropped_column CASCADE; From 0c1b31cdb5d2e1175567cc6703049f4565274367 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?C=C3=A9dric=20Villemain?= Date: Tue, 22 Jul 2025 18:49:26 +0200 Subject: [PATCH 02/13] Fix UPDATE stmts with indirection & array/jsonb subscripting with more than 1 field (#7675) DESCRIPTION: Fixes problematic UPDATE statements with indirection and array/jsonb subscripting with more than one field. Fixes #4092, #7674 and #5621. Issues #7674 and #4092 involve an UPDATE with out of order columns and a sublink (SELECT) in the source, e.g. `UPDATE T SET (col3, col1, col4) = (SELECT 3, 1, 4)` where an incorrect value could get written to a column because query deparsing generated an incorrect SQL statement. To address this the fix adds an additional check to `ruleutils` to ensure that the target list of an UPDATE statement is in an order so that deparsing can be done safely. It is needed when the source of the UPDATE has a sublink, because Postgres `rewrite` will have put the target list in attribute order, but for deparsing to produce a correct SQL text the target list needs to be in order of the references (or `paramids`) to the target list of the sublink(s). Issue #5621 involves an UPDATE with array/jsonb subscripting that can behave incorrectly with more than one field, again because Citus query deparsing is receiving a post-`rewrite` query tree. The fix also adds a check to `ruleutils` to enable correct query deparsing of the UPDATE. --------- Co-authored-by: Ibrahim Halatci Co-authored-by: Colm McHugh --- .../distributed/deparser/citus_ruleutils.c | 253 ++++++ .../distributed/deparser/ruleutils_15.c | 4 + .../distributed/deparser/ruleutils_16.c | 4 + .../distributed/deparser/ruleutils_17.c | 4 + src/include/distributed/citus_ruleutils.h | 2 + src/test/regress/citus_tests/run_test.py | 1 + .../regress/expected/multi_modifications.out | 38 + .../regress/expected/multi_update_select.out | 717 ++++++++++++++++++ src/test/regress/expected/subscripting_op.out | 105 +++ src/test/regress/multi_1_schedule | 1 + src/test/regress/multi_schedule | 1 + src/test/regress/sql/multi_modifications.sql | 28 + src/test/regress/sql/multi_update_select.sql | 416 ++++++++++ src/test/regress/sql/subscripting_op.sql | 93 +++ 14 files changed, 1667 insertions(+) create mode 100644 src/test/regress/expected/multi_update_select.out create mode 100644 src/test/regress/expected/subscripting_op.out create mode 100644 src/test/regress/sql/multi_update_select.sql create mode 100644 src/test/regress/sql/subscripting_op.sql diff --git a/src/backend/distributed/deparser/citus_ruleutils.c b/src/backend/distributed/deparser/citus_ruleutils.c index d590495a6..a25cdf5bb 100644 --- a/src/backend/distributed/deparser/citus_ruleutils.c +++ b/src/backend/distributed/deparser/citus_ruleutils.c @@ -82,6 +82,7 @@ static void AppendStorageParametersToString(StringInfo stringBuffer, List *optionList); static const char * convert_aclright_to_string(int aclright); static void simple_quote_literal(StringInfo buf, const char *val); +static SubscriptingRef * TargetEntryExprFindSubsRef(Expr *expr); static void AddVacuumParams(ReindexStmt *reindexStmt, StringInfo buffer); static void process_acl_items(Acl *acl, const char *relationName, const char *attributeName, List **defs); @@ -1715,3 +1716,255 @@ RoleSpecString(RoleSpec *spec, bool withQuoteIdentifier) } } } + + +/* + * Recursively search an expression for a Param and return its paramid + * Intended for indirection management: UPDATE SET () = (SELECT ) + * Does not cover all options but those supported by Citus. + */ +static int +GetParamId(Node *expr) +{ + int paramid = 0; + + if (expr == NULL) + { + return paramid; + } + + /* If it's a Param, return its attnum */ + if (IsA(expr, Param)) + { + Param *param = (Param *) expr; + paramid = param->paramid; + } + /* If it's a FuncExpr, search in arguments */ + else if (IsA(expr, FuncExpr)) + { + FuncExpr *func = (FuncExpr *) expr; + ListCell *lc; + + foreach(lc, func->args) + { + paramid = GetParamId((Node *) lfirst(lc)); + if (paramid != 0) + { + break; /* Stop at the first valid paramid */ + } + } + } + + return paramid; +} + + +/* + * list_sort comparator to sort target list by paramid (in MULTIEXPR) + * Intended for indirection management: UPDATE SET () = (SELECT ) + */ +static int +target_list_cmp(const ListCell *a, const ListCell *b) +{ + TargetEntry *tleA = lfirst(a); + TargetEntry *tleB = lfirst(b); + + /* + * Deal with resjunk entries; sublinks are marked resjunk and + * are placed at the end of the target list so this logic + * ensures they stay grouped at the end of the target list: + */ + if (tleA->resjunk || tleB->resjunk) + { + return tleA->resjunk - tleB->resjunk; + } + + int la = GetParamId((Node *) tleA->expr); + int lb = GetParamId((Node *) tleB->expr); + + /* + * Should be looking at legitimate param ids + */ + Assert(la > 0); + Assert(lb > 0); + + /* + * Return -1, 0 or 1 depending on if la is less than, + * equal to or greater than lb + */ + return (la > lb) - (la < lb); +} + + +/* + * Used by get_update_query_targetlist_def() (in ruleutils) to reorder the target + * list on the left side of the update: + * SET () = (SELECT ) + * Reordering the SELECT side only does not work, consider a case like: + * SET (col_1, col3) = (SELECT 1, 3), (col_2) = (SELECT 2) + * Without ensure_update_targetlist_in_param_order(), this will lead to an incorrect + * deparsed query: + * SET (col_1, col2) = (SELECT 1, 3), (col_3) = (SELECT 2) + */ +void +ensure_update_targetlist_in_param_order(List *targetList) +{ + bool need_to_sort_target_list = false; + int previous_paramid = 0; + ListCell *l; + + foreach(l, targetList) + { + TargetEntry *tle = (TargetEntry *) lfirst(l); + + if (!tle->resjunk) + { + int paramid = GetParamId((Node *) tle->expr); + if (paramid < previous_paramid) + { + need_to_sort_target_list = true; + break; + } + + previous_paramid = paramid; + } + } + + if (need_to_sort_target_list) + { + list_sort(targetList, target_list_cmp); + } +} + + +/* + * ExpandMergedSubscriptingRefEntries takes a list of target entries and expands + * each one that references a SubscriptingRef node that indicates multiple (field) + * updates on the same attribute, which is applicable for array/json types atm. + */ +List * +ExpandMergedSubscriptingRefEntries(List *targetEntryList) +{ + List *newTargetEntryList = NIL; + ListCell *tgtCell = NULL; + + foreach(tgtCell, targetEntryList) + { + TargetEntry *targetEntry = (TargetEntry *) lfirst(tgtCell); + List *expandedTargetEntries = NIL; + + Expr *expr = targetEntry->expr; + while (expr) + { + SubscriptingRef *subsRef = TargetEntryExprFindSubsRef(expr); + if (!subsRef) + { + break; + } + + /* + * Remove refexpr from the SubscriptingRef that we are about to + * wrap in a new TargetEntry and save it for the next one. + */ + Expr *refexpr = subsRef->refexpr; + subsRef->refexpr = NULL; + + /* + * Wrap the Expr that holds SubscriptingRef (directly or indirectly) + * in a new TargetEntry; note that it doesn't have a refexpr anymore. + */ + TargetEntry *newTargetEntry = copyObject(targetEntry); + newTargetEntry->expr = expr; + expandedTargetEntries = lappend(expandedTargetEntries, newTargetEntry); + + /* now inspect the refexpr that SubscriptingRef at hand were holding */ + expr = refexpr; + } + + if (expandedTargetEntries == NIL) + { + /* return original entry since it doesn't hold a SubscriptingRef node */ + newTargetEntryList = lappend(newTargetEntryList, targetEntry); + } + else + { + /* + * Need to concat expanded target list entries in reverse order + * to preserve ordering of the original target entry list. + */ + List *reversedTgtEntries = NIL; + ListCell *revCell = NULL; + foreach(revCell, expandedTargetEntries) + { + TargetEntry *tgtEntry = (TargetEntry *) lfirst(revCell); + reversedTgtEntries = lcons(tgtEntry, reversedTgtEntries); + } + newTargetEntryList = list_concat(newTargetEntryList, reversedTgtEntries); + } + } + + return newTargetEntryList; +} + + +/* + * TargetEntryExprFindSubsRef searches given Expr --assuming that it is part + * of a target list entry-- to see if it directly (i.e.: itself) or indirectly + * (e.g.: behind some level of coercions) holds a SubscriptingRef node. + * + * Returns the original SubscriptingRef node on success or NULL otherwise. + * + * Note that it wouldn't add much value to use expression_tree_walker here + * since we are only interested in a subset of the fields of a few certain + * node types. + */ +static SubscriptingRef * +TargetEntryExprFindSubsRef(Expr *expr) +{ + Node *node = (Node *) expr; + while (node) + { + if (IsA(node, FieldStore)) + { + /* + * ModifyPartialQuerySupported doesn't allow INSERT/UPDATE via + * FieldStore. If we decide supporting such commands, then we + * should take the first element of "newvals" list into account + * here. This is because, to support such commands, we will need + * to expand merged FieldStore into separate target entries too. + * + * For this reason, this block is not reachable atm and need to + * uncomment the following if we decide supporting such commands. + * + * """ + * FieldStore *fieldStore = (FieldStore *) node; + * node = (Node *) linitial(fieldStore->newvals); + * """ + */ + ereport(ERROR, (errmsg("unexpectedly got FieldStore object when " + "generating shard query"))); + } + else if (IsA(node, CoerceToDomain)) + { + CoerceToDomain *coerceToDomain = (CoerceToDomain *) node; + if (coerceToDomain->coercionformat != COERCE_IMPLICIT_CAST) + { + /* not an implicit coercion, cannot reach to a SubscriptingRef */ + break; + } + + node = (Node *) coerceToDomain->arg; + } + else if (IsA(node, SubscriptingRef)) + { + return (SubscriptingRef *) node; + } + else + { + /* got a node that we are not interested in */ + break; + } + } + + return NULL; +} diff --git a/src/backend/distributed/deparser/ruleutils_15.c b/src/backend/distributed/deparser/ruleutils_15.c index 9004f7bbc..2fd62a3f8 100644 --- a/src/backend/distributed/deparser/ruleutils_15.c +++ b/src/backend/distributed/deparser/ruleutils_15.c @@ -3509,6 +3509,8 @@ get_update_query_targetlist_def(Query *query, List *targetList, SubLink *cur_ma_sublink; List *ma_sublinks; + targetList = ExpandMergedSubscriptingRefEntries(targetList); + /* * Prepare to deal with MULTIEXPR assignments: collect the source SubLinks * into a list. We expect them to appear, in ID order, in resjunk tlist @@ -3532,6 +3534,8 @@ get_update_query_targetlist_def(Query *query, List *targetList, } } } + + ensure_update_targetlist_in_param_order(targetList); } next_ma_cell = list_head(ma_sublinks); cur_ma_sublink = NULL; diff --git a/src/backend/distributed/deparser/ruleutils_16.c b/src/backend/distributed/deparser/ruleutils_16.c index 65bbd1720..d3b0b2da5 100644 --- a/src/backend/distributed/deparser/ruleutils_16.c +++ b/src/backend/distributed/deparser/ruleutils_16.c @@ -3525,6 +3525,8 @@ get_update_query_targetlist_def(Query *query, List *targetList, SubLink *cur_ma_sublink; List *ma_sublinks; + targetList = ExpandMergedSubscriptingRefEntries(targetList); + /* * Prepare to deal with MULTIEXPR assignments: collect the source SubLinks * into a list. We expect them to appear, in ID order, in resjunk tlist @@ -3548,6 +3550,8 @@ get_update_query_targetlist_def(Query *query, List *targetList, } } } + + ensure_update_targetlist_in_param_order(targetList); } next_ma_cell = list_head(ma_sublinks); cur_ma_sublink = NULL; diff --git a/src/backend/distributed/deparser/ruleutils_17.c b/src/backend/distributed/deparser/ruleutils_17.c index f0710e684..1d0491ce4 100644 --- a/src/backend/distributed/deparser/ruleutils_17.c +++ b/src/backend/distributed/deparser/ruleutils_17.c @@ -3542,6 +3542,8 @@ get_update_query_targetlist_def(Query *query, List *targetList, SubLink *cur_ma_sublink; List *ma_sublinks; + targetList = ExpandMergedSubscriptingRefEntries(targetList); + /* * Prepare to deal with MULTIEXPR assignments: collect the source SubLinks * into a list. We expect them to appear, in ID order, in resjunk tlist @@ -3565,6 +3567,8 @@ get_update_query_targetlist_def(Query *query, List *targetList, } } } + + ensure_update_targetlist_in_param_order(targetList); } next_ma_cell = list_head(ma_sublinks); cur_ma_sublink = NULL; diff --git a/src/include/distributed/citus_ruleutils.h b/src/include/distributed/citus_ruleutils.h index 3a9c36482..28a6198c9 100644 --- a/src/include/distributed/citus_ruleutils.h +++ b/src/include/distributed/citus_ruleutils.h @@ -44,6 +44,7 @@ extern bool contain_nextval_expression_walker(Node *node, void *context); extern char * pg_get_replica_identity_command(Oid tableRelationId); extern List * pg_get_row_level_security_commands(Oid relationId); extern const char * RoleSpecString(RoleSpec *spec, bool withQuoteIdentifier); +extern List * ExpandMergedSubscriptingRefEntries(List *targetEntryList); extern char * flatten_reloptions(Oid relid); /* Function declarations for version dependent PostgreSQL ruleutils functions */ @@ -60,5 +61,6 @@ extern char * generate_operator_name(Oid operid, Oid arg1, Oid arg2); extern List * getOwnedSequences_internal(Oid relid, AttrNumber attnum, char deptype); extern void AppendOptionListToString(StringInfo stringData, List *options); +extern void ensure_update_targetlist_in_param_order(List *targetList); #endif /* CITUS_RULEUTILS_H */ diff --git a/src/test/regress/citus_tests/run_test.py b/src/test/regress/citus_tests/run_test.py index 8d11c7e66..193bdf09f 100755 --- a/src/test/regress/citus_tests/run_test.py +++ b/src/test/regress/citus_tests/run_test.py @@ -147,6 +147,7 @@ DEPS = { "multi_mx_modifying_xacts": TestDeps(None, ["multi_mx_create_table"]), "multi_mx_router_planner": TestDeps(None, ["multi_mx_create_table"]), "multi_mx_copy_data": TestDeps(None, ["multi_mx_create_table"]), + "multi_mx_modifications": TestDeps(None, ["multi_mx_create_table"]), "multi_mx_schema_support": TestDeps(None, ["multi_mx_copy_data"]), "multi_simple_queries": TestDeps("base_schedule"), "create_single_shard_table": TestDeps("minimal_schedule"), diff --git a/src/test/regress/expected/multi_modifications.out b/src/test/regress/expected/multi_modifications.out index 93f6c8c45..cebef0526 100644 --- a/src/test/regress/expected/multi_modifications.out +++ b/src/test/regress/expected/multi_modifications.out @@ -330,6 +330,12 @@ UPDATE limit_orders SET (kind, limit_price) = ('buy', 999) WHERE id = 246 RETURN 246 | GM | 30 | Mon Jul 02 16:32:15 2007 | buy | 999 (1 row) +UPDATE limit_orders SET (kind, limit_price) = (SELECT 'buy'::order_side, 999) WHERE id = 246 RETURNING *; + id | symbol | bidder_id | placed_at | kind | limit_price +--------------------------------------------------------------------- + 246 | GM | 30 | Mon Jul 02 16:32:15 2007 | buy | 999 +(1 row) + -- Test that on unique contraint violations, we fail fast \set VERBOSITY terse INSERT INTO limit_orders VALUES (275, 'ADR', 140, '2007-07-02 16:32:15', 'sell', 43.67); @@ -435,6 +441,8 @@ UPDATE limit_orders SET limit_price = 0.00 FROM bidders -- should succeed with a CTE WITH deleted_orders AS (INSERT INTO limit_orders VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)) UPDATE limit_orders SET symbol = 'GM'; +WITH deleted_orders AS (INSERT INTO limit_orders SELECT 400, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43) +UPDATE limit_orders SET symbol = 'GM'; SELECT symbol, bidder_id FROM limit_orders WHERE id = 246; symbol | bidder_id --------------------------------------------------------------------- @@ -927,6 +935,17 @@ SELECT * FROM summary_table ORDER BY id; 2 | 400 | 450.0000000000000000 | | (2 rows) +-- try different order of update targets +UPDATE summary_table SET (average_value, min_value) = + (SELECT avg(value), min(value) FROM raw_table WHERE id = 2) +WHERE id = 2; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +--------------------------------------------------------------------- + 1 | | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + UPDATE summary_table SET min_value = 100 WHERE id IN (SELECT id FROM raw_table WHERE id = 1 and value > 100) AND id = 1; SELECT * FROM summary_table ORDER BY id; @@ -1103,6 +1122,16 @@ SELECT * FROM reference_summary_table ORDER BY id; 2 | 400 | 450.0000000000000000 | | (2 rows) +UPDATE reference_summary_table SET (average_value, min_value) = + (SELECT avg(value), min(value) FROM reference_raw_table WHERE id = 2) +WHERE id = 2; +SELECT * FROM reference_summary_table ORDER BY id; + id | min_value | average_value | count | uniques +--------------------------------------------------------------------- + 1 | | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + -- no need partition colum equalities on reference tables UPDATE reference_summary_table SET (count) = (SELECT id AS inner_id FROM reference_raw_table WHERE value = 500) @@ -1329,5 +1358,14 @@ DROP TABLE raw_table; DROP TABLE summary_table; DROP TABLE reference_raw_table; DROP TABLE reference_summary_table; +DROP TABLE limit_orders; +DROP TABLE multiple_hash; +DROP TABLE range_partitioned; +DROP TABLE append_partitioned; +DROP TABLE bidders; +DROP FUNCTION stable_append; +DROP FUNCTION immutable_append; +DROP FUNCTION temp_strict_func; +DROP TYPE order_side; DROP SCHEMA multi_modifications CASCADE; NOTICE: drop cascades to table multi_modifications.local diff --git a/src/test/regress/expected/multi_update_select.out b/src/test/regress/expected/multi_update_select.out new file mode 100644 index 000000000..9aac0222f --- /dev/null +++ b/src/test/regress/expected/multi_update_select.out @@ -0,0 +1,717 @@ +CREATE SCHEMA multi_update_select; +SET search_path TO multi_update_select; +SET citus.next_shard_id TO 751000; +-- specific tests related to get_update_query_targetlist_def +-- we test only queries with sublinks, like: +-- ( ... SET (...) = (SELECT ...)) +-- Reference tables +CREATE TABLE test_ref_indirection ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_reference_table('test_ref_indirection'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE test_ref_indirection_new ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_reference_table('test_ref_indirection_new'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +-- Distributed tables +CREATE TABLE test_dist_indirection ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_distributed_table('test_dist_indirection', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE test_dist_indirection_new ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_distributed_table('test_dist_indirection_new', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- those should work: +INSERT INTO test_ref_indirection (id, col_bool, col_date, col_int, col_text) + SELECT 1, true, '1970-01-01'::date, 1, 'one'; +INSERT INTO test_dist_indirection (id, col_bool, col_date, col_int, col_text) + SELECT 1, true, '1970-01-01'::date, 1, 'one'; +INSERT INTO test_ref_indirection (id, col_text, col_bool, col_date, col_int) + SELECT 2, 'two', false, '1970-01-01'::date, 2; +INSERT INTO test_dist_indirection (id, col_text, col_bool, col_date, col_int) + SELECT 2, 'two', false, '1970-01-01'::date, 2; +INSERT INTO test_ref_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty'; +INSERT INTO test_dist_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty'; +INSERT INTO test_ref_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty'; +INSERT INTO test_dist_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty'; +INSERT INTO test_ref_indirection_new SELECT * FROM test_ref_indirection; +INSERT INTO test_dist_indirection_new SELECT * FROM test_dist_indirection; +SELECT * FROM test_ref_indirection ORDER BY id; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1970 | 1 | one + 2 | f | 01-01-1970 | 2 | two + 3 | f | 01-01-1970 | 0 | empty + 4 | f | 01-01-1970 | 0 | empty +(4 rows) + +SELECT * FROM test_dist_indirection ORDER BY id; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1970 | 1 | one + 2 | f | 01-01-1970 | 2 | two + 3 | f | 01-01-1970 | 0 | empty + 4 | f | 01-01-1970 | 0 | empty +(4 rows) + +SELECT * FROM test_ref_indirection_new ORDER BY id; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1970 | 1 | one + 2 | f | 01-01-1970 | 2 | two + 3 | f | 01-01-1970 | 0 | empty + 4 | f | 01-01-1970 | 0 | empty +(4 rows) + +SELECT * FROM test_dist_indirection_new ORDER BY id; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1970 | 1 | one + 2 | f | 01-01-1970 | 2 | two + 3 | f | 01-01-1970 | 0 | empty + 4 | f | 01-01-1970 | 0 | empty +(4 rows) + +-- now UPDATEs +UPDATE test_ref_indirection + SET (col_bool, col_date, col_int, col_text) + = (SELECT true, '1970-01-01'::date, 1, 'ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1970 | 1 | ok + 2 | t | 01-01-1970 | 1 | ok + 3 | t | 01-01-1970 | 1 | ok + 4 | t | 01-01-1970 | 1 | ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_bool, col_date, col_int, col_text) + = (SELECT true, '1970-01-01'::date, 1, 'ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1970 | 1 | ok + 2 | t | 01-01-1970 | 1 | ok + 3 | t | 01-01-1970 | 1 | ok + 4 | t | 01-01-1970 | 1 | ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_bool, col_date) = (select false, '1971-01-01'::date) + , (col_int, col_text) = (select 2, '2 ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1971 | 2 | 2 ok + 2 | f | 01-01-1971 | 2 | 2 ok + 3 | f | 01-01-1971 | 2 | 2 ok + 4 | f | 01-01-1971 | 2 | 2 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_bool, col_date) = (select false, '1971-01-01'::date) + , (col_int, col_text) = (select 2, '2 ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1971 | 2 | 2 ok + 2 | f | 01-01-1971 | 2 | 2 ok + 3 | f | 01-01-1971 | 2 | 2 ok + 4 | f | 01-01-1971 | 2 | 2 ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_bool, col_int) = (select true, 3) + , (col_text) = (select '3 ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1971 | 3 | 3 ok + 2 | t | 01-01-1971 | 3 | 3 ok + 3 | t | 01-01-1971 | 3 | 3 ok + 4 | t | 01-01-1971 | 3 | 3 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_bool, col_int) = (select true, 3) + , (col_text) = (select '3 ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1971 | 3 | 3 ok + 2 | t | 01-01-1971 | 3 | 3 ok + 3 | t | 01-01-1971 | 3 | 3 ok + 4 | t | 01-01-1971 | 3 | 3 ok +(4 rows) + +-- but those should work since 13.X +UPDATE test_ref_indirection + SET (col_date, col_text, col_int, col_bool) + = (SELECT '1972-01-01'::date, '4 ok', 4, false) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1972 | 4 | 4 ok + 2 | f | 01-01-1972 | 4 | 4 ok + 3 | f | 01-01-1972 | 4 | 4 ok + 4 | f | 01-01-1972 | 4 | 4 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_date, col_text, col_int, col_bool) + = (SELECT '1972-01-01'::date, '4 ok', 4, false) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1972 | 4 | 4 ok + 2 | f | 01-01-1972 | 4 | 4 ok + 3 | f | 01-01-1972 | 4 | 4 ok + 4 | f | 01-01-1972 | 4 | 4 ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_int, col_text) = (select 5, '5 ok') + , (col_bool) = (select true) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1972 | 5 | 5 ok + 2 | t | 01-01-1972 | 5 | 5 ok + 3 | t | 01-01-1972 | 5 | 5 ok + 4 | t | 01-01-1972 | 5 | 5 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_int, col_text) = (select 5, '5 ok') + , (col_bool) = (select true) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1972 | 5 | 5 ok + 2 | t | 01-01-1972 | 5 | 5 ok + 3 | t | 01-01-1972 | 5 | 5 ok + 4 | t | 01-01-1972 | 5 | 5 ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_int, col_date) = (select 6, '1973-01-01'::date) + , (col_text, col_bool) = (select '6 ok', false) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1973 | 6 | 6 ok + 2 | f | 01-01-1973 | 6 | 6 ok + 3 | f | 01-01-1973 | 6 | 6 ok + 4 | f | 01-01-1973 | 6 | 6 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_int, col_date) = (select 6, '1973-01-01'::date) + , (col_text, col_bool) = (select '6 ok', false) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1973 | 6 | 6 ok + 2 | f | 01-01-1973 | 6 | 6 ok + 3 | f | 01-01-1973 | 6 | 6 ok + 4 | f | 01-01-1973 | 6 | 6 ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok') + , (col_bool) = (select true) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1974 | 7 | 7 ok + 2 | t | 01-01-1974 | 7 | 7 ok + 3 | t | 01-01-1974 | 7 | 7 ok + 4 | t | 01-01-1974 | 7 | 7 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok') + , (col_bool) = (select true) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1974 | 7 | 7 ok + 2 | t | 01-01-1974 | 7 | 7 ok + 3 | t | 01-01-1974 | 7 | 7 ok + 4 | t | 01-01-1974 | 7 | 7 ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok') + , (col_int) = (select 8) + , (col_bool) = (select false) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1975 | 8 | 8 ok + 2 | f | 01-01-1975 | 8 | 8 ok + 3 | f | 01-01-1975 | 8 | 8 ok + 4 | f | 01-01-1975 | 8 | 8 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok') + , (col_int) = (select 8) + , (col_bool) = (select false) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1975 | 8 | 8 ok + 2 | f | 01-01-1975 | 8 | 8 ok + 3 | f | 01-01-1975 | 8 | 8 ok + 4 | f | 01-01-1975 | 8 | 8 ok +(4 rows) + +-- +-- more restrictive ones, just in case we miss a wrong value +-- +-- those should work +UPDATE test_ref_indirection + SET (col_bool, col_text) = (SELECT true, '9 ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1975 | 8 | 9 ok + 2 | t | 01-01-1975 | 8 | 9 ok + 3 | t | 01-01-1975 | 8 | 9 ok + 4 | t | 01-01-1975 | 8 | 9 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_bool, col_text) = (SELECT true, '9 ok') +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1975 | 8 | 9 ok + 2 | t | 01-01-1975 | 8 | 9 ok + 3 | t | 01-01-1975 | 8 | 9 ok + 4 | t | 01-01-1975 | 8 | 9 ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_bool, col_text) = (SELECT false, '10 ok') +WHERE id = 1 +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1975 | 8 | 10 ok +(1 row) + +UPDATE test_dist_indirection + SET (col_bool, col_text) = (SELECT false, '10 ok') +WHERE id = 1 +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | f | 01-01-1975 | 8 | 10 ok +(1 row) + +UPDATE test_ref_indirection + SET (col_text, col_bool) = (SELECT '11 ok', true) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1975 | 8 | 11 ok + 2 | t | 01-01-1975 | 8 | 11 ok + 3 | t | 01-01-1975 | 8 | 11 ok + 4 | t | 01-01-1975 | 8 | 11 ok +(4 rows) + +UPDATE test_dist_indirection + SET (col_text, col_bool) = (SELECT '11 ok', true) +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 1 | t | 01-01-1975 | 8 | 11 ok + 2 | t | 01-01-1975 | 8 | 11 ok + 3 | t | 01-01-1975 | 8 | 11 ok + 4 | t | 01-01-1975 | 8 | 11 ok +(4 rows) + +UPDATE test_ref_indirection + SET (col_text, col_bool) = (SELECT '12 ok', false) +WHERE id = 2 +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 2 | f | 01-01-1975 | 8 | 12 ok +(1 row) + +UPDATE test_dist_indirection + SET (col_text, col_bool) = (SELECT '12 ok', false) +WHERE id = 2 +RETURNING *; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 2 | f | 01-01-1975 | 8 | 12 ok +(1 row) + +-- several updates in CTE shoult not work +with qq3 as ( + update test_ref_indirection + SET (col_text, col_bool) + = (SELECT '13', true) + where id = 3 + returning * +), +qq4 as ( + update test_ref_indirection + SET (col_text, col_bool) + = (SELECT '14', false) + where id = 4 + returning * +) +select * from qq3 union all select * from qq4; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 3 | t | 01-01-1975 | 8 | 13 + 4 | f | 01-01-1975 | 8 | 14 +(2 rows) + +with qq3 as ( + update test_dist_indirection + SET (col_text, col_bool) + = (SELECT '13', true) + where id = 3 + returning * +), +qq4 as ( + update test_dist_indirection + SET (col_text, col_bool) + = (SELECT '14', false) + where id = 4 + returning * +) +select * from qq3 union all select * from qq4; + id | col_bool | col_date | col_int | col_text +--------------------------------------------------------------------- + 3 | t | 01-01-1975 | 8 | 13 + 4 | f | 01-01-1975 | 8 | 14 +(2 rows) + +DROP TABLE test_dist_indirection; +DROP TABLE test_dist_indirection_new; +DROP TABLE test_ref_indirection; +DROP TABLE test_ref_indirection_new; +-- https://github.com/citusdata/citus/issues/4092 +CREATE TABLE update_test ( + a INT DEFAULT 10, + b INT, + c TEXT +); +SELECT create_reference_table('update_test'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO update_test VALUES (11, 41, 'car'); +INSERT INTO update_test VALUES (100, 20, 'bike'); +INSERT INTO update_test VALUES (100, 20, 'tractor'); +SELECT * FROM update_test; + a | b | c +--------------------------------------------------------------------- + 11 | 41 | car + 100 | 20 | bike + 100 | 20 | tractor +(3 rows) + +UPDATE update_test +SET (b,a) = (select a,b from update_test where b = 41 and c = 'car') +WHERE a = 100 AND b = 20 +RETURNING *; + a | b | c +--------------------------------------------------------------------- + 41 | 11 | bike + 41 | 11 | tractor +(2 rows) + +-- Test that multiple out of order columns and multiple sublinks are handled correctly. +CREATE TABLE upd2_test (a1 int, b1 int, c1 int, d1 int, e1 int, f1 int, g1 int); +SELECT create_reference_table('upd2_test'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO upd2_test SELECT 1, 1, 1, 1, 1, 1, 1 FROM generate_series(1,5) c(i); +UPDATE upd2_test set (b1, a1) = (SELECT 200, 100), (g1, f1, e1) = (SELECT 700, 600, 500), (d1, c1) = (SELECT 400, 300); +SELECT * FROM upd2_test; + a1 | b1 | c1 | d1 | e1 | f1 | g1 +--------------------------------------------------------------------- + 100 | 200 | 300 | 400 | 500 | 600 | 700 + 100 | 200 | 300 | 400 | 500 | 600 | 700 + 100 | 200 | 300 | 400 | 500 | 600 | 700 + 100 | 200 | 300 | 400 | 500 | 600 | 700 + 100 | 200 | 300 | 400 | 500 | 600 | 700 +(5 rows) + +UPDATE upd2_test set (g1, a1) = (SELECT 77, 11), (f1, b1) = (SELECT 66, 22), (e1, c1) = (SELECT 55, 33), (d1) = (SELECT 44); +SELECT * FROM upd2_test; + a1 | b1 | c1 | d1 | e1 | f1 | g1 +--------------------------------------------------------------------- + 11 | 22 | 33 | 44 | 55 | 66 | 77 + 11 | 22 | 33 | 44 | 55 | 66 | 77 + 11 | 22 | 33 | 44 | 55 | 66 | 77 + 11 | 22 | 33 | 44 | 55 | 66 | 77 + 11 | 22 | 33 | 44 | 55 | 66 | 77 +(5 rows) + +UPDATE upd2_test set (g1, a1) = (SELECT 7, 1), (f1) = (SELECT 6), (c1, e1) = (SELECT 3, 5), (b1) = (SELECT 2), (d1) = (SELECT 4); +SELECT * FROM upd2_test; + a1 | b1 | c1 | d1 | e1 | f1 | g1 +--------------------------------------------------------------------- + 1 | 2 | 3 | 4 | 5 | 6 | 7 + 1 | 2 | 3 | 4 | 5 | 6 | 7 + 1 | 2 | 3 | 4 | 5 | 6 | 7 + 1 | 2 | 3 | 4 | 5 | 6 | 7 + 1 | 2 | 3 | 4 | 5 | 6 | 7 +(5 rows) + +-- Test out of order updates on distributed table +CREATE TABLE dist_test(a1 int, b1 numeric, c1 text, d1 int); +SELECT create_distributed_table('dist_test', 'a1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4); +-- Router plan: +UPDATE dist_test +SET (d1, c1, b1) = (SELECT 7, 'tractor', 4.2) +WHERE a1=1 +RETURNING *; + a1 | b1 | c1 | d1 +--------------------------------------------------------------------- + 1 | 4.2 | tractor | 7 + 1 | 4.2 | tractor | 7 +(2 rows) + +-- Pushdown plan: +UPDATE dist_test +SET (d1, c1, b1) = (SELECT X, 'car', Y) +FROM (SELECT r.a * d1 as X, r.b * b1 as Y FROM update_test r, dist_test WHERE r.c=c1) upd +WHERE dist_test.a1 > 2 +RETURNING *; + a1 | b1 | c1 | d1 | x | y +--------------------------------------------------------------------- + 3 | 46.2 | car | 287 | 287 | 46.2 + 3 | 46.2 | car | 287 | 287 | 46.2 + 3 | 46.2 | car | 287 | 287 | 46.2 +(3 rows) + +-- Test subscripting updates +CREATE TABLE jsonb_subscript_update (id INT, data JSONB); +SELECT create_distributed_table('jsonb_subscript_update', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'); +UPDATE jsonb_subscript_update +SET data['b'] = updated_vals.b::TEXT::jsonb, + data['c'] = updated_vals.c::TEXT::jsonb, + data['d'] = updated_vals.d::TEXT::jsonb +FROM ( + SELECT id, + data['a'] AS a, + data['a']::NUMERIC + 1 AS b, + data['a']::NUMERIC + 2 AS c, + data['a']::NUMERIC + 3 AS d + FROM jsonb_subscript_update +) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id; +SELECT * FROM jsonb_subscript_update ORDER BY 1,2; + id | data +--------------------------------------------------------------------- + 1 | {"a": 1, "b": 2, "c": 3, "d": 4} + 2 | {"a": 2, "b": 3, "c": 4, "d": 5} +(2 rows) + +TRUNCATE jsonb_subscript_update; +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'); +-- test router update with jsonb subscript +UPDATE jsonb_subscript_update +SET data['b'] = updated_vals.b::TEXT::jsonb, + data['c'] = updated_vals.c::TEXT::jsonb, + data['d'] = updated_vals.d::TEXT::jsonb +FROM ( + SELECT id, + data['a'] AS a, + data['a']::NUMERIC + 1 AS b, + data['a']::NUMERIC + 2 AS c, + data['a']::NUMERIC + 3 AS d + FROM jsonb_subscript_update +) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id + AND jsonb_subscript_update.id = 1; +SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2; + id | data +--------------------------------------------------------------------- + 1 | {"a": 1, "b": 2, "c": 3, "d": 4} +(1 row) + +TRUNCATE jsonb_subscript_update; +-- Test updates on nested json objects +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": {"c":20, "d" : 200}}'), (2, '{"a": {"d":10, "c" : 100}}'); +BEGIN; +UPDATE jsonb_subscript_update +SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB +FROM + (SELECT id, + DATA['a']['c'] AS a, + DATA['a']['c']::NUMERIC + 1 AS b, + DATA['a']['c']::NUMERIC + 2 AS c, + DATA['a']['d']::NUMERIC + 3 AS d + FROM jsonb_subscript_update) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id; +SELECT * FROM jsonb_subscript_update ORDER BY 1,2; + id | data +--------------------------------------------------------------------- + 1 | {"a": {"c": 223, "d": 200}} + 2 | {"a": {"c": 113, "d": 10}} +(2 rows) + +ROLLBACK; +BEGIN; +-- Router plan +UPDATE jsonb_subscript_update +SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB +FROM + (SELECT id, + DATA['a']['c'] AS a, + DATA['a']['c']::NUMERIC + 1 AS b, + DATA['a']['c']::NUMERIC + 2 AS c, + DATA['a']['d']::NUMERIC + 3 AS d + FROM jsonb_subscript_update) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id + AND jsonb_subscript_update.id = 1; +SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2; + id | data +--------------------------------------------------------------------- + 1 | {"a": {"c": 223, "d": 200}} +(1 row) + +ROLLBACK; +TRUNCATE jsonb_subscript_update; +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'), (4, '{"a": 4, "b": 10}'); +ALTER TABLE jsonb_subscript_update ADD CONSTRAINT pkey PRIMARY KEY (id, data); +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}') +ON CONFLICT (id, data) +DO UPDATE SET data['d']=(jsonb_subscript_update.data['a']::INT*100)::TEXT::JSONB, + data['b']=(jsonb_subscript_update.data['a']::INT*-100)::TEXT::JSONB; +SELECT * FROM jsonb_subscript_update ORDER BY 1,2; + id | data +--------------------------------------------------------------------- + 1 | {"a": 1, "b": -100, "d": 100} + 2 | {"a": 2, "b": -200, "d": 200} + 4 | {"a": 4, "b": 10} +(3 rows) + +CREATE TABLE nested_obj_update(id INT, data JSONB, text_col TEXT); +SELECT create_distributed_table('nested_obj_update', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO nested_obj_update VALUES + (1, '{"a": [1,2,3], "b": [4,5,6], "c": [7,8,9], "d": [1,2,1,2]}', '4'), + (2, '{"a": [10,20,30], "b": [41,51,61], "c": [72,82,92], "d": [11,21,11,21]}', '6'); +BEGIN; +-- Pushdown plan +UPDATE nested_obj_update +SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB, + data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB, + data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB, + text_col = (nested_obj_update.id*1000)::TEXT, + data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB, + data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB, + data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB +FROM ( + SELECT id, + data['a'][0] AS a, + data['b'][0]::NUMERIC + 1 AS b, + data['c'][0]::NUMERIC + 2 AS c, + data['c'][1]::NUMERIC + 3 AS d + FROM nested_obj_update +) updated_vals +WHERE nested_obj_update.id = updated_vals.id; +SELECT * FROM nested_obj_update ORDER BY 1,2,3; + id | data | text_col +--------------------------------------------------------------------- + 1 | {"a": [4, 2, 3], "b": [4, 5, 18], "c": [33, 8, 9], "d": [1, 2, 1, 2, 2, null, 1]} | 1000 + 2 | {"a": [60, 20, 30], "b": [41, 51, 148], "c": [255, 82, 92], "d": [11, 21, 11, 21, 4, null, 2]} | 2000 +(2 rows) + +ROLLBACK; +BEGIN; +-- Router plan +UPDATE nested_obj_update +SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB, + data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB, + data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB, + text_col = (nested_obj_update.id*1000)::TEXT, + data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB, + data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB, + data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB +FROM ( + SELECT id, + data['a'][0] AS a, + data['b'][0]::NUMERIC + 1 AS b, + data['c'][0]::NUMERIC + 2 AS c, + data['c'][1]::NUMERIC + 3 AS d + FROM nested_obj_update +) updated_vals +WHERE nested_obj_update.id = updated_vals.id + AND nested_obj_update.id = 2; +SELECT * FROM nested_obj_update WHERE id = 2 ORDER BY 1,2,3; + id | data | text_col +--------------------------------------------------------------------- + 2 | {"a": [60, 20, 30], "b": [41, 51, 148], "c": [255, 82, 92], "d": [11, 21, 11, 21, 4, null, 2]} | 2000 +(1 row) + +ROLLBACK; +-- suppress cascade messages +SET client_min_messages to ERROR; +DROP SCHEMA multi_update_select CASCADE; +RESET client_min_messages; diff --git a/src/test/regress/expected/subscripting_op.out b/src/test/regress/expected/subscripting_op.out new file mode 100644 index 000000000..d0a30c9e1 --- /dev/null +++ b/src/test/regress/expected/subscripting_op.out @@ -0,0 +1,105 @@ +\set VERBOSITY terse +SET citus.next_shard_id TO 1520000; +CREATE SCHEMA subscripting_op; +SET search_path TO subscripting_op; +CREATE TABLE arr_subs_update(id INT, arr INT[], text_col TEXT, int_col_1 INT, int_col_2 INT); +SELECT create_distributed_table('arr_subs_update', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO arr_subs_update + VALUES (1, '{1,2,3}', 'foo', 50, 60), + (2, '{4,5,6}', 'bar', 60, 70), + (3, '{7,8,9}', 'baz', 70, 80); +BEGIN; +UPDATE arr_subs_update +SET arr[1] = updated_vals.b, + arr[3] = updated_vals.d, + int_col_2 = 400, + arr[2] = updated_vals.c +FROM ( + SELECT id, + arr[0] AS a, + arr[1]::NUMERIC + 1 AS b, + arr[2]::NUMERIC + 2 AS c, + arr[3]::NUMERIC + 3 AS d + FROM arr_subs_update +) updated_vals +WHERE arr_subs_update.id = updated_vals.id; +SELECT * FROM arr_subs_update ORDER BY 1,2,3,4; + id | arr | text_col | int_col_1 | int_col_2 +--------------------------------------------------------------------- + 1 | {2,4,6} | foo | 50 | 400 + 2 | {5,7,9} | bar | 60 | 400 + 3 | {8,10,12} | baz | 70 | 400 +(3 rows) + +ROLLBACK; +BEGIN; +-- Test fast path router plan for subscripting update +UPDATE arr_subs_update +SET arr[1] = updated_vals.b, + arr[3] = updated_vals.d, + int_col_2 = 400, + arr[2] = updated_vals.c +FROM ( + SELECT id, + arr[0] AS a, + arr[1]::NUMERIC + 1 AS b, + arr[2]::NUMERIC + 2 AS c, + arr[3]::NUMERIC + 3 AS d + FROM arr_subs_update +) updated_vals +WHERE arr_subs_update.id = updated_vals.id + AND arr_subs_update.id = 1; +SELECT * FROM arr_subs_update +WHERE id=1 ORDER BY 1,2,3,4; + id | arr | text_col | int_col_1 | int_col_2 +--------------------------------------------------------------------- + 1 | {2,4,6} | foo | 50 | 400 +(1 row) + +ROLLBACK; +-- test if we can properly expand target list entries when there are dropped columns +ALTER TABLE arr_subs_update DROP COLUMN int_col_1; +BEGIN; +UPDATE arr_subs_update +SET arr[1] = updated_vals.b, + arr[3] = updated_vals.d, + int_col_2 = 400, + arr[2] = updated_vals.c +FROM ( + SELECT id, + arr[0] AS a, + arr[1]::NUMERIC + 1 AS b, + arr[2]::NUMERIC + 2 AS c, + arr[3]::NUMERIC + 3 AS d + FROM arr_subs_update +) updated_vals +WHERE arr_subs_update.id = updated_vals.id; +SELECT * FROM arr_subs_update ORDER BY 1,2,3,4; + id | arr | text_col | int_col_2 +--------------------------------------------------------------------- + 1 | {2,4,6} | foo | 400 + 2 | {5,7,9} | bar | 400 + 3 | {8,10,12} | baz | 400 +(3 rows) + +ROLLBACK; +TRUNCATE arr_subs_update; +INSERT INTO arr_subs_update VALUES (1, '{1,2,3}', 'foo', 60), (2, '{4,5,6}', 'bar', 70); +ALTER TABLE arr_subs_update ADD CONSTRAINT pkey PRIMARY KEY (id, arr); +INSERT INTO arr_subs_update VALUES (1, '{1,2,3}') +ON CONFLICT (id, arr) +DO UPDATE SET arr[0]=100, arr[1]=200, arr[5]=500; +SELECT * FROM arr_subs_update ORDER BY 1,2,3,4; + id | arr | text_col | int_col_2 +--------------------------------------------------------------------- + 1 | [0:5]={100,200,2,3,NULL,500} | foo | 60 + 2 | {4,5,6} | bar | 70 +(2 rows) + +SET client_min_messages TO WARNING; +DROP SCHEMA subscripting_op CASCADE; diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 2ce74e9a7..6a54e82ad 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -210,6 +210,7 @@ test: multi_outer_join # --- test: multi_complex_count_distinct multi_select_distinct test: multi_modifications +test: multi_update_select test: multi_distribution_metadata test: multi_prune_shard_list test: multi_upsert multi_simple_queries multi_data_types diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index e89d9075d..0b1d4ce67 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -93,6 +93,7 @@ test: multi_average_expression multi_working_columns multi_having_pushdown havin test: multi_array_agg multi_limit_clause multi_orderby_limit_pushdown test: multi_jsonb_agg multi_jsonb_object_agg multi_json_agg multi_json_object_agg bool_agg ch_bench_having chbenchmark_all_queries expression_reference_join anonymous_columns test: ch_bench_subquery_repartition +test: subscripting_op test: multi_agg_type_conversion multi_count_type_conversion recursive_relation_planning_restriction_pushdown test: multi_partition_pruning single_hash_repartition_join unsupported_lateral_subqueries test: multi_join_pruning multi_hash_pruning intermediate_result_pruning diff --git a/src/test/regress/sql/multi_modifications.sql b/src/test/regress/sql/multi_modifications.sql index 2a00e7992..958791e44 100644 --- a/src/test/regress/sql/multi_modifications.sql +++ b/src/test/regress/sql/multi_modifications.sql @@ -234,6 +234,7 @@ SELECT kind, limit_price FROM limit_orders WHERE id = 246; -- multi-column UPDATE with RETURNING UPDATE limit_orders SET (kind, limit_price) = ('buy', 999) WHERE id = 246 RETURNING *; +UPDATE limit_orders SET (kind, limit_price) = (SELECT 'buy'::order_side, 999) WHERE id = 246 RETURNING *; -- Test that on unique contraint violations, we fail fast \set VERBOSITY terse @@ -337,6 +338,9 @@ UPDATE limit_orders SET limit_price = 0.00 FROM bidders WITH deleted_orders AS (INSERT INTO limit_orders VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)) UPDATE limit_orders SET symbol = 'GM'; +WITH deleted_orders AS (INSERT INTO limit_orders SELECT 400, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43) +UPDATE limit_orders SET symbol = 'GM'; + SELECT symbol, bidder_id FROM limit_orders WHERE id = 246; -- updates referencing just a var are supported @@ -584,6 +588,13 @@ WHERE id = 2; SELECT * FROM summary_table ORDER BY id; +-- try different order of update targets +UPDATE summary_table SET (average_value, min_value) = + (SELECT avg(value), min(value) FROM raw_table WHERE id = 2) +WHERE id = 2; + +SELECT * FROM summary_table ORDER BY id; + UPDATE summary_table SET min_value = 100 WHERE id IN (SELECT id FROM raw_table WHERE id = 1 and value > 100) AND id = 1; @@ -712,6 +723,12 @@ WHERE id = 2; SELECT * FROM reference_summary_table ORDER BY id; +UPDATE reference_summary_table SET (average_value, min_value) = + (SELECT avg(value), min(value) FROM reference_raw_table WHERE id = 2) +WHERE id = 2; + +SELECT * FROM reference_summary_table ORDER BY id; + -- no need partition colum equalities on reference tables UPDATE reference_summary_table SET (count) = (SELECT id AS inner_id FROM reference_raw_table WHERE value = 500) @@ -902,4 +919,15 @@ DROP TABLE raw_table; DROP TABLE summary_table; DROP TABLE reference_raw_table; DROP TABLE reference_summary_table; +DROP TABLE limit_orders; +DROP TABLE multiple_hash; +DROP TABLE range_partitioned; +DROP TABLE append_partitioned; +DROP TABLE bidders; + +DROP FUNCTION stable_append; +DROP FUNCTION immutable_append; +DROP FUNCTION temp_strict_func; +DROP TYPE order_side; + DROP SCHEMA multi_modifications CASCADE; diff --git a/src/test/regress/sql/multi_update_select.sql b/src/test/regress/sql/multi_update_select.sql new file mode 100644 index 000000000..d9ed85f87 --- /dev/null +++ b/src/test/regress/sql/multi_update_select.sql @@ -0,0 +1,416 @@ +CREATE SCHEMA multi_update_select; +SET search_path TO multi_update_select; + +SET citus.next_shard_id TO 751000; + +-- specific tests related to get_update_query_targetlist_def +-- we test only queries with sublinks, like: +-- ( ... SET (...) = (SELECT ...)) + +-- Reference tables +CREATE TABLE test_ref_indirection ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_reference_table('test_ref_indirection'); + +CREATE TABLE test_ref_indirection_new ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_reference_table('test_ref_indirection_new'); + +-- Distributed tables +CREATE TABLE test_dist_indirection ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_distributed_table('test_dist_indirection', 'id'); + +CREATE TABLE test_dist_indirection_new ( + id bigint primary key + , col_bool bool , col_date date , col_int integer , col_text text + ); +SELECT create_distributed_table('test_dist_indirection_new', 'id'); + +-- those should work: +INSERT INTO test_ref_indirection (id, col_bool, col_date, col_int, col_text) + SELECT 1, true, '1970-01-01'::date, 1, 'one'; +INSERT INTO test_dist_indirection (id, col_bool, col_date, col_int, col_text) + SELECT 1, true, '1970-01-01'::date, 1, 'one'; + +INSERT INTO test_ref_indirection (id, col_text, col_bool, col_date, col_int) + SELECT 2, 'two', false, '1970-01-01'::date, 2; +INSERT INTO test_dist_indirection (id, col_text, col_bool, col_date, col_int) + SELECT 2, 'two', false, '1970-01-01'::date, 2; + +INSERT INTO test_ref_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty'; +INSERT INTO test_dist_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty'; +INSERT INTO test_ref_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty'; +INSERT INTO test_dist_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty'; + +INSERT INTO test_ref_indirection_new SELECT * FROM test_ref_indirection; +INSERT INTO test_dist_indirection_new SELECT * FROM test_dist_indirection; + +SELECT * FROM test_ref_indirection ORDER BY id; +SELECT * FROM test_dist_indirection ORDER BY id; + +SELECT * FROM test_ref_indirection_new ORDER BY id; +SELECT * FROM test_dist_indirection_new ORDER BY id; + +-- now UPDATEs +UPDATE test_ref_indirection + SET (col_bool, col_date, col_int, col_text) + = (SELECT true, '1970-01-01'::date, 1, 'ok') +RETURNING *; +UPDATE test_dist_indirection + SET (col_bool, col_date, col_int, col_text) + = (SELECT true, '1970-01-01'::date, 1, 'ok') +RETURNING *; + +UPDATE test_ref_indirection + SET (col_bool, col_date) = (select false, '1971-01-01'::date) + , (col_int, col_text) = (select 2, '2 ok') +RETURNING *; +UPDATE test_dist_indirection + SET (col_bool, col_date) = (select false, '1971-01-01'::date) + , (col_int, col_text) = (select 2, '2 ok') +RETURNING *; + +UPDATE test_ref_indirection + SET (col_bool, col_int) = (select true, 3) + , (col_text) = (select '3 ok') +RETURNING *; +UPDATE test_dist_indirection + SET (col_bool, col_int) = (select true, 3) + , (col_text) = (select '3 ok') +RETURNING *; + +-- but those should work since 13.X +UPDATE test_ref_indirection + SET (col_date, col_text, col_int, col_bool) + = (SELECT '1972-01-01'::date, '4 ok', 4, false) +RETURNING *; +UPDATE test_dist_indirection + SET (col_date, col_text, col_int, col_bool) + = (SELECT '1972-01-01'::date, '4 ok', 4, false) +RETURNING *; + +UPDATE test_ref_indirection + SET (col_int, col_text) = (select 5, '5 ok') + , (col_bool) = (select true) +RETURNING *; +UPDATE test_dist_indirection + SET (col_int, col_text) = (select 5, '5 ok') + , (col_bool) = (select true) +RETURNING *; + +UPDATE test_ref_indirection + SET (col_int, col_date) = (select 6, '1973-01-01'::date) + , (col_text, col_bool) = (select '6 ok', false) +RETURNING *; +UPDATE test_dist_indirection + SET (col_int, col_date) = (select 6, '1973-01-01'::date) + , (col_text, col_bool) = (select '6 ok', false) +RETURNING *; + +UPDATE test_ref_indirection + SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok') + , (col_bool) = (select true) +RETURNING *; +UPDATE test_dist_indirection + SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok') + , (col_bool) = (select true) +RETURNING *; + +UPDATE test_ref_indirection + SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok') + , (col_int) = (select 8) + , (col_bool) = (select false) +RETURNING *; +UPDATE test_dist_indirection + SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok') + , (col_int) = (select 8) + , (col_bool) = (select false) +RETURNING *; + +-- +-- more restrictive ones, just in case we miss a wrong value +-- +-- those should work +UPDATE test_ref_indirection + SET (col_bool, col_text) = (SELECT true, '9 ok') +RETURNING *; +UPDATE test_dist_indirection + SET (col_bool, col_text) = (SELECT true, '9 ok') +RETURNING *; + +UPDATE test_ref_indirection + SET (col_bool, col_text) = (SELECT false, '10 ok') +WHERE id = 1 +RETURNING *; +UPDATE test_dist_indirection + SET (col_bool, col_text) = (SELECT false, '10 ok') +WHERE id = 1 +RETURNING *; + +UPDATE test_ref_indirection + SET (col_text, col_bool) = (SELECT '11 ok', true) +RETURNING *; +UPDATE test_dist_indirection + SET (col_text, col_bool) = (SELECT '11 ok', true) +RETURNING *; + +UPDATE test_ref_indirection + SET (col_text, col_bool) = (SELECT '12 ok', false) +WHERE id = 2 +RETURNING *; +UPDATE test_dist_indirection + SET (col_text, col_bool) = (SELECT '12 ok', false) +WHERE id = 2 +RETURNING *; + +-- several updates in CTE shoult not work +with qq3 as ( + update test_ref_indirection + SET (col_text, col_bool) + = (SELECT '13', true) + where id = 3 + returning * +), +qq4 as ( + update test_ref_indirection + SET (col_text, col_bool) + = (SELECT '14', false) + where id = 4 + returning * +) +select * from qq3 union all select * from qq4; +with qq3 as ( + update test_dist_indirection + SET (col_text, col_bool) + = (SELECT '13', true) + where id = 3 + returning * +), +qq4 as ( + update test_dist_indirection + SET (col_text, col_bool) + = (SELECT '14', false) + where id = 4 + returning * +) +select * from qq3 union all select * from qq4; + +DROP TABLE test_dist_indirection; +DROP TABLE test_dist_indirection_new; +DROP TABLE test_ref_indirection; +DROP TABLE test_ref_indirection_new; + +-- https://github.com/citusdata/citus/issues/4092 +CREATE TABLE update_test ( + a INT DEFAULT 10, + b INT, + c TEXT +); + +SELECT create_reference_table('update_test'); +INSERT INTO update_test VALUES (11, 41, 'car'); +INSERT INTO update_test VALUES (100, 20, 'bike'); +INSERT INTO update_test VALUES (100, 20, 'tractor'); +SELECT * FROM update_test; + +UPDATE update_test +SET (b,a) = (select a,b from update_test where b = 41 and c = 'car') +WHERE a = 100 AND b = 20 +RETURNING *; + +-- Test that multiple out of order columns and multiple sublinks are handled correctly. +CREATE TABLE upd2_test (a1 int, b1 int, c1 int, d1 int, e1 int, f1 int, g1 int); +SELECT create_reference_table('upd2_test'); + +INSERT INTO upd2_test SELECT 1, 1, 1, 1, 1, 1, 1 FROM generate_series(1,5) c(i); + +UPDATE upd2_test set (b1, a1) = (SELECT 200, 100), (g1, f1, e1) = (SELECT 700, 600, 500), (d1, c1) = (SELECT 400, 300); +SELECT * FROM upd2_test; + +UPDATE upd2_test set (g1, a1) = (SELECT 77, 11), (f1, b1) = (SELECT 66, 22), (e1, c1) = (SELECT 55, 33), (d1) = (SELECT 44); +SELECT * FROM upd2_test; + +UPDATE upd2_test set (g1, a1) = (SELECT 7, 1), (f1) = (SELECT 6), (c1, e1) = (SELECT 3, 5), (b1) = (SELECT 2), (d1) = (SELECT 4); +SELECT * FROM upd2_test; + +-- Test out of order updates on distributed table +CREATE TABLE dist_test(a1 int, b1 numeric, c1 text, d1 int); +SELECT create_distributed_table('dist_test', 'a1'); +INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4); +INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4); + +-- Router plan: +UPDATE dist_test +SET (d1, c1, b1) = (SELECT 7, 'tractor', 4.2) +WHERE a1=1 +RETURNING *; + +-- Pushdown plan: +UPDATE dist_test +SET (d1, c1, b1) = (SELECT X, 'car', Y) +FROM (SELECT r.a * d1 as X, r.b * b1 as Y FROM update_test r, dist_test WHERE r.c=c1) upd +WHERE dist_test.a1 > 2 +RETURNING *; + +-- Test subscripting updates +CREATE TABLE jsonb_subscript_update (id INT, data JSONB); +SELECT create_distributed_table('jsonb_subscript_update', 'id'); + +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'); + +UPDATE jsonb_subscript_update +SET data['b'] = updated_vals.b::TEXT::jsonb, + data['c'] = updated_vals.c::TEXT::jsonb, + data['d'] = updated_vals.d::TEXT::jsonb +FROM ( + SELECT id, + data['a'] AS a, + data['a']::NUMERIC + 1 AS b, + data['a']::NUMERIC + 2 AS c, + data['a']::NUMERIC + 3 AS d + FROM jsonb_subscript_update +) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id; + +SELECT * FROM jsonb_subscript_update ORDER BY 1,2; + +TRUNCATE jsonb_subscript_update; +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'); + +-- test router update with jsonb subscript +UPDATE jsonb_subscript_update +SET data['b'] = updated_vals.b::TEXT::jsonb, + data['c'] = updated_vals.c::TEXT::jsonb, + data['d'] = updated_vals.d::TEXT::jsonb +FROM ( + SELECT id, + data['a'] AS a, + data['a']::NUMERIC + 1 AS b, + data['a']::NUMERIC + 2 AS c, + data['a']::NUMERIC + 3 AS d + FROM jsonb_subscript_update +) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id + AND jsonb_subscript_update.id = 1; + +SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2; + +TRUNCATE jsonb_subscript_update; + +-- Test updates on nested json objects +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": {"c":20, "d" : 200}}'), (2, '{"a": {"d":10, "c" : 100}}'); + +BEGIN; +UPDATE jsonb_subscript_update +SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB +FROM + (SELECT id, + DATA['a']['c'] AS a, + DATA['a']['c']::NUMERIC + 1 AS b, + DATA['a']['c']::NUMERIC + 2 AS c, + DATA['a']['d']::NUMERIC + 3 AS d + FROM jsonb_subscript_update) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id; + +SELECT * FROM jsonb_subscript_update ORDER BY 1,2; +ROLLBACK; + +BEGIN; +-- Router plan +UPDATE jsonb_subscript_update +SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB +FROM + (SELECT id, + DATA['a']['c'] AS a, + DATA['a']['c']::NUMERIC + 1 AS b, + DATA['a']['c']::NUMERIC + 2 AS c, + DATA['a']['d']::NUMERIC + 3 AS d + FROM jsonb_subscript_update) updated_vals +WHERE jsonb_subscript_update.id = updated_vals.id + AND jsonb_subscript_update.id = 1; + +SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2; +ROLLBACK; + +TRUNCATE jsonb_subscript_update; +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'), (4, '{"a": 4, "b": 10}'); + +ALTER TABLE jsonb_subscript_update ADD CONSTRAINT pkey PRIMARY KEY (id, data); + +INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}') +ON CONFLICT (id, data) +DO UPDATE SET data['d']=(jsonb_subscript_update.data['a']::INT*100)::TEXT::JSONB, + data['b']=(jsonb_subscript_update.data['a']::INT*-100)::TEXT::JSONB; + +SELECT * FROM jsonb_subscript_update ORDER BY 1,2; + +CREATE TABLE nested_obj_update(id INT, data JSONB, text_col TEXT); +SELECT create_distributed_table('nested_obj_update', 'id'); +INSERT INTO nested_obj_update VALUES + (1, '{"a": [1,2,3], "b": [4,5,6], "c": [7,8,9], "d": [1,2,1,2]}', '4'), + (2, '{"a": [10,20,30], "b": [41,51,61], "c": [72,82,92], "d": [11,21,11,21]}', '6'); + +BEGIN; +-- Pushdown plan +UPDATE nested_obj_update +SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB, + data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB, + data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB, + text_col = (nested_obj_update.id*1000)::TEXT, + data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB, + data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB, + data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB +FROM ( + SELECT id, + data['a'][0] AS a, + data['b'][0]::NUMERIC + 1 AS b, + data['c'][0]::NUMERIC + 2 AS c, + data['c'][1]::NUMERIC + 3 AS d + FROM nested_obj_update +) updated_vals +WHERE nested_obj_update.id = updated_vals.id; + +SELECT * FROM nested_obj_update ORDER BY 1,2,3; +ROLLBACK; + +BEGIN; +-- Router plan +UPDATE nested_obj_update +SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB, + data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB, + data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB, + text_col = (nested_obj_update.id*1000)::TEXT, + data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB, + data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB, + data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB +FROM ( + SELECT id, + data['a'][0] AS a, + data['b'][0]::NUMERIC + 1 AS b, + data['c'][0]::NUMERIC + 2 AS c, + data['c'][1]::NUMERIC + 3 AS d + FROM nested_obj_update +) updated_vals +WHERE nested_obj_update.id = updated_vals.id + AND nested_obj_update.id = 2; + +SELECT * FROM nested_obj_update WHERE id = 2 ORDER BY 1,2,3; +ROLLBACK; + +-- suppress cascade messages +SET client_min_messages to ERROR; +DROP SCHEMA multi_update_select CASCADE; +RESET client_min_messages; + diff --git a/src/test/regress/sql/subscripting_op.sql b/src/test/regress/sql/subscripting_op.sql new file mode 100644 index 000000000..3ff3ad1d1 --- /dev/null +++ b/src/test/regress/sql/subscripting_op.sql @@ -0,0 +1,93 @@ +\set VERBOSITY terse + +SET citus.next_shard_id TO 1520000; + +CREATE SCHEMA subscripting_op; +SET search_path TO subscripting_op; + +CREATE TABLE arr_subs_update(id INT, arr INT[], text_col TEXT, int_col_1 INT, int_col_2 INT); +SELECT create_distributed_table('arr_subs_update', 'id'); +INSERT INTO arr_subs_update + VALUES (1, '{1,2,3}', 'foo', 50, 60), + (2, '{4,5,6}', 'bar', 60, 70), + (3, '{7,8,9}', 'baz', 70, 80); + +BEGIN; +UPDATE arr_subs_update +SET arr[1] = updated_vals.b, + arr[3] = updated_vals.d, + int_col_2 = 400, + arr[2] = updated_vals.c +FROM ( + SELECT id, + arr[0] AS a, + arr[1]::NUMERIC + 1 AS b, + arr[2]::NUMERIC + 2 AS c, + arr[3]::NUMERIC + 3 AS d + FROM arr_subs_update +) updated_vals +WHERE arr_subs_update.id = updated_vals.id; + +SELECT * FROM arr_subs_update ORDER BY 1,2,3,4; + +ROLLBACK; + +BEGIN; +-- Test fast path router plan for subscripting update +UPDATE arr_subs_update +SET arr[1] = updated_vals.b, + arr[3] = updated_vals.d, + int_col_2 = 400, + arr[2] = updated_vals.c +FROM ( + SELECT id, + arr[0] AS a, + arr[1]::NUMERIC + 1 AS b, + arr[2]::NUMERIC + 2 AS c, + arr[3]::NUMERIC + 3 AS d + FROM arr_subs_update +) updated_vals +WHERE arr_subs_update.id = updated_vals.id + AND arr_subs_update.id = 1; + +SELECT * FROM arr_subs_update +WHERE id=1 ORDER BY 1,2,3,4; + +ROLLBACK; + +-- test if we can properly expand target list entries when there are dropped columns +ALTER TABLE arr_subs_update DROP COLUMN int_col_1; + +BEGIN; +UPDATE arr_subs_update +SET arr[1] = updated_vals.b, + arr[3] = updated_vals.d, + int_col_2 = 400, + arr[2] = updated_vals.c +FROM ( + SELECT id, + arr[0] AS a, + arr[1]::NUMERIC + 1 AS b, + arr[2]::NUMERIC + 2 AS c, + arr[3]::NUMERIC + 3 AS d + FROM arr_subs_update +) updated_vals +WHERE arr_subs_update.id = updated_vals.id; + +SELECT * FROM arr_subs_update ORDER BY 1,2,3,4; + +ROLLBACK; + +TRUNCATE arr_subs_update; +INSERT INTO arr_subs_update VALUES (1, '{1,2,3}', 'foo', 60), (2, '{4,5,6}', 'bar', 70); + +ALTER TABLE arr_subs_update ADD CONSTRAINT pkey PRIMARY KEY (id, arr); + +INSERT INTO arr_subs_update VALUES (1, '{1,2,3}') +ON CONFLICT (id, arr) +DO UPDATE SET arr[0]=100, arr[1]=200, arr[5]=500; + +SELECT * FROM arr_subs_update ORDER BY 1,2,3,4; + +SET client_min_messages TO WARNING; +DROP SCHEMA subscripting_op CASCADE; From 9ccf758bb8933a03f60175560b8b9cf13012a0bf Mon Sep 17 00:00:00 2001 From: Colm Date: Wed, 23 Jul 2025 10:44:26 +0100 Subject: [PATCH 03/13] Fix PG15 compiler error introduced in commit 245a62df3e96f3e (#8069) Commit 245a62df3e96f3e included an assertion on a struct field that is in PG16+, without PG_VERSION_NUM check. This commit removes the offending line of code. The same assertion is present later in the function with the PG_VERSION_NUM check, so the offending line of code is redundant. --- src/backend/distributed/planner/multi_router_planner.c | 1 - 1 file changed, 1 deletion(-) diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 59a1d7bb4..907fe1eaf 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -2138,7 +2138,6 @@ ConvertToQueryOnShard(Query *query, Oid citusTableOid, Oid shardId) Assert(list_length(query->rtable) == 1); RangeTblEntry *citusTableRte = (RangeTblEntry *) linitial(query->rtable); Assert(citusTableRte->relid == citusTableOid); - Assert(list_length(query->rteperminfos) == 1); const char *citusTableName = get_rel_name(citusTableOid); Assert(citusTableName != NULL); From 9327df8446e42354577993d8707c45aa0cc9e585 Mon Sep 17 00:00:00 2001 From: Mehmet YILMAZ Date: Wed, 23 Jul 2025 15:15:55 +0300 Subject: [PATCH 04/13] Add PG 18Beta2 Build compatibility (#8060) Fixes #8061 Add PG 18Beta2 Build compatibility Revert "Don't lock partitions pruned by initial pruning Relevant PG commit: 1722d5eb05d8e5d2e064cd1798abcae4f296ca9d https://github.com/postgres/postgres/commit/1722d5e --- .../distributed/executor/local_executor.c | 20 ------ .../distributed/executor/multi_executor.c | 16 ----- .../partitioned_intermediate_results.c | 16 ----- .../distributed/planner/multi_explain.c | 66 +------------------ src/backend/distributed/shared_library_init.c | 60 ++++++----------- .../distributed/utils/background_jobs.c | 15 ----- 6 files changed, 22 insertions(+), 171 deletions(-) diff --git a/src/backend/distributed/executor/local_executor.c b/src/backend/distributed/executor/local_executor.c index 07bd89116..0730e792a 100644 --- a/src/backend/distributed/executor/local_executor.c +++ b/src/backend/distributed/executor/local_executor.c @@ -764,24 +764,6 @@ ExecuteTaskPlan(PlannedStmt *taskPlan, char *queryString, localPlacementIndex) : CreateDestReceiver(DestNone); - /* Create a QueryDesc for the query */ - #if PG_VERSION_NUM >= PG_VERSION_18 - - /* PG18+: nine‐arg CreateQueryDesc with a CachedPlan slot */ - QueryDesc *queryDesc = CreateQueryDesc( - taskPlan, /* PlannedStmt *plannedstmt */ - NULL, /* CachedPlan *cplan (none) */ - queryString, /* const char *sourceText */ - GetActiveSnapshot(), /* Snapshot snapshot */ - InvalidSnapshot, /* Snapshot crosscheck_snapshot */ - destReceiver, /* DestReceiver *dest */ - paramListInfo, /* ParamListInfo params */ - queryEnv, /* QueryEnvironment *queryEnv */ - 0 /* int instrument_options */ - ); - #else - - /* PG15–17: eight‐arg CreateQueryDesc without CachedPlan */ QueryDesc *queryDesc = CreateQueryDesc( taskPlan, /* PlannedStmt *plannedstmt */ queryString, /* const char *sourceText */ @@ -792,8 +774,6 @@ ExecuteTaskPlan(PlannedStmt *taskPlan, char *queryString, queryEnv, /* QueryEnvironment *queryEnv */ 0 /* int instrument_options */ ); - #endif - ExecutorStart(queryDesc, eflags); diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 3815382e0..dba302e7c 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -701,21 +701,6 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, /* don't display the portal in pg_cursors, it is for internal use only */ portal->visible = false; -#if PG_VERSION_NUM >= PG_VERSION_18 - - /* PostgreSQL 18+ adds a seventh “plansource” argument */ - PortalDefineQuery( - portal, - NULL, /* no prepared statement name */ - "", /* query text */ - CMDTAG_SELECT, /* command tag */ - list_make1(queryPlan),/* list of PlannedStmt* */ - NULL, /* no CachedPlan */ - NULL /* no CachedPlanSource */ - ); -#else - - /* PostgreSQL 17-: six-arg signature */ PortalDefineQuery( portal, NULL, /* no prepared statement name */ @@ -724,7 +709,6 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, list_make1(queryPlan),/* list of PlannedStmt* */ NULL /* no CachedPlan */ ); -#endif PortalStart(portal, params, eflags, GetActiveSnapshot()); diff --git a/src/backend/distributed/executor/partitioned_intermediate_results.c b/src/backend/distributed/executor/partitioned_intermediate_results.c index 55b01840c..60cdcbdfe 100644 --- a/src/backend/distributed/executor/partitioned_intermediate_results.c +++ b/src/backend/distributed/executor/partitioned_intermediate_results.c @@ -315,21 +315,6 @@ StartPortalForQueryExecution(const char *queryString) /* don't display the portal in pg_cursors, it is for internal use only */ portal->visible = false; -#if PG_VERSION_NUM >= PG_VERSION_18 - - /* PG 18+: new CachedPlanSource slot */ - PortalDefineQuery( - portal, - NULL, /* no prepared‐stmt name */ - queryString, /* the SQL text */ - CMDTAG_SELECT, /* we’re running a SELECT */ - list_make1(queryPlan), /* plan trees */ - NULL, /* no CachedPlan */ - NULL /* no CachedPlanSource */ - ); -#else - - /* PG 15–17: six‐arg signature */ PortalDefineQuery( portal, NULL, @@ -338,7 +323,6 @@ StartPortalForQueryExecution(const char *queryString) list_make1(queryPlan), NULL /* no CachedPlan */ ); -#endif int eflags = 0; PortalStart(portal, NULL, eflags, GetActiveSnapshot()); diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index c0affb50e..f357663a6 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -543,22 +543,7 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es) } #endif -#if PG_VERSION_NUM >= PG_VERSION_18 - ExplainOnePlan( - plan, /* PlannedStmt *plannedstmt */ - NULL, /* CachedPlan *cplan */ - NULL, /* CachedPlanSource *plansource */ - 0, /* query_index */ - into, /* IntoClause *into */ - es, /* struct ExplainState *es */ - queryString, /* const char *queryString */ - params, /* ParamListInfo params */ - NULL, /* QueryEnvironment *queryEnv */ - &planduration, /* const instr_time *planduration */ - (es->buffers ? &bufusage : NULL),/* const BufferUsage *bufusage */ - (es->memory ? &mem_counters : NULL) /* const MemoryContextCounters *mem_counters */ - ); -#elif PG_VERSION_NUM >= PG_VERSION_17 +#if PG_VERSION_NUM >= PG_VERSION_17 ExplainOnePlan( plan, into, @@ -1606,22 +1591,7 @@ CitusExplainOneQuery(Query *query, int cursorOptions, IntoClause *into, } #endif -#if PG_VERSION_NUM >= PG_VERSION_18 - ExplainOnePlan( - plan, /* PlannedStmt *plannedstmt */ - NULL, /* no CachedPlan */ - NULL, /* no CachedPlanSource */ - 0, /* query_index */ - into, /* IntoClause *into */ - es, /* struct ExplainState *es */ - queryString, /* const char *queryString */ - params, /* ParamListInfo params */ - queryEnv, /* QueryEnvironment *queryEnv */ - &planduration, /* const instr_time *planduration */ - (es->buffers ? &bufusage : NULL), /* const BufferUsage *bufusage */ - (es->memory ? &mem_counters : NULL) /* const MemoryContextCounters *mem_counters */ - ); -#elif PG_VERSION_NUM >= PG_VERSION_17 +#if PG_VERSION_NUM >= PG_VERSION_17 /* PostgreSQL 17 signature (9 args: includes mem_counters) */ ExplainOnePlan( @@ -2107,22 +2077,7 @@ ExplainOneQuery(Query *query, int cursorOptions, } #endif -#if PG_VERSION_NUM >= PG_VERSION_18 - ExplainOnePlan( - plan, /* PlannedStmt *plannedstmt */ - NULL, /* CachedPlan *cplan */ - NULL, /* CachedPlanSource *plansource */ - 0, /* query_index */ - into, /* IntoClause *into */ - es, /* struct ExplainState *es */ - queryString, /* const char *queryString */ - params, /* ParamListInfo params */ - queryEnv, /* QueryEnvironment *queryEnv */ - &planduration, /* const instr_time *planduration */ - (es->buffers ? &bufusage : NULL), - (es->memory ? &mem_counters: NULL) - ); -#elif PG_VERSION_NUM >= PG_VERSION_17 +#if PG_VERSION_NUM >= PG_VERSION_17 ExplainOnePlan( plan, into, @@ -2146,7 +2101,6 @@ ExplainOneQuery(Query *query, int cursorOptions, (es->buffers ? &bufusage : NULL) ); #endif - } } @@ -2208,19 +2162,6 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es UpdateActiveSnapshotCommandId(); /* Create a QueryDesc for the query */ - #if PG_VERSION_NUM >= PG_VERSION_18 - queryDesc = CreateQueryDesc( - plannedstmt, /* PlannedStmt *plannedstmt */ - NULL, /* CachedPlan *cplan (none) */ - queryString, /* const char *sourceText */ - GetActiveSnapshot(), /* Snapshot snapshot */ - InvalidSnapshot, /* Snapshot crosscheck_snapshot */ - dest, /* DestReceiver *dest */ - params, /* ParamListInfo params */ - queryEnv, /* QueryEnvironment *queryEnv */ - instrument_option /* int instrument_options */ - ); - #else queryDesc = CreateQueryDesc( plannedstmt, /* PlannedStmt *plannedstmt */ queryString, /* const char *sourceText */ @@ -2231,7 +2172,6 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es queryEnv, /* QueryEnvironment *queryEnv */ instrument_option /* int instrument_options */ ); - #endif /* Select execution options */ if (es->analyze) diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 61b16610b..165aea05f 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -386,51 +386,29 @@ static const struct config_enum_entry metadata_sync_mode_options[] = { /*----------------------------------------------------------------------* -* On PG 18+ the hook signatures changed; we wrap the old Citus handlers -* in fresh functions that match the new typedefs exactly. +* On PG 18+ the hook signature changed; we wrap the old Citus handler +* in a fresh function that matches the new typedef exactly. *----------------------------------------------------------------------*/ -#if PG_VERSION_NUM >= PG_VERSION_18 -static bool -citus_executor_start_adapter(QueryDesc *queryDesc, int eflags) -{ - /* PG18+ expects a bool return */ - CitusExecutorStart(queryDesc, eflags); - return true; -} - - static void citus_executor_run_adapter(QueryDesc *queryDesc, ScanDirection direction, - uint64 count) -{ - /* PG18+ has no run_once flag - * call the original Citus hook (which still expects the old 4-arg form) */ - CitusExecutorRun(queryDesc, direction, count, true); -} - - -#else - -/* PG15–17: adapter signatures must match the *old* typedefs */ -static void -citus_executor_start_adapter(QueryDesc *queryDesc, int eflags) -{ - CitusExecutorStart(queryDesc, eflags); -} - - -static void -citus_executor_run_adapter(QueryDesc *queryDesc, - ScanDirection direction, - uint64 count, - bool run_once) -{ - CitusExecutorRun(queryDesc, direction, count, run_once); -} - - + uint64 count +#if PG_VERSION_NUM < PG_VERSION_18 + , bool run_once #endif + ) +{ + /* PG18+ has no run_once flag */ + CitusExecutorRun(queryDesc, + direction, + count, +#if PG_VERSION_NUM >= PG_VERSION_18 + true +#else + run_once +#endif + ); +} /* shared library initialization function */ @@ -507,7 +485,7 @@ _PG_init(void) set_rel_pathlist_hook = multi_relation_restriction_hook; get_relation_info_hook = multi_get_relation_info_hook; set_join_pathlist_hook = multi_join_restriction_hook; - ExecutorStart_hook = citus_executor_start_adapter; + ExecutorStart_hook = CitusExecutorStart; ExecutorRun_hook = citus_executor_run_adapter; ExplainOneQuery_hook = CitusExplainOneQuery; prev_ExecutorEnd = ExecutorEnd_hook; diff --git a/src/backend/distributed/utils/background_jobs.c b/src/backend/distributed/utils/background_jobs.c index 6d491a6b3..2d0f03a4c 100644 --- a/src/backend/distributed/utils/background_jobs.c +++ b/src/backend/distributed/utils/background_jobs.c @@ -1907,20 +1907,6 @@ ExecuteSqlString(const char *sql) /* Don't display the portal in pg_cursors */ portal->visible = false; - #if PG_VERSION_NUM >= PG_VERSION_18 - - /* PG18+ added a seventh “plansource” argument */ - PortalDefineQuery( - portal, - NULL, /* no prepared‐stmt name */ - sql, /* the query text */ - commandTag, /* the CommandTag */ - plantree_list, /* List of PlannedStmt* */ - NULL, /* no CachedPlan */ - NULL /* no CachedPlanSource */ - ); - #else - /* PG17-: six‐arg signature */ PortalDefineQuery( portal, @@ -1930,7 +1916,6 @@ ExecuteSqlString(const char *sql) plantree_list, /* List of PlannedStmt* */ NULL /* no CachedPlan */ ); - #endif PortalStart(portal, NULL, 0, InvalidSnapshot); int16 format[] = { 1 }; From f1160b0892e1b5c56bc25c5a83c6c623eae0c84c Mon Sep 17 00:00:00 2001 From: Colm Date: Thu, 24 Jul 2025 14:19:39 +0100 Subject: [PATCH 05/13] Fix assert failure introduced in 245a62df3e96f3eef The assert on the number of shards incorrectly used the value of citus.shard_replication_factor; it should check the table's metadata to determine the replication factor of its data, and not assume it is the current GUC value. --- .../distributed/planner/multi_router_planner.c | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 907fe1eaf..59124c5bf 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -2060,14 +2060,6 @@ CheckAndBuildDelayedFastPathPlan(DistributedPlanningContext *planContext, Task *task = (Task *) linitial(tasks); List *placements = task->taskPlacementList; int32 localGroupId = GetLocalGroupId(); - - /* - * Today FastPathRouterQuery() doesn't set delayFastPathPlanning to true for - * reference tables. We should be looking at 1 placement, or ShardReplicationFactor - * of them. - */ - Assert(list_length(placements) == 1 || list_length(placements) == - ShardReplicationFactor); ShardPlacement *primaryPlacement = (ShardPlacement *) linitial(placements); bool isLocalExecution = !IsDummyPlacement(primaryPlacement) && @@ -2081,6 +2073,14 @@ CheckAndBuildDelayedFastPathPlan(DistributedPlanningContext *planContext, RelationShard *relationShard = (RelationShard *) linitial(relationShards); Assert(relationShard->shardId == primaryPlacement->shardId); + /* + * Today FastPathRouterQuery() doesn't set delayFastPathPlanning to true for + * reference tables. We should be looking at 1 placement, or their replication + * factor. + */ + Assert(list_length(placements) == 1 || list_length(placements) == + TableShardReplicationFactor(relationShard->relationId)); + canBuildLocalPlan = ConvertToQueryOnShard(planContext->query, relationShard->relationId, relationShard->shardId); From a2e3c797e843b47b8f4dfb4ba8b311317fc0ff81 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 25 Jul 2025 17:54:28 +0300 Subject: [PATCH 06/13] Bump black from 23.11.0 to 24.3.0 in /.devcontainer (#8075) Bumps [black](https://github.com/psf/black) from 23.11.0 to 24.3.0.
Release notes

Sourced from black's releases.

24.3.0

Highlights

This release is a milestone: it fixes Black's first CVE security vulnerability. If you run Black on untrusted input, or if you habitually put thousands of leading tab characters in your docstrings, you are strongly encouraged to upgrade immediately to fix CVE-2024-21503.

This release also fixes a bug in Black's AST safety check that allowed Black to make incorrect changes to certain f-strings that are valid in Python 3.12 and higher.

Stable style

  • Don't move comments along with delimiters, which could cause crashes (#4248)
  • Strengthen AST safety check to catch more unsafe changes to strings. Previous versions of Black would incorrectly format the contents of certain unusual f-strings containing nested strings with the same quote type. Now, Black will crash on such strings until support for the new f-string syntax is implemented. (#4270)
  • Fix a bug where line-ranges exceeding the last code line would not work as expected (#4273)

Performance

  • Fix catastrophic performance on docstrings that contain large numbers of leading tab characters. This fixes CVE-2024-21503. (#4278)

Documentation

  • Note what happens when --check is used with --quiet (#4236)

24.2.0

Stable style

  • Fixed a bug where comments where mistakenly removed along with redundant parentheses (#4218)

Preview style

  • Move the hug_parens_with_braces_and_square_brackets feature to the unstable style due to an outstanding crash and proposed formatting tweaks (#4198)
  • Fixed a bug where base expressions caused inconsistent formatting of ** in tenary expression (#4154)
  • Checking for newline before adding one on docstring that is almost at the line limit (#4185)
  • Remove redundant parentheses in case statement if guards (#4214).

Configuration

... (truncated)

Changelog

Sourced from black's changelog.

24.3.0

Highlights

This release is a milestone: it fixes Black's first CVE security vulnerability. If you run Black on untrusted input, or if you habitually put thousands of leading tab characters in your docstrings, you are strongly encouraged to upgrade immediately to fix CVE-2024-21503.

This release also fixes a bug in Black's AST safety check that allowed Black to make incorrect changes to certain f-strings that are valid in Python 3.12 and higher.

Stable style

  • Don't move comments along with delimiters, which could cause crashes (#4248)
  • Strengthen AST safety check to catch more unsafe changes to strings. Previous versions of Black would incorrectly format the contents of certain unusual f-strings containing nested strings with the same quote type. Now, Black will crash on such strings until support for the new f-string syntax is implemented. (#4270)
  • Fix a bug where line-ranges exceeding the last code line would not work as expected (#4273)

Performance

  • Fix catastrophic performance on docstrings that contain large numbers of leading tab characters. This fixes CVE-2024-21503. (#4278)

Documentation

  • Note what happens when --check is used with --quiet (#4236)

24.2.0

Stable style

  • Fixed a bug where comments where mistakenly removed along with redundant parentheses (#4218)

Preview style

  • Move the hug_parens_with_braces_and_square_brackets feature to the unstable style due to an outstanding crash and proposed formatting tweaks (#4198)
  • Fixed a bug where base expressions caused inconsistent formatting of ** in tenary expression (#4154)
  • Checking for newline before adding one on docstring that is almost at the line limit (#4185)
  • Remove redundant parentheses in case statement if guards (#4214).

... (truncated)

Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=black&package-manager=pip&previous-version=23.11.0&new-version=24.3.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself) You can disable automated security fix PRs for this repo from the [Security Alerts page](https://github.com/citusdata/citus/network/alerts).
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .devcontainer/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.devcontainer/requirements.txt b/.devcontainer/requirements.txt index 7300b3b89..b1f8d2857 100644 --- a/.devcontainer/requirements.txt +++ b/.devcontainer/requirements.txt @@ -1,4 +1,4 @@ -black==23.11.0 +black==24.3.0 click==8.1.7 isort==5.12.0 mypy-extensions==1.0.0 From 3e2b6f61faab8ff38357baa3648b5db7dc98f38b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 25 Jul 2025 20:48:36 +0300 Subject: [PATCH 07/13] Bump certifi from 2024.2.2 to 2024.7.4 in /src/test/regress (#8076) Bumps [certifi](https://github.com/certifi/python-certifi) from 2024.2.2 to 2024.7.4.
Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=certifi&package-manager=pip&previous-version=2024.2.2&new-version=2024.7.4)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself) You can disable automated security fix PRs for this repo from the [Security Alerts page](https://github.com/citusdata/citus/network/alerts).
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- src/test/regress/Pipfile.lock | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/test/regress/Pipfile.lock b/src/test/regress/Pipfile.lock index 35fcd4ac8..7f1e34135 100644 --- a/src/test/regress/Pipfile.lock +++ b/src/test/regress/Pipfile.lock @@ -119,11 +119,12 @@ }, "certifi": { "hashes": [ - "sha256:0569859f95fc761b18b45ef421b1290a0f65f147e92a1e5eb3e635f9a5e4e66f", - "sha256:dc383c07b76109f368f6106eee2b593b04a011ea4d55f652c6ca24a754d1cdd1" + "sha256:5a1e7645bc0ec61a09e26c36f6106dd4cf40c6db3a1fb6352b0244e7fb057c7b", + "sha256:c198e21b1289c2ab85ee4e67bb4b4ef3ead0892059901a8d5b622f24a1101e90" ], + "index": "pypi", "markers": "python_version >= '3.6'", - "version": "==2024.2.2" + "version": "==2024.7.4" }, "cffi": { "hashes": [ From 6b9962c0c03da7b5ee24945c9c1ef24b01f52d7e Mon Sep 17 00:00:00 2001 From: ibrahim halatci Date: Tue, 29 Jul 2025 13:24:42 +0300 Subject: [PATCH 08/13] [doc] wrong code comments for function PopUnassignedPlacementExecution (#8079) Fixes #7621 DESCRIPTION: function comment correction --- src/backend/distributed/executor/adaptive_executor.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 895f01ae7..846ba6427 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -3804,7 +3804,7 @@ PopAssignedPlacementExecution(WorkerSession *session) /* - * PopAssignedPlacementExecution finds an executable task from the queue of assigned tasks. + * PopUnAssignedPlacementExecution finds an executable task from the queue of unassigned tasks. */ static TaskPlacementExecution * PopUnassignedPlacementExecution(WorkerPool *workerPool) From f31bcb42199116cfdbb8be86ed4de0c07c82a576 Mon Sep 17 00:00:00 2001 From: Mehmet YILMAZ Date: Tue, 29 Jul 2025 15:52:36 +0300 Subject: [PATCH 09/13] PG18 - Assert("HaveRegisteredOrActiveSnapshot() fix for cluster creation (#8073) fixes #8072 fixes #8055 https://github.com/postgres/postgres/commit/706054b11b959c865c0c7935c34d92370d7168d4 before fix when try to create cluster with assert on `citus_dev make test1 --destroy` ``` TRAP: failed Assert("HaveRegisteredOrActiveSnapshot()"), File: "heapam.c", Line: 232, PID: 75572 postgres: citus citus [local] SELECT(ExceptionalCondition+0x6e)[0x5585e16123e6] postgres: citus citus [local] SELECT(heap_insert+0x220)[0x5585e10709af] postgres: citus citus [local] SELECT(simple_heap_insert+0x33)[0x5585e1071a20] postgres: citus citus [local] SELECT(CatalogTupleInsert+0x32)[0x5585e1135843] /home/citus/.pgenv/pgsql-18beta2/lib/citus.so(+0x11e0aa)[0x7fa26f1ca0aa] /home/citus/.pgenv/pgsql-18beta2/lib/citus.so(+0x11b607)[0x7fa26f1c7607] /home/citus/.pgenv/pgsql-18beta2/lib/citus.so(+0x11bf25)[0x7fa26f1c7f25] /home/citus/.pgenv/pgsql-18beta2/lib/citus.so(+0x11d4e2)[0x7fa26f1c94e2] postgres: citus citus [local] SELECT(+0x1c267d)[0x5585e10e967d] postgres: citus citus [local] SELECT(+0x1c6ba0)[0x5585e10edba0] postgres: citus citus [local] SELECT(+0x1c7b80)[0x5585e10eeb80] postgres: citus citus [local] SELECT(CommitTransactionCommand+0xd)[0x5585e10eef0a] postgres: citus citus [local] SELECT(+0x575b3d)[0x5585e149cb3d] postgres: citus citus [local] SELECT(+0x5788ce)[0x5585e149f8ce] postgres: citus citus [local] SELECT(PostgresMain+0xae7)[0x5585e14a2088] postgres: citus citus [local] SELECT(BackendMain+0x51)[0x5585e149ab36] postgres: citus citus [local] SELECT(postmaster_child_launch+0x101)[0x5585e13d6b32] postgres: citus citus [local] SELECT(+0x4b273f)[0x5585e13d973f] postgres: citus citus [local] SELECT(+0x4b49f3)[0x5585e13db9f3] postgres: citus citus [local] SELECT(PostmasterMain+0x1089)[0x5585e13dcee2] postgres: citus citus [local] SELECT(main+0x1d7)[0x5585e12e3428] /lib/x86_64-linux-gnu/libc.so.6(+0x29d90)[0x7fa271421d90] /lib/x86_64-linux-gnu/libc.so.6(__libc_start_main+0x80)[0x7fa271421e40] ``` --- .../distributed/metadata/node_metadata.c | 2 +- .../transaction/transaction_recovery.c | 2 +- src/include/pg_version_compat.h | 20 +++++++++++++++++++ 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 6c7a98587..2412a88a2 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -2930,7 +2930,7 @@ InsertNodeRow(int nodeid, char *nodeName, int32 nodePort, NodeMetadata *nodeMeta TupleDesc tupleDescriptor = RelationGetDescr(pgDistNode); HeapTuple heapTuple = heap_form_tuple(tupleDescriptor, values, isNulls); - CatalogTupleInsert(pgDistNode, heapTuple); + CATALOG_INSERT_WITH_SNAPSHOT(pgDistNode, heapTuple); CitusInvalidateRelcacheByRelid(DistNodeRelationId()); diff --git a/src/backend/distributed/transaction/transaction_recovery.c b/src/backend/distributed/transaction/transaction_recovery.c index fb5509def..a4ad3e094 100644 --- a/src/backend/distributed/transaction/transaction_recovery.c +++ b/src/backend/distributed/transaction/transaction_recovery.c @@ -106,7 +106,7 @@ LogTransactionRecord(int32 groupId, char *transactionName, FullTransactionId out TupleDesc tupleDescriptor = RelationGetDescr(pgDistTransaction); HeapTuple heapTuple = heap_form_tuple(tupleDescriptor, values, isNulls); - CatalogTupleInsert(pgDistTransaction, heapTuple); + CATALOG_INSERT_WITH_SNAPSHOT(pgDistTransaction, heapTuple); CommandCounterIncrement(); diff --git a/src/include/pg_version_compat.h b/src/include/pg_version_compat.h index 385aecd38..997ad4b58 100644 --- a/src/include/pg_version_compat.h +++ b/src/include/pg_version_compat.h @@ -13,6 +13,10 @@ #include "pg_version_constants.h" +/* we need these for PG-18’s PushActiveSnapshot/PopActiveSnapshot APIs */ +#include "access/xact.h" +#include "utils/snapmgr.h" + #if PG_VERSION_NUM >= PG_VERSION_18 #define create_foreignscan_path_compat(a, b, c, d, e, f, g, h, i, j, k) \ create_foreignscan_path( \ @@ -36,6 +40,14 @@ /* PG-18 unified row-compare operator codes under COMPARE_* */ #define ROWCOMPARE_NE COMPARE_NE +#define CATALOG_INSERT_WITH_SNAPSHOT(rel, tup) \ + do { \ + Snapshot __snap = GetTransactionSnapshot(); \ + PushActiveSnapshot(__snap); \ + CatalogTupleInsert((rel), (tup)); \ + PopActiveSnapshot(); \ + } while (0) + #elif PG_VERSION_NUM >= PG_VERSION_17 #define create_foreignscan_path_compat(a, b, c, d, e, f, g, h, i, j, k) \ create_foreignscan_path( \ @@ -43,6 +55,10 @@ (e), (f), \ (g), (h), (i), (j), (k) \ ) + +/* no-op wrapper on older PGs */ +#define CATALOG_INSERT_WITH_SNAPSHOT(rel, tup) \ + CatalogTupleInsert((rel), (tup)) #endif #if PG_VERSION_NUM >= PG_VERSION_17 @@ -453,6 +469,10 @@ getStxstattarget_compat(HeapTuple tup) k) create_foreignscan_path(a, b, c, d, e, f, g, h, \ i, k) +/* no-op wrapper on older PGs */ +#define CATALOG_INSERT_WITH_SNAPSHOT(rel, tup) \ + CatalogTupleInsert((rel), (tup)) + #define getProcNo_compat(a) (a->pgprocno) #define getLxid_compat(a) (a->lxid) From 889aa92ac019c539d404a0958098904e99f3ee62 Mon Sep 17 00:00:00 2001 From: Teja Mupparti <44680808+tejeswarm@users.noreply.github.com> Date: Wed, 30 Jul 2025 11:29:50 -0700 Subject: [PATCH 10/13] EXPLAIN ANALYZE - Prevent execution of the plan during the plan-print (#8017) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit DESCRIPTION: Fixed a bug in EXPLAIN ANALYZE to prevent unintended (duplicate) execution of the (sub)plans during the explain phase. Fixes #4212 ### 🐞 Bug #4212 : Redundant (Subplan) Execution in `EXPLAIN ANALYZE` codepath #### 🔍 Background In the standard PostgreSQL execution path, `ExplainOnePlan()` is responsible for two distinct operations depending on whether `EXPLAIN ANALYZE` is requested: 1. **Execute the plan** ```c if (es->analyze) ExecutorRun(queryDesc, direction, 0L, true); ``` 2. **Print the plan tree** ```c ExplainPrintPlan(es, queryDesc); ``` When printing the plan, the executor should **not run the plan again**. Execution is only expected to happen once—at the top level when `es->analyze = true`. --- #### ⚠️ Issue in Citus In the Citus implementation of `CustomScanMethods.ExplainCustomScan = CitusExplainScan`, which is a custom scan explain callback function used to print explain information of a Citus plan incorrectly performs **redundant execution** inside the explain path of `ExplainPrintPlan()` ```c ExplainOnePlan() ExplainPrintPlan() ExplainNode() CitusExplainScan() if (distributedPlan->subPlanList != NIL) { ExplainSubPlans(distributedPlan, es); { PlannedStmt *plan = subPlan->plan; ExplainOnePlan(plan, ...); // ⚠️ May re-execute subplan if es->analyze is true } } ``` This causes the subplans to be **executed again**, even though they have already been executed during the top-level plan execution. This behavior violates the expectation in PostgreSQL where `EXPLAIN ANALYZE` should **execute each node exactly once** for analysis. --- #### ✅ Fix (proposed) Save the output of Subplans during `ExecuteSubPlans()`, and later use it in `ExplainSubPlans()` --- .../distributed/executor/adaptive_executor.c | 2 +- .../executor/insert_select_executor.c | 3 +- .../distributed/executor/merge_executor.c | 3 +- .../distributed/executor/multi_executor.c | 10 +- .../distributed/executor/subplan_execution.c | 56 +++- .../distributed/planner/multi_explain.c | 295 +++++++++++++++--- .../distributed/sql/citus--13.1-1--13.2-1.sql | 1 + .../sql/downgrades/citus--13.2-1--13.1-1.sql | 1 + .../13.2-1.sql | 10 + .../9.4-1.sql | 2 + .../latest.sql | 5 +- .../distributed/utils/citus_copyfuncs.c | 25 ++ .../distributed/utils/citus_outfuncs.c | 42 +++ src/include/distributed/multi_executor.h | 4 +- .../distributed/multi_physical_planner.h | 21 ++ src/include/distributed/subplan_execution.h | 2 +- src/test/regress/expected/multi_explain.out | 163 +++++++++- src/test/regress/expected/multi_explain_0.out | 163 +++++++++- src/test/regress/expected/multi_extension.out | 4 +- src/test/regress/expected/stat_counters.out | 9 +- src/test/regress/sql/multi_explain.sql | 26 ++ src/test/regress/sql/stat_counters.sql | 9 +- 22 files changed, 775 insertions(+), 81 deletions(-) create mode 100644 src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/13.2-1.sql diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 846ba6427..677535591 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -760,7 +760,7 @@ AdaptiveExecutorPreExecutorRun(CitusScanState *scanState) */ LockPartitionsForDistributedPlan(distributedPlan); - ExecuteSubPlans(distributedPlan); + ExecuteSubPlans(distributedPlan, RequestedForExplainAnalyze(scanState)); scanState->finishedPreScan = true; } diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 9ed1962fa..58c172c66 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -42,6 +42,7 @@ #include "distributed/merge_planner.h" #include "distributed/metadata_cache.h" #include "distributed/multi_executor.h" +#include "distributed/multi_explain.h" #include "distributed/multi_partitioning_utils.h" #include "distributed/multi_physical_planner.h" #include "distributed/multi_router_planner.h" @@ -121,7 +122,7 @@ NonPushableInsertSelectExecScan(CustomScanState *node) bool binaryFormat = CanUseBinaryCopyFormatForTargetList(selectQuery->targetList); - ExecuteSubPlans(distSelectPlan); + ExecuteSubPlans(distSelectPlan, RequestedForExplainAnalyze(scanState)); /* * We have a separate directory for each transaction, so choosing diff --git a/src/backend/distributed/executor/merge_executor.c b/src/backend/distributed/executor/merge_executor.c index d0f01dcf2..56bde62bc 100644 --- a/src/backend/distributed/executor/merge_executor.c +++ b/src/backend/distributed/executor/merge_executor.c @@ -23,6 +23,7 @@ #include "distributed/merge_executor.h" #include "distributed/merge_planner.h" #include "distributed/multi_executor.h" +#include "distributed/multi_explain.h" #include "distributed/multi_partitioning_utils.h" #include "distributed/multi_router_planner.h" #include "distributed/repartition_executor.h" @@ -132,7 +133,7 @@ ExecuteSourceAtWorkerAndRepartition(CitusScanState *scanState) ereport(DEBUG1, (errmsg("Executing subplans of the source query and " "storing the results at the respective node(s)"))); - ExecuteSubPlans(distSourcePlan); + ExecuteSubPlans(distSourcePlan, RequestedForExplainAnalyze(scanState)); /* * We have a separate directory for each transaction, so choosing diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index dba302e7c..eb6bdf111 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -688,7 +688,7 @@ ExecuteQueryIntoDestReceiver(Query *query, ParamListInfo params, DestReceiver *d * ExecutePlanIntoDestReceiver executes a query plan and sends results to the given * DestReceiver. */ -void +uint64 ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, DestReceiver *dest) { @@ -713,6 +713,8 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, PortalStart(portal, params, eflags, GetActiveSnapshot()); + QueryCompletion qc = { 0 }; + #if PG_VERSION_NUM >= PG_VERSION_18 /* PG 18+: six-arg signature (drop the run_once bool) */ @@ -721,7 +723,7 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, false, /* isTopLevel */ dest, /* DestReceiver *dest */ dest, /* DestReceiver *altdest */ - NULL); /* QueryCompletion *qc */ + &qc); /* QueryCompletion *qc */ #else /* PG 17-: original seven-arg signature */ @@ -731,10 +733,12 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, true, /* run_once */ dest, /* DestReceiver *dest */ dest, /* DestReceiver *altdest */ - NULL); /* QueryCompletion *qc */ + &qc); /* QueryCompletion *qc */ #endif PortalDrop(portal, false); + + return qc.nprocessed; } diff --git a/src/backend/distributed/executor/subplan_execution.c b/src/backend/distributed/executor/subplan_execution.c index ef2838343..108d130ec 100644 --- a/src/backend/distributed/executor/subplan_execution.c +++ b/src/backend/distributed/executor/subplan_execution.c @@ -30,13 +30,22 @@ int MaxIntermediateResult = 1048576; /* maximum size in KB the intermediate resu /* when this is true, we enforce intermediate result size limit in all executors */ int SubPlanLevel = 0; +/* + * SubPlanExplainAnalyzeContext is both a memory context for storing + * subplans’ EXPLAIN ANALYZE output and a flag indicating that execution + * is running under EXPLAIN ANALYZE for subplans. + */ +MemoryContext SubPlanExplainAnalyzeContext = NULL; +SubPlanExplainOutputData *SubPlanExplainOutput; +extern uint8 TotalExplainOutputCapacity; +extern uint8 NumTasksOutput; /* * ExecuteSubPlans executes a list of subplans from a distributed plan * by sequentially executing each plan from the top. */ void -ExecuteSubPlans(DistributedPlan *distributedPlan) +ExecuteSubPlans(DistributedPlan *distributedPlan, bool explainAnalyzeEnabled) { uint64 planId = distributedPlan->planId; List *subPlanList = distributedPlan->subPlanList; @@ -47,6 +56,19 @@ ExecuteSubPlans(DistributedPlan *distributedPlan) return; } + /* + * If the root DistributedPlan has EXPLAIN ANALYZE enabled, + * its subplans should also have EXPLAIN ANALYZE enabled. + */ + if (explainAnalyzeEnabled) + { + SubPlanExplainAnalyzeContext = GetMemoryChunkContext(distributedPlan); + } + else + { + SubPlanExplainAnalyzeContext = NULL; + } + HTAB *intermediateResultsHash = MakeIntermediateResultHTAB(); RecordSubplanExecutionsOnNodes(intermediateResultsHash, distributedPlan); @@ -79,7 +101,23 @@ ExecuteSubPlans(DistributedPlan *distributedPlan) TimestampTz startTimestamp = GetCurrentTimestamp(); - ExecutePlanIntoDestReceiver(plannedStmt, params, copyDest); + uint64 nprocessed; + + PG_TRY(); + { + nprocessed = + ExecutePlanIntoDestReceiver(plannedStmt, params, copyDest); + } + PG_CATCH(); + { + SubPlanExplainAnalyzeContext = NULL; + SubPlanExplainOutput = NULL; + TotalExplainOutputCapacity = 0; + NumTasksOutput = 0; + PG_RE_THROW(); + } + PG_END_TRY(); + /* * EXPLAIN ANALYZE instrumentations. Calculating these are very light-weight, @@ -94,10 +132,24 @@ ExecuteSubPlans(DistributedPlan *distributedPlan) subPlan->durationMillisecs += durationMicrosecs * MICRO_TO_MILLI_SECOND; subPlan->bytesSentPerWorker = RemoteFileDestReceiverBytesSent(copyDest); + subPlan->ntuples = nprocessed; subPlan->remoteWorkerCount = list_length(remoteWorkerNodeList); subPlan->writeLocalFile = entry->writeLocalFile; SubPlanLevel--; + + /* + * Save the EXPLAIN ANALYZE output(s) for later extraction in ExplainSubPlans(). + * Because the SubPlan context isn’t available during distributed execution, + * pass the pointer as a global variable in SubPlanExplainOutput. + */ + subPlan->totalExplainOutput = SubPlanExplainOutput; + subPlan->numTasksOutput = NumTasksOutput; + SubPlanExplainOutput = NULL; + TotalExplainOutputCapacity = 0; + NumTasksOutput = 0; FreeExecutorState(estate); } + + SubPlanExplainAnalyzeContext = NULL; } diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index f357663a6..4d27939f7 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -26,6 +26,7 @@ #include "commands/tablecmds.h" #include "executor/tstoreReceiver.h" #include "lib/stringinfo.h" +#include "nodes/nodeFuncs.h" #include "nodes/plannodes.h" #include "nodes/primnodes.h" #include "nodes/print.h" @@ -73,6 +74,7 @@ #include "distributed/placement_connection.h" #include "distributed/recursive_planning.h" #include "distributed/remote_commands.h" +#include "distributed/subplan_execution.h" #include "distributed/tuple_destination.h" #include "distributed/tuplestore.h" #include "distributed/version_compat.h" @@ -83,6 +85,7 @@ bool ExplainDistributedQueries = true; bool ExplainAllTasks = false; int ExplainAnalyzeSortMethod = EXPLAIN_ANALYZE_SORT_BY_TIME; +extern MemoryContext SubPlanExplainAnalyzeContext; /* * If enabled, EXPLAIN ANALYZE output & other statistics of last worker task @@ -90,6 +93,11 @@ int ExplainAnalyzeSortMethod = EXPLAIN_ANALYZE_SORT_BY_TIME; */ static char *SavedExplainPlan = NULL; static double SavedExecutionDurationMillisec = 0.0; +static double SavedExplainPlanNtuples = 0; +static double SavedExplainPlanNloops = 0; +extern SubPlanExplainOutputData *SubPlanExplainOutput; +uint8 TotalExplainOutputCapacity = 0; +uint8 NumTasksOutput = 0; /* struct to save explain flags */ typedef struct @@ -215,7 +223,8 @@ static const char * ExplainFormatStr(ExplainFormat format); #if PG_VERSION_NUM >= PG_VERSION_17 static const char * ExplainSerializeStr(ExplainSerializeOption serializeOption); #endif -static void ExplainWorkerPlan(PlannedStmt *plannedStmt, DestReceiver *dest, +static void ExplainWorkerPlan(PlannedStmt *plannedStmt, DistributedSubPlan *subPlan, + DestReceiver *dest, ExplainState *es, const char *queryString, ParamListInfo params, QueryEnvironment *queryEnv, @@ -224,7 +233,9 @@ static void ExplainWorkerPlan(PlannedStmt *plannedStmt, DestReceiver *dest, const BufferUsage *bufusage, const MemoryContextCounters *mem_counters, #endif - double *executionDurationMillisec); + double *executionDurationMillisec, + double *executionTuples, + double *executionLoops); static ExplainFormat ExtractFieldExplainFormat(Datum jsonbDoc, const char *fieldName, ExplainFormat defaultValue); #if PG_VERSION_NUM >= PG_VERSION_17 @@ -256,7 +267,8 @@ static double elapsed_time(instr_time *starttime); static void ExplainPropertyBytes(const char *qlabel, int64 bytes, ExplainState *es); static uint64 TaskReceivedTupleData(Task *task); static bool ShowReceivedTupleData(CitusScanState *scanState, ExplainState *es); - +static bool PlanStateAnalyzeWalker(PlanState *planState, void *ctx); +static void ExtractAnalyzeStats(DistributedSubPlan *subPlan, PlanState *planState); /* exports for SQL callable functions */ PG_FUNCTION_INFO_V1(worker_last_saved_explain_analyze); @@ -432,6 +444,84 @@ NonPushableMergeCommandExplainScan(CustomScanState *node, List *ancestors, } +/* + * ExtractAnalyzeStats parses the EXPLAIN ANALYZE output of the pre-executed + * subplans and injects the parsed statistics into queryDesc->planstate->instrument. + */ +static void +ExtractAnalyzeStats(DistributedSubPlan *subPlan, PlanState *planState) +{ + if (!planState) + { + return; + } + + Instrumentation *instr = planState->instrument; + if (!IsA(planState, CustomScanState)) + { + instr->ntuples = subPlan->ntuples; + instr->nloops = 1; /* subplan nodes are executed only once */ + return; + } + + Assert(IsA(planState, CustomScanState)); + + if (subPlan->numTasksOutput <= 0) + { + return; + } + + ListCell *lc; + int tasksOutput = 0; + double tasksNtuples = 0; + double tasksNloops = 0; + memset(instr, 0, sizeof(Instrumentation)); + DistributedPlan *newdistributedPlan = + ((CitusScanState *) planState)->distributedPlan; + + /* + * Inject the earlier executed results—extracted from the workers' EXPLAIN output— + * into the newly created tasks. + */ + foreach(lc, newdistributedPlan->workerJob->taskList) + { + Task *task = (Task *) lfirst(lc); + uint32 taskId = task->taskId; + + if (tasksOutput > subPlan->numTasksOutput) + { + break; + } + + if (!subPlan->totalExplainOutput[taskId].explainOutput) + { + continue; + } + + /* + * Now feed the earlier saved output, which will be used + * by RemoteExplain() when printing tasks + */ + MemoryContext taskContext = GetMemoryChunkContext(task); + task->totalReceivedTupleData = + subPlan->totalExplainOutput[taskId].totalReceivedTupleData; + task->fetchedExplainAnalyzeExecutionDuration = + subPlan->totalExplainOutput[taskId].executionDuration; + task->fetchedExplainAnalyzePlan = + MemoryContextStrdup(taskContext, + subPlan->totalExplainOutput[taskId].explainOutput); + tasksNtuples += subPlan->totalExplainOutput[taskId].executionNtuples; + tasksNloops = subPlan->totalExplainOutput[taskId].executionNloops; + + subPlan->totalExplainOutput[taskId].explainOutput = NULL; + tasksOutput++; + } + + instr->ntuples = tasksNtuples; + instr->nloops = tasksNloops; +} + + /* * ExplainSubPlans generates EXPLAIN output for subplans for CTEs * and complex subqueries. Because the planning for these queries @@ -450,7 +540,6 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es) { DistributedSubPlan *subPlan = (DistributedSubPlan *) lfirst(subPlanCell); PlannedStmt *plan = subPlan->plan; - IntoClause *into = NULL; ParamListInfo params = NULL; /* @@ -534,6 +623,11 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es) ExplainOpenGroup("PlannedStmt", "PlannedStmt", false, es); + DestReceiver *dest = None_Receiver; /* No query execution */ + double executionDurationMillisec = 0.0; + double executionTuples = 0; + double executionLoops = 0; + /* Capture memory stats on PG17+ */ #if PG_VERSION_NUM >= PG_VERSION_17 if (es->memory) @@ -541,31 +635,21 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es) MemoryContextSwitchTo(saved_ctx); MemoryContextMemConsumed(planner_ctx, &mem_counters); } -#endif -#if PG_VERSION_NUM >= PG_VERSION_17 - ExplainOnePlan( - plan, - into, - es, - queryString, - params, - NULL, /* QueryEnvironment *queryEnv */ - &planduration, - (es->buffers ? &bufusage : NULL), - (es->memory ? &mem_counters : NULL) - ); + /* Execute EXPLAIN without ANALYZE */ + ExplainWorkerPlan(plan, subPlan, dest, es, queryString, params, NULL, + &planduration, + (es->buffers ? &bufusage : NULL), + (es->memory ? &mem_counters : NULL), + &executionDurationMillisec, + &executionTuples, + &executionLoops); #else - ExplainOnePlan( - plan, - into, - es, - queryString, - params, - NULL, /* QueryEnvironment *queryEnv */ - &planduration, - (es->buffers ? &bufusage : NULL) - ); + + /* Execute EXPLAIN without ANALYZE */ + ExplainWorkerPlan(plan, subPlan, dest, es, queryString, params, NULL, + &planduration, &executionDurationMillisec, + &executionTuples, &executionLoops); #endif ExplainCloseGroup("PlannedStmt", "PlannedStmt", false, es); @@ -1236,17 +1320,19 @@ worker_last_saved_explain_analyze(PG_FUNCTION_ARGS) if (SavedExplainPlan != NULL) { int columnCount = tupleDescriptor->natts; - if (columnCount != 2) + if (columnCount != 4) { - ereport(ERROR, (errmsg("expected 3 output columns in definition of " + ereport(ERROR, (errmsg("expected 4 output columns in definition of " "worker_last_saved_explain_analyze, but got %d", columnCount))); } - bool columnNulls[2] = { false }; - Datum columnValues[2] = { + bool columnNulls[4] = { false }; + Datum columnValues[4] = { CStringGetTextDatum(SavedExplainPlan), - Float8GetDatum(SavedExecutionDurationMillisec) + Float8GetDatum(SavedExecutionDurationMillisec), + Float8GetDatum(SavedExplainPlanNtuples), + Float8GetDatum(SavedExplainPlanNloops) }; tuplestore_putvalues(tupleStore, tupleDescriptor, columnValues, columnNulls); @@ -1267,6 +1353,8 @@ worker_save_query_explain_analyze(PG_FUNCTION_ARGS) text *queryText = PG_GETARG_TEXT_P(0); char *queryString = text_to_cstring(queryText); double executionDurationMillisec = 0.0; + double executionTuples = 0; + double executionLoops = 0; Datum explainOptions = PG_GETARG_DATUM(1); ExplainState *es = NewExplainState(); @@ -1383,16 +1471,19 @@ worker_save_query_explain_analyze(PG_FUNCTION_ARGS) } /* do the actual EXPLAIN ANALYZE */ - ExplainWorkerPlan(plan, tupleStoreDest, es, queryString, boundParams, NULL, + ExplainWorkerPlan(plan, NULL, tupleStoreDest, es, queryString, boundParams, NULL, &planDuration, (es->buffers ? &bufusage : NULL), (es->memory ? &mem_counters : NULL), - &executionDurationMillisec); + &executionDurationMillisec, + &executionTuples, + &executionLoops); #else /* do the actual EXPLAIN ANALYZE */ - ExplainWorkerPlan(plan, tupleStoreDest, es, queryString, boundParams, NULL, - &planDuration, &executionDurationMillisec); + ExplainWorkerPlan(plan, NULL, tupleStoreDest, es, queryString, boundParams, NULL, + &planDuration, &executionDurationMillisec, + &executionTuples, &executionLoops); #endif ExplainEndOutput(es); @@ -1403,6 +1494,8 @@ worker_save_query_explain_analyze(PG_FUNCTION_ARGS) SavedExplainPlan = pstrdup(es->str->data); SavedExecutionDurationMillisec = executionDurationMillisec; + SavedExplainPlanNtuples = executionTuples; + SavedExplainPlanNloops = executionLoops; MemoryContextSwitchTo(oldContext); @@ -1632,11 +1725,13 @@ CreateExplainAnlyzeDestination(Task *task, TupleDestination *taskDest) tupleDestination->originalTask = task; tupleDestination->originalTaskDestination = taskDest; - TupleDesc lastSavedExplainAnalyzeTupDesc = CreateTemplateTupleDesc(2); + TupleDesc lastSavedExplainAnalyzeTupDesc = CreateTemplateTupleDesc(4); TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 1, "explain analyze", TEXTOID, 0, 0); TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 2, "duration", FLOAT8OID, 0, 0); + TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 3, "ntuples", FLOAT8OID, 0, 0); + TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 4, "nloops", FLOAT8OID, 0, 0); tupleDestination->lastSavedExplainAnalyzeTupDesc = lastSavedExplainAnalyzeTupDesc; @@ -1647,6 +1742,51 @@ CreateExplainAnlyzeDestination(Task *task, TupleDestination *taskDest) } +/* + * EnsureExplainOutputCapacity is to ensure capacity for new entries. Input + * parameter requiredSize is minimum number of elements needed. + */ +static void +EnsureExplainOutputCapacity(int requiredSize) +{ + if (requiredSize < TotalExplainOutputCapacity) + { + return; + } + + int newCapacity = + (TotalExplainOutputCapacity == 0) ? 32 : TotalExplainOutputCapacity * 2; + + while (newCapacity <= requiredSize) + { + newCapacity *= 2; + } + + if (SubPlanExplainOutput == NULL) + { + SubPlanExplainOutput = + (SubPlanExplainOutputData *) MemoryContextAllocZero( + SubPlanExplainAnalyzeContext, + newCapacity * + sizeof(SubPlanExplainOutputData)); + } + else + { + /* Use repalloc and manually zero the new memory */ + int oldSize = TotalExplainOutputCapacity * sizeof(SubPlanExplainOutputData); + int newSize = newCapacity * sizeof(SubPlanExplainOutputData); + + SubPlanExplainOutput = + (SubPlanExplainOutputData *) repalloc(SubPlanExplainOutput, newSize); + + /* Zero out the newly allocated memory */ + MemSet((char *) SubPlanExplainOutput + oldSize, 0, newSize - oldSize); + } + + TotalExplainOutputCapacity = newCapacity; +} + + /* * ExplainAnalyzeDestPutTuple implements TupleDestination->putTuple * for ExplainAnalyzeDestination. @@ -1656,6 +1796,8 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task, int placementIndex, int queryNumber, HeapTuple heapTuple, uint64 tupleLibpqSize) { + uint32 taskId = task->taskId; + ExplainAnalyzeDestination *tupleDestination = (ExplainAnalyzeDestination *) self; if (queryNumber == 0) { @@ -1663,6 +1805,13 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task, originalTupDest->putTuple(originalTupDest, task, placementIndex, 0, heapTuple, tupleLibpqSize); tupleDestination->originalTask->totalReceivedTupleData += tupleLibpqSize; + + if (SubPlanExplainAnalyzeContext) + { + EnsureExplainOutputCapacity(taskId + 1); + SubPlanExplainOutput[taskId].totalReceivedTupleData = + tupleDestination->originalTask->totalReceivedTupleData; + } } else if (queryNumber == 1) { @@ -1678,6 +1827,8 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task, } Datum executionDuration = heap_getattr(heapTuple, 2, tupDesc, &isNull); + Datum executionTuples = heap_getattr(heapTuple, 3, tupDesc, &isNull); + Datum executionLoops = heap_getattr(heapTuple, 4, tupDesc, &isNull); if (isNull) { @@ -1687,6 +1838,8 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task, char *fetchedExplainAnalyzePlan = TextDatumGetCString(explainAnalyze); double fetchedExplainAnalyzeExecutionDuration = DatumGetFloat8(executionDuration); + double fetchedExplainAnalyzeTuples = DatumGetFloat8(executionTuples); + double fetchedExplainAnalyzeLoops = DatumGetFloat8(executionLoops); /* * Allocate fetchedExplainAnalyzePlan in the same context as the Task, since we are @@ -1712,6 +1865,20 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task, placementIndex; tupleDestination->originalTask->fetchedExplainAnalyzeExecutionDuration = fetchedExplainAnalyzeExecutionDuration; + + /* We should build tupleDestination in subPlan similar to the above */ + if (SubPlanExplainAnalyzeContext) + { + EnsureExplainOutputCapacity(taskId + 1); + SubPlanExplainOutput[taskId].explainOutput = + MemoryContextStrdup(SubPlanExplainAnalyzeContext, + fetchedExplainAnalyzePlan); + SubPlanExplainOutput[taskId].executionDuration = + fetchedExplainAnalyzeExecutionDuration; + SubPlanExplainOutput[taskId].executionNtuples = fetchedExplainAnalyzeTuples; + SubPlanExplainOutput[taskId].executionNloops = fetchedExplainAnalyzeLoops; + NumTasksOutput++; + } } else { @@ -1774,7 +1941,14 @@ ExplainAnalyzeDestTupleDescForQuery(TupleDestination *self, int queryNumber) bool RequestedForExplainAnalyze(CitusScanState *node) { - return (node->customScanState.ss.ps.state->es_instrument != 0); + /* + * When running a distributed plan—either the root plan or a subplan’s + * distributed fragment—we need to know if we’re under EXPLAIN ANALYZE. + * Subplans can’t receive the EXPLAIN ANALYZE flag directly, so we use + * SubPlanExplainAnalyzeContext as a flag to indicate that context. + */ + return (node->customScanState.ss.ps.state->es_instrument != 0) || + (SubPlanLevel > 0 && SubPlanExplainAnalyzeContext); } @@ -1933,7 +2107,8 @@ FetchPlanQueryForExplainAnalyze(const char *queryString, ParamListInfo params) } appendStringInfoString(fetchQuery, - "SELECT explain_analyze_output, execution_duration " + "SELECT explain_analyze_output, execution_duration, " + "execution_ntuples, execution_nloops " "FROM worker_last_saved_explain_analyze()"); return fetchQuery->data; @@ -2105,6 +2280,20 @@ ExplainOneQuery(Query *query, int cursorOptions, } +/* + * PlanStateAnalyzeWalker Tree walker callback that visits each PlanState node in the + * plan tree and extracts analyze statistics from CustomScanState tasks using + * ExtractAnalyzeStats. Always returns false to recurse into all children. + */ +static bool +PlanStateAnalyzeWalker(PlanState *planState, void *ctx) +{ + DistributedSubPlan *subplan = (DistributedSubPlan *) ctx; + ExtractAnalyzeStats(subplan, planState); + return false; +} + + /* * ExplainWorkerPlan produces explain output into es. If es->analyze, it also executes * the given plannedStmt and sends the results to dest. It puts total time to execute in @@ -2119,20 +2308,25 @@ ExplainOneQuery(Query *query, int cursorOptions, * destination. */ static void -ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es, +ExplainWorkerPlan(PlannedStmt *plannedstmt, DistributedSubPlan *subPlan, DestReceiver *dest, ExplainState *es, const char *queryString, ParamListInfo params, QueryEnvironment *queryEnv, const instr_time *planduration, #if PG_VERSION_NUM >= PG_VERSION_17 const BufferUsage *bufusage, const MemoryContextCounters *mem_counters, #endif - double *executionDurationMillisec) + double *executionDurationMillisec, + double *executionTuples, + double *executionLoops) { QueryDesc *queryDesc; instr_time starttime; double totaltime = 0; int eflags; int instrument_option = 0; + /* Sub-plan already executed; skipping execution */ + bool executeQuery = (es->analyze && !subPlan); + bool executeSubplan = (es->analyze && subPlan); Assert(plannedstmt->commandType != CMD_UTILITY); @@ -2174,7 +2368,7 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es ); /* Select execution options */ - if (es->analyze) + if (executeQuery) eflags = 0; /* default run-to-completion flags */ else eflags = EXEC_FLAG_EXPLAIN_ONLY; @@ -2183,7 +2377,7 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es ExecutorStart(queryDesc, eflags); /* Execute the plan for statistics if asked for */ - if (es->analyze) + if (executeQuery) { ScanDirection dir = ForwardScanDirection; @@ -2206,6 +2400,12 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es ExplainOpenGroup("Query", NULL, true, es); + if (executeSubplan) + { + ExtractAnalyzeStats(subPlan, queryDesc->planstate); + planstate_tree_walker(queryDesc->planstate, PlanStateAnalyzeWalker, (void *) subPlan); + } + /* Create textual dump of plan tree */ ExplainPrintPlan(es, queryDesc); @@ -2278,6 +2478,13 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es */ INSTR_TIME_SET_CURRENT(starttime); + if (executeQuery) + { + Instrumentation *instr = queryDesc->planstate->instrument; + *executionTuples = instr->ntuples; + *executionLoops = instr->nloops; + } + ExecutorEnd(queryDesc); FreeQueryDesc(queryDesc); @@ -2285,7 +2492,7 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es PopActiveSnapshot(); /* We need a CCI just in case query expanded to multiple plans */ - if (es->analyze) + if (executeQuery) CommandCounterIncrement(); totaltime += elapsed_time(&starttime); diff --git a/src/backend/distributed/sql/citus--13.1-1--13.2-1.sql b/src/backend/distributed/sql/citus--13.1-1--13.2-1.sql index 0373d3c40..2f507eb24 100644 --- a/src/backend/distributed/sql/citus--13.1-1--13.2-1.sql +++ b/src/backend/distributed/sql/citus--13.1-1--13.2-1.sql @@ -1,2 +1,3 @@ -- citus--13.1-1--13.2-1 -- bump version to 13.2-1 +#include "udfs/worker_last_saved_explain_analyze/13.2-1.sql" diff --git a/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql b/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql index 6f4ecd1ef..2212600f4 100644 --- a/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql +++ b/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql @@ -1,2 +1,3 @@ -- citus--13.2-1--13.1-1 -- downgrade version to 13.1-1 +#include "../udfs/worker_last_saved_explain_analyze/9.4-1.sql" diff --git a/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/13.2-1.sql b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/13.2-1.sql new file mode 100644 index 000000000..805dc83cc --- /dev/null +++ b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/13.2-1.sql @@ -0,0 +1,10 @@ + +DROP FUNCTION pg_catalog.worker_last_saved_explain_analyze(); + +CREATE OR REPLACE FUNCTION pg_catalog.worker_last_saved_explain_analyze() + RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION, + execution_ntuples DOUBLE PRECISION, execution_nloops DOUBLE PRECISION) + LANGUAGE C STRICT + AS 'citus'; +COMMENT ON FUNCTION pg_catalog.worker_last_saved_explain_analyze() IS + 'Returns the saved explain analyze output for the last run query'; diff --git a/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql index 17a5a15c5..037a17b92 100644 --- a/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql +++ b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql @@ -1,4 +1,6 @@ +DROP FUNCTION IF EXISTS pg_catalog.worker_last_saved_explain_analyze(); + CREATE OR REPLACE FUNCTION pg_catalog.worker_last_saved_explain_analyze() RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION) LANGUAGE C STRICT diff --git a/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/latest.sql b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/latest.sql index 17a5a15c5..805dc83cc 100644 --- a/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/latest.sql +++ b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/latest.sql @@ -1,6 +1,9 @@ +DROP FUNCTION pg_catalog.worker_last_saved_explain_analyze(); + CREATE OR REPLACE FUNCTION pg_catalog.worker_last_saved_explain_analyze() - RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION) + RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION, + execution_ntuples DOUBLE PRECISION, execution_nloops DOUBLE PRECISION) LANGUAGE C STRICT AS 'citus'; COMMENT ON FUNCTION pg_catalog.worker_last_saved_explain_analyze() IS diff --git a/src/backend/distributed/utils/citus_copyfuncs.c b/src/backend/distributed/utils/citus_copyfuncs.c index 51716cff3..aca376df9 100644 --- a/src/backend/distributed/utils/citus_copyfuncs.c +++ b/src/backend/distributed/utils/citus_copyfuncs.c @@ -147,6 +147,31 @@ CopyNodeDistributedSubPlan(COPYFUNC_ARGS) COPY_SCALAR_FIELD(subPlanId); COPY_NODE_FIELD(plan); + COPY_SCALAR_FIELD(bytesSentPerWorker); + COPY_SCALAR_FIELD(remoteWorkerCount); + COPY_SCALAR_FIELD(durationMillisecs); + COPY_SCALAR_FIELD(writeLocalFile); + + if (newnode->totalExplainOutput) + { + MemSet(newnode->totalExplainOutput, 0, sizeof(newnode->totalExplainOutput)); + } + + /* copy each SubPlanExplainOutput element */ + for (int i = 0; i < from->numTasksOutput; i++) + { + /* copy the explainOutput string pointer */ + COPY_STRING_FIELD(totalExplainOutput[i].explainOutput); + + /* copy the executionDuration (double) */ + COPY_SCALAR_FIELD(totalExplainOutput[i].executionDuration); + + /* copy the totalReceivedTupleData (uint64) */ + COPY_SCALAR_FIELD(totalExplainOutput[i].totalReceivedTupleData); + } + + COPY_SCALAR_FIELD(numTasksOutput); + COPY_SCALAR_FIELD(ntuples); } diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index 751063789..c19b0c3d4 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -215,6 +215,48 @@ OutDistributedSubPlan(OUTFUNC_ARGS) WRITE_UINT_FIELD(subPlanId); WRITE_NODE_FIELD(plan); + WRITE_UINT64_FIELD(bytesSentPerWorker); + WRITE_INT_FIELD(remoteWorkerCount); + WRITE_FLOAT_FIELD(durationMillisecs, "%.2f"); + WRITE_BOOL_FIELD(writeLocalFile); + + appendStringInfoString(str, " totalExplainOutput ["); + for (int i = 0; i < node->numTasksOutput; i++) + { + const SubPlanExplainOutputData *e = &node->totalExplainOutput[i]; + + /* skip empty slots */ + if (e->explainOutput == NULL && + e->executionDuration == 0 + && e->totalReceivedTupleData == 0) + { + continue; + } + + if (i > 0) + { + appendStringInfoChar(str, ' '); + } + + appendStringInfoChar(str, '('); + + /* string pointer – prints quoted or NULL */ + WRITE_STRING_FIELD(totalExplainOutput[i].explainOutput); + + /* double field */ + WRITE_FLOAT_FIELD(totalExplainOutput[i].executionDuration, "%.2f"); + + /* 64-bit unsigned – use the uint64 macro */ + WRITE_UINT64_FIELD(totalExplainOutput[i].totalReceivedTupleData); + + appendStringInfoChar(str, ')'); + } + + appendStringInfoChar(str, ']'); + + WRITE_INT_FIELD(numTasksOutput); + WRITE_FLOAT_FIELD(ntuples, "%.2f"); + } void diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 6708d9a64..b0b0288de 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -146,8 +146,8 @@ extern void ExecuteQueryStringIntoDestReceiver(const char *queryString, ParamLis DestReceiver *dest); extern void ExecuteQueryIntoDestReceiver(Query *query, ParamListInfo params, DestReceiver *dest); -extern void ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, - DestReceiver *dest); +extern uint64 ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, + DestReceiver *dest); extern void SetLocalMultiShardModifyModeToSequential(void); extern void EnsureSequentialMode(ObjectType objType); extern void SetLocalForceMaxQueryParallelization(void); diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index e5ec2205d..1040b4149 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -490,6 +490,24 @@ typedef struct DistributedPlan } DistributedPlan; +/* + * SubPlanExplainOutputData Holds the EXPLAIN ANALYZE output and collected + * statistics for a single task executed by a worker during distributed + * query execution. + * explainOutput — raw EXPLAIN ANALYZE output for the task + * executionDuration — wall‑clock time taken to run the task + * totalReceivedTupleData — total bytes of tuple data received from the worker + */ +typedef struct SubPlanExplainOutputData +{ + char *explainOutput; + double executionDuration; + double executionNtuples; + double executionNloops; + uint64 totalReceivedTupleData; +} SubPlanExplainOutputData; + + /* * DistributedSubPlan contains a subplan of a distributed plan. Subplans are * executed before the distributed query and their results are written to @@ -508,6 +526,9 @@ typedef struct DistributedSubPlan uint32 remoteWorkerCount; double durationMillisecs; bool writeLocalFile; + SubPlanExplainOutputData *totalExplainOutput; + uint32 numTasksOutput; /* actual size of the above array */ + double ntuples; /* total tuples produced */ } DistributedSubPlan; diff --git a/src/include/distributed/subplan_execution.h b/src/include/distributed/subplan_execution.h index d68db43ce..045e77bc6 100644 --- a/src/include/distributed/subplan_execution.h +++ b/src/include/distributed/subplan_execution.h @@ -17,7 +17,7 @@ extern int MaxIntermediateResult; extern int SubPlanLevel; -extern void ExecuteSubPlans(DistributedPlan *distributedPlan); +extern void ExecuteSubPlans(DistributedPlan *distributedPlan, bool explainAnalyzeEnabled); /** * IntermediateResultsHashEntry is used to store which nodes need to receive diff --git a/src/test/regress/expected/multi_explain.out b/src/test/regress/expected/multi_explain.out index bfcf29c4d..49027b217 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -2492,15 +2492,15 @@ Custom Scan (Citus Adaptive) (actual rows=1 loops=1) -> Distributed Subplan XXX_1 Intermediate Data Size: 100 bytes Result destination: Write locally - -> Custom Scan (Citus Adaptive) (actual rows=20 loops=1) + -> Custom Scan (Citus Adaptive) (actual rows=10 loops=1) Task Count: 4 - Tuple data received from nodes: 160 bytes + Tuple data received from nodes: 80 bytes Tasks Shown: One of 4 -> Task - Tuple data received from node: 64 bytes + Tuple data received from node: 32 bytes Node: host=localhost port=xxxxx dbname=regression - -> Insert on dist_table_570017 citus_table_alias (actual rows=8 loops=1) - -> Seq Scan on dist_table_570017 dist_table (actual rows=8 loops=1) + -> Insert on dist_table_570017 citus_table_alias (actual rows=4 loops=1) + -> Seq Scan on dist_table_570017 dist_table (actual rows=4 loops=1) Filter: (a IS NOT NULL) -> Distributed Subplan XXX_2 Intermediate Data Size: 150 bytes @@ -3228,6 +3228,159 @@ Custom Scan (Citus Adaptive) (actual rows=0 loops=1) -> Update on tbl_570036 tbl (actual rows=0 loops=1) -> Seq Scan on tbl_570036 tbl (actual rows=0 loops=1) Filter: (a = 1) +-- EXPLAIN ANALYZE shouldn't execute SubPlans twice (bug #4212) +SET search_path TO multi_explain; +CREATE TABLE test_subplans (x int primary key, y int); +SELECT create_distributed_table('test_subplans','x'); + +EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; +Custom Scan (Citus Adaptive) (actual rows=1 loops=1) + -> Distributed Subplan XXX_1 + Intermediate Data Size: 18 bytes + Result destination: Write locally + -> Custom Scan (Citus Adaptive) (actual rows=1 loops=1) + Task Count: 1 + Tuple data received from nodes: 16 bytes + Tasks Shown: All + -> Task + Tuple data received from node: 16 bytes + Node: host=localhost port=xxxxx dbname=regression + -> Insert on test_subplans_570038 (actual rows=1 loops=1) + -> Result (actual rows=1 loops=1) + Task Count: 1 + Tuple data received from nodes: 8 bytes + Tasks Shown: All + -> Task + Tuple data received from node: 8 bytes + Node: host=localhost port=xxxxx dbname=regression + -> Function Scan on read_intermediate_result intermediate_result (actual rows=1 loops=1) +-- Only one row must exist +SELECT * FROM test_subplans; +1|2 +-- Will fail with duplicate pk +EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; +ERROR: duplicate key value violates unique constraint "test_subplans_pkey_570038" +DETAIL: Key (x)=(1) already exists. +CONTEXT: while executing command on localhost:xxxxx +-- Test JSON format +TRUNCATE test_subplans; +EXPLAIN (FORMAT JSON, COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; +[ + { + "Plan": { + "Node Type": "Custom Scan", + "Custom Plan Provider": "Citus Adaptive", + "Parallel Aware": false, + "Async Capable": false, + "Actual Rows": 1, + "Actual Loops": 1, + "Distributed Query": { + "Subplans": [ + { + "Intermediate Data Size": "18 bytes", + "Result destination": "Write locally", + "PlannedStmt": [ + { + "Plan": { + "Node Type": "Custom Scan", + "Custom Plan Provider": "Citus Adaptive", + "Parallel Aware": false, + "Async Capable": false, + "Actual Rows": 1, + "Actual Loops": 1, + "Distributed Query": { + "Job": { + "Task Count": 1, + "Tuple data received from nodes": "16 bytes", + "Tasks Shown": "All", + "Tasks": [ + { + "Tuple data received from node": "16 bytes", + "Node": "host=localhost port=xxxxx dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "ModifyTable", + "Operation": "Insert", + "Parallel Aware": false, + "Async Capable": false, + "Relation Name": "test_subplans_570038", + "Alias": "test_subplans_570038", + "Actual Rows": 1, + "Actual Loops": 1, + "Plans": [ + { + "Node Type": "Result", + "Parent Relationship": "Outer", + "Parallel Aware": false, + "Async Capable": false, + "Actual Rows": 1, + "Actual Loops": 1 + } + ] + }, + "Triggers": [ + ] + } + ] + + ] + } + ] + } + } + }, + "Triggers": [ + ] + } + ] + } + ], + "Job": { + "Task Count": 1, + "Tuple data received from nodes": "8 bytes", + "Tasks Shown": "All", + "Tasks": [ + { + "Tuple data received from node": "8 bytes", + "Node": "host=localhost port=xxxxx dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "Function Scan", + "Parallel Aware": false, + "Async Capable": false, + "Function Name": "read_intermediate_result", + "Alias": "intermediate_result", + "Actual Rows": 1, + "Actual Loops": 1 + }, + "Triggers": [ + ] + } + ] + + ] + } + ] + } + } + }, + "Triggers": [ + ] + } +] +-- Only one row must exist +SELECT * FROM test_subplans; +1|2 -- check when auto explain + analyze is enabled, we do not allow local execution. CREATE SCHEMA test_auto_explain; SET search_path TO 'test_auto_explain'; diff --git a/src/test/regress/expected/multi_explain_0.out b/src/test/regress/expected/multi_explain_0.out index 4d3acd14d..00a8309a9 100644 --- a/src/test/regress/expected/multi_explain_0.out +++ b/src/test/regress/expected/multi_explain_0.out @@ -2484,15 +2484,15 @@ Custom Scan (Citus Adaptive) (actual rows=1 loops=1) -> Distributed Subplan XXX_1 Intermediate Data Size: 100 bytes Result destination: Write locally - -> Custom Scan (Citus Adaptive) (actual rows=20 loops=1) + -> Custom Scan (Citus Adaptive) (actual rows=10 loops=1) Task Count: 4 - Tuple data received from nodes: 160 bytes + Tuple data received from nodes: 80 bytes Tasks Shown: One of 4 -> Task - Tuple data received from node: 64 bytes + Tuple data received from node: 32 bytes Node: host=localhost port=xxxxx dbname=regression - -> Insert on dist_table_570017 citus_table_alias (actual rows=8 loops=1) - -> Seq Scan on dist_table_570017 dist_table (actual rows=8 loops=1) + -> Insert on dist_table_570017 citus_table_alias (actual rows=4 loops=1) + -> Seq Scan on dist_table_570017 dist_table (actual rows=4 loops=1) Filter: (a IS NOT NULL) -> Distributed Subplan XXX_2 Intermediate Data Size: 150 bytes @@ -3217,6 +3217,159 @@ Custom Scan (Citus Adaptive) (actual rows=0 loops=1) -> Update on tbl_570036 tbl (actual rows=0 loops=1) -> Seq Scan on tbl_570036 tbl (actual rows=0 loops=1) Filter: (a = 1) +-- EXPLAIN ANALYZE shouldn't execute SubPlans twice (bug #4212) +SET search_path TO multi_explain; +CREATE TABLE test_subplans (x int primary key, y int); +SELECT create_distributed_table('test_subplans','x'); + +EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; +Custom Scan (Citus Adaptive) (actual rows=1 loops=1) + -> Distributed Subplan XXX_1 + Intermediate Data Size: 18 bytes + Result destination: Write locally + -> Custom Scan (Citus Adaptive) (actual rows=1 loops=1) + Task Count: 1 + Tuple data received from nodes: 16 bytes + Tasks Shown: All + -> Task + Tuple data received from node: 16 bytes + Node: host=localhost port=xxxxx dbname=regression + -> Insert on test_subplans_570038 (actual rows=1 loops=1) + -> Result (actual rows=1 loops=1) + Task Count: 1 + Tuple data received from nodes: 8 bytes + Tasks Shown: All + -> Task + Tuple data received from node: 8 bytes + Node: host=localhost port=xxxxx dbname=regression + -> Function Scan on read_intermediate_result intermediate_result (actual rows=1 loops=1) +-- Only one row must exist +SELECT * FROM test_subplans; +1|2 +-- Will fail with duplicate pk +EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; +ERROR: duplicate key value violates unique constraint "test_subplans_pkey_570038" +DETAIL: Key (x)=(1) already exists. +CONTEXT: while executing command on localhost:xxxxx +-- Test JSON format +TRUNCATE test_subplans; +EXPLAIN (FORMAT JSON, COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; +[ + { + "Plan": { + "Node Type": "Custom Scan", + "Custom Plan Provider": "Citus Adaptive", + "Parallel Aware": false, + "Async Capable": false, + "Actual Rows": 1, + "Actual Loops": 1, + "Distributed Query": { + "Subplans": [ + { + "Intermediate Data Size": "18 bytes", + "Result destination": "Write locally", + "PlannedStmt": [ + { + "Plan": { + "Node Type": "Custom Scan", + "Custom Plan Provider": "Citus Adaptive", + "Parallel Aware": false, + "Async Capable": false, + "Actual Rows": 1, + "Actual Loops": 1, + "Distributed Query": { + "Job": { + "Task Count": 1, + "Tuple data received from nodes": "16 bytes", + "Tasks Shown": "All", + "Tasks": [ + { + "Tuple data received from node": "16 bytes", + "Node": "host=localhost port=xxxxx dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "ModifyTable", + "Operation": "Insert", + "Parallel Aware": false, + "Async Capable": false, + "Relation Name": "test_subplans_570038", + "Alias": "test_subplans_570038", + "Actual Rows": 1, + "Actual Loops": 1, + "Plans": [ + { + "Node Type": "Result", + "Parent Relationship": "Outer", + "Parallel Aware": false, + "Async Capable": false, + "Actual Rows": 1, + "Actual Loops": 1 + } + ] + }, + "Triggers": [ + ] + } + ] + + ] + } + ] + } + } + }, + "Triggers": [ + ] + } + ] + } + ], + "Job": { + "Task Count": 1, + "Tuple data received from nodes": "8 bytes", + "Tasks Shown": "All", + "Tasks": [ + { + "Tuple data received from node": "8 bytes", + "Node": "host=localhost port=xxxxx dbname=regression", + "Remote Plan": [ + [ + { + "Plan": { + "Node Type": "Function Scan", + "Parallel Aware": false, + "Async Capable": false, + "Function Name": "read_intermediate_result", + "Alias": "intermediate_result", + "Actual Rows": 1, + "Actual Loops": 1 + }, + "Triggers": [ + ] + } + ] + + ] + } + ] + } + } + }, + "Triggers": [ + ] + } +] +-- Only one row must exist +SELECT * FROM test_subplans; +1|2 -- check when auto explain + analyze is enabled, we do not allow local execution. CREATE SCHEMA test_auto_explain; SET search_path TO 'test_auto_explain'; diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 4e8e927f4..defe41f0d 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -1503,7 +1503,9 @@ ALTER EXTENSION citus UPDATE TO '13.2-1'; SELECT * FROM multi_extension.print_extension_changes(); previous_object | current_object --------------------------------------------------------------------- -(0 rows) + function worker_last_saved_explain_analyze() TABLE(explain_analyze_output text, execution_duration double precision) | + | function worker_last_saved_explain_analyze() TABLE(explain_analyze_output text, execution_duration double precision, execution_ntuples double precision, execution_nloops double precision) +(2 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version diff --git a/src/test/regress/expected/stat_counters.out b/src/test/regress/expected/stat_counters.out index a27eb3241..25327d4f7 100644 --- a/src/test/regress/expected/stat_counters.out +++ b/src/test/regress/expected/stat_counters.out @@ -721,13 +721,11 @@ CALL exec_query_and_check_query_counters($$ 0, 0 ); -- same with explain analyze --- --- this time, query_execution_multi_shard is incremented twice because of #4212 CALL exec_query_and_check_query_counters($$ EXPLAIN (ANALYZE) SELECT * FROM (SELECT * FROM dist_table OFFSET 0) q $$, - 1, 2 + 1, 1 ); CALL exec_query_and_check_query_counters($$ DELETE FROM dist_table WHERE a = 1 @@ -1041,9 +1039,6 @@ PL/pgSQL function exec_query_and_check_query_counters(text,bigint,bigint) line X -- A similar one but without the insert, so we would normally expect 2 increments -- for query_execution_single_shard and 2 for query_execution_multi_shard instead -- of 3 since the insert is not there anymore. --- --- But this time we observe more counter increments because we execute the subplans --- twice because of #4212. CALL exec_query_and_check_query_counters($$ EXPLAIN (ANALYZE) -- single-shard subplan (whole cte) @@ -1057,7 +1052,7 @@ CALL exec_query_and_check_query_counters($$ FROM (SELECT * FROM dist_table_1 ORDER BY a LIMIT 16) q -- multi-shard subplan (subquery q) JOIN cte ON q.a = cte.a $$, - 3, 4 + 2, 2 ); -- safe to push-down CALL exec_query_and_check_query_counters($$ diff --git a/src/test/regress/sql/multi_explain.sql b/src/test/regress/sql/multi_explain.sql index 65ca6f5da..c6502fec8 100644 --- a/src/test/regress/sql/multi_explain.sql +++ b/src/test/regress/sql/multi_explain.sql @@ -1166,6 +1166,32 @@ PREPARE q2(int_wrapper_type) AS WITH a AS (UPDATE tbl SET b = $1 WHERE a = 1 RET EXPLAIN (COSTS false) EXECUTE q2('(1)'); EXPLAIN :default_analyze_flags EXECUTE q2('(1)'); +-- EXPLAIN ANALYZE shouldn't execute SubPlans twice (bug #4212) +SET search_path TO multi_explain; +CREATE TABLE test_subplans (x int primary key, y int); +SELECT create_distributed_table('test_subplans','x'); + +EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; + +-- Only one row must exist +SELECT * FROM test_subplans; + +-- Will fail with duplicate pk +EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; + +-- Test JSON format +TRUNCATE test_subplans; +EXPLAIN (FORMAT JSON, COSTS off, ANALYZE on, TIMING off, SUMMARY off) +WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *) +SELECT * FROM a; + +-- Only one row must exist +SELECT * FROM test_subplans; + -- check when auto explain + analyze is enabled, we do not allow local execution. CREATE SCHEMA test_auto_explain; SET search_path TO 'test_auto_explain'; diff --git a/src/test/regress/sql/stat_counters.sql b/src/test/regress/sql/stat_counters.sql index 3376ba6c7..18f4b8aac 100644 --- a/src/test/regress/sql/stat_counters.sql +++ b/src/test/regress/sql/stat_counters.sql @@ -476,13 +476,11 @@ CALL exec_query_and_check_query_counters($$ ); -- same with explain analyze --- --- this time, query_execution_multi_shard is incremented twice because of #4212 CALL exec_query_and_check_query_counters($$ EXPLAIN (ANALYZE) SELECT * FROM (SELECT * FROM dist_table OFFSET 0) q $$, - 1, 2 + 1, 1 ); CALL exec_query_and_check_query_counters($$ @@ -807,9 +805,6 @@ CALL exec_query_and_check_query_counters($$ -- A similar one but without the insert, so we would normally expect 2 increments -- for query_execution_single_shard and 2 for query_execution_multi_shard instead -- of 3 since the insert is not there anymore. --- --- But this time we observe more counter increments because we execute the subplans --- twice because of #4212. CALL exec_query_and_check_query_counters($$ EXPLAIN (ANALYZE) -- single-shard subplan (whole cte) @@ -823,7 +818,7 @@ CALL exec_query_and_check_query_counters($$ FROM (SELECT * FROM dist_table_1 ORDER BY a LIMIT 16) q -- multi-shard subplan (subquery q) JOIN cte ON q.a = cte.a $$, - 3, 4 + 2, 2 ); -- safe to push-down From c183634207e522818f00f4a257f83a0f1b439f5e Mon Sep 17 00:00:00 2001 From: Onur Tirtir Date: Thu, 31 Jul 2025 13:30:12 +0300 Subject: [PATCH 11/13] Move "DROP FUNCTION" for older version of UDF to correct file (#8085) We never update an older version of a SQL object for consistency across release tags, so this commit moves "DROP FUNCTION .." for the older version of "pg_catalog.worker_last_saved_explain_analyze();" to the appropriate migration script. See https://github.com/citusdata/citus/pull/8017. --- .../distributed/sql/downgrades/citus--13.2-1--13.1-1.sql | 2 ++ .../sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql | 2 -- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql b/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql index 2212600f4..de26b790a 100644 --- a/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql +++ b/src/backend/distributed/sql/downgrades/citus--13.2-1--13.1-1.sql @@ -1,3 +1,5 @@ -- citus--13.2-1--13.1-1 -- downgrade version to 13.1-1 + +DROP FUNCTION IF EXISTS pg_catalog.worker_last_saved_explain_analyze(); #include "../udfs/worker_last_saved_explain_analyze/9.4-1.sql" diff --git a/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql index 037a17b92..17a5a15c5 100644 --- a/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql +++ b/src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/9.4-1.sql @@ -1,6 +1,4 @@ -DROP FUNCTION IF EXISTS pg_catalog.worker_last_saved_explain_analyze(); - CREATE OR REPLACE FUNCTION pg_catalog.worker_last_saved_explain_analyze() RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION) LANGUAGE C STRICT From f0789bd388be193962b32161989c48e8cd3afa8c Mon Sep 17 00:00:00 2001 From: manaldush Date: Tue, 5 Aug 2025 13:03:35 +0300 Subject: [PATCH 12/13] Fix memory corruptions that could happen when a Citus downgrade is followed by an upgrade (#7950) DESCRIPTION: Fixes potential memory corruptions that could happen when a Citus downgrade is followed by a Citus upgrade. In case of citus downgrade and further upgrade citus crash with core dump. The reason is that citus hardcoded number of columns in pg_dist_partition table, but in case of downgrade and following update table can have more columns, and some of then can be marked as dropped. Patch suggest decision for this problem with using tupleDescriptor->nattrs(postgres internal approach). Fixes #7933. --------- Co-authored-by: Onur Tirtir --- .../distributed/cdc/cdc_decoder_utils.c | 10 +- .../distributed/metadata/metadata_cache.c | 54 +++++--- .../distributed/metadata/metadata_sync.c | 8 +- .../distributed/metadata/metadata_utility.c | 115 ++++++++++++------ .../distributed/utils/colocation_utils.c | 31 +++-- src/include/distributed/metadata_utility.h | 1 + 6 files changed, 149 insertions(+), 70 deletions(-) diff --git a/src/backend/distributed/cdc/cdc_decoder_utils.c b/src/backend/distributed/cdc/cdc_decoder_utils.c index b571d18b9..9053d1b68 100644 --- a/src/backend/distributed/cdc/cdc_decoder_utils.c +++ b/src/backend/distributed/cdc/cdc_decoder_utils.c @@ -346,12 +346,12 @@ CdcIsReferenceTableViaCatalog(Oid relationId) return false; } - Datum datumArray[Natts_pg_dist_partition]; - bool isNullArray[Natts_pg_dist_partition]; - Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); + heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray); if (isNullArray[Anum_pg_dist_partition_partmethod - 1] || @@ -363,6 +363,8 @@ CdcIsReferenceTableViaCatalog(Oid relationId) */ heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); return false; } @@ -374,6 +376,8 @@ CdcIsReferenceTableViaCatalog(Oid relationId) heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); /* * A table is a reference table when its partition method is 'none' diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 79cc61092..8fd39d3b7 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -729,12 +729,13 @@ PartitionMethodViaCatalog(Oid relationId) return DISTRIBUTE_BY_INVALID; } - Datum datumArray[Natts_pg_dist_partition]; - bool isNullArray[Natts_pg_dist_partition]; - Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); + heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray); if (isNullArray[Anum_pg_dist_partition_partmethod - 1]) @@ -742,6 +743,8 @@ PartitionMethodViaCatalog(Oid relationId) /* partition method cannot be NULL, still let's make sure */ heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); return DISTRIBUTE_BY_INVALID; } @@ -750,6 +753,8 @@ PartitionMethodViaCatalog(Oid relationId) heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); return partitionMethodChar; } @@ -768,12 +773,12 @@ PartitionColumnViaCatalog(Oid relationId) return NULL; } - Datum datumArray[Natts_pg_dist_partition]; - bool isNullArray[Natts_pg_dist_partition]; - Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); + heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray); if (isNullArray[Anum_pg_dist_partition_partkey - 1]) @@ -781,6 +786,8 @@ PartitionColumnViaCatalog(Oid relationId) /* partition key cannot be NULL, still let's make sure */ heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); return NULL; } @@ -795,6 +802,8 @@ PartitionColumnViaCatalog(Oid relationId) heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); return partitionColumn; } @@ -813,12 +822,13 @@ ColocationIdViaCatalog(Oid relationId) return INVALID_COLOCATION_ID; } - Datum datumArray[Natts_pg_dist_partition]; - bool isNullArray[Natts_pg_dist_partition]; - Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); + heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray); if (isNullArray[Anum_pg_dist_partition_colocationid - 1]) @@ -826,6 +836,8 @@ ColocationIdViaCatalog(Oid relationId) /* colocation id cannot be NULL, still let's make sure */ heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); return INVALID_COLOCATION_ID; } @@ -834,6 +846,8 @@ ColocationIdViaCatalog(Oid relationId) heap_freetuple(partitionTuple); table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); return colocationId; } @@ -1741,10 +1755,11 @@ BuildCitusTableCacheEntry(Oid relationId) } MemoryContext oldContext = NULL; - Datum datumArray[Natts_pg_dist_partition]; - bool isNullArray[Natts_pg_dist_partition]; TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); + heap_deform_tuple(distPartitionTuple, tupleDescriptor, datumArray, isNullArray); CitusTableCacheEntry *cacheEntry = @@ -1797,7 +1812,7 @@ BuildCitusTableCacheEntry(Oid relationId) cacheEntry->replicationModel = DatumGetChar(replicationModelDatum); } - if (isNullArray[Anum_pg_dist_partition_autoconverted - 1]) + if (isNullArray[GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor)]) { /* * We don't expect this to happen, but set it to false (the default value) @@ -1808,7 +1823,7 @@ BuildCitusTableCacheEntry(Oid relationId) else { cacheEntry->autoConverted = DatumGetBool( - datumArray[Anum_pg_dist_partition_autoconverted - 1]); + datumArray[GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor)]); } heap_freetuple(distPartitionTuple); @@ -1852,6 +1867,9 @@ BuildCitusTableCacheEntry(Oid relationId) table_close(pgDistPartition, NoLock); + pfree(datumArray); + pfree(isNullArray); + cacheEntry->isValid = true; return cacheEntry; @@ -5011,10 +5029,13 @@ CitusTableTypeIdList(CitusTableType citusTableType) TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); HeapTuple heapTuple = systable_getnext(scanDescriptor); + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); while (HeapTupleIsValid(heapTuple)) { - bool isNullArray[Natts_pg_dist_partition]; - Datum datumArray[Natts_pg_dist_partition]; + memset(datumArray, 0, tupleDescriptor->natts * sizeof(Datum)); + memset(isNullArray, 0, tupleDescriptor->natts * sizeof(bool)); + heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray); Datum partMethodDatum = datumArray[Anum_pg_dist_partition_partmethod - 1]; @@ -5038,6 +5059,9 @@ CitusTableTypeIdList(CitusTableType citusTableType) heapTuple = systable_getnext(scanDescriptor); } + pfree(datumArray); + pfree(isNullArray); + systable_endscan(scanDescriptor); table_close(pgDistPartition, AccessShareLock); diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index f73856169..e3b655ab0 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -573,13 +573,17 @@ FetchRelationIdFromPgPartitionHeapTuple(HeapTuple heapTuple, TupleDesc tupleDesc { Assert(heapTuple->t_tableOid == DistPartitionRelationId()); - bool isNullArray[Natts_pg_dist_partition]; - Datum datumArray[Natts_pg_dist_partition]; + Datum *datumArray = (Datum *) palloc(tupleDesc->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDesc->natts * sizeof(bool)); + heap_deform_tuple(heapTuple, tupleDesc, datumArray, isNullArray); Datum relationIdDatum = datumArray[Anum_pg_dist_partition_logicalrelid - 1]; Oid relationId = DatumGetObjectId(relationIdDatum); + pfree(datumArray); + pfree(isNullArray); + return relationId; } diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index 0c3dbbda3..2b8bd0d1c 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -812,6 +812,7 @@ GenerateSizeQueryOnMultiplePlacements(List *shardIntervalList, { partitionedShardNames = lappend(partitionedShardNames, quotedShardName); } + /* for non-partitioned tables, we will use Postgres' size functions */ else { @@ -1919,23 +1920,22 @@ InsertIntoPgDistPartition(Oid relationId, char distributionMethod, { char *distributionColumnString = NULL; - Datum newValues[Natts_pg_dist_partition]; - bool newNulls[Natts_pg_dist_partition]; - /* open system catalog and insert new tuple */ Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock); + TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + + Datum *newValues = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum)); + bool *newNulls = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); /* form new tuple for pg_dist_partition */ - memset(newValues, 0, sizeof(newValues)); - memset(newNulls, false, sizeof(newNulls)); - newValues[Anum_pg_dist_partition_logicalrelid - 1] = ObjectIdGetDatum(relationId); newValues[Anum_pg_dist_partition_partmethod - 1] = CharGetDatum(distributionMethod); newValues[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId); newValues[Anum_pg_dist_partition_repmodel - 1] = CharGetDatum(replicationModel); - newValues[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(autoConverted); + newValues[GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor)] = + BoolGetDatum(autoConverted); /* set partkey column to NULL for reference tables */ if (distributionMethod != DISTRIBUTE_BY_NONE) @@ -1951,7 +1951,7 @@ InsertIntoPgDistPartition(Oid relationId, char distributionMethod, newNulls[Anum_pg_dist_partition_partkey - 1] = true; } - HeapTuple newTuple = heap_form_tuple(RelationGetDescr(pgDistPartition), newValues, + HeapTuple newTuple = heap_form_tuple(tupleDescriptor, newValues, newNulls); /* finally insert tuple, build index entries & register cache invalidation */ @@ -1963,6 +1963,9 @@ InsertIntoPgDistPartition(Oid relationId, char distributionMethod, CommandCounterIncrement(); table_close(pgDistPartition, NoLock); + + pfree(newValues); + pfree(newNulls); } @@ -2154,13 +2157,13 @@ UpdatePlacementGroupId(uint64 placementId, int groupId) ScanKeyData scanKey[1]; int scanKeyCount = 1; bool indexOK = true; - Datum values[Natts_pg_dist_placement]; - bool isnull[Natts_pg_dist_placement]; - bool replace[Natts_pg_dist_placement]; bool colIsNull = false; Relation pgDistPlacement = table_open(DistPlacementRelationId(), RowExclusiveLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement); + Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum)); + bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_placementid, BTEqualStrategyNumber, F_INT8EQ, Int64GetDatum(placementId)); @@ -2177,8 +2180,6 @@ UpdatePlacementGroupId(uint64 placementId, int groupId) placementId))); } - memset(replace, 0, sizeof(replace)); - values[Anum_pg_dist_placement_groupid - 1] = Int32GetDatum(groupId); isnull[Anum_pg_dist_placement_groupid - 1] = false; replace[Anum_pg_dist_placement_groupid - 1] = true; @@ -2197,6 +2198,10 @@ UpdatePlacementGroupId(uint64 placementId, int groupId) systable_endscan(scanDescriptor); table_close(pgDistPlacement, NoLock); + + pfree(values); + pfree(isnull); + pfree(replace); } @@ -2210,12 +2215,13 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted) ScanKeyData scanKey[1]; int scanKeyCount = 1; bool indexOK = true; - Datum values[Natts_pg_dist_partition]; - bool isnull[Natts_pg_dist_partition]; - bool replace[Natts_pg_dist_partition]; Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum)); + bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid, BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(citusTableId)); @@ -2231,11 +2237,10 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted) citusTableId))); } - memset(replace, 0, sizeof(replace)); - - values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(autoConverted); - isnull[Anum_pg_dist_partition_autoconverted - 1] = false; - replace[Anum_pg_dist_partition_autoconverted - 1] = true; + int autoconvertedindex = GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor); + values[autoconvertedindex] = BoolGetDatum(autoConverted); + isnull[autoconvertedindex] = false; + replace[autoconvertedindex] = true; heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace); @@ -2247,6 +2252,10 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted) systable_endscan(scanDescriptor); table_close(pgDistPartition, NoLock); + + pfree(values); + pfree(isnull); + pfree(replace); } @@ -2286,12 +2295,13 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut ScanKeyData scanKey[1]; int scanKeyCount = 1; bool indexOK = true; - Datum values[Natts_pg_dist_partition]; - bool isnull[Natts_pg_dist_partition]; - bool replace[Natts_pg_dist_partition]; Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum)); + bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid, BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(relationId)); @@ -2307,8 +2317,6 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut relationId))); } - memset(replace, 0, sizeof(replace)); - replace[Anum_pg_dist_partition_partmethod - 1] = true; values[Anum_pg_dist_partition_partmethod - 1] = CharGetDatum(distributionMethod); isnull[Anum_pg_dist_partition_partmethod - 1] = false; @@ -2317,9 +2325,10 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId); isnull[Anum_pg_dist_partition_colocationid - 1] = false; - replace[Anum_pg_dist_partition_autoconverted - 1] = true; - values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(false); - isnull[Anum_pg_dist_partition_autoconverted - 1] = false; + int autoconvertedindex = GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor); + replace[autoconvertedindex] = true; + values[autoconvertedindex] = BoolGetDatum(false); + isnull[autoconvertedindex] = false; char *distributionColumnString = nodeToString((Node *) distributionColumn); @@ -2337,6 +2346,10 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut systable_endscan(scanDescriptor); table_close(pgDistPartition, NoLock); + + pfree(values); + pfree(isnull); + pfree(replace); } @@ -2380,12 +2393,13 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca ScanKeyData scanKey[1]; int scanKeyCount = 1; bool indexOK = true; - Datum values[Natts_pg_dist_partition]; - bool isnull[Natts_pg_dist_partition]; - bool replace[Natts_pg_dist_partition]; Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum)); + bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid, BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(relationId)); @@ -2401,8 +2415,6 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca relationId))); } - memset(replace, 0, sizeof(replace)); - values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId); isnull[Anum_pg_dist_partition_colocationid - 1] = false; replace[Anum_pg_dist_partition_colocationid - 1] = true; @@ -2411,9 +2423,10 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca isnull[Anum_pg_dist_partition_repmodel - 1] = false; replace[Anum_pg_dist_partition_repmodel - 1] = true; - values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(autoConverted); - isnull[Anum_pg_dist_partition_autoconverted - 1] = false; - replace[Anum_pg_dist_partition_autoconverted - 1] = true; + int autoconvertedindex = GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor); + values[autoconvertedindex] = BoolGetDatum(autoConverted); + isnull[autoconvertedindex] = false; + replace[autoconvertedindex] = true; heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace); @@ -2424,6 +2437,10 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca systable_endscan(scanDescriptor); table_close(pgDistPartition, NoLock); + + pfree(values); + pfree(isnull); + pfree(replace); } @@ -3149,8 +3166,8 @@ ScheduleBackgroundTask(int64 jobId, Oid owner, char *command, int dependingTaskC values[Anum_pg_dist_background_task_nodes_involved - 1] = IntArrayToDatum(nodesInvolvedCount, nodesInvolved); - nulls[Anum_pg_dist_background_task_nodes_involved - 1] = (nodesInvolvedCount == - 0); + nulls[Anum_pg_dist_background_task_nodes_involved - 1] = + (nodesInvolvedCount == 0); HeapTuple newTuple = heap_form_tuple(RelationGetDescr(pgDistBackgroundTask), values, nulls); @@ -4420,3 +4437,23 @@ UnblockDependingBackgroundTasks(BackgroundTask *task) table_close(pgDistBackgroundTasksDepend, NoLock); } + + +/* + * GetAutoConvertedAttrIndexInPgDistPartition returns attrnum for autoconverted attr. + * + * autoconverted attr was added to table pg_dist_partition using alter operation after + * the version where Citus started supporting downgrades, and it's only column that we've + * introduced to pg_dist_partition since then. + * + * And in case of a downgrade + upgrade, tupleDesc->natts becomes greater than + * Natts_pg_dist_partition and when this happens, then we know that attrnum autoconverted is + * not Anum_pg_dist_partition_autoconverted anymore but tupleDesc->natts - 1. + */ +int +GetAutoConvertedAttrIndexInPgDistPartition(TupleDesc tupleDesc) +{ + return TupleDescSize(tupleDesc) == Natts_pg_dist_partition + ? (Anum_pg_dist_partition_autoconverted - 1) + : tupleDesc->natts - 1; +} diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index 5f031b2b5..af507d5b9 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -815,13 +815,14 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId, bool indexOK = true; int scanKeyCount = 1; ScanKeyData scanKey[1]; - Datum values[Natts_pg_dist_partition]; - bool isNull[Natts_pg_dist_partition]; - bool replace[Natts_pg_dist_partition]; Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock); TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition); + Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum)); + bool *isNull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool)); + ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid, BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(distributedRelationId)); @@ -838,10 +839,6 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId, distributedRelationName))); } - memset(values, 0, sizeof(values)); - memset(isNull, false, sizeof(isNull)); - memset(replace, false, sizeof(replace)); - values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId); isNull[Anum_pg_dist_partition_colocationid - 1] = false; replace[Anum_pg_dist_partition_colocationid - 1] = true; @@ -858,6 +855,10 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId, systable_endscan(scanDescriptor); table_close(pgDistPartition, NoLock); + pfree(values); + pfree(isNull); + pfree(replace); + bool shouldSyncMetadata = ShouldSyncTableMetadata(distributedRelationId); if (shouldSyncMetadata && !localOnly) { @@ -998,10 +999,12 @@ ColocationGroupTableList(uint32 colocationId, uint32 count) indexOK, NULL, scanKeyCount, scanKey); HeapTuple heapTuple = systable_getnext(scanDescriptor); + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); while (HeapTupleIsValid(heapTuple)) { - bool isNullArray[Natts_pg_dist_partition]; - Datum datumArray[Natts_pg_dist_partition]; + memset(datumArray, 0, tupleDescriptor->natts * sizeof(Datum)); + memset(isNullArray, 0, tupleDescriptor->natts * sizeof(bool)); heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray); Oid colocatedTableId = DatumGetObjectId( datumArray[Anum_pg_dist_partition_logicalrelid - 1]); @@ -1020,6 +1023,8 @@ ColocationGroupTableList(uint32 colocationId, uint32 count) break; } } + pfree(datumArray); + pfree(isNullArray); systable_endscan(scanDescriptor); table_close(pgDistPartition, AccessShareLock); @@ -1192,10 +1197,12 @@ ColocatedTableId(int32 colocationId) indexOK, NULL, scanKeyCount, scanKey); HeapTuple heapTuple = systable_getnext(scanDescriptor); + Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum)); + bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool)); while (HeapTupleIsValid(heapTuple)) { - bool isNullArray[Natts_pg_dist_partition]; - Datum datumArray[Natts_pg_dist_partition]; + memset(datumArray, 0, tupleDescriptor->natts * sizeof(Datum)); + memset(isNullArray, 0, tupleDescriptor->natts * sizeof(bool)); heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray); colocatedTableId = DatumGetObjectId( datumArray[Anum_pg_dist_partition_logicalrelid - 1]); @@ -1223,6 +1230,8 @@ ColocatedTableId(int32 colocationId) heapTuple = systable_getnext(scanDescriptor); } + pfree(datumArray); + pfree(isNullArray); systable_endscan(scanDescriptor); table_close(pgDistPartition, AccessShareLock); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 38c13eb51..a507138d2 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -466,4 +466,5 @@ extern bool IsBackgroundJobStatusTerminal(BackgroundJobStatus status); extern bool IsBackgroundTaskStatusTerminal(BackgroundTaskStatus status); extern Oid BackgroundJobStatusOid(BackgroundJobStatus status); extern Oid BackgroundTaskStatusOid(BackgroundTaskStatus status); +extern int GetAutoConvertedAttrIndexInPgDistPartition(TupleDesc tupleDEsc); #endif /* METADATA_UTILITY_H */ From 3d8fd337e5c3161a828c84b11fd5d97008ee6b8f Mon Sep 17 00:00:00 2001 From: eaydingol <60466783+eaydingol@users.noreply.github.com> Date: Wed, 6 Aug 2025 16:13:14 +0300 Subject: [PATCH 13/13] Check outer table partition column (#8092) DESCRIPTION: Introduce a new check to push down a query including union and outer join to fix #8091 . In "SafeToPushdownUnionSubquery", we check if the distribution column of the outer relation is in the target list. --- .../relation_restriction_equivalence.c | 82 +++++++++++ src/test/regress/citus_tests/run_test.py | 9 ++ ...lect_behavioral_analytics_create_table.out | 0 .../multi_subquery_in_where_clause.out | 2 +- .../regress/expected/multi_subquery_misc.out | 3 +- .../regress/expected/multi_subquery_union.out | 4 +- src/test/regress/expected/union_pushdown.out | 135 ++++++++++++++++++ .../sql/multi_subquery_in_where_clause.sql | 2 +- src/test/regress/sql/multi_subquery_misc.sql | 3 +- src/test/regress/sql/multi_subquery_union.sql | 5 +- src/test/regress/sql/union_pushdown.sql | 44 ++++++ 11 files changed, 278 insertions(+), 11 deletions(-) create mode 100644 src/test/regress/expected/multi_insert_select_behavioral_analytics_create_table.out diff --git a/src/backend/distributed/planner/relation_restriction_equivalence.c b/src/backend/distributed/planner/relation_restriction_equivalence.c index 89516640a..94c99ef20 100644 --- a/src/backend/distributed/planner/relation_restriction_equivalence.c +++ b/src/backend/distributed/planner/relation_restriction_equivalence.c @@ -171,6 +171,14 @@ static bool FindQueryContainingRTEIdentityInternal(Node *node, static int ParentCountPriorToAppendRel(List *appendRelList, AppendRelInfo *appendRelInfo); +static bool PartitionColumnSelectedForOuterJoin(Query *query, + RelationRestrictionContext * + restrictionContext, + JoinRestrictionContext * + joinRestrictionContext); + +static bool PartitionColumnIsInTargetList(Query *query, JoinRestriction *joinRestriction, + RelationRestrictionContext *restrictionContext); /* * AllDistributionKeysInQueryAreEqual returns true if either @@ -391,6 +399,80 @@ SafeToPushdownUnionSubquery(Query *originalQuery, return false; } + if (!PartitionColumnSelectedForOuterJoin(originalQuery, + restrictionContext, + joinRestrictionContext)) + { + /* outer join does not select partition column of outer relation */ + return false; + } + return true; +} + + +/* + * PartitionColumnSelectedForOuterJoin checks whether the partition column of + * the outer relation is selected in the target list of the query. + * + * If there is no outer join, it returns true. + */ +static bool +PartitionColumnSelectedForOuterJoin(Query *query, + RelationRestrictionContext *restrictionContext, + JoinRestrictionContext *joinRestrictionContext) +{ + ListCell *joinRestrictionCell; + foreach(joinRestrictionCell, joinRestrictionContext->joinRestrictionList) + { + JoinRestriction *joinRestriction = (JoinRestriction *) lfirst( + joinRestrictionCell); + + /* Restriction context includes alternative plans, sufficient to check for left joins.*/ + if (joinRestriction->joinType != JOIN_LEFT) + { + continue; + } + + if (!PartitionColumnIsInTargetList(query, joinRestriction, restrictionContext)) + { + /* outer join does not select partition column of outer relation */ + return false; + } + } + + return true; +} + + +/* + * PartitionColumnIsInTargetList checks whether the partition column of + * the given relation is included in the target list of the query. + */ +static bool +PartitionColumnIsInTargetList(Query *query, JoinRestriction *joinRestriction, + RelationRestrictionContext *restrictionContext) +{ + Relids relids = joinRestriction->outerrelRelids; + int relationId = -1; + Index partitionKeyIndex = InvalidAttrNumber; + while ((relationId = bms_next_member(relids, relationId)) >= 0) + { + RangeTblEntry *rte = joinRestriction->plannerInfo->simple_rte_array[relationId]; + if (rte->rtekind != RTE_RELATION) + { + /* skip if it is not a relation */ + continue; + } + int targetRTEIndex = GetRTEIdentity(rte); + PartitionKeyForRTEIdentityInQuery(query, targetRTEIndex, + &partitionKeyIndex); + if (partitionKeyIndex == 0) + { + /* partition key is not in the target list */ + return false; + } + } + return true; } diff --git a/src/test/regress/citus_tests/run_test.py b/src/test/regress/citus_tests/run_test.py index 193bdf09f..fff261372 100755 --- a/src/test/regress/citus_tests/run_test.py +++ b/src/test/regress/citus_tests/run_test.py @@ -226,6 +226,15 @@ DEPS = { repeatable=False, ), "pg17": TestDeps("minimal_schedule", ["multi_behavioral_analytics_create_table"]), + "multi_subquery_misc": TestDeps( + "minimal_schedule", ["multi_behavioral_analytics_create_table"] + ), + "multi_subquery_union": TestDeps( + "minimal_schedule", ["multi_behavioral_analytics_create_table"] + ), + "multi_subquery_in_where_clause": TestDeps( + "minimal_schedule", ["multi_behavioral_analytics_create_table"] + ), } diff --git a/src/test/regress/expected/multi_insert_select_behavioral_analytics_create_table.out b/src/test/regress/expected/multi_insert_select_behavioral_analytics_create_table.out new file mode 100644 index 000000000..e69de29bb diff --git a/src/test/regress/expected/multi_subquery_in_where_clause.out b/src/test/regress/expected/multi_subquery_in_where_clause.out index 834cef505..c6c5a2b2a 100644 --- a/src/test/regress/expected/multi_subquery_in_where_clause.out +++ b/src/test/regress/expected/multi_subquery_in_where_clause.out @@ -1,7 +1,7 @@ -- -- multi subquery in where queries aims to expand existing subquery pushdown -- regression tests to cover more cases specifically subqueries in WHERE clause --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests -- subqueries in WHERE with greater operator diff --git a/src/test/regress/expected/multi_subquery_misc.out b/src/test/regress/expected/multi_subquery_misc.out index 3c8abc67d..32f5ab801 100644 --- a/src/test/regress/expected/multi_subquery_misc.out +++ b/src/test/regress/expected/multi_subquery_misc.out @@ -2,7 +2,7 @@ -- (i) Prepared statements -- (ii) PL/PGSQL functions -- (iii) SQL functions --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests SET citus.enable_router_execution TO false; PREPARE prepared_subquery_1 AS @@ -352,6 +352,7 @@ ORDER BY 2 DESC; -- Similar to the above queries, but -- this time the joins are not removed because -- target list contains all the entries +SET citus.enable_router_execution TO true; SELECT * FROM users_table t1 diff --git a/src/test/regress/expected/multi_subquery_union.out b/src/test/regress/expected/multi_subquery_union.out index 2206e5a4a..7dfd389b3 100644 --- a/src/test/regress/expected/multi_subquery_union.out +++ b/src/test/regress/expected/multi_subquery_union.out @@ -1,7 +1,7 @@ -- -- multi subquery toplevel union queries aims to expand existing subquery pushdown -- regression tests to cover more cases --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests -- SET citus.next_shard_id TO 1400000; -- a very simple union query @@ -1246,5 +1246,3 @@ SELECT user_id FROM users_table UNION SELECT u.user_id FROM users_table, users_udf() u; ERROR: cannot perform distributed planning on this query because parameterized queries for SQL functions referencing distributed tables are not supported HINT: Consider using PL/pgSQL functions instead. -DROP TABLE events_reference_table; -DROP TABLE users_reference_table; diff --git a/src/test/regress/expected/union_pushdown.out b/src/test/regress/expected/union_pushdown.out index 4ae83c972..bd078b1fb 100644 --- a/src/test/regress/expected/union_pushdown.out +++ b/src/test/regress/expected/union_pushdown.out @@ -1469,5 +1469,140 @@ $$); f (1 row) +CREATE TABLE dist1 (a int, b int); +CREATE TABLE dist2 (a int, b int); +SET citus.shard_count to 4; +SELECT create_distributed_table('dist1', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist2', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO dist1 VALUES (1, 1), (2, 1), (3, 1), (4, 1), (5, 1); +INSERT INTO dist2 VALUES (5, 2), (6, 2), (7, 2), (8, 2), (9, 2); +-- safe to pushdown +SELECT * FROM +( + SELECT * FROM dist1 JOIN dist2 USING (a) + UNION + SELECT * FROM dist1 JOIN dist2 USING (a) +) AS t1 ORDER BY 1; + a | b | b +--------------------------------------------------------------------- + 5 | 1 | 2 +(1 row) + +-- not safe to pushdown, the distribution key from the outer part of the outer join is not in the target list +SELECT * FROM +( + SELECT dist2.a FROM dist1 LEFT JOIN dist2 USING (a) + UNION + SELECT dist2.a FROM dist2 +) AS t1 ORDER BY 1; + a +--------------------------------------------------------------------- + 5 + 6 + 7 + 8 + 9 + +(6 rows) + +set client_min_messages to DEBUG3; +-- not safe to pushdown, as is, sub-plan is generated +-- the distribution key from the outer part of the outer join is not in the target list +SELECT * FROM +( + SELECT dist1.a FROM dist1 RIGHT JOIN dist2 USING (a) + UNION + SELECT dist2.a FROM dist2 +) AS t1 ORDER BY 1; +DEBUG: no shard pruning constraints on dist1 found +DEBUG: shard count after pruning for dist1: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on dist1 found +DEBUG: shard count after pruning for dist1: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on dist1 found +DEBUG: shard count after pruning for dist1: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_1 for subquery SELECT dist1.a FROM (union_pushdown.dist1 RIGHT JOIN union_pushdown.dist2 USING (a)) +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: generating subplan XXX_2 for subquery SELECT a FROM union_pushdown.dist2 +DEBUG: Creating router plan +DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer) UNION SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t1 ORDER BY a +DEBUG: Creating router plan + a +--------------------------------------------------------------------- + 5 + 6 + 7 + 8 + 9 + +(6 rows) + +-- safe to pushdown, the distribution key from the outer side of the RIGHT join is in the target list +SELECT * FROM +( + SELECT dist2.a + FROM dist1 RIGHT JOIN dist2 USING (a) + UNION + SELECT dist2.a FROM dist2 +) AS t1 +ORDER BY 1; +DEBUG: no shard pruning constraints on dist1 found +DEBUG: shard count after pruning for dist1: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: no shard pruning constraints on dist1 found +DEBUG: shard count after pruning for dist1: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: no shard pruning constraints on dist2 found +DEBUG: shard count after pruning for dist2: 4 +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx +DEBUG: assigned task to node localhost:xxxxx + a +--------------------------------------------------------------------- + 5 + 6 + 7 + 8 + 9 +(5 rows) + SET client_min_messages TO WARNING; DROP SCHEMA union_pushdown CASCADE; diff --git a/src/test/regress/sql/multi_subquery_in_where_clause.sql b/src/test/regress/sql/multi_subquery_in_where_clause.sql index ecd4cbffa..0c7343627 100644 --- a/src/test/regress/sql/multi_subquery_in_where_clause.sql +++ b/src/test/regress/sql/multi_subquery_in_where_clause.sql @@ -1,7 +1,7 @@ -- -- multi subquery in where queries aims to expand existing subquery pushdown -- regression tests to cover more cases specifically subqueries in WHERE clause --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests diff --git a/src/test/regress/sql/multi_subquery_misc.sql b/src/test/regress/sql/multi_subquery_misc.sql index 4b81491b1..2cdcc810a 100644 --- a/src/test/regress/sql/multi_subquery_misc.sql +++ b/src/test/regress/sql/multi_subquery_misc.sql @@ -3,7 +3,7 @@ -- (ii) PL/PGSQL functions -- (iii) SQL functions --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests SET citus.enable_router_execution TO false; @@ -213,6 +213,7 @@ ORDER BY 2 DESC; -- Similar to the above queries, but -- this time the joins are not removed because -- target list contains all the entries +SET citus.enable_router_execution TO true; SELECT * FROM users_table t1 diff --git a/src/test/regress/sql/multi_subquery_union.sql b/src/test/regress/sql/multi_subquery_union.sql index 3d35609e5..d4407646a 100644 --- a/src/test/regress/sql/multi_subquery_union.sql +++ b/src/test/regress/sql/multi_subquery_union.sql @@ -1,7 +1,7 @@ -- -- multi subquery toplevel union queries aims to expand existing subquery pushdown -- regression tests to cover more cases --- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- the tables that are used depends to multi_behavioral_analytics_create_table.sql -- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests -- SET citus.next_shard_id TO 1400000; @@ -898,6 +898,3 @@ LANGUAGE sql stable; SELECT user_id FROM users_table UNION SELECT u.user_id FROM users_table, users_udf() u; - -DROP TABLE events_reference_table; -DROP TABLE users_reference_table; diff --git a/src/test/regress/sql/union_pushdown.sql b/src/test/regress/sql/union_pushdown.sql index 1bb63eb62..09bf218d3 100644 --- a/src/test/regress/sql/union_pushdown.sql +++ b/src/test/regress/sql/union_pushdown.sql @@ -1109,5 +1109,49 @@ SELECT k, COUNT(*) FROM v GROUP BY k ORDER BY k; $$); +CREATE TABLE dist1 (a int, b int); +CREATE TABLE dist2 (a int, b int); +SET citus.shard_count to 4; +SELECT create_distributed_table('dist1', 'a'); +SELECT create_distributed_table('dist2', 'a'); +INSERT INTO dist1 VALUES (1, 1), (2, 1), (3, 1), (4, 1), (5, 1); +INSERT INTO dist2 VALUES (5, 2), (6, 2), (7, 2), (8, 2), (9, 2); + +-- safe to pushdown +SELECT * FROM +( + SELECT * FROM dist1 JOIN dist2 USING (a) + UNION + SELECT * FROM dist1 JOIN dist2 USING (a) +) AS t1 ORDER BY 1; + +-- not safe to pushdown, the distribution key from the outer part of the outer join is not in the target list +SELECT * FROM +( + SELECT dist2.a FROM dist1 LEFT JOIN dist2 USING (a) + UNION + SELECT dist2.a FROM dist2 +) AS t1 ORDER BY 1; + +set client_min_messages to DEBUG3; +-- not safe to pushdown, as is, sub-plan is generated +-- the distribution key from the outer part of the outer join is not in the target list +SELECT * FROM +( + SELECT dist1.a FROM dist1 RIGHT JOIN dist2 USING (a) + UNION + SELECT dist2.a FROM dist2 +) AS t1 ORDER BY 1; + +-- safe to pushdown, the distribution key from the outer side of the RIGHT join is in the target list +SELECT * FROM +( + SELECT dist2.a + FROM dist1 RIGHT JOIN dist2 USING (a) + UNION + SELECT dist2.a FROM dist2 +) AS t1 +ORDER BY 1; + SET client_min_messages TO WARNING; DROP SCHEMA union_pushdown CASCADE;