From 29975cd85f0216da40e3f6eb42ad5716ea9a9a49 Mon Sep 17 00:00:00 2001 From: Teja Mupparti Date: Sat, 25 Mar 2023 15:54:02 -0700 Subject: [PATCH] Implement repartition MERGE --- .../distributed/executor/citus_custom_scan.c | 42 +- .../distributed_intermediate_results.c | 12 + .../executor/insert_select_executor.c | 9 +- .../distributed/executor/merge_executor.c | 259 ++++ .../executor/multi_server_executor.c | 6 + .../distributed/planner/distributed_planner.c | 9 +- .../planner/insert_select_planner.c | 3 +- .../distributed/planner/merge_planner.c | 1070 ++++++++++++----- .../distributed/planner/multi_explain.c | 53 + .../planner/multi_router_planner.c | 28 +- .../distributed/planner/recursive_planning.c | 3 +- src/include/distributed/citus_custom_scan.h | 1 + .../distributed/insert_select_executor.h | 6 +- .../distributed/insert_select_planner.h | 1 + src/include/distributed/merge_executor.h | 17 + src/include/distributed/merge_planner.h | 18 +- .../distributed/multi_server_executor.h | 3 +- src/include/distributed/recursive_planning.h | 1 + src/test/regress/expected/merge.out | 703 +++++++++-- .../regress/expected/merge_repartition.out | 926 ++++++++++++++ .../regress/expected/merge_repartition_0.out | 6 + src/test/regress/expected/pg15.out | 11 +- src/test/regress/expected/pgmerge.out | 9 +- src/test/regress/multi_schedule | 3 +- src/test/regress/sql/merge.sql | 315 +++-- src/test/regress/sql/merge_repartition.sql | 401 ++++++ src/test/regress/sql/pg15.sql | 2 +- src/test/regress/sql/pgmerge.sql | 3 +- 28 files changed, 3337 insertions(+), 583 deletions(-) create mode 100644 src/backend/distributed/executor/merge_executor.c create mode 100644 src/include/distributed/merge_executor.h create mode 100644 src/test/regress/expected/merge_repartition.out create mode 100644 src/test/regress/expected/merge_repartition_0.out create mode 100644 src/test/regress/sql/merge_repartition.sql diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 28486f23d..74c8f0bea 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -27,6 +27,8 @@ #include "distributed/listutils.h" #include "distributed/local_executor.h" #include "distributed/local_plan_cache.h" +#include "distributed/merge_executor.h" +#include "distributed/merge_planner.h" #include "distributed/multi_executor.h" #include "distributed/multi_server_executor.h" #include "distributed/multi_router_planner.h" @@ -53,6 +55,7 @@ extern AllowedDistributionColumn AllowedDistributionColumnValue; static Node * AdaptiveExecutorCreateScan(CustomScan *scan); static Node * NonPushableInsertSelectCreateScan(CustomScan *scan); static Node * DelayedErrorCreateScan(CustomScan *scan); +static Node * NonPushableMergeCommandCreateScan(CustomScan *scan); /* functions that are common to different scans */ static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags); @@ -88,6 +91,11 @@ CustomScanMethods DelayedErrorCustomScanMethods = { DelayedErrorCreateScan }; +CustomScanMethods NonPushableMergeCommandCustomScanMethods = { + "Citus MERGE INTO ...", + NonPushableMergeCommandCreateScan +}; + /* * Define executor methods for the different executor types. @@ -111,6 +119,16 @@ static CustomExecMethods NonPushableInsertSelectCustomExecMethods = { }; +static CustomExecMethods NonPushableMergeCommandCustomExecMethods = { + .CustomName = "NonPushableMergeCommandScan", + .BeginCustomScan = CitusBeginScan, + .ExecCustomScan = NonPushableMergeCommandExecScan, + .EndCustomScan = CitusEndScan, + .ReScanCustomScan = CitusReScan, + .ExplainCustomScan = NonPushableMergeCommandExplainScan +}; + + /* * IsCitusCustomState returns if a given PlanState node is a CitusCustomState node. */ @@ -124,7 +142,8 @@ IsCitusCustomState(PlanState *planState) CustomScanState *css = castNode(CustomScanState, planState); if (css->methods == &AdaptiveExecutorCustomExecMethods || - css->methods == &NonPushableInsertSelectCustomExecMethods) + css->methods == &NonPushableInsertSelectCustomExecMethods || + css->methods == &NonPushableMergeCommandCustomExecMethods) { return true; } @@ -142,6 +161,7 @@ RegisterCitusCustomScanMethods(void) RegisterCustomScanMethods(&AdaptiveExecutorCustomScanMethods); RegisterCustomScanMethods(&NonPushableInsertSelectCustomScanMethods); RegisterCustomScanMethods(&DelayedErrorCustomScanMethods); + RegisterCustomScanMethods(&NonPushableMergeCommandCustomScanMethods); } @@ -723,6 +743,26 @@ DelayedErrorCreateScan(CustomScan *scan) } +/* + * NonPushableMergeCommandCreateScan creates the scan state for executing + * MERGE INTO ... into a distributed table with repartition of source rows. + */ +static Node * +NonPushableMergeCommandCreateScan(CustomScan *scan) +{ + CitusScanState *scanState = palloc0(sizeof(CitusScanState)); + + scanState->executorType = MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY; + scanState->customScanState.ss.ps.type = T_CustomScanState; + scanState->distributedPlan = GetDistributedPlan(scan); + scanState->customScanState.methods = &NonPushableMergeCommandCustomExecMethods; + scanState->finishedPreScan = false; + scanState->finishedRemoteScan = false; + + return (Node *) scanState; +} + + /* * CitusEndScan is used to clean up tuple store of the given custom scan state. */ diff --git a/src/backend/distributed/executor/distributed_intermediate_results.c b/src/backend/distributed/executor/distributed_intermediate_results.c index e7a5830e6..c10303e18 100644 --- a/src/backend/distributed/executor/distributed_intermediate_results.c +++ b/src/backend/distributed/executor/distributed_intermediate_results.c @@ -610,6 +610,18 @@ QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer) StringInfo fragmentNamesArrayString = makeStringInfo(); int fragmentCount = 0; NodePair *nodePair = &fragmentsTransfer->nodes; + uint32 sourceNodeId = nodePair->sourceNodeId; + + /* + * If the placement is dummy, for example, queries that generate + * intermediate results at the coordinator that need to be redistributed + * to worker nodes, we need the local id. + */ + if (sourceNodeId == LOCAL_NODE_ID) + { + nodePair->sourceNodeId = GetLocalNodeId(); + } + WorkerNode *sourceNode = LookupNodeByNodeIdOrError(nodePair->sourceNodeId); appendStringInfoString(fragmentNamesArrayString, "ARRAY["); diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index a69ae0f22..613cb3d3a 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -67,11 +67,6 @@ static HTAB * ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId, static List * BuildColumnNameListFromTargetList(Oid targetRelationId, List *insertTargetList); static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList); -static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery, - CitusTableCacheEntry *targetRelation, - List **redistributedResults, - bool useBinaryFormat); -static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn); static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries); @@ -535,7 +530,7 @@ IsSupportedRedistributionTarget(Oid targetRelationId) * a result name which should be inserted into * targetRelation->sortedShardIntervalArray[shardIndex]. */ -static List * +List * RedistributedInsertSelectTaskList(Query *insertSelectQuery, CitusTableCacheEntry *targetRelation, List **redistributedResults, @@ -632,7 +627,7 @@ RedistributedInsertSelectTaskList(Query *insertSelectQuery, * PartitionColumnIndex finds the index of given partition column in the * given target list. */ -static int +int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn) { TargetEntry *insertTargetEntry = NULL; diff --git a/src/backend/distributed/executor/merge_executor.c b/src/backend/distributed/executor/merge_executor.c new file mode 100644 index 000000000..0eeb51e3f --- /dev/null +++ b/src/backend/distributed/executor/merge_executor.c @@ -0,0 +1,259 @@ +/*------------------------------------------------------------------------- + * + * merge_executor.c + * + * Executor logic for MERGE SQL statement. + * + * Copyright (c) Citus Data, Inc. + *------------------------------------------------------------------------- + */ + +#include "postgres.h" +#include "miscadmin.h" + +#include "distributed/distributed_execution_locks.h" +#include "distributed/insert_select_executor.h" +#include "distributed/intermediate_results.h" +#include "distributed/listutils.h" +#include "distributed/merge_executor.h" +#include "distributed/merge_planner.h" +#include "distributed/multi_executor.h" +#include "distributed/multi_partitioning_utils.h" +#include "distributed/multi_router_planner.h" +#include "distributed/subplan_execution.h" + +#include "nodes/execnodes.h" +#include "nodes/makefuncs.h" +#include "nodes/nodeFuncs.h" + + +static int sourceResultPartitionColumnIndex(Query *mergeQuery, + List *sourceTargetList, + CitusTableCacheEntry *targetRelation); +static bool ExtractEqualOpExprWalker(Node *node, List **equalOpExprList); + +/* + * NonPushableMergeCommandExecScan performs an MERGE INTO distributed_table + * USING (source-query) ... command. This can be done either by aggregating + * task results at the coordinator and repartitioning the results, or by + * repartitioning task results and directly transferring data between nodes. + */ +TupleTableSlot * +NonPushableMergeCommandExecScan(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + + if (!scanState->finishedRemoteScan) + { + EState *executorState = ScanStateGetExecutorState(scanState); + DistributedPlan *distributedPlan = scanState->distributedPlan; + Query *mergeQuery = copyObject(distributedPlan->insertSelectQuery); + RangeTblEntry *targetRte = ExtractResultRelationRTE(mergeQuery); + RangeTblEntry *sourceRte = ExtractSourceRangeTableEntry(mergeQuery); + Oid targetRelationId = targetRte->relid; + bool hasReturning = distributedPlan->expectResults; + Query *selectQuery = sourceRte->subquery; + PlannedStmt *selectPlan = copyObject(distributedPlan->selectPlanForInsertSelect); + + /* + * If we are dealing with partitioned table, we also need to lock its + * partitions. Here we only lock targetRelation, we acquire necessary + * locks on source tables during execution of those source queries. + */ + if (PartitionedTable(targetRelationId)) + { + LockPartitionRelations(targetRelationId, RowExclusiveLock); + } + + bool randomAccess = true; + bool interTransactions = false; + DistributedPlan *distSelectPlan = + GetDistributedPlan((CustomScan *) selectPlan->planTree); + Job *distSelectJob = distSelectPlan->workerJob; + List *distSelectTaskList = distSelectJob->taskList; + bool binaryFormat = + CanUseBinaryCopyFormatForTargetList(selectQuery->targetList); + + ereport(DEBUG1, (errmsg("Executing subplans of the source"))); + ExecuteSubPlans(distSelectPlan); + + /* + * We have a separate directory for each transaction, so choosing + * the same result prefix won't cause filename conflicts. Results + * directory name also includes node id and database id, so we don't + * need to include them in the filename. We include job id here for + * the case "MERGE USING " is executed recursively. + */ + StringInfo distResultPrefixString = makeStringInfo(); + appendStringInfo(distResultPrefixString, + "repartitioned_results_" UINT64_FORMAT, + distSelectJob->jobId); + char *distResultPrefix = distResultPrefixString->data; + CitusTableCacheEntry *targetRelation = GetCitusTableCacheEntry(targetRelationId); + ereport(DEBUG1, (errmsg("Executing task list and " + "redistributing the source rows"))); + + /* + * partitionColumnIndex determines the column in the selectTaskList to + * use for (re)partitioning of the source result, which will colocate + * the result data with the target. + */ + int partitionColumnIndex = + sourceResultPartitionColumnIndex(mergeQuery, + selectQuery->targetList, + targetRelation); + + if (partitionColumnIndex == -1) + { + ereport(ERROR, (errmsg("Missing required join condition between " + "source and target's distribution column"), + errhint("Without a join condition on the " + "target's distribution column, " + "the source rows cannot be efficiently " + "redistributed, and the NOT-MATCHED " + "condition cannot be evaluated " + "unambiguously. This can result in " + "incorrect or unexpected results when " + "attempting to merge tables in a distributed setting"))); + } + else + { + ereport(DEBUG1, + (errmsg("Using column - index:%d from the source list " + "to redistribute", partitionColumnIndex))); + } + + + /* + * Below call executes the given task list and partitions the results using shard + * ranges and partition method of targetRelation, and then colocates the result files + * with shards. It also moves the fragments in the cluster so they are colocated + * with the shards of target relation. These transfers are done by calls to + * fetch_intermediate_results() between nodes. + */ + List **redistributedResults = + RedistributeTaskListResults(distResultPrefix, + distSelectTaskList, partitionColumnIndex, + targetRelation, binaryFormat); + + ereport(DEBUG1, (errmsg("Executing final MERGE on workers"))); + + /* + * At this point source query has been executed on workers and results + * have been fetched in such a way that they are colocated with corresponding + * target shard(s). Create and execute a list of tasks of form + * MERGE INTO ... USING SELECT * FROM read_intermediate_results(...); + */ + List *taskList = RedistributedInsertSelectTaskList(mergeQuery, + targetRelation, + redistributedResults, + binaryFormat); + + scanState->tuplestorestate = + tuplestore_begin_heap(randomAccess, interTransactions, work_mem); + TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(scanState); + TupleDestination *tupleDest = + CreateTupleStoreTupleDest(scanState->tuplestorestate, + tupleDescriptor); + uint64 rowsMerged = + ExecuteTaskListIntoTupleDest(ROW_MODIFY_COMMUTATIVE, taskList, + tupleDest, + hasReturning); + executorState->es_processed = rowsMerged; + scanState->finishedRemoteScan = true; + } + + TupleTableSlot *resultSlot = ReturnTupleFromTuplestore(scanState); + + return resultSlot; +} + + +/* + * ExtractEqualOpExprWalker traverses the ON clause and gathers + * all expressions of the following types: ... ON t.id = s.id + * or t.id = + */ +static bool +ExtractEqualOpExprWalker(Node *node, List **equalOpExprList) +{ + if (node == NULL) + { + return false; + } + + if (NodeIsEqualsOpExpr(node)) + { + OpExpr *eqExpr = (OpExpr *) node; + (*equalOpExprList) = lappend(*equalOpExprList, eqExpr); + } + + bool walkerResult = + expression_tree_walker(node, ExtractEqualOpExprWalker, + equalOpExprList); + return walkerResult; +} + + +/* + * sourceResultPartitionColumnIndex collects all the "=" expressions from the + * provided quals and verifies if there is a join, either left or right, with + * the distribution column of the given target. Once a match is found, it returns + * the index of that match in the source's target list. + */ +static int +sourceResultPartitionColumnIndex(Query *mergeQuery, List *sourceTargetList, + CitusTableCacheEntry *targetRelation) +{ + Node *quals = mergeQuery->jointree->quals; + + if (IsA(quals, List)) + { + quals = (Node *) make_ands_explicit((List *) quals); + } + + /* Get all the equal(=) expressions from the ON clause */ + List *equalOpExprList = NIL; + ExtractEqualOpExprWalker((Node *) quals, &equalOpExprList); + + /* + * Now that we have gathered all the _equal_ expressions in the list, we proceed + * to match the one that joins with the distribution column of the target. + */ + Var *targetColumn = targetRelation->partitionColumn; + Var *sourceJoinColumn = NULL; + Node *joinClause = NULL; + foreach_ptr(joinClause, equalOpExprList) + { + Node *leftOperand; + Node *rightOperand; + if (!BinaryOpExpression((Expr *) joinClause, &leftOperand, &rightOperand)) + { + continue; + } + + if (equal(leftOperand, targetColumn) && IsA(rightOperand, Var)) + { + sourceJoinColumn = (Var *) rightOperand; + } + else if (equal(rightOperand, targetColumn) && IsA(leftOperand, Var)) + { + sourceJoinColumn = (Var *) leftOperand; + } + } + + if (!sourceJoinColumn) + { + return -1; + } + + ErrorIfInsertNotMatchTargetDistributionColumn(targetRelation->relationId, + mergeQuery, + sourceJoinColumn); + + /* + * Find the index in the source list that will be utilized + * for colocation-redistribution. + */ + return PartitionColumnIndex(sourceTargetList, sourceJoinColumn); +} diff --git a/src/backend/distributed/executor/multi_server_executor.c b/src/backend/distributed/executor/multi_server_executor.c index caf6797da..d0c16b974 100644 --- a/src/backend/distributed/executor/multi_server_executor.c +++ b/src/backend/distributed/executor/multi_server_executor.c @@ -24,6 +24,7 @@ #include "distributed/multi_executor.h" #include "distributed/multi_physical_planner.h" #include "distributed/multi_server_executor.h" +#include "distributed/multi_router_planner.h" #include "distributed/coordinator_protocol.h" #include "distributed/subplan_execution.h" #include "distributed/tuple_destination.h" @@ -49,6 +50,11 @@ JobExecutorType(DistributedPlan *distributedPlan) if (distributedPlan->insertSelectQuery != NULL) { + if (IsMergeQuery(distributedPlan->insertSelectQuery)) + { + return MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY; + } + /* * We go through * MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT because diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 6c5d0f32a..e3138fa51 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -990,7 +990,8 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina case MERGE_QUERY: { distributedPlan = - CreateMergePlan(originalQuery, query, plannerRestrictionContext); + CreateMergePlan(planId, originalQuery, query, plannerRestrictionContext, + boundParams); break; } @@ -1388,6 +1389,12 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan) break; } + case MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY: + { + customScan->methods = &NonPushableMergeCommandCustomScanMethods; + break; + } + default: { customScan->methods = &DelayedErrorCustomScanMethods; diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index cae71845b..8af62730c 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -85,7 +85,6 @@ static DeferredErrorMessage * InsertPartitionColumnMatchesSelect(Query *query, static DistributedPlan * CreateNonPushableInsertSelectPlan(uint64 planId, Query *parse, ParamListInfo boundParams); static DeferredErrorMessage * NonPushableInsertSelectSupported(Query *insertSelectQuery); -static Query * WrapSubquery(Query *subquery); static void RelabelTargetEntryList(List *selectTargetList, List *insertTargetList); static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, Oid targetRelationId); @@ -1635,7 +1634,7 @@ InsertSelectResultIdPrefix(uint64 planId) * WrapSubquery wraps the given query as a subquery in a newly constructed * "SELECT * FROM (...subquery...) citus_insert_select_subquery" query. */ -static Query * +Query * WrapSubquery(Query *subquery) { ParseState *pstate = make_parsestate(NULL); diff --git a/src/backend/distributed/planner/merge_planner.c b/src/backend/distributed/planner/merge_planner.c index 86163e131..aed462321 100644 --- a/src/backend/distributed/planner/merge_planner.c +++ b/src/backend/distributed/planner/merge_planner.c @@ -13,36 +13,39 @@ #include "postgres.h" #include "nodes/makefuncs.h" +#include "nodes/nodeFuncs.h" #include "optimizer/optimizer.h" #include "parser/parsetree.h" +#include "tcop/tcopprot.h" #include "utils/lsyscache.h" #include "distributed/citus_clauses.h" +#include "distributed/citus_custom_scan.h" +#include "distributed/insert_select_planner.h" #include "distributed/listutils.h" +#include "distributed/local_distributed_join_planner.h" #include "distributed/merge_planner.h" #include "distributed/multi_logical_optimizer.h" #include "distributed/multi_router_planner.h" #include "distributed/pg_dist_node_metadata.h" #include "distributed/pg_version_constants.h" #include "distributed/query_pushdown_planning.h" +#include "distributed/query_colocation_checker.h" #if PG_VERSION_NUM >= PG_VERSION_15 -static DeferredErrorMessage * ErrorIfDistTablesNotColocated(Query *parse, - List * - distTablesList, - PlannerRestrictionContext - * - plannerRestrictionContext); -static DeferredErrorMessage * ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, - Query *parse, - List *rangeTableList, - PlannerRestrictionContext * - restrictionContext); +static void ErrorIfMergeQueryQualAndTargetListNotSupported(Oid targetRelationId, + Query *originalQuery); +static void ErrorIfMergeNotSupported(Query *query, Oid targetRelationId, + List *rangeTableList); +static void ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, List *rangeTableList); static bool IsDistributionColumnInMergeSource(Expr *columnExpression, Query *query, bool skipOuterVars); -static DeferredErrorMessage * InsertDistributionColumnMatchesSource(Oid targetRelationId, - Query *query); +static DeferredErrorMessage * DeferErrorIfRoutableMergeNotSupported(Query *query, + List *rangeTableList, + PlannerRestrictionContext + * + plannerRestrictionContext); static DeferredErrorMessage * MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, FromExpr *joinTree, @@ -50,55 +53,105 @@ static DeferredErrorMessage * MergeQualAndTargetListFunctionsSupported(Oid List *targetList, CmdType commandType); +static DistributedPlan * CreateRouterMergePlan(Oid targetRelationId, Query *originalQuery, + Query *query, + List *rangeTableList, + PlannerRestrictionContext * + plannerRestrictionContext); +static void ErrorIfRepartitionMergeNotSupported(Oid targetRelationId, Query *mergeQuery, + Query *sourceQuery); +static void ConvertSourceRTEIntoSubquery(Query *mergeQuery, + RangeTblEntry *sourceRte, + PlannerRestrictionContext * + plannerRestrictionContext); +static void ConvertCteRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte); +static void ConvertRelationRTEIntoSubquery(Query *mergeQuery, + RangeTblEntry *sourceRte, + PlannerRestrictionContext * + plannerRestrictionContext); +static void ErrorIfUnsupportedCTEs(Query *query); +static void ContainsUnsupportedCTEs(Query *query); +static bool MergeQueryCTEWalker(Node *node, void *context); +static DistributedPlan * CreateRepartitionMergePlan(Oid targetRelationId, uint64 planId, + Query *originalQuery, + Query *query, + PlannerRestrictionContext * + plannerRestrictionContext, + ParamListInfo boundParams); + /* - * ErrorIfDistTablesNotColocated Checks to see if - * - * - There are a minimum of two distributed tables (source and a target). - * - All the distributed tables are indeed colocated. - * - * If any of the conditions are not met, it raises an exception. + * ContainsUnsupportedCTEs checks the CTE if it's modifying or recursive CTE, if true, + * raises an exception. */ -static DeferredErrorMessage * -ErrorIfDistTablesNotColocated(Query *parse, List *distTablesList, - PlannerRestrictionContext * - plannerRestrictionContext) +static void +ContainsUnsupportedCTEs(Query *query) { - /* All MERGE tables must be distributed */ - if (list_length(distTablesList) < 2) + if (query->hasModifyingCTE) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "For MERGE command, both the source and target " - "must be distributed", NULL, NULL); + ereport(ERROR, (errmsg("CTEs with modifying actions are not yet " + "supported in MERGE"))); } - /* All distributed tables must be colocated */ - if (!AllDistributedRelationsInRTEListColocated(distTablesList)) + CommonTableExpr *candidateCte = NULL; + foreach_ptr(candidateCte, query->cteList) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "For MERGE command, all the distributed tables " - "must be colocated", NULL, NULL); + if (candidateCte->cterecursive) + { + ereport(ERROR, (errmsg("Recursive CTEs are not yet " + "supported in MERGE"))); + } + } +} + + +/* + * MergeQueryCTEWalker descends into the MERGE query to check for any subqueries + */ +static bool +MergeQueryCTEWalker(Node *node, void *context) +{ + if (node == NULL) + { + return false; } - return NULL; + if (IsA(node, Query)) + { + Query *query = (Query *) node; + + ContainsUnsupportedCTEs(query); + + query_tree_walker(query, MergeQueryCTEWalker, NULL, 0); + + /* we're done, no need to recurse anymore for this query */ + return false; + } + + return expression_tree_walker(node, MergeQueryCTEWalker, context); +} + + +/* + * ErrorIfUnsupportedCTEs checks for unsupported CTEs, such as, modifying and recursive + */ +static void +ErrorIfUnsupportedCTEs(Query *query) +{ + ContainsUnsupportedCTEs(query); + query_tree_walker(query, MergeQueryCTEWalker, NULL, 0); } /* * ErrorIfMergeHasUnsupportedTables checks if all the tables(target, source or any CTE * present) in the MERGE command are local i.e. a combination of Citus local and Non-Citus - * tables (regular Postgres tables), or distributed tables with some restrictions, please - * see header of routine ErrorIfDistTablesNotColocated for details, raises an exception - * for all other combinations. + * tables (regular Postgres tables), or distributed tables with some restrictions + * raises an exception for all other combinations. */ -static DeferredErrorMessage * -ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *rangeTableList, - PlannerRestrictionContext *restrictionContext) +static void +ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, List *rangeTableList) { - List *distTablesList = NIL; - bool foundLocalTables = false; - bool foundReferenceTables = false; - RangeTblEntry *rangeTableEntry = NULL; foreach_ptr(rangeTableEntry, rangeTableList) { @@ -133,17 +186,18 @@ ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *range case RTE_NAMEDTUPLESTORE: case RTE_RESULT: { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command is not supported with " - "Tuplestores and results", - NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("MERGE command is not supported with " + "Tuplestores and results"))); + break; } default: { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command: Unrecognized range table entry.", - NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "MERGE command: Unrecognized range table entry(%d) ", + rangeTableEntry->rtekind))); } } @@ -163,12 +217,10 @@ ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *range if (relationId == targetRelationId) { /* Usually we don't reach this exception as the Postgres parser catches it */ - StringInfo errorMessage = makeStringInfo(); - appendStringInfo(errorMessage, "MERGE command is not allowed on " - "relation type(relkind:%c)", - rangeTableEntry->relkind); - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - errorMessage->data, NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("MERGE command is not allowed on " + "relation type(relkind:%c)", + rangeTableEntry->relkind))); } break; } @@ -183,18 +235,14 @@ ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *range default: { - StringInfo errorMessage = makeStringInfo(); - appendStringInfo(errorMessage, "Unexpected table type(relkind:%c) " - "in MERGE command", - rangeTableEntry->relkind); - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - errorMessage->data, NULL, NULL); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("Unexpected table type(relkind:%c) " + "in MERGE command", rangeTableEntry->relkind))); } } /* - * For now, save all distributed tables, later (below) we will - * check for supported combination(s). + * Check for unsupported distributed tables */ if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) { @@ -202,81 +250,32 @@ ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *range if (IsCitusTableType(relationId, APPEND_DISTRIBUTED) || IsCitusTableType(relationId, RANGE_DISTRIBUTED)) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "For MERGE command, all the distributed tables " - "must be colocated, for append/range distribution, " - "colocation is not supported", NULL, - "Consider using hash distribution instead"); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("For MERGE command, append/range distribution " + "table is not supported yet"))); } - - distTablesList = lappend(distTablesList, rangeTableEntry); } - else if (IsCitusTableType(relationId, REFERENCE_TABLE)) + else if (IsCitusTableType(relationId, REFERENCE_TABLE) && + relationId == targetRelationId) { /* Reference table as a target is not allowed */ - if (relationId == targetRelationId) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "Reference table as target " - "is not allowed in " - "MERGE command", NULL, NULL); - } - - foundReferenceTables = true; + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("Reference table as target is " + "not allowed in MERGE command"))); } else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) { - /* Citus local tables */ - foundLocalTables = true; + /* + * All the tables are local/reference, supported as long as + * coordinator is in the metadata. + */ + if (FindCoordinatorNodeId() == -1) + { + ereport(ERROR, (errmsg("Coordinator node is not in " + "the metadata. TODO better message"))); + } } - else if (!IsCitusTable(relationId)) - { - /* Regular Postgres table */ - foundLocalTables = true; - } - - /* Any other Citus table type missing ? */ } - - /* Ensure all tables are indeed local (or a combination of reference and local) */ - if (list_length(distTablesList) == 0) - { - /* - * All the tables are local/reference, supported as long as - * coordinator is in the metadata. - */ - if (FindCoordinatorNodeId() == -1) - { - elog(ERROR, "Coordinator node is not in the metadata. TODO better meesage"); - } - - /* All the tables are local/reference, supported */ - return NULL; - } - - if (foundLocalTables) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command is not supported with " - "combination of distributed/local tables yet", - NULL, NULL); - } - - if (foundReferenceTables) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE command is not supported with " - "combination of distributed/reference yet", - NULL, - "If target is distributed, source " - "must be distributed and co-located"); - } - - - /* Ensure all distributed tables are indeed co-located */ - return ErrorIfDistTablesNotColocated(parse, - distTablesList, - restrictionContext); } @@ -320,104 +319,6 @@ IsDistributionColumnInMergeSource(Expr *columnExpression, Query *query, bool } -/* - * InsertDistributionColumnMatchesSource check to see if MERGE is inserting a - * value into the target which is not from the source table, if so, it - * raises an exception. - * Note: Inserting random values other than the joined column values will - * result in unexpected behaviour of rows ending up in incorrect shards, to - * prevent such mishaps, we disallow such inserts here. - */ -static DeferredErrorMessage * -InsertDistributionColumnMatchesSource(Oid targetRelationId, Query *query) -{ - Assert(IsMergeQuery(query)); - - if (!IsCitusTableType(targetRelationId, DISTRIBUTED_TABLE)) - { - return NULL; - } - - if (!HasDistributionKey(targetRelationId)) - { - return NULL; - } - - bool foundDistributionColumn = false; - MergeAction *action = NULL; - foreach_ptr(action, query->mergeActionList) - { - /* Skip MATCHED clause as INSERTS are not allowed in it*/ - if (action->matched) - { - continue; - } - - /* NOT MATCHED can have either INSERT or DO NOTHING */ - if (action->commandType == CMD_NOTHING) - { - return NULL; - } - - if (action->targetList == NIL) - { - /* INSERT DEFAULT VALUES is not allowed */ - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "cannot perform MERGE INSERT with DEFAULTS", - NULL, NULL); - } - - Assert(action->commandType == CMD_INSERT); - Var *targetKey = PartitionColumn(targetRelationId, 1); - - TargetEntry *targetEntry = NULL; - foreach_ptr(targetEntry, action->targetList) - { - AttrNumber originalAttrNo = targetEntry->resno; - - /* skip processing of target table non-partition columns */ - if (originalAttrNo != targetKey->varattno) - { - continue; - } - - foundDistributionColumn = true; - - if (IsA(targetEntry->expr, Var)) - { - if (IsDistributionColumnInMergeSource(targetEntry->expr, query, true)) - { - return NULL; - } - else - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE INSERT must use the source table " - "distribution column value", - NULL, NULL); - } - } - else - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE INSERT must refer a source column " - "for distribution column ", - NULL, NULL); - } - } - - if (!foundDistributionColumn) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "MERGE INSERT must have distribution column as value", - NULL, NULL); - } - } - - return NULL; -} - - /* * MergeQualAndTargetListFunctionsSupported Checks WHEN/ON clause actions to see what functions * are allowed, if we are updating distribution column, etc. @@ -554,24 +455,469 @@ MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, FromExpr *joinTre } -#endif +/* + * RepartitionMergeSupported checks if certain conditions cannot accommodate the + * strategy of repartition and redistribution of source rows, the routine will verify + * them and subsequently raises an exception. + */ +static void +ErrorIfRepartitionMergeNotSupported(Oid targetRelationId, Query *mergeQuery, + Query *sourceQuery) +{ + if (!IsCitusTableType(targetRelationId, DISTRIBUTED_TABLE)) + { + ereport(ERROR, + (errmsg("MERGE involving repartition of rows " + "is supported only if the target is distributed"))); + } + + RTEListProperties *queryRteListProperties = GetRTEListPropertiesForQuery(mergeQuery); + if (queryRteListProperties->hasPostgresLocalTable) + { + ereport(ERROR, (errmsg("MERGE INTO an distributed table from " + "Postgres table is not yet supported"))); + } + + queryRteListProperties = GetRTEListPropertiesForQuery(sourceQuery); + if (!queryRteListProperties->hasCitusTable) + { + ereport(ERROR, (errmsg("To MERGE into a distributed table, source must " + "be Citus table(s)"))); + } +} /* - * MergeQuerySupported does check for a MERGE command in the query, if it finds + * ConvertCteRTEIntoSubquery takes a RTE_CTE and converts it into a RTE_SUBQUERY. + */ +static void +ConvertCteRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte) +{ +/* CTE cterefcount number of RTEs referencing this CTE */ +/* CTE bool self_reference is this a recursive self-reference? */ + + CommonTableExpr *sourceCte = NULL; + CommonTableExpr *candidateCte = NULL; + List *cteList = NIL; + foreach_ptr(candidateCte, mergeQuery->cteList) + { + if (strcmp(candidateCte->ctename, sourceRte->ctename) == 0) + { + /* The source CTE that will be converted to a subquery */ + sourceCte = candidateCte; + } + else + { + /* Save all other CTEs */ + cteList = lappend(cteList, candidateCte); + } + } + + Assert(sourceCte); + + Query *cteQuery = (Query *) copyObject(sourceCte->ctequery); + + sourceRte->rtekind = RTE_SUBQUERY; + + /* + * As we are delinking the CTE from main query, we have to walk through the + * tree and decrement the ctelevelsup, but by wrapping a subquery, we avoid + * adjusting the ctelevelsup in RTE's + */ + sourceRte->subquery = WrapSubquery(cteQuery); + + /* Copy the rest of the CTEs(if any) and remove them from main query */ + sourceRte->subquery->cteList = copyObject(cteList); + mergeQuery->cteList = NIL; + + /* Zero out CTE-specific fields */ + sourceRte->security_barrier = false; + sourceRte->ctename = NULL; + sourceRte->ctelevelsup = 0; + sourceRte->self_reference = false; + sourceRte->coltypes = NIL; + sourceRte->coltypmods = NIL; + sourceRte->colcollations = NIL; +} + + +/* + * ConvertRelationRTEIntoSubquery takes a RTE_RELATION and converts it into a RTE_SUBQUERY, + * which is basically a SELECT * FROM the relation. + */ +static void +ConvertRelationRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte, + PlannerRestrictionContext *plannerRestrictionContext) +{ + Query *sourceResultsQuery = makeNode(Query); + RangeTblRef *newRangeTableRef = makeNode(RangeTblRef); + + RelationRestriction *relationRestriction = + RelationRestrictionForRelation(sourceRte, plannerRestrictionContext); + + /* + * Hack: RequiredAttrNumbersForRelation uses the planner's query which is + * not capturing the required attributes of the source table from the MERGE + * action list. Here, we temporarily pass the original query to get the + * correct target list of the source rows. + */ + Query *saveQuery = relationRestriction->plannerInfo->parse; + relationRestriction->plannerInfo->parse = mergeQuery; + List *requiredAttributes = + RequiredAttrNumbersForRelation(sourceRte, plannerRestrictionContext); + + /* Restore the planner's parse tree */ + relationRestriction->plannerInfo->parse = saveQuery; + + sourceResultsQuery->commandType = CMD_SELECT; + + /* we copy the input rteRelation to preserve the rteIdentity */ + RangeTblEntry *newRangeTableEntry = copyObject(sourceRte); + sourceResultsQuery->rtable = list_make1(newRangeTableEntry); + + /* set the FROM expression to the subquery */ + newRangeTableRef->rtindex = SINGLE_RTE_INDEX; + sourceResultsQuery->jointree = makeFromExpr(list_make1(newRangeTableRef), NULL); + sourceResultsQuery->targetList = + CreateAllTargetListForRelation(sourceRte->relid, requiredAttributes); + List *restrictionList = + GetRestrictInfoListForRelation(sourceRte, plannerRestrictionContext); + List *copyRestrictionList = copyObject(restrictionList); + Expr *andedBoundExpressions = make_ands_explicit(copyRestrictionList); + sourceResultsQuery->jointree->quals = (Node *) andedBoundExpressions; + + /* + * Originally the quals were pointing to the RTE and its varno + * was pointing to its index in rtable. However now we converted the RTE + * to a subquery and the quals should be pointing to that subquery, which + * is the only RTE in its rtable, hence we update the varnos so that they + * point to the subquery RTE. + * Originally: rtable: [rte1, current_rte, rte3...] + * Now: rtable: [rte1, subquery[current_rte], rte3...] --subquery[current_rte] refers to its rtable. + */ + Node *quals = sourceResultsQuery->jointree->quals; + UpdateVarNosInNode(quals, SINGLE_RTE_INDEX); + + /* replace the function with the constructed subquery */ + sourceRte->rtekind = RTE_SUBQUERY; + sourceRte->subquery = sourceResultsQuery; + sourceRte->inh = false; +} + + +/* + * ConvertSourceRTEIntoSubquery converts MERGE's source RTE into a subquery, + * whose result rows are repartitioned during runtime. + */ +static void +ConvertSourceRTEIntoSubquery(Query *mergeQuery, RangeTblEntry *sourceRte, + PlannerRestrictionContext *plannerRestrictionContext) +{ + switch (sourceRte->rtekind) + { + case RTE_SUBQUERY: + { + /* If RTE is already a subquery, this is a no-op */ + return; + } + + case RTE_RELATION: + { + ConvertRelationRTEIntoSubquery(mergeQuery, + sourceRte, plannerRestrictionContext); + return; + } + + case RTE_CTE: + { + ConvertCteRTEIntoSubquery(mergeQuery, sourceRte); + return; + } + + default: + { + ereport(ERROR, (errmsg("Unexpected source relation type(rtekind:%d)", + sourceRte->rtekind))); + } + } +} + + +/* + * ExtractSourceRangeTableEntry returns the range table entry of source table or query. + */ +RangeTblEntry * +ExtractSourceRangeTableEntry(Query *query) +{ + Assert(IsMergeQuery(query)); + + List *fromList = query->jointree->fromlist; + + /* We should have only one RTE(MergeStmt->sourceRelation) in the from-list */ + if (list_length(fromList) != 1) + { + ereport(ERROR, (errmsg("Unexpected source list in MERGE sql USING clause"))); + } + + RangeTblRef *reference = linitial(fromList); + + /* + * The planner sometimes generates JoinExprs internally; these can + * have rtindex = 0 if there are no join alias variables referencing + * such joins. + */ + if (reference->rtindex == 0) + { + ereport(ERROR, (errmsg("Source is not an explicit query"), + errhint("Source query is a Join expression, " + "try converting into a query as SELECT * " + "FROM (..Join..)"))); + } + + Assert(reference->rtindex >= 1); + RangeTblEntry *subqueryRte = rt_fetch(reference->rtindex, query->rtable); + + return subqueryRte; +} + + +/* + * CreateRouterMergePlan attempts to create a pushable plan for the given MERGE + * SQL statement. If planning fails ->planningError is set to a description + * of the failure. + */ +static DistributedPlan * +CreateRouterMergePlan(Oid targetRelationId, Query *originalQuery, Query *query, + List *rangeTableList, + PlannerRestrictionContext *plannerRestrictionContext) +{ + DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); + + Assert(originalQuery->commandType == CMD_MERGE); + Assert(OidIsValid(targetRelationId)); + + distributedPlan->planningError = DeferErrorIfRoutableMergeNotSupported(originalQuery, + rangeTableList, + plannerRestrictionContext); + if (distributedPlan->planningError != NULL) + { + return distributedPlan; + } + + ErrorIfInsertNotMatchTargetDistributionColumn(targetRelationId, originalQuery, NULL); + + Job *job = RouterJob(originalQuery, plannerRestrictionContext, + &distributedPlan->planningError); + + if (distributedPlan->planningError != NULL) + { + return distributedPlan; + } + + ereport(DEBUG1, (errmsg("Creating MERGE router plan"))); + + distributedPlan->workerJob = job; + distributedPlan->targetRelationId = targetRelationId; + distributedPlan->modLevel = RowModifyLevelForQuery(query); + + /* There is no coordinator query for MERGE */ + distributedPlan->combineQuery = NULL; + + /* MERGE doesn't support RETURNING clause */ + distributedPlan->expectResults = false; + distributedPlan->fastPathRouterPlan = + plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; + + return distributedPlan; +} + + +/* + * CreateRepartitionMergePlan + */ +static DistributedPlan * +CreateRepartitionMergePlan(Oid targetRelationId, uint64 planId, Query *originalQuery, + Query *query, + PlannerRestrictionContext *plannerRestrictionContext, + ParamListInfo boundParams) +{ + Query *mergeQuery = copyObject(originalQuery); + RangeTblEntry *sourceRte = ExtractSourceRangeTableEntry(mergeQuery); + DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); + + + ereport(DEBUG1, (errmsg("Creating MERGE repartition plan"))); + ConvertSourceRTEIntoSubquery(mergeQuery, sourceRte, plannerRestrictionContext); + Query *sourceQuery = sourceRte->subquery; + + ErrorIfRepartitionMergeNotSupported(targetRelationId, mergeQuery, sourceQuery); + + /* + * Make a copy of the source query, since following code scribbles it + * but we need to keep the original for EXPLAIN. + */ + Query *sourceQueryCopy = copyObject(sourceQuery); + + /* plan the subquery, this may be another distributed query */ + int cursorOptions = CURSOR_OPT_PARALLEL_OK; + PlannedStmt *sourceRowsPlan = pg_plan_query(sourceQueryCopy, NULL, cursorOptions, + boundParams); + + if (!IsCitusCustomScan(sourceRowsPlan->planTree)) + { + /* + * Ideally, with all the earlier checks, we shouldn't be here, this + * is an extra safety-net as merge-executor expects citus tables. + */ + ereport(ERROR, (errmsg("To MERGE into a distributed table, " + "source has to be Citus table"))); + } + + /* There is no coordinator query for MERGE */ + distributedPlan->combineQuery = NULL; + + /* MERGE doesn't support RETURNING clause */ + distributedPlan->expectResults = false; + + distributedPlan->modLevel = RowModifyLevelForQuery(mergeQuery); + distributedPlan->targetRelationId = targetRelationId; + distributedPlan->intermediateResultIdPrefix = InsertSelectResultIdPrefix(planId); + distributedPlan->insertSelectQuery = mergeQuery; + distributedPlan->selectPlanForInsertSelect = sourceRowsPlan; + distributedPlan->fastPathRouterPlan = + plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; + + return distributedPlan; +} + + +/* + * ErrorIfMergeNotSupported Checks for conditions that are not supported in either + * the routable or repartition strategies. It checks for + * - Supported table types and their combinations + * - Check the target lists and quals of both the query and merge actions + * - Supported CTEs + */ +static void +ErrorIfMergeNotSupported(Query *query, Oid targetRelationId, List *rangeTableList) +{ + ErrorIfMergeHasUnsupportedTables(targetRelationId, rangeTableList); + ErrorIfMergeQueryQualAndTargetListNotSupported(targetRelationId, query); + ErrorIfUnsupportedCTEs(query); +} + + +/* + * DeferErrorIfRoutableMergeNotSupported Checks for conditions that prevent pushable planning, if + * found, raises a deferred error, which then continues to try repartitioning strategy. + */ +static DeferredErrorMessage * +DeferErrorIfRoutableMergeNotSupported(Query *query, List *rangeTableList, + PlannerRestrictionContext *plannerRestrictionContext) +{ + List *distTablesList = NIL; + List *refTablesList = NIL; + List *localTablesList = NIL; + RangeTblEntry *rangeTableEntry = NULL; + + foreach_ptr(rangeTableEntry, rangeTableList) + { + Oid relationId = rangeTableEntry->relid; + + if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) + { + distTablesList = lappend(distTablesList, rangeTableEntry); + } + else if (IsCitusTableType(relationId, REFERENCE_TABLE)) + { + refTablesList = lappend(refTablesList, rangeTableEntry); + } + else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) + { + localTablesList = lappend(localTablesList, rangeTableEntry); + } + } + + if (list_length(distTablesList) > 0 && list_length(refTablesList) > 0) + { + ereport(DEBUG1, (errmsg( + "A mix of distributed and reference table, try repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "A mix of distributed and reference table, " + "routable query is not possible", NULL, NULL); + } + + if (list_length(distTablesList) > 0 && list_length(localTablesList) > 0) + { + ereport(DEBUG1, (errmsg( + "A mix of distributed and local table, try repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "A mix of distributed and citus-local table, " + "routable query is not possible", NULL, NULL); + } + + /* + * If all tables are either local or reference tables, no need to proceed further down + * as the below checks are applicable for distributed tables only + */ + if (list_length(distTablesList) == 0) + { + return NULL; + } + + /* Only one distributed tablle is involved in the MERGE */ + if (list_length(distTablesList) == 1) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "There is only one distributed table, merge is not " + "pushable, try repartitioning", NULL, NULL); + } + + /* Ensure all distributed tables are indeed co-located */ + if (!AllDistributedRelationsInRTEListColocated(distTablesList)) + { + ereport(DEBUG1, (errmsg("Distributed tables are not co-located, try " + "repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "For MERGE command, all the distributed tables " + "must be colocated", NULL, NULL); + } + + DeferredErrorMessage *deferredError = + DeferErrorIfUnsupportedSubqueryPushdown(query, + plannerRestrictionContext); + if (deferredError) + { + ereport(DEBUG1, (errmsg("Sub-query is not pushable, try repartitioning"))); + return deferredError; + } + + if (HasDangerousJoinUsing(query->rtable, (Node *) query->jointree)) + { + ereport(DEBUG1, (errmsg( + "Query has ambigious joins, merge is not pushable, try repartitioning"))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "a join with USING causes an internal naming " + "conflict, use ON instead", NULL, NULL); + } + + return NULL; +} + + +/* + * ErrorIfMergeQueryQualAndTargetListNotSupported does check for a MERGE command in the query, if it finds * one, it will verify the below criteria - * - Supported tables and combinations in ErrorIfMergeHasUnsupportedTables - * - Distributed tables requirements in ErrorIfDistTablesNotColocated + * - Distributed tables co-location requirements * - Checks target-lists and functions-in-quals in TargetlistAndFunctionsSupported */ -DeferredErrorMessage * -MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQuery, - PlannerRestrictionContext *plannerRestrictionContext) +static void +ErrorIfMergeQueryQualAndTargetListNotSupported(Oid targetRelationId, Query *originalQuery) { /* function is void for pre-15 versions of Postgres */ #if PG_VERSION_NUM < PG_VERSION_15 - return NULL; + ereport(ERROR, (errmsg("MERGE is not supported in pre-15 Postgres versions"))); #else @@ -585,42 +931,20 @@ MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQ */ if (contain_mutable_functions((Node *) originalQuery)) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "non-IMMUTABLE functions are not yet supported " - "in MERGE sql with distributed tables ", - NULL, NULL); + ereport(ERROR, (errmsg("non-IMMUTABLE functions are not yet " + "supported in MERGE sql with distributed tables"))); } - List *rangeTableList = ExtractRangeTableEntryList(originalQuery); - - /* - * Fast path queries cannot have merge command, and we prevent the remaining here. - * In Citus we have limited support for MERGE, it's allowed only if all - * the tables(target, source or any CTE) tables are are local i.e. a - * combination of Citus local and Non-Citus tables (regular Postgres tables) - * or distributed tables with some restrictions, please see header of routine - * ErrorIfDistTablesNotColocated for details. - */ DeferredErrorMessage *deferredError = - ErrorIfMergeHasUnsupportedTables(resultRelationId, - originalQuery, - rangeTableList, - plannerRestrictionContext); - if (deferredError) - { - /* MERGE's unsupported combination, raise the exception */ - RaiseDeferredError(deferredError, ERROR); - } + MergeQualAndTargetListFunctionsSupported(targetRelationId, + originalQuery->jointree, + originalQuery->jointree->quals, + originalQuery->targetList, + originalQuery->commandType); - deferredError = MergeQualAndTargetListFunctionsSupported(resultRelationId, - originalQuery->jointree, - originalQuery->jointree-> - quals, - originalQuery->targetList, - originalQuery->commandType); if (deferredError) { - return deferredError; + RaiseDeferredError(deferredError, ERROR); } /* @@ -631,7 +955,7 @@ MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQ foreach_ptr(action, originalQuery->mergeActionList) { Assert(originalQuery->returningList == NULL); - deferredError = MergeQualAndTargetListFunctionsSupported(resultRelationId, + deferredError = MergeQualAndTargetListFunctionsSupported(targetRelationId, originalQuery->jointree, action->qual, action->targetList, @@ -643,85 +967,187 @@ MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQ } } - deferredError = - InsertDistributionColumnMatchesSource(resultRelationId, originalQuery); - if (deferredError) + #endif +} + + +#endif + +/* + * ErrorIfInsertNotMatchTargetDistributionColumn checks to see if MERGE is inserting a + * value into the target which is not from the source table, if so, it + * raises an exception. + * Note: Inserting random values other than the joined column values will + * result in unexpected behaviour of rows ending up in incorrect shards, to + * prevent such mishaps, we disallow such inserts here. + */ +void +ErrorIfInsertNotMatchTargetDistributionColumn(Oid targetRelationId, Query *query, + Var *sourceJoinColumn) +{ + /* function is void for pre-15 versions of Postgres */ + #if PG_VERSION_NUM < PG_VERSION_15 + + ereport(ERROR, (errmsg("MERGE is not supported in pre-15 Postgres versions"))); + + #else + + Assert(IsMergeQuery(query)); + + if (!IsCitusTableType(targetRelationId, DISTRIBUTED_TABLE)) { - /* MERGE's unsupported scenario, raise the exception */ - RaiseDeferredError(deferredError, ERROR); + return; } - if (multiShardQuery) + if (!HasDistributionKey(targetRelationId)) { - deferredError = - DeferErrorIfUnsupportedSubqueryPushdown(originalQuery, - plannerRestrictionContext); - if (deferredError) + return; + } + + bool foundDistributionColumn = false; + MergeAction *action = NULL; + foreach_ptr(action, query->mergeActionList) + { + /* Skip MATCHED clause as INSERTS are not allowed in it*/ + if (action->matched) { - return deferredError; + continue; + } + + /* NOT MATCHED can have either INSERT or DO NOTHING */ + if (action->commandType == CMD_NOTHING) + { + return; + } + + if (action->targetList == NIL) + { + /* INSERT DEFAULT VALUES is not allowed */ + ereport(ERROR, (errmsg("cannot perform MERGE INSERT with DEFAULTS"))); + } + + Assert(action->commandType == CMD_INSERT); + Var *targetKey = PartitionColumn(targetRelationId, 1); + + TargetEntry *targetEntry = NULL; + foreach_ptr(targetEntry, action->targetList) + { + AttrNumber originalAttrNo = targetEntry->resno; + + /* skip processing of target table non-distribution columns */ + if (originalAttrNo != targetKey->varattno) + { + continue; + } + + foundDistributionColumn = true; + + /* Found a Var inserting into target's distribution column */ + if (!IsA(targetEntry->expr, Var)) + { + /* Inserting constant VALUES is not allowed */ + ereport(ERROR, (errmsg("MERGE INSERT must refer a source column " + "for distribution column"))); + } + + /* + * Check if the Var is appropriate to be inserted into distribution + * column of the target + */ + if (sourceJoinColumn) + { + /* + * Source and target are non-colocated, source is being + * is repartitioned and redistributed on the Join-column + * with the target, we *must* insert this id value only. + */ + if (!equal((Var *) targetEntry->expr, sourceJoinColumn)) + { + ereport(ERROR, + (errmsg("MERGE INSERT must use the source's joining column " + "for target's distribution column"))); + } + } + else + { + /* + * Source and target are colocated and joined on their respective + * distribution columns, we *must* insert source's distribution id + */ + if (!IsDistributionColumnInMergeSource(targetEntry->expr, query, true)) + { + ereport(ERROR, (errmsg("MERGE INSERT must use the source table " + "distribution column value"))); + } + } + + /* Allowed value */ + return; + } + + if (!foundDistributionColumn) + { + ereport(ERROR, + (errmsg("MERGE INSERT must have distribution column as value"))); } } - if (HasDangerousJoinUsing(originalQuery->rtable, (Node *) originalQuery->jointree)) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "a join with USING causes an internal naming " - "conflict, use ON instead", NULL, NULL); - } - - return NULL; - #endif } /* - * CreateMergePlan attempts to create a plan for the given MERGE SQL - * statement. If planning fails ->planningError is set to a description - * of the failure. + * CreateMergePlan + * 1) Check for conditions that are not supported in MERGE command. + * 2) Try to create a pushable plan + * - Check for conditions suitable for a routable plan, if not found, + * raise deferred error + * 3) Try to create repartition and redistribution plan + * - Check for conditions that prevent repartition strategy, if found, + * raise an exception and quit. */ DistributedPlan * -CreateMergePlan(Query *originalQuery, Query *query, - PlannerRestrictionContext *plannerRestrictionContext) +CreateMergePlan(uint64 planId, Query *originalQuery, Query *query, + PlannerRestrictionContext *plannerRestrictionContext, + ParamListInfo boundParams) { - DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); - bool multiShardQuery = false; + /* function is void for pre-15 versions of Postgres */ + #if PG_VERSION_NUM < PG_VERSION_15 + + ereport(ERROR, (errmsg("MERGE is not supported in pre-15 Postgres versions"))); + + #else + Oid targetRelationId = ModifyQueryResultRelationId(originalQuery); - Assert(originalQuery->commandType == CMD_MERGE); - Assert(OidIsValid(targetRelationId)); + /* + * Step 1: Look for definitive error conditions applicable to both Routable + * and Repartition strategies. + */ + List *rangeTableList = ExtractRangeTableEntryList(originalQuery); + ErrorIfMergeNotSupported(originalQuery, targetRelationId, rangeTableList); - distributedPlan->targetRelationId = targetRelationId; - distributedPlan->modLevel = RowModifyLevelForQuery(query); - distributedPlan->planningError = MergeQuerySupported(targetRelationId, - originalQuery, - multiShardQuery, - plannerRestrictionContext); + /* Step 2: Try pushable merge plan */ + DistributedPlan *distributedPlan = + CreateRouterMergePlan(targetRelationId, originalQuery, query, + rangeTableList, plannerRestrictionContext); + /* Step 3: If the routing plan failed, try for repartition strategy */ if (distributedPlan->planningError != NULL) { - return distributedPlan; + RaiseDeferredError(distributedPlan->planningError, DEBUG1); + + /* If MERGE is not routable, try repartitioning */ + distributedPlan = + CreateRepartitionMergePlan(targetRelationId, planId, + originalQuery, query, + plannerRestrictionContext, + boundParams); } - Job *job = RouterJob(originalQuery, plannerRestrictionContext, - &distributedPlan->planningError); - - if (distributedPlan->planningError != NULL) - { - return distributedPlan; - } - - ereport(DEBUG1, (errmsg("Creating MERGE router plan"))); - - distributedPlan->workerJob = job; - distributedPlan->combineQuery = NULL; - - /* MERGE doesn't support RETURNING clause */ - distributedPlan->expectResults = false; - distributedPlan->fastPathRouterPlan = - plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; - return distributedPlan; + + #endif } diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index c23509df1..ccd042afb 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -33,6 +33,7 @@ #include "distributed/insert_select_planner.h" #include "distributed/insert_select_executor.h" #include "distributed/listutils.h" +#include "distributed/merge_planner.h" #include "distributed/multi_executor.h" #include "distributed/multi_explain.h" #include "distributed/multi_logical_optimizer.h" @@ -281,6 +282,58 @@ NonPushableInsertSelectExplainScan(CustomScanState *node, List *ancestors, } +/* + * NonPushableMergeSqlExplainScan is a custom scan explain callback function + * which is used to print explain information of a Citus plan for MERGE INTO + * distributed_table USING (source query/table), where source can be any query + * whose results are repartitioned to colocated with the target table. + */ +void +NonPushableMergeCommandExplainScan(CustomScanState *node, List *ancestors, + struct ExplainState *es) +{ + CitusScanState *scanState = (CitusScanState *) node; + DistributedPlan *distributedPlan = scanState->distributedPlan; + Query *mergeQuery = distributedPlan->insertSelectQuery; + RangeTblEntry *sourceRte = ExtractSourceRangeTableEntry(mergeQuery); + + /* + * Create a copy because ExplainOneQuery can modify the query, and later + * executions of prepared statements might require it. See + * https://github.com/citusdata/citus/issues/3947 for what can happen. + */ + Query *sourceQueryCopy = copyObject(sourceRte->subquery); + + if (es->analyze) + { + ereport(ERROR, (errmsg("EXPLAIN ANALYZE is currently not supported for " + "MERGE INTO ... commands with repartitioning"))); + } + + Oid targetRelationId = ModifyQueryResultRelationId(mergeQuery); + StringInfo mergeMethodMessage = makeStringInfo(); + appendStringInfo(mergeMethodMessage, + "MERGE INTO %s method", get_rel_name(targetRelationId)); + + ExplainPropertyText(mergeMethodMessage->data, "source repartition", es); + + ExplainOpenGroup("Source Query", "Source Query", false, es); + + /* explain the MERGE source query */ + IntoClause *into = NULL; + ParamListInfo params = NULL; + + /* + * With PG14, we need to provide a string here, for now we put an empty + * string, which is valid according to postgres. + */ + char *queryString = pstrdup(""); + ExplainOneQuery(sourceQueryCopy, 0, into, es, queryString, params, NULL); + + ExplainCloseGroup("Source Query", "Source Query", false, es); +} + + /* * ExplainSubPlans generates EXPLAIN output for subplans for CTEs * and complex subqueries. Because the planning for these queries diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 0c6ec9dca..2506c68f6 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -1879,24 +1879,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon if (*planningError) { - /* - * For MERGE, we do _not_ plan any other router job than the MERGE job itself, - * let's not continue further down the lane in distributed planning, simply - * bail out. - */ - if (IsMergeQuery(originalQuery)) - { - if (ContainsSingleShardTable(originalQuery)) - { - WrapRouterErrorForSingleShardTable(*planningError); - } - - RaiseDeferredError(*planningError, ERROR); - } - else - { - return NULL; - } + return NULL; } Job *job = CreateJob(originalQuery); @@ -2368,14 +2351,7 @@ PlanRouterQuery(Query *originalQuery, Assert(UpdateOrDeleteOrMergeQuery(originalQuery)); - if (IsMergeQuery(originalQuery)) - { - targetRelationId = ModifyQueryResultRelationId(originalQuery); - planningError = MergeQuerySupported(targetRelationId, originalQuery, - isMultiShardQuery, - plannerRestrictionContext); - } - else + if (!IsMergeQuery(originalQuery)) { planningError = ModifyQuerySupported(originalQuery, originalQuery, isMultiShardQuery, diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index 936b17364..f582fd9df 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -188,7 +188,6 @@ static Query * BuildReadIntermediateResultsQuery(List *targetEntryList, List *columnAliasList, Const *resultIdConst, Oid functionOid, bool useBinaryCopyFormat); -static void UpdateVarNosInNode(Node *node, Index newVarNo); static Query * CreateOuterSubquery(RangeTblEntry *rangeTableEntry, List *outerSubqueryTargetList); static List * GenerateRequiredColNamesFromTargetList(List *targetList); @@ -1891,7 +1890,7 @@ GenerateRequiredColNamesFromTargetList(List *targetList) * UpdateVarNosInNode iterates the Vars in the * given node and updates the varno's as the newVarNo. */ -static void +void UpdateVarNosInNode(Node *node, Index newVarNo) { List *varList = pull_var_clause(node, PVC_RECURSE_AGGREGATES | diff --git a/src/include/distributed/citus_custom_scan.h b/src/include/distributed/citus_custom_scan.h index f31138ac2..a3da4958c 100644 --- a/src/include/distributed/citus_custom_scan.h +++ b/src/include/distributed/citus_custom_scan.h @@ -34,6 +34,7 @@ typedef struct CitusScanState extern CustomScanMethods AdaptiveExecutorCustomScanMethods; extern CustomScanMethods NonPushableInsertSelectCustomScanMethods; extern CustomScanMethods DelayedErrorCustomScanMethods; +extern CustomScanMethods NonPushableMergeCommandCustomScanMethods; extern void RegisterCitusCustomScanMethods(void); diff --git a/src/include/distributed/insert_select_executor.h b/src/include/distributed/insert_select_executor.h index 6e84b80f2..81326d9a7 100644 --- a/src/include/distributed/insert_select_executor.h +++ b/src/include/distributed/insert_select_executor.h @@ -21,6 +21,10 @@ extern bool EnableRepartitionedInsertSelect; extern TupleTableSlot * NonPushableInsertSelectExecScan(CustomScanState *node); extern bool IsSupportedRedistributionTarget(Oid targetRelationId); extern bool IsRedistributablePlan(Plan *selectPlan); - +extern List * RedistributedInsertSelectTaskList(Query *insertSelectQuery, + CitusTableCacheEntry *targetRelation, + List **redistributedResults, + bool useBinaryFormat); +extern int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn); #endif /* INSERT_SELECT_EXECUTOR_H */ diff --git a/src/include/distributed/insert_select_planner.h b/src/include/distributed/insert_select_planner.h index 74b8a0708..771d1d60f 100644 --- a/src/include/distributed/insert_select_planner.h +++ b/src/include/distributed/insert_select_planner.h @@ -44,6 +44,7 @@ extern DistributedPlan * CreateInsertSelectIntoLocalTablePlan(uint64 planId, plannerRestrictionContext); extern char * InsertSelectResultIdPrefix(uint64 planId); extern bool PlanningInsertSelect(void); +extern Query * WrapSubquery(Query *subquery); #endif /* INSERT_SELECT_PLANNER_H */ diff --git a/src/include/distributed/merge_executor.h b/src/include/distributed/merge_executor.h new file mode 100644 index 000000000..0bc31ab74 --- /dev/null +++ b/src/include/distributed/merge_executor.h @@ -0,0 +1,17 @@ +/*------------------------------------------------------------------------- + * + * merge_executor.h + * + * Declarations for public functions and types related to executing + * MERGE INTO ... SQL commands. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ +#ifndef MERGE_EXECUTOR_H +#define MERGE_EXECUTOR_H + +extern TupleTableSlot * NonPushableMergeCommandExecScan(CustomScanState *node); + +#endif /* MERGE_EXECUTOR_H */ diff --git a/src/include/distributed/merge_planner.h b/src/include/distributed/merge_planner.h index b4ec1852f..ff40ca821 100644 --- a/src/include/distributed/merge_planner.h +++ b/src/include/distributed/merge_planner.h @@ -19,16 +19,20 @@ #include "distributed/errormessage.h" #include "distributed/multi_physical_planner.h" -extern DeferredErrorMessage * MergeQuerySupported(Oid resultRelationId, - Query *originalQuery, - bool multiShardQuery, - PlannerRestrictionContext * - plannerRestrictionContext); -extern DistributedPlan * CreateMergePlan(Query *originalQuery, Query *query, +extern DistributedPlan * CreateMergePlan(uint64 planId, Query *originalQuery, + Query *query, PlannerRestrictionContext * - plannerRestrictionContext); + plannerRestrictionContext, + ParamListInfo boundParams); extern bool IsLocalTableModification(Oid targetRelationId, Query *query, uint64 shardId, RTEListProperties *rteProperties); +extern void NonPushableMergeCommandExplainScan(CustomScanState *node, List *ancestors, + struct ExplainState *es); +extern void ErrorIfInsertNotMatchTargetDistributionColumn(Oid targetRelationId, + Query *query, + Var *sourceJoinColumn); +extern RangeTblEntry * ExtractSourceRangeTableEntry(Query *query); + #endif /* MERGE_PLANNER_H */ diff --git a/src/include/distributed/multi_server_executor.h b/src/include/distributed/multi_server_executor.h index 56dd6d808..f49ef60c2 100644 --- a/src/include/distributed/multi_server_executor.h +++ b/src/include/distributed/multi_server_executor.h @@ -29,7 +29,8 @@ typedef enum { MULTI_EXECUTOR_INVALID_FIRST = 0, MULTI_EXECUTOR_ADAPTIVE = 1, - MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT = 2 + MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT = 2, + MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY = 3 } MultiExecutorType; diff --git a/src/include/distributed/recursive_planning.h b/src/include/distributed/recursive_planning.h index e849d7158..8943443aa 100644 --- a/src/include/distributed/recursive_planning.h +++ b/src/include/distributed/recursive_planning.h @@ -46,6 +46,7 @@ extern void ReplaceRTERelationWithRteSubquery(RangeTblEntry *rangeTableEntry, extern bool IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry); extern bool IsRelationLocalTableOrMatView(Oid relationId); extern bool ContainsReferencesToOuterQuery(Query *query); +extern void UpdateVarNosInNode(Node *node, Index newVarNo); #endif /* RECURSIVE_PLANNING_H */ diff --git a/src/test/regress/expected/merge.out b/src/test/regress/expected/merge.out index 3cf776ded..c2a5c4757 100644 --- a/src/test/regress/expected/merge.out +++ b/src/test/regress/expected/merge.out @@ -20,13 +20,14 @@ SET citus.next_shard_id TO 4000000; SET citus.explain_all_tasks TO true; SET citus.shard_replication_factor TO 1; SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); -NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata ?column? --------------------------------------------------------------------- 1 (1 row) +RESET client_min_messages; CREATE TABLE source ( order_id INT, @@ -1425,19 +1426,13 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut (1 row) BEGIN; -SET citus.log_remote_commands to true; MERGE INTO target_cj t -USING source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2 +USING (SELECT * FROM source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src2 WHEN NOT MATCHED THEN DO NOTHING; -NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx'); -DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx -NOTICE: issuing MERGE INTO merge_schema.target_cj_xxxxxxx t USING (merge_schema.source_cj1_xxxxxxx s1 JOIN merge_schema.source_cj2_xxxxxxx s2 ON ((s1.sid1 OPERATOR(pg_catalog.=) s2.sid2))) ON ((t.tid OPERATOR(pg_catalog.=) s1.sid1) AND (t.tid OPERATOR(pg_catalog.=) 2)) WHEN MATCHED THEN UPDATE SET src = s2.src2 WHEN NOT MATCHED THEN DO NOTHING -DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx -SET citus.log_remote_commands to false; SELECT * FROM target_cj ORDER BY 1; tid | src | val --------------------------------------------------------------------- @@ -1451,8 +1446,8 @@ ROLLBACK; BEGIN; -- try accessing columns from either side of the source join MERGE INTO target_cj t -USING source_cj1 s2 - INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10 +USING (SELECT * FROM source_cj1 s2 + INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src1, val = val2 @@ -1520,7 +1515,7 @@ SELECT * FROM target_cj ORDER BY 1; ROLLBACK; -- Test PREPARE -PREPARE foo(int) AS +PREPARE merge_prepare(int) AS MERGE INTO target_cj target USING (SELECT * FROM source_cj1) sub ON target.tid = sub.sid1 AND target.tid = $1 @@ -1538,11 +1533,11 @@ SELECT * FROM target_cj ORDER BY 1; (4 rows) BEGIN; -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); SELECT * FROM target_cj ORDER BY 1; tid | src | val --------------------------------------------------------------------- @@ -1556,7 +1551,7 @@ ROLLBACK; BEGIN; SET citus.log_remote_commands to true; SET client_min_messages TO DEBUG1; -EXECUTE foo(2); +EXECUTE merge_prepare(2); DEBUG: DEBUG: DEBUG: @@ -1569,7 +1564,7 @@ DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx NOTICE: issuing MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx RESET client_min_messages; -EXECUTE foo(2); +EXECUTE merge_prepare(2); NOTICE: issuing MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx SET citus.log_remote_commands to false; @@ -2526,9 +2521,442 @@ WHERE pg_result.t1 IS NULL OR local_ref.t1 IS NULL; 0 (1 row) +-- Now make target as distributed, keep reference as source +TRUNCATE reftarget_local; +TRUNCATE refsource_ref; +INSERT INTO reftarget_local VALUES(1, 0); +INSERT INTO reftarget_local VALUES(3, 100); +INSERT INTO refsource_ref VALUES(1, 1); +INSERT INTO refsource_ref VALUES(2, 2); +INSERT INTO refsource_ref VALUES(3, 3); +SELECT create_distributed_table('reftarget_local', 't1'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.reftarget_local$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO reftarget_local +USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1 +WHEN MATCHED AND reftarget_local.t2 = 100 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET t2 = t2 + 100 +WHEN NOT MATCHED THEN + INSERT VALUES(foo.s1); +SELECT * INTO dist_reftarget FROM reftarget_local ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_reftarget c, pg_result p +WHERE c.t1 = p.t1 +ORDER BY 1,2; + t1 | t2 | t1 | t2 +--------------------------------------------------------------------- + 1 | 100 | 1 | 100 + 2 | | 2 | +(2 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_result FULL OUTER JOIN dist_reftarget ON pg_result.t1 = dist_reftarget.t1 +WHERE pg_result.t1 IS NULL OR dist_reftarget.t1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- +-- Distributed (target), Reference(source) +-- +CREATE TABLE demo_distributed(id1 int, val1 int); +CREATE TABLE demo_source_table(id2 int, val2 int); +CREATE FUNCTION setup_demo_data() RETURNS VOID AS $$ +INSERT INTO demo_distributed VALUES(1, 100); +INSERT INTO demo_distributed VALUES(7, 100); +INSERT INTO demo_distributed VALUES(15, 100); +INSERT INTO demo_distributed VALUES(100, 0); +INSERT INTO demo_distributed VALUES(300, 100); +INSERT INTO demo_distributed VALUES(400, 0); + +INSERT INTO demo_source_table VALUES(1, 77); +INSERT INTO demo_source_table VALUES(15, 77); +INSERT INTO demo_source_table VALUES(75, 77); +INSERT INTO demo_source_table VALUES(100, 77); +INSERT INTO demo_source_table VALUES(300, 77); +INSERT INTO demo_source_table VALUES(400, 77); +INSERT INTO demo_source_table VALUES(500, 77); +$$ +LANGUAGE SQL; +CREATE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING demo_source_table s ON s.id2 = t.id1 +WHEN MATCHED AND t.val1= 0 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET val1 = val1 + s.val2 +WHEN NOT MATCHED THEN + INSERT VALUES(s.id2, s.val2); +$$ +LANGUAGE SQL; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT create_distributed_table('demo_distributed', 'id1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_reference_table('demo_source_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + id1 | val1 | id1 | val1 +--------------------------------------------------------------------- + 1 | 177 | 1 | 177 + 7 | 100 | 7 | 100 + 15 | 177 | 15 | 177 + 75 | 77 | 75 | 77 + 300 | 177 | 300 | 177 + 500 | 77 | 500 | 77 +(6 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- Now convert source as distributed, but non-colocated with target +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +NOTICE: creating a new table for merge_schema.demo_distributed +NOTICE: moving the data of merge_schema.demo_distributed +NOTICE: dropping the old merge_schema.demo_distributed +NOTICE: renaming the new table to merge_schema.demo_distributed + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +SELECT undistribute_table('demo_source_table'); +NOTICE: creating a new table for merge_schema.demo_source_table +NOTICE: moving the data of merge_schema.demo_source_table +NOTICE: dropping the old merge_schema.demo_source_table +NOTICE: renaming the new table to merge_schema.demo_source_table + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT id2,val2 FROM demo_source_table UNION SELECT val2,id2 FROM demo_source_table) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = val1 + 1; +$$ +LANGUAGE SQL; +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; +SELECT create_distributed_table('demo_distributed', 'id1'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_distributed$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_source_table$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + id1 | val1 | id1 | val1 +--------------------------------------------------------------------- + 1 | 101 | 1 | 101 + 7 | 100 | 7 | 100 + 15 | 101 | 15 | 101 + 100 | 1 | 100 | 1 + 300 | 101 | 300 | 101 + 400 | 1 | 400 | 1 +(6 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- Test with LIMIT +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); +$$ +LANGUAGE SQL; +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +NOTICE: creating a new table for merge_schema.demo_distributed +NOTICE: moving the data of merge_schema.demo_distributed +NOTICE: dropping the old merge_schema.demo_distributed +NOTICE: renaming the new table to merge_schema.demo_distributed + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +SELECT undistribute_table('demo_source_table'); +NOTICE: creating a new table for merge_schema.demo_source_table +NOTICE: moving the data of merge_schema.demo_source_table +NOTICE: dropping the old merge_schema.demo_source_table +NOTICE: renaming the new table to merge_schema.demo_source_table + undistribute_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; +SELECT create_distributed_table('demo_distributed', 'id1'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_distributed$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_schema.demo_source_table$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; +SELECT setup_demo_data(); + setup_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT merge_demo_data(); + merge_demo_data +--------------------------------------------------------------------- + +(1 row) + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + id1 | val1 | id1 | val1 +--------------------------------------------------------------------- + 1 | 999 | 1 | 999 + 7 | 100 | 7 | 100 + 15 | 999 | 15 | 999 + 75 | 999 | 75 | 999 + 100 | 0 | 100 | 0 + 300 | 100 | 300 | 100 + 400 | 0 | 400 | 0 +(7 rows) + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- Test explain with repartition +SET citus.explain_all_tasks TO false; +EXPLAIN (COSTS OFF) +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus MERGE INTO ...) + MERGE INTO demo_distributed method: source repartition + -> Custom Scan (Citus Adaptive) + -> Distributed Subplan XXX_1 + -> Limit + -> Sort + Sort Key: remote_scan.id2 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Limit + -> Sort + Sort Key: id2 + -> Seq Scan on demo_source_table_4000135 demo_source_table + -> Distributed Subplan XXX_2 + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Seq Scan on demo_source_table_4000135 demo_source_table + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Merge Left Join + Merge Cond: (intermediate_result.id2 = intermediate_result_1.id2) + -> Sort + Sort Key: intermediate_result.id2 + -> Function Scan on read_intermediate_result intermediate_result + -> Sort + Sort Key: intermediate_result_1.id2 + -> Function Scan on read_intermediate_result intermediate_result_1 +(35 rows) + -- -- Error and Unsupported scenarios -- +-- Test explain analyze with repartition +EXPLAIN ANALYZE +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); +ERROR: EXPLAIN ANALYZE is currently not supported for MERGE INTO ... commands with repartitioning +-- Source without a table +MERGE INTO target_cj t +USING (VALUES (1, 1), (2, 1), (3, 3)) as s (sid, val) +ON t.tid = s.sid AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET val = s.val +WHEN NOT MATCHED THEN + DO NOTHING; +ERROR: To MERGE into a distributed table, source must be Citus table(s) +-- Incomplete source +MERGE INTO target_cj t +USING (source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = val2) s +ON t.tid = s.sid1 AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET src = src2 +WHEN NOT MATCHED THEN + DO NOTHING; +ERROR: Source is not an explicit query +HINT: Source query is a Join expression, try converting into a query as SELECT * FROM (..Join..) -- Reference as a target and local as source MERGE INTO refsource_ref USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1 @@ -2537,40 +2965,20 @@ WHEN MATCHED THEN WHEN NOT MATCHED THEN INSERT VALUES(foo.t1); ERROR: Reference table as target is not allowed in MERGE command --- Reference as a source and distributed as target -MERGE INTO target_set t -USING refsource_ref AS s ON t.t1 = s.s1 -WHEN MATCHED THEN - DO NOTHING; -ERROR: MERGE command is not supported with combination of distributed/reference yet -HINT: If target is distributed, source must be distributed and co-located MERGE INTO target_set USING source_set AS foo ON target_set.t1 = foo.s1 WHEN MATCHED THEN UPDATE SET ctid = '(0,100)'; ERROR: cannot assign to system column "ctid" -MERGE INTO target_set -USING (SELECT s1,s2 FROM source_set UNION SELECT s2,s1 FROM source_set) AS foo ON target_set.t1 = foo.s1 -WHEN MATCHED THEN - UPDATE SET t2 = t2 + 1; -ERROR: cannot pushdown the subquery since not all subqueries in the UNION have the partition column in the same position -DETAIL: Each leaf query of the UNION should return the partition column in the same position and all joins must be on the partition column -MERGE INTO target_set -USING (SELECT 2 as s3, source_set.* FROM (SELECT * FROM source_set LIMIT 1) as foo LEFT JOIN source_set USING( s1)) AS foo -ON target_set.t1 = foo.s1 -WHEN MATCHED THEN UPDATE SET t2 = t2 + 1 -WHEN NOT MATCHED THEN INSERT VALUES(s1, s3); -ERROR: cannot push down this subquery -DETAIL: Limit clause is currently unsupported when a subquery references a column from another query -- modifying CTE not supported EXPLAIN -WITH cte_1 AS (DELETE FROM target_json) +WITH cte_1 AS (DELETE FROM target_json RETURNING *) MERGE INTO target_json sda -USING source_json sdn +USING cte_1 sdn ON sda.id = sdn.id WHEN NOT matched THEN INSERT (id, z) VALUES (sdn.id, 5); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: CTEs with modifying actions are not yet supported in MERGE -- Grouping sets not supported MERGE INTO citus_target t USING (SELECT count(*), id FROM citus_source GROUP BY GROUPING SETS (id, val)) subq @@ -2581,8 +2989,8 @@ WHEN NOT MATCHED THEN INSERT VALUES (subq.id, 99) WHEN MATCHED AND t.id < 350 THEN DELETE; -ERROR: cannot push down this subquery -DETAIL: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +HINT: Consider using an equality filter on the distributed table's partition column. WITH subq AS ( SELECT count(*), id FROM citus_source GROUP BY GROUPING SETS (id, val) @@ -2596,8 +3004,8 @@ WHEN NOT MATCHED THEN INSERT VALUES (subq.id, 99) WHEN MATCHED AND t.id < 350 THEN DELETE; -ERROR: cannot push down this subquery -DETAIL: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +HINT: Consider using an equality filter on the distributed table's partition column. -- try inserting unmatched distribution column value MERGE INTO citus_target t USING citus_source s @@ -2617,6 +3025,7 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT (id) VALUES(1000); ERROR: MERGE INSERT must refer a source column for distribution column +-- Colocated merge MERGE INTO t1 t USING s1 s ON t.id = s.id @@ -2629,6 +3038,13 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT (val) VALUES(s.val); ERROR: MERGE INSERT must have distribution column as value +-- Non-colocated merge +MERGE INTO t1 t +USING s1 s +ON t.id = s.val +WHEN NOT MATCHED THEN + INSERT (id) VALUES(s.id); +ERROR: MERGE INSERT must use the source's joining column for target's distribution column -- try updating the distribution key column BEGIN; MERGE INTO target_cj t @@ -2698,7 +3114,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1.id, s1.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE involving repartition of rows is supported only if the target is distributed -- Now both s1 and t1 are distributed tables SELECT undistribute_table('t1'); NOTICE: creating a new table for merge_schema.t1 @@ -2764,17 +3180,7 @@ WHEN MATCHED AND (merge_when_and_write()) THEN UPDATE SET val = t1.val + s1.val; ERROR: non-IMMUTABLE functions are not yet supported in MERGE sql with distributed tables ROLLBACK; --- Joining on partition columns with sub-query -MERGE INTO t1 - USING (SELECT * FROM s1) sub ON (sub.val = t1.id) -- sub.val is not a distribution column - WHEN MATCHED AND sub.val = 0 THEN - DELETE - WHEN MATCHED THEN - UPDATE SET val = t1.val + 1 - WHEN NOT MATCHED THEN - INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- Joining on partition columns with CTE +-- Joining on non-partition columns with CTE source, but INSERT incorrect column WITH s1_res AS ( SELECT * FROM s1 ) @@ -2786,7 +3192,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: MERGE INSERT must use the source's joining column for target's distribution column -- Constant Join condition WITH s1_res AS ( SELECT * FROM s1 @@ -2799,8 +3205,9 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- With a single WHEN clause, which causes a non-left join +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting +-- Join condition without target distribution column WITH s1_res AS ( SELECT * FROM s1 ) @@ -2808,7 +3215,8 @@ WITH s1_res AS ( WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting -- -- Reference tables -- @@ -2902,7 +3310,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1.id, s1.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported MERGE INTO t1 USING (SELECT * FROM s1) sub ON (sub.id = t1.id) WHEN MATCHED AND sub.val = 0 THEN @@ -2911,7 +3319,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported CREATE TABLE pg(val int); SELECT create_distributed_table('s1', 'id'); NOTICE: Copying data from local table... @@ -2932,7 +3340,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported -- Mix Postgres table in CTE WITH pg_res AS ( SELECT * FROM pg @@ -2945,7 +3353,7 @@ MERGE INTO t1 UPDATE SET val = t1.val + 1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (sub.id, sub.val); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported -- Match more than one source row should fail same as Postgres behavior SELECT undistribute_table('t1'); NOTICE: creating a new table for merge_schema.t1 @@ -3000,7 +3408,7 @@ WHEN NOT MATCHED THEN INSERT VALUES(mv_source.id, mv_source.val); ERROR: cannot execute MERGE on relation "mv_source" DETAIL: This operation is not supported for materialized views. --- Distributed tables *must* be colocated +-- Do not allow constant values into the distribution column CREATE TABLE dist_target(id int, val varchar); SELECT create_distributed_table('dist_target', 'id'); create_distributed_table @@ -3015,31 +3423,6 @@ SELECT create_distributed_table('dist_source', 'id', colocate_with => 'none'); (1 row) -MERGE INTO dist_target -USING dist_source -ON dist_target.id = dist_source.id -WHEN MATCHED THEN -UPDATE SET val = dist_source.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated --- Distributed tables *must* be joined on distribution column -CREATE TABLE dist_colocated(id int, val int); -SELECT create_distributed_table('dist_colocated', 'id', colocate_with => 'dist_target'); - create_distributed_table ---------------------------------------------------------------------- - -(1 row) - -MERGE INTO dist_target -USING dist_colocated -ON dist_target.id = dist_colocated.val -- val is not the distribution column -WHEN MATCHED THEN -UPDATE SET val = dist_colocated.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_colocated.id, dist_colocated.val); -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- Both the source and target must be distributed MERGE INTO dist_target USING (SELECT 100 id) AS source ON dist_target.id = source.id AND dist_target.val = 'const' @@ -3047,7 +3430,7 @@ WHEN MATCHED THEN UPDATE SET val = 'source' WHEN NOT MATCHED THEN INSERT VALUES(source.id, 'source'); -ERROR: For MERGE command, both the source and target must be distributed +ERROR: To MERGE into a distributed table, source must be Citus table(s) -- Non-hash distributed tables (append/range). CREATE VIEW show_tables AS SELECT logicalrelid, partmethod @@ -3086,8 +3469,7 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet SELECT undistribute_table('dist_source'); NOTICE: creating a new table for merge_schema.dist_source NOTICE: moving the data of merge_schema.dist_source @@ -3120,8 +3502,7 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet -- Both are append tables SELECT undistribute_table('dist_target'); NOTICE: creating a new table for merge_schema.dist_target @@ -3173,8 +3554,7 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet -- Both are range tables SELECT undistribute_table('dist_target'); NOTICE: creating a new table for merge_schema.dist_target @@ -3226,8 +3606,7 @@ WHEN MATCHED THEN UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); -ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported -HINT: Consider using hash distribution instead +ERROR: For MERGE command, append/range distribution table is not supported yet -- test merge with single-shard tables CREATE SCHEMA query_single_shard_table; SET search_path TO query_single_shard_table; @@ -3295,44 +3674,82 @@ INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; -- with a colocated table MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN DELETE; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan -- with non-colocated single-shard table MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b; -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b); -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting -- with a distributed table MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a) WHEN MATCHED THEN UPDATE SET b = distributed_table.b WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b); -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +DEBUG: Using column - index:0 from the source list to redistribute +DEBUG: Executing final MERGE on workers -- with a reference table MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a) WHEN MATCHED THEN UPDATE SET b = reference_table.b; -ERROR: MERGE command is not supported with combination of distributed/reference yet -HINT: If target is distributed, source must be distributed and co-located +DEBUG: A mix of distributed and reference table, try repartitioning +DEBUG: A mix of distributed and reference table, routable query is not possible +DEBUG: Creating MERGE repartition plan +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); @@ -3340,38 +3757,67 @@ ERROR: Reference table as target is not allowed in MERGE command -- with a citus local table MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a) WHEN MATCHED THEN UPDATE SET b = citus_local_table.b; -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: A mix of distributed and local table, try repartitioning +DEBUG: A mix of distributed and citus-local table, routable query is not possible +DEBUG: Creating MERGE repartition plan +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a) WHEN MATCHED THEN DELETE; -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: A mix of distributed and local table, try repartitioning +DEBUG: A mix of distributed and citus-local table, routable query is not possible +DEBUG: Creating MERGE repartition plan +ERROR: MERGE involving repartition of rows is supported only if the target is distributed -- with a postgres table MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a) WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b; -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: There is only one distributed table, merge is not pushable, try repartitioning +DEBUG: Creating MERGE repartition plan +ERROR: MERGE INTO an distributed table from Postgres table is not yet supported MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); -ERROR: MERGE command is not supported with combination of distributed/local tables yet +DEBUG: There is only one distributed table, merge is not pushable, try repartitioning +DEBUG: Creating MERGE repartition plan +ERROR: MERGE involving repartition of rows is supported only if the target is distributed -- using ctes WITH cte AS ( SELECT * FROM nullkey_c1_t1 ) MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan WITH cte AS ( SELECT * FROM distributed_table ) MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting WITH cte AS materialized ( SELECT * FROM distributed_table ) MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; -ERROR: For MERGE command, all the distributed tables must be colocated +DEBUG: Distributed tables are not co-located, try repartitioning +DEBUG: For MERGE command, all the distributed tables must be colocated +DEBUG: Creating MERGE repartition plan +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Executing subplans of the source +DEBUG: Executing task list and redistributing the source rows +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting SET client_min_messages TO WARNING; DROP SCHEMA query_single_shard_table CASCADE; RESET client_min_messages; @@ -3386,7 +3832,7 @@ CONTEXT: SQL statement "SELECT citus_drop_all_shards(v_obj.objid, v_obj.schema_ PL/pgSQL function citus_drop_trigger() line XX at PERFORM DROP FUNCTION merge_when_and_write(); DROP SCHEMA merge_schema CASCADE; -NOTICE: drop cascades to 90 other objects +NOTICE: drop cascades to 95 other objects DETAIL: drop cascades to function insert_data() drop cascades to table local_local drop cascades to table target @@ -3460,26 +3906,25 @@ drop cascades to table source_serial drop cascades to table target_serial drop cascades to table target_set drop cascades to table source_set -drop cascades to table reftarget_local_4000113 drop cascades to table refsource_ref drop cascades to table pg_result drop cascades to table refsource_ref_4000112 drop cascades to table pg_ref -drop cascades to table reftarget_local drop cascades to table local_ref +drop cascades to table reftarget_local +drop cascades to table dist_reftarget +drop cascades to function setup_demo_data() +drop cascades to function merge_demo_data() +drop cascades to table demo_distributed +drop cascades to table demo_source_table +drop cascades to table pg_demo_result +drop cascades to table dist_demo_result drop cascades to function add_s(integer,integer) drop cascades to table pg -drop cascades to table t1_4000133 -drop cascades to table s1_4000134 +drop cascades to table t1_4000158 +drop cascades to table s1_4000159 drop cascades to table t1 drop cascades to table s1 -drop cascades to table dist_colocated drop cascades to table dist_target drop cascades to table dist_source drop cascades to view show_tables -SELECT 1 FROM master_remove_node('localhost', :master_port); - ?column? ---------------------------------------------------------------------- - 1 -(1 row) - diff --git a/src/test/regress/expected/merge_repartition.out b/src/test/regress/expected/merge_repartition.out new file mode 100644 index 000000000..47b944bcf --- /dev/null +++ b/src/test/regress/expected/merge_repartition.out @@ -0,0 +1,926 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. +DROP SCHEMA IF EXISTS merge_repartition_schema CASCADE; +NOTICE: schema "merge_repartition_schema" does not exist, skipping +CREATE SCHEMA merge_repartition_schema; +SET search_path TO merge_repartition_schema; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 5000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +RESET client_min_messages; +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('citus_target'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('citus_source'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(50001, 100000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(50001, 100000) i; +$$ +LANGUAGE SQL; +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; +-- +-- Target and source are distributed, and non-colocated +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target and source are distributed, and colocated but not joined on distribution column +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); +MERGE INTO citus_target t +USING (SELECT * FROM citus_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target and source are distributed, colocated, joined on distribution column +-- but with nondistribution values +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT id,const FROM pg_source UNION SELECT const,id FROM pg_source ) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); +MERGE INTO citus_target t +USING (SELECT id,const FROM citus_source UNION SELECT const,id FROM citus_source) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Repartition with a predicate on target_table_name rows in ON clause +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source WHERE id < 95000) s +ON t.id = s.id AND t.id < 90000 +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING (SELECT * FROM citus_source WHERE id < 95000) s +ON t.id = s.id AND t.id < 90000 +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Test CTE and non-colocated tables +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +WITH cte AS ( + SELECT * FROM pg_source +) +MERGE INTO pg_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +WITH cte AS ( + SELECT * FROM citus_source +) +MERGE INTO citus_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Test nested CTEs +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +WITH cte1 AS ( + SELECT * FROM pg_source ORDER BY 1 LIMIT 90000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO pg_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +WITH cte1 AS ( + SELECT * FROM citus_source ORDER BY 1 LIMIT 90000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO citus_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target and source are distributed and colocated +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_source', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT 999 as newval, pg_source.* FROM (SELECT * FROM pg_source ORDER BY 1 LIMIT 60000) as src LEFT JOIN pg_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 55000 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); +MERGE INTO citus_target t +USING (SELECT 999 as newval, citus_source.* FROM (SELECT * FROM citus_source ORDER BY 1 LIMIT 60000) as src LEFT JOIN citus_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 55000 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target is distributed and source is reference +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_reference_table('citus_source'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target is distributed and reference as source in a sub-query +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_reference_table('citus_source'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_source$$) + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source UNION SELECT * FROM pg_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING (SELECT * FROM citus_source UNION SELECT * FROM citus_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +-- +-- Target is distributed and citus-local as source +-- +SELECT cleanup_data(); +NOTICE: creating a new table for merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: moving the data of merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: dropping the old merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: renaming the new table to merge_repartition_schema.citus_target +CONTEXT: SQL function "cleanup_data" statement 5 +NOTICE: creating a new table for merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: moving the data of merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: dropping the old merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 +NOTICE: renaming the new table to merge_repartition_schema.citus_source +CONTEXT: SQL function "cleanup_data" statement 6 + cleanup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT setup_data(); + setup_data +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('citus_target', 'id'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$merge_repartition_schema.citus_target$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_add_local_table_to_metadata('citus_source'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); +NOTICE: The average of pg_target.id is equal to citus_target.id +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 1 +NOTICE: The average of pg_target.val is equal to citus_target.val +CONTEXT: PL/pgSQL function check_data(text,text,text,text) line XX at RAISE +SQL function "compare_data" statement 2 + compare_data +--------------------------------------------------------------------- + +(1 row) + +DROP SCHEMA merge_repartition_schema CASCADE; +NOTICE: drop cascades to 9 other objects +DETAIL: drop cascades to table pg_target +drop cascades to table pg_source +drop cascades to function cleanup_data() +drop cascades to function setup_data() +drop cascades to function check_data(text,text,text,text) +drop cascades to function compare_data() +drop cascades to table citus_target +drop cascades to table citus_source_5000072 +drop cascades to table citus_source diff --git a/src/test/regress/expected/merge_repartition_0.out b/src/test/regress/expected/merge_repartition_0.out new file mode 100644 index 000000000..a7e3fbf20 --- /dev/null +++ b/src/test/regress/expected/merge_repartition_0.out @@ -0,0 +1,6 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q diff --git a/src/test/regress/expected/pg15.out b/src/test/regress/expected/pg15.out index 4d1040a7e..ca2368622 100644 --- a/src/test/regress/expected/pg15.out +++ b/src/test/regress/expected/pg15.out @@ -418,14 +418,16 @@ SELECT create_distributed_table('tbl2', 'x'); MERGE INTO tbl1 USING tbl2 ON (true) WHEN MATCHED THEN DELETE; -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns --- also, not inside subqueries & ctes +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting +-- also, inside subqueries & ctes WITH targq AS ( SELECT * FROM tbl2 ) MERGE INTO tbl1 USING targq ON (true) WHEN MATCHED THEN DELETE; -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting WITH foo AS ( MERGE INTO tbl1 USING tbl2 ON (true) WHEN MATCHED THEN DELETE @@ -441,7 +443,8 @@ USING tbl2 ON (true) WHEN MATCHED THEN DO NOTHING; -ERROR: MERGE command is only supported when all distributed tables are co-located and joined on their distribution columns +ERROR: Missing required join condition between source and target's distribution column +HINT: Without a join condition on the target's distribution column, the source rows cannot be efficiently redistributed, and the NOT-MATCHED condition cannot be evaluated unambiguously. This can result in incorrect or unexpected results when attempting to merge tables in a distributed setting MERGE INTO tbl1 t USING tbl2 ON (true) diff --git a/src/test/regress/expected/pgmerge.out b/src/test/regress/expected/pgmerge.out index 6bdb7f771..895bf0680 100644 --- a/src/test/regress/expected/pgmerge.out +++ b/src/test/regress/expected/pgmerge.out @@ -15,13 +15,14 @@ SET search_path TO pgmerge_schema; SET citus.use_citus_managed_tables to true; \set SHOW_CONTEXT errors SET citus.next_shard_id TO 4001000; +SET client_min_messages = warning; SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); -NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata ?column? --------------------------------------------------------------------- 1 (1 row) +RESET client_min_messages; CREATE USER regress_merge_privs; CREATE USER regress_merge_no_privs; DROP TABLE IF EXISTS target; @@ -2133,9 +2134,3 @@ drop cascades to table source2 drop cascades to function merge_trigfunc() DROP USER regress_merge_privs; DROP USER regress_merge_no_privs; -SELECT 1 FROM master_remove_node('localhost', :master_port); - ?column? ---------------------------------------------------------------------- - 1 -(1 row) - diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index a78ee6088..8de26f382 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -105,8 +105,7 @@ test: background_task_queue_monitor test: clock # MERGE tests -test: merge -test: pgmerge +test: merge merge_repartition pgmerge # --------- # test that no tests leaked intermediate results. This should always be last diff --git a/src/test/regress/sql/merge.sql b/src/test/regress/sql/merge.sql index f10ab6c99..fc7a21c77 100644 --- a/src/test/regress/sql/merge.sql +++ b/src/test/regress/sql/merge.sql @@ -21,7 +21,9 @@ SET citus.next_shard_id TO 4000000; SET citus.explain_all_tasks TO true; SET citus.shard_replication_factor TO 1; SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; CREATE TABLE source ( @@ -929,23 +931,21 @@ SELECT create_distributed_table('source_cj1', 'sid1'); SELECT create_distributed_table('source_cj2', 'sid2'); BEGIN; -SET citus.log_remote_commands to true; MERGE INTO target_cj t -USING source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2 +USING (SELECT * FROM source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = sid2) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src2 WHEN NOT MATCHED THEN DO NOTHING; -SET citus.log_remote_commands to false; SELECT * FROM target_cj ORDER BY 1; ROLLBACK; BEGIN; -- try accessing columns from either side of the source join MERGE INTO target_cj t -USING source_cj1 s2 - INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10 +USING (SELECT * FROM source_cj1 s2 + INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10) s ON t.tid = sid1 AND t.tid = 2 WHEN MATCHED THEN UPDATE SET src = src1, val = val2 @@ -983,7 +983,7 @@ ROLLBACK; -- Test PREPARE -PREPARE foo(int) AS +PREPARE merge_prepare(int) AS MERGE INTO target_cj target USING (SELECT * FROM source_cj1) sub ON target.tid = sub.sid1 AND target.tid = $1 @@ -995,11 +995,11 @@ WHEN NOT MATCHED THEN SELECT * FROM target_cj ORDER BY 1; BEGIN; -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); -EXECUTE foo(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); +EXECUTE merge_prepare(2); SELECT * FROM target_cj ORDER BY 1; ROLLBACK; @@ -1007,10 +1007,10 @@ BEGIN; SET citus.log_remote_commands to true; SET client_min_messages TO DEBUG1; -EXECUTE foo(2); +EXECUTE merge_prepare(2); RESET client_min_messages; -EXECUTE foo(2); +EXECUTE merge_prepare(2); SET citus.log_remote_commands to false; SELECT * FROM target_cj ORDER BY 1; @@ -1623,10 +1623,230 @@ SELECT count(*) FROM pg_result FULL OUTER JOIN local_ref ON pg_result.t1 = local_ref.t1 WHERE pg_result.t1 IS NULL OR local_ref.t1 IS NULL; +-- Now make target as distributed, keep reference as source +TRUNCATE reftarget_local; +TRUNCATE refsource_ref; +INSERT INTO reftarget_local VALUES(1, 0); +INSERT INTO reftarget_local VALUES(3, 100); +INSERT INTO refsource_ref VALUES(1, 1); +INSERT INTO refsource_ref VALUES(2, 2); +INSERT INTO refsource_ref VALUES(3, 3); + +SELECT create_distributed_table('reftarget_local', 't1'); + +MERGE INTO reftarget_local +USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1 +WHEN MATCHED AND reftarget_local.t2 = 100 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET t2 = t2 + 100 +WHEN NOT MATCHED THEN + INSERT VALUES(foo.s1); +SELECT * INTO dist_reftarget FROM reftarget_local ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_reftarget c, pg_result p +WHERE c.t1 = p.t1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_result FULL OUTER JOIN dist_reftarget ON pg_result.t1 = dist_reftarget.t1 +WHERE pg_result.t1 IS NULL OR dist_reftarget.t1 IS NULL; + +-- +-- Distributed (target), Reference(source) +-- +CREATE TABLE demo_distributed(id1 int, val1 int); +CREATE TABLE demo_source_table(id2 int, val2 int); + +CREATE FUNCTION setup_demo_data() RETURNS VOID AS $$ +INSERT INTO demo_distributed VALUES(1, 100); +INSERT INTO demo_distributed VALUES(7, 100); +INSERT INTO demo_distributed VALUES(15, 100); +INSERT INTO demo_distributed VALUES(100, 0); +INSERT INTO demo_distributed VALUES(300, 100); +INSERT INTO demo_distributed VALUES(400, 0); + +INSERT INTO demo_source_table VALUES(1, 77); +INSERT INTO demo_source_table VALUES(15, 77); +INSERT INTO demo_source_table VALUES(75, 77); +INSERT INTO demo_source_table VALUES(100, 77); +INSERT INTO demo_source_table VALUES(300, 77); +INSERT INTO demo_source_table VALUES(400, 77); +INSERT INTO demo_source_table VALUES(500, 77); +$$ +LANGUAGE SQL; + +CREATE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING demo_source_table s ON s.id2 = t.id1 +WHEN MATCHED AND t.val1= 0 THEN + DELETE +WHEN MATCHED THEN + UPDATE SET val1 = val1 + s.val2 +WHEN NOT MATCHED THEN + INSERT VALUES(s.id2, s.val2); +$$ +LANGUAGE SQL; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT create_distributed_table('demo_distributed', 'id1'); +SELECT create_reference_table('demo_source_table'); + +SELECT setup_demo_data(); +SELECT merge_demo_data(); + +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + +-- Now convert source as distributed, but non-colocated with target +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +SELECT undistribute_table('demo_source_table'); + +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT id2,val2 FROM demo_source_table UNION SELECT val2,id2 FROM demo_source_table) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = val1 + 1; +$$ +LANGUAGE SQL; + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; + +SELECT create_distributed_table('demo_distributed', 'id1'); +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + +-- Test with LIMIT + +CREATE OR REPLACE FUNCTION merge_demo_data() RETURNS VOID AS $$ +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); +$$ +LANGUAGE SQL; + +DROP TABLE pg_demo_result, dist_demo_result; +SELECT undistribute_table('demo_distributed'); +SELECT undistribute_table('demo_source_table'); + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO pg_demo_result FROM demo_distributed ORDER BY 1, 2; + +SELECT create_distributed_table('demo_distributed', 'id1'); +SELECT create_distributed_table('demo_source_table', 'id2', colocate_with=>'none'); + +TRUNCATE demo_distributed; +TRUNCATE demo_source_table; + +SELECT setup_demo_data(); +SELECT merge_demo_data(); +SELECT * INTO dist_demo_result FROM demo_distributed ORDER BY 1, 2; + +-- Should be equal +SELECT c.*, p.* +FROM dist_demo_result c, pg_demo_result p +WHERE c.id1 = p.id1 +ORDER BY 1,2; + +-- Must return zero rows +SELECT count(*) +FROM pg_demo_result p FULL OUTER JOIN dist_demo_result d ON p.id1 = d.id1 +WHERE p.id1 IS NULL OR d.id1 IS NULL; + +-- Test explain with repartition +SET citus.explain_all_tasks TO false; +EXPLAIN (COSTS OFF) +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); + -- -- Error and Unsupported scenarios -- +-- Test explain analyze with repartition +EXPLAIN ANALYZE +MERGE INTO demo_distributed t +USING (SELECT 999 as s3, demo_source_table.* FROM (SELECT * FROM demo_source_table ORDER BY 1 LIMIT 3) as foo LEFT JOIN demo_source_table USING(id2)) AS s +ON t.id1 = s.id2 +WHEN MATCHED THEN + UPDATE SET val1 = s3 +WHEN NOT MATCHED THEN + INSERT VALUES(id2, s3); + +-- Source without a table +MERGE INTO target_cj t +USING (VALUES (1, 1), (2, 1), (3, 3)) as s (sid, val) +ON t.tid = s.sid AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET val = s.val +WHEN NOT MATCHED THEN + DO NOTHING; + +-- Incomplete source +MERGE INTO target_cj t +USING (source_cj1 s1 INNER JOIN source_cj2 s2 ON sid1 = val2) s +ON t.tid = s.sid1 AND t.tid = 2 +WHEN MATCHED THEN + UPDATE SET src = src2 +WHEN NOT MATCHED THEN + DO NOTHING; + -- Reference as a target and local as source MERGE INTO refsource_ref USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1 @@ -1635,34 +1855,16 @@ WHEN MATCHED THEN WHEN NOT MATCHED THEN INSERT VALUES(foo.t1); --- Reference as a source and distributed as target -MERGE INTO target_set t -USING refsource_ref AS s ON t.t1 = s.s1 -WHEN MATCHED THEN - DO NOTHING; - MERGE INTO target_set USING source_set AS foo ON target_set.t1 = foo.s1 WHEN MATCHED THEN UPDATE SET ctid = '(0,100)'; -MERGE INTO target_set -USING (SELECT s1,s2 FROM source_set UNION SELECT s2,s1 FROM source_set) AS foo ON target_set.t1 = foo.s1 -WHEN MATCHED THEN - UPDATE SET t2 = t2 + 1; - -MERGE INTO target_set -USING (SELECT 2 as s3, source_set.* FROM (SELECT * FROM source_set LIMIT 1) as foo LEFT JOIN source_set USING( s1)) AS foo -ON target_set.t1 = foo.s1 -WHEN MATCHED THEN UPDATE SET t2 = t2 + 1 -WHEN NOT MATCHED THEN INSERT VALUES(s1, s3); - - -- modifying CTE not supported EXPLAIN -WITH cte_1 AS (DELETE FROM target_json) +WITH cte_1 AS (DELETE FROM target_json RETURNING *) MERGE INTO target_json sda -USING source_json sdn +USING cte_1 sdn ON sda.id = sdn.id WHEN NOT matched THEN INSERT (id, z) VALUES (sdn.id, 5); @@ -1711,6 +1913,7 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT (id) VALUES(1000); +-- Colocated merge MERGE INTO t1 t USING s1 s ON t.id = s.id @@ -1723,6 +1926,13 @@ ON t.id = s.id WHEN NOT MATCHED THEN INSERT (val) VALUES(s.val); +-- Non-colocated merge +MERGE INTO t1 t +USING s1 s +ON t.id = s.val +WHEN NOT MATCHED THEN + INSERT (id) VALUES(s.id); + -- try updating the distribution key column BEGIN; MERGE INTO target_cj t @@ -1811,17 +2021,7 @@ WHEN MATCHED AND (merge_when_and_write()) THEN ROLLBACK; --- Joining on partition columns with sub-query -MERGE INTO t1 - USING (SELECT * FROM s1) sub ON (sub.val = t1.id) -- sub.val is not a distribution column - WHEN MATCHED AND sub.val = 0 THEN - DELETE - WHEN MATCHED THEN - UPDATE SET val = t1.val + 1 - WHEN NOT MATCHED THEN - INSERT (id, val) VALUES (sub.id, sub.val); - --- Joining on partition columns with CTE +-- Joining on non-partition columns with CTE source, but INSERT incorrect column WITH s1_res AS ( SELECT * FROM s1 ) @@ -1847,7 +2047,7 @@ MERGE INTO t1 WHEN NOT MATCHED THEN INSERT (id, val) VALUES (s1_res.id, s1_res.val); --- With a single WHEN clause, which causes a non-left join +-- Join condition without target distribution column WITH s1_res AS ( SELECT * FROM s1 ) @@ -1954,34 +2154,12 @@ WHEN MATCHED THEN WHEN NOT MATCHED THEN INSERT VALUES(mv_source.id, mv_source.val); --- Distributed tables *must* be colocated +-- Do not allow constant values into the distribution column CREATE TABLE dist_target(id int, val varchar); SELECT create_distributed_table('dist_target', 'id'); CREATE TABLE dist_source(id int, val varchar); SELECT create_distributed_table('dist_source', 'id', colocate_with => 'none'); -MERGE INTO dist_target -USING dist_source -ON dist_target.id = dist_source.id -WHEN MATCHED THEN -UPDATE SET val = dist_source.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_source.id, dist_source.val); - --- Distributed tables *must* be joined on distribution column -CREATE TABLE dist_colocated(id int, val int); -SELECT create_distributed_table('dist_colocated', 'id', colocate_with => 'dist_target'); - -MERGE INTO dist_target -USING dist_colocated -ON dist_target.id = dist_colocated.val -- val is not the distribution column -WHEN MATCHED THEN -UPDATE SET val = dist_colocated.val -WHEN NOT MATCHED THEN -INSERT VALUES(dist_colocated.id, dist_colocated.val); - - --- Both the source and target must be distributed MERGE INTO dist_target USING (SELECT 100 id) AS source ON dist_target.id = source.id AND dist_target.val = 'const' @@ -2166,4 +2344,3 @@ SET search_path TO merge_schema; DROP SERVER foreign_server CASCADE; DROP FUNCTION merge_when_and_write(); DROP SCHEMA merge_schema CASCADE; -SELECT 1 FROM master_remove_node('localhost', :master_port); diff --git a/src/test/regress/sql/merge_repartition.sql b/src/test/regress/sql/merge_repartition.sql new file mode 100644 index 000000000..f906eba41 --- /dev/null +++ b/src/test/regress/sql/merge_repartition.sql @@ -0,0 +1,401 @@ +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15 +\gset +\if :server_version_ge_15 +\else +\q +\endif + +-- We create two sets of source and target tables, one set in Postgres and +-- the other in Citus distributed. We run the _exact_ MERGE SQL on both sets +-- and compare the final results of the target tables in Postgres and Citus. +-- The results should match. This process is repeated for various combinations +-- of MERGE SQL. + +DROP SCHEMA IF EXISTS merge_repartition_schema CASCADE; +CREATE SCHEMA merge_repartition_schema; +SET search_path TO merge_repartition_schema; +SET citus.shard_count TO 4; +SET citus.next_shard_id TO 5000000; +SET citus.explain_all_tasks TO true; +SET citus.shard_replication_factor TO 1; +SET citus.max_adaptive_executor_pool_size TO 1; +SET client_min_messages = warning; +SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; + + +CREATE TABLE pg_target(id int, val int); +CREATE TABLE pg_source(id int, val int, const int); +CREATE TABLE citus_target(id int, val int); +CREATE TABLE citus_source(id int, val int, const int); +SELECT citus_add_local_table_to_metadata('citus_target'); +SELECT citus_add_local_table_to_metadata('citus_source'); + +CREATE OR REPLACE FUNCTION cleanup_data() RETURNS VOID AS $$ + TRUNCATE pg_target; + TRUNCATE pg_source; + TRUNCATE citus_target; + TRUNCATE citus_source; + SELECT undistribute_table('citus_target'); + SELECT undistribute_table('citus_source'); +$$ +LANGUAGE SQL; +-- +-- Load same set of data to both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION setup_data() RETURNS VOID AS $$ + INSERT INTO pg_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO pg_target SELECT i, 1 FROM generate_series(50001, 100000) i; + INSERT INTO citus_source SELECT i, i+1, 1 FROM generate_series(1, 100000) i; + INSERT INTO citus_target SELECT i, 1 FROM generate_series(50001, 100000) i; +$$ +LANGUAGE SQL; + +-- +-- Compares the final target tables, merge-modified data, of both Postgres and Citus tables +-- +CREATE OR REPLACE FUNCTION check_data(table1_name text, column1_name text, table2_name text, column2_name text) +RETURNS VOID AS $$ +DECLARE + table1_avg numeric; + table2_avg numeric; +BEGIN + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column1_name, table1_name) INTO table1_avg; + EXECUTE format('SELECT COALESCE(AVG(%I), 0) FROM %I', column2_name, table2_name) INTO table2_avg; + + IF table1_avg > table2_avg THEN + RAISE EXCEPTION 'The average of %.% is greater than %.%', table1_name, column1_name, table2_name, column2_name; + ELSIF table1_avg < table2_avg THEN + RAISE EXCEPTION 'The average of %.% is less than %.%', table1_name, column1_name, table2_name, column2_name; + ELSE + RAISE NOTICE 'The average of %.% is equal to %.%', table1_name, column1_name, table2_name, column2_name; + END IF; +END; +$$ LANGUAGE plpgsql; + +CREATE OR REPLACE FUNCTION compare_data() RETURNS VOID AS $$ + SELECT check_data('pg_target', 'id', 'citus_target', 'id'); + SELECT check_data('pg_target', 'val', 'citus_target', 'val'); +$$ +LANGUAGE SQL; + +-- +-- Target and source are distributed, and non-colocated +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- +-- Target and source are distributed, and colocated but not joined on distribution column +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); + +MERGE INTO citus_target t +USING (SELECT * FROM citus_source) subq +ON (subq.val = t.id) +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = subq.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(subq.val, subq.id); + +SELECT compare_data(); + +-- +-- Target and source are distributed, colocated, joined on distribution column +-- but with nondistribution values +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'citus_target'); + +MERGE INTO pg_target t +USING (SELECT id,const FROM pg_source UNION SELECT const,id FROM pg_source ) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); + +MERGE INTO citus_target t +USING (SELECT id,const FROM citus_source UNION SELECT const,id FROM citus_source) AS s +ON t.id = s.id +WHEN MATCHED THEN + UPDATE SET val = s.const + 1 +WHEN NOT MATCHED THEN + INSERT VALUES(id, const); + +SELECT compare_data(); + +-- +-- Repartition with a predicate on target_table_name rows in ON clause +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source WHERE id < 95000) s +ON t.id = s.id AND t.id < 90000 +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING (SELECT * FROM citus_source WHERE id < 95000) s +ON t.id = s.id AND t.id < 90000 +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- +-- Test CTE and non-colocated tables +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +WITH cte AS ( + SELECT * FROM pg_source +) +MERGE INTO pg_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +WITH cte AS ( + SELECT * FROM citus_source +) +MERGE INTO citus_target t +USING cte s +ON s.id = t.id +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +SELECT compare_data(); + +-- +-- Test nested CTEs +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id', colocate_with=>'none'); + +WITH cte1 AS ( + SELECT * FROM pg_source ORDER BY 1 LIMIT 90000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO pg_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +WITH cte1 AS ( + SELECT * FROM citus_source ORDER BY 1 LIMIT 90000 +), +cte2 AS( + SELECT * FROM cte1 +), +cte3 AS( + SELECT * FROM cte2 +) +MERGE INTO citus_target t +USING cte3 s +ON (s.id=t.id) +WHEN MATCHED AND t.id > 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES (s.id, s.val); + +SELECT compare_data(); + +-- +-- Target and source are distributed and colocated +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_distributed_table('citus_source', 'id'); + +MERGE INTO pg_target t +USING (SELECT 999 as newval, pg_source.* FROM (SELECT * FROM pg_source ORDER BY 1 LIMIT 60000) as src LEFT JOIN pg_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 55000 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); + +MERGE INTO citus_target t +USING (SELECT 999 as newval, citus_source.* FROM (SELECT * FROM citus_source ORDER BY 1 LIMIT 60000) as src LEFT JOIN citus_source USING(id)) AS s +ON t.id = s.id +WHEN MATCHED AND t.id <= 55000 THEN + UPDATE SET val = newval +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(id, newval); + +SELECT compare_data(); + +-- +-- Target is distributed and source is reference +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_reference_table('citus_source'); + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +-- +-- Target is distributed and reference as source in a sub-query +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT create_reference_table('citus_source'); + +MERGE INTO pg_target t +USING (SELECT * FROM pg_source UNION SELECT * FROM pg_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING (SELECT * FROM citus_source UNION SELECT * FROM citus_source) AS s ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + t.val +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); +SELECT compare_data(); + +-- +-- Target is distributed and citus-local as source +-- +SELECT cleanup_data(); +SELECT setup_data(); +SELECT create_distributed_table('citus_target', 'id'); +SELECT citus_add_local_table_to_metadata('citus_source'); + +MERGE INTO pg_target t +USING pg_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +MERGE INTO citus_target t +USING citus_source s +ON t.id = s.id +WHEN MATCHED AND t.id <= 75000 THEN + UPDATE SET val = s.val + 1 +WHEN MATCHED THEN + DELETE +WHEN NOT MATCHED THEN + INSERT VALUES(s.id, s.val); + +SELECT compare_data(); + +DROP SCHEMA merge_repartition_schema CASCADE; diff --git a/src/test/regress/sql/pg15.sql b/src/test/regress/sql/pg15.sql index b82b0d745..3639c07bb 100644 --- a/src/test/regress/sql/pg15.sql +++ b/src/test/regress/sql/pg15.sql @@ -262,7 +262,7 @@ SELECT create_distributed_table('tbl2', 'x'); MERGE INTO tbl1 USING tbl2 ON (true) WHEN MATCHED THEN DELETE; --- also, not inside subqueries & ctes +-- also, inside subqueries & ctes WITH targq AS ( SELECT * FROM tbl2 ) diff --git a/src/test/regress/sql/pgmerge.sql b/src/test/regress/sql/pgmerge.sql index 9b828f27e..ab1f4a40d 100644 --- a/src/test/regress/sql/pgmerge.sql +++ b/src/test/regress/sql/pgmerge.sql @@ -19,7 +19,9 @@ SET citus.use_citus_managed_tables to true; SET citus.next_shard_id TO 4001000; +SET client_min_messages = warning; SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0); +RESET client_min_messages; CREATE USER regress_merge_privs; CREATE USER regress_merge_no_privs; @@ -1374,4 +1376,3 @@ REVOKE ALL ON SCHEMA pgmerge_schema FROM regress_merge_no_privs; DROP SCHEMA pgmerge_schema CASCADE; DROP USER regress_merge_privs; DROP USER regress_merge_no_privs; -SELECT 1 FROM master_remove_node('localhost', :master_port);