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 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/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/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 895f01ae7..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; } @@ -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) 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/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/local_executor.c b/src/backend/distributed/executor/local_executor.c index 2ced0a43f..0730e792a 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; @@ -754,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 */ @@ -782,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/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 3815382e0..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) { @@ -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,11 +709,12 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params, list_make1(queryPlan),/* list of PlannedStmt* */ NULL /* no CachedPlan */ ); -#endif 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) */ @@ -737,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 */ @@ -747,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/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/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/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/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/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_explain.c b/src/backend/distributed/planner/multi_explain.c index c0affb50e..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,46 +635,21 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es) MemoryContextSwitchTo(saved_ctx); MemoryContextMemConsumed(planner_ctx, &mem_counters); } -#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 - 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); @@ -1251,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); @@ -1282,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(); @@ -1398,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); @@ -1418,6 +1494,8 @@ worker_save_query_explain_analyze(PG_FUNCTION_ARGS) SavedExplainPlan = pstrdup(es->str->data); SavedExecutionDurationMillisec = executionDurationMillisec; + SavedExplainPlanNtuples = executionTuples; + SavedExplainPlanNloops = executionLoops; MemoryContextSwitchTo(oldContext); @@ -1606,22 +1684,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( @@ -1662,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; @@ -1677,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. @@ -1686,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) { @@ -1693,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) { @@ -1708,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) { @@ -1717,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 @@ -1742,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 { @@ -1804,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); } @@ -1963,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; @@ -2107,22 +2252,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,11 +2276,24 @@ ExplainOneQuery(Query *query, int cursorOptions, (es->buffers ? &bufusage : NULL) ); #endif - } } +/* + * 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 @@ -2165,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); @@ -2208,19 +2356,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,10 +2366,9 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es queryEnv, /* QueryEnvironment *queryEnv */ instrument_option /* int instrument_options */ ); - #endif /* Select execution options */ - if (es->analyze) + if (executeQuery) eflags = 0; /* default run-to-completion flags */ else eflags = EXEC_FLAG_EXPLAIN_ONLY; @@ -2243,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; @@ -2266,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); @@ -2338,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); @@ -2345,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/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 19d386343..59124c5bf 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,262 @@ 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(); + 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); + + /* + * 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); + 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); + + 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 +2219,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 +2232,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 +2264,8 @@ GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, shardId, job->parametersInJobQueryResolved, isLocalTableModification, - job->partitionKeyValue, job->colocationId); + job->partitionKeyValue, job->colocationId, + delayedFastPath); } } @@ -2096,7 +2360,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 +2432,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 +2716,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/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/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index b4c64e070..165aea05f 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); @@ -385,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 */ @@ -506,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; @@ -1426,6 +1405,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 +2841,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/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..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,2 +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/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/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/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/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 }; diff --git a/src/backend/distributed/utils/citus_copyfuncs.c b/src/backend/distributed/utils/citus_copyfuncs.c index 4b4a334c8..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); } @@ -287,6 +312,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/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/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/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/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/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 */ 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 475a41b37..1040b4149 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; @@ -474,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 @@ -492,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/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/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/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) 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": [ diff --git a/src/test/regress/citus_tests/run_test.py b/src/test/regress/citus_tests/run_test.py index 8d11c7e66..fff261372 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"), @@ -225,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/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/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/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_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_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/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/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/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/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/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/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_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/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/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; 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/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_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/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/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 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; 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;