mirror of https://github.com/citusdata/citus.git
Implement repartition MERGE
parent
b58665773b
commit
29975cd85f
|
@ -27,6 +27,8 @@
|
||||||
#include "distributed/listutils.h"
|
#include "distributed/listutils.h"
|
||||||
#include "distributed/local_executor.h"
|
#include "distributed/local_executor.h"
|
||||||
#include "distributed/local_plan_cache.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_executor.h"
|
||||||
#include "distributed/multi_server_executor.h"
|
#include "distributed/multi_server_executor.h"
|
||||||
#include "distributed/multi_router_planner.h"
|
#include "distributed/multi_router_planner.h"
|
||||||
|
@ -53,6 +55,7 @@ extern AllowedDistributionColumn AllowedDistributionColumnValue;
|
||||||
static Node * AdaptiveExecutorCreateScan(CustomScan *scan);
|
static Node * AdaptiveExecutorCreateScan(CustomScan *scan);
|
||||||
static Node * NonPushableInsertSelectCreateScan(CustomScan *scan);
|
static Node * NonPushableInsertSelectCreateScan(CustomScan *scan);
|
||||||
static Node * DelayedErrorCreateScan(CustomScan *scan);
|
static Node * DelayedErrorCreateScan(CustomScan *scan);
|
||||||
|
static Node * NonPushableMergeCommandCreateScan(CustomScan *scan);
|
||||||
|
|
||||||
/* functions that are common to different scans */
|
/* functions that are common to different scans */
|
||||||
static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags);
|
static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags);
|
||||||
|
@ -88,6 +91,11 @@ CustomScanMethods DelayedErrorCustomScanMethods = {
|
||||||
DelayedErrorCreateScan
|
DelayedErrorCreateScan
|
||||||
};
|
};
|
||||||
|
|
||||||
|
CustomScanMethods NonPushableMergeCommandCustomScanMethods = {
|
||||||
|
"Citus MERGE INTO ...",
|
||||||
|
NonPushableMergeCommandCreateScan
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Define executor methods for the different executor types.
|
* 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.
|
* IsCitusCustomState returns if a given PlanState node is a CitusCustomState node.
|
||||||
*/
|
*/
|
||||||
|
@ -124,7 +142,8 @@ IsCitusCustomState(PlanState *planState)
|
||||||
|
|
||||||
CustomScanState *css = castNode(CustomScanState, planState);
|
CustomScanState *css = castNode(CustomScanState, planState);
|
||||||
if (css->methods == &AdaptiveExecutorCustomExecMethods ||
|
if (css->methods == &AdaptiveExecutorCustomExecMethods ||
|
||||||
css->methods == &NonPushableInsertSelectCustomExecMethods)
|
css->methods == &NonPushableInsertSelectCustomExecMethods ||
|
||||||
|
css->methods == &NonPushableMergeCommandCustomExecMethods)
|
||||||
{
|
{
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -142,6 +161,7 @@ RegisterCitusCustomScanMethods(void)
|
||||||
RegisterCustomScanMethods(&AdaptiveExecutorCustomScanMethods);
|
RegisterCustomScanMethods(&AdaptiveExecutorCustomScanMethods);
|
||||||
RegisterCustomScanMethods(&NonPushableInsertSelectCustomScanMethods);
|
RegisterCustomScanMethods(&NonPushableInsertSelectCustomScanMethods);
|
||||||
RegisterCustomScanMethods(&DelayedErrorCustomScanMethods);
|
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.
|
* CitusEndScan is used to clean up tuple store of the given custom scan state.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -610,6 +610,18 @@ QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer)
|
||||||
StringInfo fragmentNamesArrayString = makeStringInfo();
|
StringInfo fragmentNamesArrayString = makeStringInfo();
|
||||||
int fragmentCount = 0;
|
int fragmentCount = 0;
|
||||||
NodePair *nodePair = &fragmentsTransfer->nodes;
|
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);
|
WorkerNode *sourceNode = LookupNodeByNodeIdOrError(nodePair->sourceNodeId);
|
||||||
|
|
||||||
appendStringInfoString(fragmentNamesArrayString, "ARRAY[");
|
appendStringInfoString(fragmentNamesArrayString, "ARRAY[");
|
||||||
|
|
|
@ -67,11 +67,6 @@ static HTAB * ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId,
|
||||||
static List * BuildColumnNameListFromTargetList(Oid targetRelationId,
|
static List * BuildColumnNameListFromTargetList(Oid targetRelationId,
|
||||||
List *insertTargetList);
|
List *insertTargetList);
|
||||||
static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList);
|
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);
|
static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries);
|
||||||
|
|
||||||
|
|
||||||
|
@ -535,7 +530,7 @@ IsSupportedRedistributionTarget(Oid targetRelationId)
|
||||||
* a result name which should be inserted into
|
* a result name which should be inserted into
|
||||||
* targetRelation->sortedShardIntervalArray[shardIndex].
|
* targetRelation->sortedShardIntervalArray[shardIndex].
|
||||||
*/
|
*/
|
||||||
static List *
|
List *
|
||||||
RedistributedInsertSelectTaskList(Query *insertSelectQuery,
|
RedistributedInsertSelectTaskList(Query *insertSelectQuery,
|
||||||
CitusTableCacheEntry *targetRelation,
|
CitusTableCacheEntry *targetRelation,
|
||||||
List **redistributedResults,
|
List **redistributedResults,
|
||||||
|
@ -632,7 +627,7 @@ RedistributedInsertSelectTaskList(Query *insertSelectQuery,
|
||||||
* PartitionColumnIndex finds the index of given partition column in the
|
* PartitionColumnIndex finds the index of given partition column in the
|
||||||
* given target list.
|
* given target list.
|
||||||
*/
|
*/
|
||||||
static int
|
int
|
||||||
PartitionColumnIndex(List *insertTargetList, Var *partitionColumn)
|
PartitionColumnIndex(List *insertTargetList, Var *partitionColumn)
|
||||||
{
|
{
|
||||||
TargetEntry *insertTargetEntry = NULL;
|
TargetEntry *insertTargetEntry = NULL;
|
||||||
|
|
|
@ -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 <source query>" 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 = <const>
|
||||||
|
*/
|
||||||
|
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);
|
||||||
|
}
|
|
@ -24,6 +24,7 @@
|
||||||
#include "distributed/multi_executor.h"
|
#include "distributed/multi_executor.h"
|
||||||
#include "distributed/multi_physical_planner.h"
|
#include "distributed/multi_physical_planner.h"
|
||||||
#include "distributed/multi_server_executor.h"
|
#include "distributed/multi_server_executor.h"
|
||||||
|
#include "distributed/multi_router_planner.h"
|
||||||
#include "distributed/coordinator_protocol.h"
|
#include "distributed/coordinator_protocol.h"
|
||||||
#include "distributed/subplan_execution.h"
|
#include "distributed/subplan_execution.h"
|
||||||
#include "distributed/tuple_destination.h"
|
#include "distributed/tuple_destination.h"
|
||||||
|
@ -49,6 +50,11 @@ JobExecutorType(DistributedPlan *distributedPlan)
|
||||||
|
|
||||||
if (distributedPlan->insertSelectQuery != NULL)
|
if (distributedPlan->insertSelectQuery != NULL)
|
||||||
{
|
{
|
||||||
|
if (IsMergeQuery(distributedPlan->insertSelectQuery))
|
||||||
|
{
|
||||||
|
return MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY;
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* We go through
|
* We go through
|
||||||
* MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT because
|
* MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT because
|
||||||
|
|
|
@ -990,7 +990,8 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina
|
||||||
case MERGE_QUERY:
|
case MERGE_QUERY:
|
||||||
{
|
{
|
||||||
distributedPlan =
|
distributedPlan =
|
||||||
CreateMergePlan(originalQuery, query, plannerRestrictionContext);
|
CreateMergePlan(planId, originalQuery, query, plannerRestrictionContext,
|
||||||
|
boundParams);
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1388,6 +1389,12 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan)
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
case MULTI_EXECUTOR_NON_PUSHABLE_MERGE_QUERY:
|
||||||
|
{
|
||||||
|
customScan->methods = &NonPushableMergeCommandCustomScanMethods;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
|
||||||
default:
|
default:
|
||||||
{
|
{
|
||||||
customScan->methods = &DelayedErrorCustomScanMethods;
|
customScan->methods = &DelayedErrorCustomScanMethods;
|
||||||
|
|
|
@ -85,7 +85,6 @@ static DeferredErrorMessage * InsertPartitionColumnMatchesSelect(Query *query,
|
||||||
static DistributedPlan * CreateNonPushableInsertSelectPlan(uint64 planId, Query *parse,
|
static DistributedPlan * CreateNonPushableInsertSelectPlan(uint64 planId, Query *parse,
|
||||||
ParamListInfo boundParams);
|
ParamListInfo boundParams);
|
||||||
static DeferredErrorMessage * NonPushableInsertSelectSupported(Query *insertSelectQuery);
|
static DeferredErrorMessage * NonPushableInsertSelectSupported(Query *insertSelectQuery);
|
||||||
static Query * WrapSubquery(Query *subquery);
|
|
||||||
static void RelabelTargetEntryList(List *selectTargetList, List *insertTargetList);
|
static void RelabelTargetEntryList(List *selectTargetList, List *insertTargetList);
|
||||||
static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
|
static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
|
||||||
Oid targetRelationId);
|
Oid targetRelationId);
|
||||||
|
@ -1635,7 +1634,7 @@ InsertSelectResultIdPrefix(uint64 planId)
|
||||||
* WrapSubquery wraps the given query as a subquery in a newly constructed
|
* WrapSubquery wraps the given query as a subquery in a newly constructed
|
||||||
* "SELECT * FROM (...subquery...) citus_insert_select_subquery" query.
|
* "SELECT * FROM (...subquery...) citus_insert_select_subquery" query.
|
||||||
*/
|
*/
|
||||||
static Query *
|
Query *
|
||||||
WrapSubquery(Query *subquery)
|
WrapSubquery(Query *subquery)
|
||||||
{
|
{
|
||||||
ParseState *pstate = make_parsestate(NULL);
|
ParseState *pstate = make_parsestate(NULL);
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -33,6 +33,7 @@
|
||||||
#include "distributed/insert_select_planner.h"
|
#include "distributed/insert_select_planner.h"
|
||||||
#include "distributed/insert_select_executor.h"
|
#include "distributed/insert_select_executor.h"
|
||||||
#include "distributed/listutils.h"
|
#include "distributed/listutils.h"
|
||||||
|
#include "distributed/merge_planner.h"
|
||||||
#include "distributed/multi_executor.h"
|
#include "distributed/multi_executor.h"
|
||||||
#include "distributed/multi_explain.h"
|
#include "distributed/multi_explain.h"
|
||||||
#include "distributed/multi_logical_optimizer.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
|
* ExplainSubPlans generates EXPLAIN output for subplans for CTEs
|
||||||
* and complex subqueries. Because the planning for these queries
|
* and complex subqueries. Because the planning for these queries
|
||||||
|
|
|
@ -1879,24 +1879,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
|
||||||
|
|
||||||
if (*planningError)
|
if (*planningError)
|
||||||
{
|
{
|
||||||
/*
|
return NULL;
|
||||||
* 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;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Job *job = CreateJob(originalQuery);
|
Job *job = CreateJob(originalQuery);
|
||||||
|
@ -2368,14 +2351,7 @@ PlanRouterQuery(Query *originalQuery,
|
||||||
|
|
||||||
Assert(UpdateOrDeleteOrMergeQuery(originalQuery));
|
Assert(UpdateOrDeleteOrMergeQuery(originalQuery));
|
||||||
|
|
||||||
if (IsMergeQuery(originalQuery))
|
if (!IsMergeQuery(originalQuery))
|
||||||
{
|
|
||||||
targetRelationId = ModifyQueryResultRelationId(originalQuery);
|
|
||||||
planningError = MergeQuerySupported(targetRelationId, originalQuery,
|
|
||||||
isMultiShardQuery,
|
|
||||||
plannerRestrictionContext);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
{
|
||||||
planningError = ModifyQuerySupported(originalQuery, originalQuery,
|
planningError = ModifyQuerySupported(originalQuery, originalQuery,
|
||||||
isMultiShardQuery,
|
isMultiShardQuery,
|
||||||
|
|
|
@ -188,7 +188,6 @@ static Query * BuildReadIntermediateResultsQuery(List *targetEntryList,
|
||||||
List *columnAliasList,
|
List *columnAliasList,
|
||||||
Const *resultIdConst, Oid functionOid,
|
Const *resultIdConst, Oid functionOid,
|
||||||
bool useBinaryCopyFormat);
|
bool useBinaryCopyFormat);
|
||||||
static void UpdateVarNosInNode(Node *node, Index newVarNo);
|
|
||||||
static Query * CreateOuterSubquery(RangeTblEntry *rangeTableEntry,
|
static Query * CreateOuterSubquery(RangeTblEntry *rangeTableEntry,
|
||||||
List *outerSubqueryTargetList);
|
List *outerSubqueryTargetList);
|
||||||
static List * GenerateRequiredColNamesFromTargetList(List *targetList);
|
static List * GenerateRequiredColNamesFromTargetList(List *targetList);
|
||||||
|
@ -1891,7 +1890,7 @@ GenerateRequiredColNamesFromTargetList(List *targetList)
|
||||||
* UpdateVarNosInNode iterates the Vars in the
|
* UpdateVarNosInNode iterates the Vars in the
|
||||||
* given node and updates the varno's as the newVarNo.
|
* given node and updates the varno's as the newVarNo.
|
||||||
*/
|
*/
|
||||||
static void
|
void
|
||||||
UpdateVarNosInNode(Node *node, Index newVarNo)
|
UpdateVarNosInNode(Node *node, Index newVarNo)
|
||||||
{
|
{
|
||||||
List *varList = pull_var_clause(node, PVC_RECURSE_AGGREGATES |
|
List *varList = pull_var_clause(node, PVC_RECURSE_AGGREGATES |
|
||||||
|
|
|
@ -34,6 +34,7 @@ typedef struct CitusScanState
|
||||||
extern CustomScanMethods AdaptiveExecutorCustomScanMethods;
|
extern CustomScanMethods AdaptiveExecutorCustomScanMethods;
|
||||||
extern CustomScanMethods NonPushableInsertSelectCustomScanMethods;
|
extern CustomScanMethods NonPushableInsertSelectCustomScanMethods;
|
||||||
extern CustomScanMethods DelayedErrorCustomScanMethods;
|
extern CustomScanMethods DelayedErrorCustomScanMethods;
|
||||||
|
extern CustomScanMethods NonPushableMergeCommandCustomScanMethods;
|
||||||
|
|
||||||
|
|
||||||
extern void RegisterCitusCustomScanMethods(void);
|
extern void RegisterCitusCustomScanMethods(void);
|
||||||
|
|
|
@ -21,6 +21,10 @@ extern bool EnableRepartitionedInsertSelect;
|
||||||
extern TupleTableSlot * NonPushableInsertSelectExecScan(CustomScanState *node);
|
extern TupleTableSlot * NonPushableInsertSelectExecScan(CustomScanState *node);
|
||||||
extern bool IsSupportedRedistributionTarget(Oid targetRelationId);
|
extern bool IsSupportedRedistributionTarget(Oid targetRelationId);
|
||||||
extern bool IsRedistributablePlan(Plan *selectPlan);
|
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 */
|
#endif /* INSERT_SELECT_EXECUTOR_H */
|
||||||
|
|
|
@ -44,6 +44,7 @@ extern DistributedPlan * CreateInsertSelectIntoLocalTablePlan(uint64 planId,
|
||||||
plannerRestrictionContext);
|
plannerRestrictionContext);
|
||||||
extern char * InsertSelectResultIdPrefix(uint64 planId);
|
extern char * InsertSelectResultIdPrefix(uint64 planId);
|
||||||
extern bool PlanningInsertSelect(void);
|
extern bool PlanningInsertSelect(void);
|
||||||
|
extern Query * WrapSubquery(Query *subquery);
|
||||||
|
|
||||||
|
|
||||||
#endif /* INSERT_SELECT_PLANNER_H */
|
#endif /* INSERT_SELECT_PLANNER_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 */
|
|
@ -19,16 +19,20 @@
|
||||||
#include "distributed/errormessage.h"
|
#include "distributed/errormessage.h"
|
||||||
#include "distributed/multi_physical_planner.h"
|
#include "distributed/multi_physical_planner.h"
|
||||||
|
|
||||||
extern DeferredErrorMessage * MergeQuerySupported(Oid resultRelationId,
|
extern DistributedPlan * CreateMergePlan(uint64 planId, Query *originalQuery,
|
||||||
Query *originalQuery,
|
Query *query,
|
||||||
bool multiShardQuery,
|
|
||||||
PlannerRestrictionContext *
|
|
||||||
plannerRestrictionContext);
|
|
||||||
extern DistributedPlan * CreateMergePlan(Query *originalQuery, Query *query,
|
|
||||||
PlannerRestrictionContext *
|
PlannerRestrictionContext *
|
||||||
plannerRestrictionContext);
|
plannerRestrictionContext,
|
||||||
|
ParamListInfo boundParams);
|
||||||
extern bool IsLocalTableModification(Oid targetRelationId, Query *query,
|
extern bool IsLocalTableModification(Oid targetRelationId, Query *query,
|
||||||
uint64 shardId,
|
uint64 shardId,
|
||||||
RTEListProperties *rteProperties);
|
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 */
|
#endif /* MERGE_PLANNER_H */
|
||||||
|
|
|
@ -29,7 +29,8 @@ typedef enum
|
||||||
{
|
{
|
||||||
MULTI_EXECUTOR_INVALID_FIRST = 0,
|
MULTI_EXECUTOR_INVALID_FIRST = 0,
|
||||||
MULTI_EXECUTOR_ADAPTIVE = 1,
|
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;
|
} MultiExecutorType;
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -46,6 +46,7 @@ extern void ReplaceRTERelationWithRteSubquery(RangeTblEntry *rangeTableEntry,
|
||||||
extern bool IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry);
|
extern bool IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry);
|
||||||
extern bool IsRelationLocalTableOrMatView(Oid relationId);
|
extern bool IsRelationLocalTableOrMatView(Oid relationId);
|
||||||
extern bool ContainsReferencesToOuterQuery(Query *query);
|
extern bool ContainsReferencesToOuterQuery(Query *query);
|
||||||
|
extern void UpdateVarNosInNode(Node *node, Index newVarNo);
|
||||||
|
|
||||||
|
|
||||||
#endif /* RECURSIVE_PLANNING_H */
|
#endif /* RECURSIVE_PLANNING_H */
|
||||||
|
|
|
@ -20,13 +20,14 @@ SET citus.next_shard_id TO 4000000;
|
||||||
SET citus.explain_all_tasks TO true;
|
SET citus.explain_all_tasks TO true;
|
||||||
SET citus.shard_replication_factor TO 1;
|
SET citus.shard_replication_factor TO 1;
|
||||||
SET citus.max_adaptive_executor_pool_size 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);
|
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?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
RESET client_min_messages;
|
||||||
CREATE TABLE source
|
CREATE TABLE source
|
||||||
(
|
(
|
||||||
order_id INT,
|
order_id INT,
|
||||||
|
@ -1425,19 +1426,13 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SET citus.log_remote_commands to true;
|
|
||||||
MERGE INTO target_cj t
|
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
|
ON t.tid = sid1 AND t.tid = 2
|
||||||
WHEN MATCHED THEN
|
WHEN MATCHED THEN
|
||||||
UPDATE SET src = src2
|
UPDATE SET src = src2
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
DO NOTHING;
|
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;
|
SELECT * FROM target_cj ORDER BY 1;
|
||||||
tid | src | val
|
tid | src | val
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -1451,8 +1446,8 @@ ROLLBACK;
|
||||||
BEGIN;
|
BEGIN;
|
||||||
-- try accessing columns from either side of the source join
|
-- try accessing columns from either side of the source join
|
||||||
MERGE INTO target_cj t
|
MERGE INTO target_cj t
|
||||||
USING source_cj1 s2
|
USING (SELECT * FROM source_cj1 s2
|
||||||
INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10
|
INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10) s
|
||||||
ON t.tid = sid1 AND t.tid = 2
|
ON t.tid = sid1 AND t.tid = 2
|
||||||
WHEN MATCHED THEN
|
WHEN MATCHED THEN
|
||||||
UPDATE SET src = src1, val = val2
|
UPDATE SET src = src1, val = val2
|
||||||
|
@ -1520,7 +1515,7 @@ SELECT * FROM target_cj ORDER BY 1;
|
||||||
|
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
-- Test PREPARE
|
-- Test PREPARE
|
||||||
PREPARE foo(int) AS
|
PREPARE merge_prepare(int) AS
|
||||||
MERGE INTO target_cj target
|
MERGE INTO target_cj target
|
||||||
USING (SELECT * FROM source_cj1) sub
|
USING (SELECT * FROM source_cj1) sub
|
||||||
ON target.tid = sub.sid1 AND target.tid = $1
|
ON target.tid = sub.sid1 AND target.tid = $1
|
||||||
|
@ -1538,11 +1533,11 @@ SELECT * FROM target_cj ORDER BY 1;
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
SELECT * FROM target_cj ORDER BY 1;
|
SELECT * FROM target_cj ORDER BY 1;
|
||||||
tid | src | val
|
tid | src | val
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -1556,7 +1551,7 @@ ROLLBACK;
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SET citus.log_remote_commands to true;
|
SET citus.log_remote_commands to true;
|
||||||
SET client_min_messages TO DEBUG1;
|
SET client_min_messages TO DEBUG1;
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
DEBUG: <Deparsed MERGE query: 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 >
|
DEBUG: <Deparsed MERGE query: 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 >
|
||||||
DEBUG: <Deparsed MERGE query: 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 >
|
DEBUG: <Deparsed MERGE query: 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 >
|
||||||
DEBUG: <Deparsed MERGE query: 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 >
|
DEBUG: <Deparsed MERGE query: 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 >
|
||||||
|
@ -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
|
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
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
RESET client_min_messages;
|
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
|
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
|
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||||
SET citus.log_remote_commands to false;
|
SET citus.log_remote_commands to false;
|
||||||
|
@ -2526,9 +2521,442 @@ WHERE pg_result.t1 IS NULL OR local_ref.t1 IS NULL;
|
||||||
0
|
0
|
||||||
(1 row)
|
(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
|
-- 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
|
-- Reference as a target and local as source
|
||||||
MERGE INTO refsource_ref
|
MERGE INTO refsource_ref
|
||||||
USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1
|
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
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(foo.t1);
|
INSERT VALUES(foo.t1);
|
||||||
ERROR: Reference table as target is not allowed in MERGE command
|
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
|
MERGE INTO target_set
|
||||||
USING source_set AS foo ON target_set.t1 = foo.s1
|
USING source_set AS foo ON target_set.t1 = foo.s1
|
||||||
WHEN MATCHED THEN
|
WHEN MATCHED THEN
|
||||||
UPDATE SET ctid = '(0,100)';
|
UPDATE SET ctid = '(0,100)';
|
||||||
ERROR: cannot assign to system column "ctid"
|
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
|
-- modifying CTE not supported
|
||||||
EXPLAIN
|
EXPLAIN
|
||||||
WITH cte_1 AS (DELETE FROM target_json)
|
WITH cte_1 AS (DELETE FROM target_json RETURNING *)
|
||||||
MERGE INTO target_json sda
|
MERGE INTO target_json sda
|
||||||
USING source_json sdn
|
USING cte_1 sdn
|
||||||
ON sda.id = sdn.id
|
ON sda.id = sdn.id
|
||||||
WHEN NOT matched THEN
|
WHEN NOT matched THEN
|
||||||
INSERT (id, z) VALUES (sdn.id, 5);
|
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
|
-- Grouping sets not supported
|
||||||
MERGE INTO citus_target t
|
MERGE INTO citus_target t
|
||||||
USING (SELECT count(*), id FROM citus_source GROUP BY GROUPING SETS (id, val)) subq
|
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)
|
INSERT VALUES (subq.id, 99)
|
||||||
WHEN MATCHED AND t.id < 350 THEN
|
WHEN MATCHED AND t.id < 350 THEN
|
||||||
DELETE;
|
DELETE;
|
||||||
ERROR: cannot push down this subquery
|
ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP
|
||||||
DETAIL: 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
|
WITH subq AS
|
||||||
(
|
(
|
||||||
SELECT count(*), id FROM citus_source GROUP BY GROUPING SETS (id, val)
|
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)
|
INSERT VALUES (subq.id, 99)
|
||||||
WHEN MATCHED AND t.id < 350 THEN
|
WHEN MATCHED AND t.id < 350 THEN
|
||||||
DELETE;
|
DELETE;
|
||||||
ERROR: cannot push down this subquery
|
ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP
|
||||||
DETAIL: 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
|
-- try inserting unmatched distribution column value
|
||||||
MERGE INTO citus_target t
|
MERGE INTO citus_target t
|
||||||
USING citus_source s
|
USING citus_source s
|
||||||
|
@ -2617,6 +3025,7 @@ ON t.id = s.id
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id) VALUES(1000);
|
INSERT (id) VALUES(1000);
|
||||||
ERROR: MERGE INSERT must refer a source column for distribution column
|
ERROR: MERGE INSERT must refer a source column for distribution column
|
||||||
|
-- Colocated merge
|
||||||
MERGE INTO t1 t
|
MERGE INTO t1 t
|
||||||
USING s1 s
|
USING s1 s
|
||||||
ON t.id = s.id
|
ON t.id = s.id
|
||||||
|
@ -2629,6 +3038,13 @@ ON t.id = s.id
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (val) VALUES(s.val);
|
INSERT (val) VALUES(s.val);
|
||||||
ERROR: MERGE INSERT must have distribution column as value
|
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
|
-- try updating the distribution key column
|
||||||
BEGIN;
|
BEGIN;
|
||||||
MERGE INTO target_cj t
|
MERGE INTO target_cj t
|
||||||
|
@ -2698,7 +3114,7 @@ MERGE INTO t1
|
||||||
UPDATE SET val = t1.val + 1
|
UPDATE SET val = t1.val + 1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (s1.id, s1.val);
|
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
|
-- Now both s1 and t1 are distributed tables
|
||||||
SELECT undistribute_table('t1');
|
SELECT undistribute_table('t1');
|
||||||
NOTICE: creating a new table for merge_schema.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;
|
UPDATE SET val = t1.val + s1.val;
|
||||||
ERROR: non-IMMUTABLE functions are not yet supported in MERGE sql with distributed tables
|
ERROR: non-IMMUTABLE functions are not yet supported in MERGE sql with distributed tables
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
-- Joining on partition columns with sub-query
|
-- Joining on non-partition columns with CTE source, but INSERT incorrect column
|
||||||
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
|
|
||||||
WITH s1_res AS (
|
WITH s1_res AS (
|
||||||
SELECT * FROM s1
|
SELECT * FROM s1
|
||||||
)
|
)
|
||||||
|
@ -2786,7 +3192,7 @@ MERGE INTO t1
|
||||||
UPDATE SET val = t1.val + 1
|
UPDATE SET val = t1.val + 1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (s1_res.id, s1_res.val);
|
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
|
-- Constant Join condition
|
||||||
WITH s1_res AS (
|
WITH s1_res AS (
|
||||||
SELECT * FROM s1
|
SELECT * FROM s1
|
||||||
|
@ -2799,8 +3205,9 @@ MERGE INTO t1
|
||||||
UPDATE SET val = t1.val + 1
|
UPDATE SET val = t1.val + 1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (s1_res.id, s1_res.val);
|
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
|
||||||
-- With a single WHEN clause, which causes a non-left join
|
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 (
|
WITH s1_res AS (
|
||||||
SELECT * FROM s1
|
SELECT * FROM s1
|
||||||
)
|
)
|
||||||
|
@ -2808,7 +3215,8 @@ WITH s1_res AS (
|
||||||
WHEN MATCHED THEN DELETE
|
WHEN MATCHED THEN DELETE
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (s1_res.id, s1_res.val);
|
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
|
-- Reference tables
|
||||||
--
|
--
|
||||||
|
@ -2902,7 +3310,7 @@ MERGE INTO t1
|
||||||
UPDATE SET val = t1.val + 1
|
UPDATE SET val = t1.val + 1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (s1.id, s1.val);
|
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
|
MERGE INTO t1
|
||||||
USING (SELECT * FROM s1) sub ON (sub.id = t1.id)
|
USING (SELECT * FROM s1) sub ON (sub.id = t1.id)
|
||||||
WHEN MATCHED AND sub.val = 0 THEN
|
WHEN MATCHED AND sub.val = 0 THEN
|
||||||
|
@ -2911,7 +3319,7 @@ MERGE INTO t1
|
||||||
UPDATE SET val = t1.val + 1
|
UPDATE SET val = t1.val + 1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (sub.id, sub.val);
|
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);
|
CREATE TABLE pg(val int);
|
||||||
SELECT create_distributed_table('s1', 'id');
|
SELECT create_distributed_table('s1', 'id');
|
||||||
NOTICE: Copying data from local table...
|
NOTICE: Copying data from local table...
|
||||||
|
@ -2932,7 +3340,7 @@ MERGE INTO t1
|
||||||
UPDATE SET val = t1.val + 1
|
UPDATE SET val = t1.val + 1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (sub.id, sub.val);
|
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
|
-- Mix Postgres table in CTE
|
||||||
WITH pg_res AS (
|
WITH pg_res AS (
|
||||||
SELECT * FROM pg
|
SELECT * FROM pg
|
||||||
|
@ -2945,7 +3353,7 @@ MERGE INTO t1
|
||||||
UPDATE SET val = t1.val + 1
|
UPDATE SET val = t1.val + 1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (sub.id, sub.val);
|
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
|
-- Match more than one source row should fail same as Postgres behavior
|
||||||
SELECT undistribute_table('t1');
|
SELECT undistribute_table('t1');
|
||||||
NOTICE: creating a new table for merge_schema.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);
|
INSERT VALUES(mv_source.id, mv_source.val);
|
||||||
ERROR: cannot execute MERGE on relation "mv_source"
|
ERROR: cannot execute MERGE on relation "mv_source"
|
||||||
DETAIL: This operation is not supported for materialized views.
|
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);
|
CREATE TABLE dist_target(id int, val varchar);
|
||||||
SELECT create_distributed_table('dist_target', 'id');
|
SELECT create_distributed_table('dist_target', 'id');
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
@ -3015,31 +3423,6 @@ SELECT create_distributed_table('dist_source', 'id', colocate_with => 'none');
|
||||||
|
|
||||||
(1 row)
|
(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
|
MERGE INTO dist_target
|
||||||
USING (SELECT 100 id) AS source
|
USING (SELECT 100 id) AS source
|
||||||
ON dist_target.id = source.id AND dist_target.val = 'const'
|
ON dist_target.id = source.id AND dist_target.val = 'const'
|
||||||
|
@ -3047,7 +3430,7 @@ WHEN MATCHED THEN
|
||||||
UPDATE SET val = 'source'
|
UPDATE SET val = 'source'
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(source.id, 'source');
|
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).
|
-- Non-hash distributed tables (append/range).
|
||||||
CREATE VIEW show_tables AS
|
CREATE VIEW show_tables AS
|
||||||
SELECT logicalrelid, partmethod
|
SELECT logicalrelid, partmethod
|
||||||
|
@ -3086,8 +3469,7 @@ WHEN MATCHED THEN
|
||||||
UPDATE SET val = dist_source.val
|
UPDATE SET val = dist_source.val
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(dist_source.id, dist_source.val);
|
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
|
ERROR: For MERGE command, append/range distribution table is not supported yet
|
||||||
HINT: Consider using hash distribution instead
|
|
||||||
SELECT undistribute_table('dist_source');
|
SELECT undistribute_table('dist_source');
|
||||||
NOTICE: creating a new table for merge_schema.dist_source
|
NOTICE: creating a new table for merge_schema.dist_source
|
||||||
NOTICE: moving the data of 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
|
UPDATE SET val = dist_source.val
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(dist_source.id, dist_source.val);
|
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
|
ERROR: For MERGE command, append/range distribution table is not supported yet
|
||||||
HINT: Consider using hash distribution instead
|
|
||||||
-- Both are append tables
|
-- Both are append tables
|
||||||
SELECT undistribute_table('dist_target');
|
SELECT undistribute_table('dist_target');
|
||||||
NOTICE: creating a new table for merge_schema.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
|
UPDATE SET val = dist_source.val
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(dist_source.id, dist_source.val);
|
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
|
ERROR: For MERGE command, append/range distribution table is not supported yet
|
||||||
HINT: Consider using hash distribution instead
|
|
||||||
-- Both are range tables
|
-- Both are range tables
|
||||||
SELECT undistribute_table('dist_target');
|
SELECT undistribute_table('dist_target');
|
||||||
NOTICE: creating a new table for merge_schema.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
|
UPDATE SET val = dist_source.val
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(dist_source.id, dist_source.val);
|
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
|
ERROR: For MERGE command, append/range distribution table is not supported yet
|
||||||
HINT: Consider using hash distribution instead
|
|
||||||
-- test merge with single-shard tables
|
-- test merge with single-shard tables
|
||||||
CREATE SCHEMA query_single_shard_table;
|
CREATE SCHEMA query_single_shard_table;
|
||||||
SET search_path TO 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
|
-- with a colocated table
|
||||||
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
|
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 MATCHED THEN UPDATE SET b = nullkey_c1_t2.b;
|
||||||
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b>
|
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000168 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000169 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b>
|
||||||
DEBUG: Creating MERGE router plan
|
DEBUG: Creating MERGE router plan
|
||||||
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
|
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
|
||||||
WHEN MATCHED THEN DELETE;
|
WHEN MATCHED THEN DELETE;
|
||||||
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE>
|
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000168 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000169 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE>
|
||||||
DEBUG: Creating MERGE router plan
|
DEBUG: Creating MERGE router plan
|
||||||
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
|
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 MATCHED THEN UPDATE SET b = nullkey_c1_t2.b
|
||||||
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
|
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
|
||||||
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
|
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000168 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000169 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
|
||||||
DEBUG: Creating MERGE router plan
|
DEBUG: Creating MERGE router plan
|
||||||
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
|
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
|
||||||
WHEN MATCHED THEN DELETE
|
WHEN MATCHED THEN DELETE
|
||||||
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
|
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
|
||||||
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
|
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000168 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000169 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
|
||||||
DEBUG: Creating MERGE router plan
|
DEBUG: Creating MERGE router plan
|
||||||
-- with non-colocated single-shard table
|
-- with non-colocated single-shard table
|
||||||
MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a)
|
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 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)
|
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 MATCHED THEN UPDATE SET b = nullkey_c2_t1.b
|
||||||
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, 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
|
-- with a distributed table
|
||||||
MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a)
|
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 MATCHED THEN UPDATE SET b = distributed_table.b
|
||||||
WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, 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)
|
MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a)
|
||||||
WHEN MATCHED THEN DELETE
|
WHEN MATCHED THEN DELETE
|
||||||
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
|
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
|
-- with a reference table
|
||||||
MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a)
|
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;
|
WHEN MATCHED THEN UPDATE SET b = reference_table.b;
|
||||||
ERROR: MERGE command is not supported with combination of distributed/reference yet
|
DEBUG: A mix of distributed and reference table, try repartitioning
|
||||||
HINT: If target is distributed, source must be distributed and co-located
|
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)
|
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 MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
|
||||||
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, 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
|
-- with a citus local table
|
||||||
MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a)
|
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;
|
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)
|
MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a)
|
||||||
WHEN MATCHED THEN DELETE;
|
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
|
-- with a postgres table
|
||||||
MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a)
|
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;
|
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)
|
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 MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
|
||||||
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, 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
|
-- using ctes
|
||||||
WITH cte AS (
|
WITH cte AS (
|
||||||
SELECT * FROM nullkey_c1_t1
|
SELECT * FROM nullkey_c1_t1
|
||||||
)
|
)
|
||||||
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
|
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
|
||||||
WHEN MATCHED THEN UPDATE SET b = cte.b;
|
WHEN MATCHED THEN UPDATE SET b = cte.b;
|
||||||
DEBUG: <Deparsed MERGE query: WITH cte AS (SELECT nullkey_c1_t1_1.a, nullkey_c1_t1_1.b FROM query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1_1) MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b>
|
DEBUG: <Deparsed MERGE query: WITH cte AS (SELECT nullkey_c1_t1_1.a, nullkey_c1_t1_1.b FROM query_single_shard_table.nullkey_c1_t1_4000168 nullkey_c1_t1_1) MERGE INTO query_single_shard_table.nullkey_c1_t1_4000168 nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b>
|
||||||
DEBUG: Creating MERGE router plan
|
DEBUG: Creating MERGE router plan
|
||||||
WITH cte AS (
|
WITH cte AS (
|
||||||
SELECT * FROM distributed_table
|
SELECT * FROM distributed_table
|
||||||
)
|
)
|
||||||
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
|
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
|
||||||
WHEN MATCHED THEN UPDATE SET b = cte.b;
|
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 (
|
WITH cte AS materialized (
|
||||||
SELECT * FROM distributed_table
|
SELECT * FROM distributed_table
|
||||||
)
|
)
|
||||||
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
|
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
|
||||||
WHEN MATCHED THEN UPDATE SET b = cte.b;
|
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;
|
SET client_min_messages TO WARNING;
|
||||||
DROP SCHEMA query_single_shard_table CASCADE;
|
DROP SCHEMA query_single_shard_table CASCADE;
|
||||||
RESET client_min_messages;
|
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
|
PL/pgSQL function citus_drop_trigger() line XX at PERFORM
|
||||||
DROP FUNCTION merge_when_and_write();
|
DROP FUNCTION merge_when_and_write();
|
||||||
DROP SCHEMA merge_schema CASCADE;
|
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()
|
DETAIL: drop cascades to function insert_data()
|
||||||
drop cascades to table local_local
|
drop cascades to table local_local
|
||||||
drop cascades to table target
|
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_serial
|
||||||
drop cascades to table target_set
|
drop cascades to table target_set
|
||||||
drop cascades to table source_set
|
drop cascades to table source_set
|
||||||
drop cascades to table reftarget_local_4000113
|
|
||||||
drop cascades to table refsource_ref
|
drop cascades to table refsource_ref
|
||||||
drop cascades to table pg_result
|
drop cascades to table pg_result
|
||||||
drop cascades to table refsource_ref_4000112
|
drop cascades to table refsource_ref_4000112
|
||||||
drop cascades to table pg_ref
|
drop cascades to table pg_ref
|
||||||
drop cascades to table reftarget_local
|
|
||||||
drop cascades to table local_ref
|
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 function add_s(integer,integer)
|
||||||
drop cascades to table pg
|
drop cascades to table pg
|
||||||
drop cascades to table t1_4000133
|
drop cascades to table t1_4000158
|
||||||
drop cascades to table s1_4000134
|
drop cascades to table s1_4000159
|
||||||
drop cascades to table t1
|
drop cascades to table t1
|
||||||
drop cascades to table s1
|
drop cascades to table s1
|
||||||
drop cascades to table dist_colocated
|
|
||||||
drop cascades to table dist_target
|
drop cascades to table dist_target
|
||||||
drop cascades to table dist_source
|
drop cascades to table dist_source
|
||||||
drop cascades to view show_tables
|
drop cascades to view show_tables
|
||||||
SELECT 1 FROM master_remove_node('localhost', :master_port);
|
|
||||||
?column?
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
1
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
|
|
|
@ -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
|
|
@ -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
|
|
@ -418,14 +418,16 @@ SELECT create_distributed_table('tbl2', 'x');
|
||||||
|
|
||||||
MERGE INTO tbl1 USING tbl2 ON (true)
|
MERGE INTO tbl1 USING tbl2 ON (true)
|
||||||
WHEN MATCHED THEN DELETE;
|
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
|
||||||
-- also, not inside subqueries & ctes
|
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 (
|
WITH targq AS (
|
||||||
SELECT * FROM tbl2
|
SELECT * FROM tbl2
|
||||||
)
|
)
|
||||||
MERGE INTO tbl1 USING targq ON (true)
|
MERGE INTO tbl1 USING targq ON (true)
|
||||||
WHEN MATCHED THEN DELETE;
|
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 (
|
WITH foo AS (
|
||||||
MERGE INTO tbl1 USING tbl2 ON (true)
|
MERGE INTO tbl1 USING tbl2 ON (true)
|
||||||
WHEN MATCHED THEN DELETE
|
WHEN MATCHED THEN DELETE
|
||||||
|
@ -441,7 +443,8 @@ USING tbl2
|
||||||
ON (true)
|
ON (true)
|
||||||
WHEN MATCHED THEN
|
WHEN MATCHED THEN
|
||||||
DO NOTHING;
|
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
|
MERGE INTO tbl1 t
|
||||||
USING tbl2
|
USING tbl2
|
||||||
ON (true)
|
ON (true)
|
||||||
|
|
|
@ -15,13 +15,14 @@ SET search_path TO pgmerge_schema;
|
||||||
SET citus.use_citus_managed_tables to true;
|
SET citus.use_citus_managed_tables to true;
|
||||||
\set SHOW_CONTEXT errors
|
\set SHOW_CONTEXT errors
|
||||||
SET citus.next_shard_id TO 4001000;
|
SET citus.next_shard_id TO 4001000;
|
||||||
|
SET client_min_messages = warning;
|
||||||
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
|
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?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
RESET client_min_messages;
|
||||||
CREATE USER regress_merge_privs;
|
CREATE USER regress_merge_privs;
|
||||||
CREATE USER regress_merge_no_privs;
|
CREATE USER regress_merge_no_privs;
|
||||||
DROP TABLE IF EXISTS target;
|
DROP TABLE IF EXISTS target;
|
||||||
|
@ -2133,9 +2134,3 @@ drop cascades to table source2
|
||||||
drop cascades to function merge_trigfunc()
|
drop cascades to function merge_trigfunc()
|
||||||
DROP USER regress_merge_privs;
|
DROP USER regress_merge_privs;
|
||||||
DROP USER regress_merge_no_privs;
|
DROP USER regress_merge_no_privs;
|
||||||
SELECT 1 FROM master_remove_node('localhost', :master_port);
|
|
||||||
?column?
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
1
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
|
|
|
@ -105,8 +105,7 @@ test: background_task_queue_monitor
|
||||||
test: clock
|
test: clock
|
||||||
|
|
||||||
# MERGE tests
|
# MERGE tests
|
||||||
test: merge
|
test: merge merge_repartition pgmerge
|
||||||
test: pgmerge
|
|
||||||
|
|
||||||
# ---------
|
# ---------
|
||||||
# test that no tests leaked intermediate results. This should always be last
|
# test that no tests leaked intermediate results. This should always be last
|
||||||
|
|
|
@ -21,7 +21,9 @@ SET citus.next_shard_id TO 4000000;
|
||||||
SET citus.explain_all_tasks TO true;
|
SET citus.explain_all_tasks TO true;
|
||||||
SET citus.shard_replication_factor TO 1;
|
SET citus.shard_replication_factor TO 1;
|
||||||
SET citus.max_adaptive_executor_pool_size 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);
|
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
|
||||||
|
RESET client_min_messages;
|
||||||
|
|
||||||
CREATE TABLE source
|
CREATE TABLE source
|
||||||
(
|
(
|
||||||
|
@ -929,23 +931,21 @@ SELECT create_distributed_table('source_cj1', 'sid1');
|
||||||
SELECT create_distributed_table('source_cj2', 'sid2');
|
SELECT create_distributed_table('source_cj2', 'sid2');
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SET citus.log_remote_commands to true;
|
|
||||||
MERGE INTO target_cj t
|
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
|
ON t.tid = sid1 AND t.tid = 2
|
||||||
WHEN MATCHED THEN
|
WHEN MATCHED THEN
|
||||||
UPDATE SET src = src2
|
UPDATE SET src = src2
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
DO NOTHING;
|
DO NOTHING;
|
||||||
SET citus.log_remote_commands to false;
|
|
||||||
SELECT * FROM target_cj ORDER BY 1;
|
SELECT * FROM target_cj ORDER BY 1;
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
-- try accessing columns from either side of the source join
|
-- try accessing columns from either side of the source join
|
||||||
MERGE INTO target_cj t
|
MERGE INTO target_cj t
|
||||||
USING source_cj1 s2
|
USING (SELECT * FROM source_cj1 s2
|
||||||
INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10
|
INNER JOIN source_cj2 s1 ON sid1 = sid2 AND val1 = 10) s
|
||||||
ON t.tid = sid1 AND t.tid = 2
|
ON t.tid = sid1 AND t.tid = 2
|
||||||
WHEN MATCHED THEN
|
WHEN MATCHED THEN
|
||||||
UPDATE SET src = src1, val = val2
|
UPDATE SET src = src1, val = val2
|
||||||
|
@ -983,7 +983,7 @@ ROLLBACK;
|
||||||
|
|
||||||
|
|
||||||
-- Test PREPARE
|
-- Test PREPARE
|
||||||
PREPARE foo(int) AS
|
PREPARE merge_prepare(int) AS
|
||||||
MERGE INTO target_cj target
|
MERGE INTO target_cj target
|
||||||
USING (SELECT * FROM source_cj1) sub
|
USING (SELECT * FROM source_cj1) sub
|
||||||
ON target.tid = sub.sid1 AND target.tid = $1
|
ON target.tid = sub.sid1 AND target.tid = $1
|
||||||
|
@ -995,11 +995,11 @@ WHEN NOT MATCHED THEN
|
||||||
SELECT * FROM target_cj ORDER BY 1;
|
SELECT * FROM target_cj ORDER BY 1;
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
SELECT * FROM target_cj ORDER BY 1;
|
SELECT * FROM target_cj ORDER BY 1;
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
|
|
||||||
|
@ -1007,10 +1007,10 @@ BEGIN;
|
||||||
|
|
||||||
SET citus.log_remote_commands to true;
|
SET citus.log_remote_commands to true;
|
||||||
SET client_min_messages TO DEBUG1;
|
SET client_min_messages TO DEBUG1;
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
RESET client_min_messages;
|
RESET client_min_messages;
|
||||||
|
|
||||||
EXECUTE foo(2);
|
EXECUTE merge_prepare(2);
|
||||||
SET citus.log_remote_commands to false;
|
SET citus.log_remote_commands to false;
|
||||||
|
|
||||||
SELECT * FROM target_cj ORDER BY 1;
|
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
|
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;
|
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
|
-- 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
|
-- Reference as a target and local as source
|
||||||
MERGE INTO refsource_ref
|
MERGE INTO refsource_ref
|
||||||
USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1
|
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
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(foo.t1);
|
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
|
MERGE INTO target_set
|
||||||
USING source_set AS foo ON target_set.t1 = foo.s1
|
USING source_set AS foo ON target_set.t1 = foo.s1
|
||||||
WHEN MATCHED THEN
|
WHEN MATCHED THEN
|
||||||
UPDATE SET ctid = '(0,100)';
|
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
|
-- modifying CTE not supported
|
||||||
EXPLAIN
|
EXPLAIN
|
||||||
WITH cte_1 AS (DELETE FROM target_json)
|
WITH cte_1 AS (DELETE FROM target_json RETURNING *)
|
||||||
MERGE INTO target_json sda
|
MERGE INTO target_json sda
|
||||||
USING source_json sdn
|
USING cte_1 sdn
|
||||||
ON sda.id = sdn.id
|
ON sda.id = sdn.id
|
||||||
WHEN NOT matched THEN
|
WHEN NOT matched THEN
|
||||||
INSERT (id, z) VALUES (sdn.id, 5);
|
INSERT (id, z) VALUES (sdn.id, 5);
|
||||||
|
@ -1711,6 +1913,7 @@ ON t.id = s.id
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id) VALUES(1000);
|
INSERT (id) VALUES(1000);
|
||||||
|
|
||||||
|
-- Colocated merge
|
||||||
MERGE INTO t1 t
|
MERGE INTO t1 t
|
||||||
USING s1 s
|
USING s1 s
|
||||||
ON t.id = s.id
|
ON t.id = s.id
|
||||||
|
@ -1723,6 +1926,13 @@ ON t.id = s.id
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (val) VALUES(s.val);
|
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
|
-- try updating the distribution key column
|
||||||
BEGIN;
|
BEGIN;
|
||||||
MERGE INTO target_cj t
|
MERGE INTO target_cj t
|
||||||
|
@ -1811,17 +2021,7 @@ WHEN MATCHED AND (merge_when_and_write()) THEN
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
|
|
||||||
|
|
||||||
-- Joining on partition columns with sub-query
|
-- Joining on non-partition columns with CTE source, but INSERT incorrect column
|
||||||
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
|
|
||||||
WITH s1_res AS (
|
WITH s1_res AS (
|
||||||
SELECT * FROM s1
|
SELECT * FROM s1
|
||||||
)
|
)
|
||||||
|
@ -1847,7 +2047,7 @@ MERGE INTO t1
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT (id, val) VALUES (s1_res.id, s1_res.val);
|
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 (
|
WITH s1_res AS (
|
||||||
SELECT * FROM s1
|
SELECT * FROM s1
|
||||||
)
|
)
|
||||||
|
@ -1954,34 +2154,12 @@ WHEN MATCHED THEN
|
||||||
WHEN NOT MATCHED THEN
|
WHEN NOT MATCHED THEN
|
||||||
INSERT VALUES(mv_source.id, mv_source.val);
|
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);
|
CREATE TABLE dist_target(id int, val varchar);
|
||||||
SELECT create_distributed_table('dist_target', 'id');
|
SELECT create_distributed_table('dist_target', 'id');
|
||||||
CREATE TABLE dist_source(id int, val varchar);
|
CREATE TABLE dist_source(id int, val varchar);
|
||||||
SELECT create_distributed_table('dist_source', 'id', colocate_with => 'none');
|
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
|
MERGE INTO dist_target
|
||||||
USING (SELECT 100 id) AS source
|
USING (SELECT 100 id) AS source
|
||||||
ON dist_target.id = source.id AND dist_target.val = 'const'
|
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 SERVER foreign_server CASCADE;
|
||||||
DROP FUNCTION merge_when_and_write();
|
DROP FUNCTION merge_when_and_write();
|
||||||
DROP SCHEMA merge_schema CASCADE;
|
DROP SCHEMA merge_schema CASCADE;
|
||||||
SELECT 1 FROM master_remove_node('localhost', :master_port);
|
|
||||||
|
|
|
@ -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;
|
|
@ -262,7 +262,7 @@ SELECT create_distributed_table('tbl2', 'x');
|
||||||
MERGE INTO tbl1 USING tbl2 ON (true)
|
MERGE INTO tbl1 USING tbl2 ON (true)
|
||||||
WHEN MATCHED THEN DELETE;
|
WHEN MATCHED THEN DELETE;
|
||||||
|
|
||||||
-- also, not inside subqueries & ctes
|
-- also, inside subqueries & ctes
|
||||||
WITH targq AS (
|
WITH targq AS (
|
||||||
SELECT * FROM tbl2
|
SELECT * FROM tbl2
|
||||||
)
|
)
|
||||||
|
|
|
@ -19,7 +19,9 @@ SET citus.use_citus_managed_tables to true;
|
||||||
|
|
||||||
SET citus.next_shard_id TO 4001000;
|
SET citus.next_shard_id TO 4001000;
|
||||||
|
|
||||||
|
SET client_min_messages = warning;
|
||||||
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
|
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
|
||||||
|
RESET client_min_messages;
|
||||||
|
|
||||||
CREATE USER regress_merge_privs;
|
CREATE USER regress_merge_privs;
|
||||||
CREATE USER regress_merge_no_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 SCHEMA pgmerge_schema CASCADE;
|
||||||
DROP USER regress_merge_privs;
|
DROP USER regress_merge_privs;
|
||||||
DROP USER regress_merge_no_privs;
|
DROP USER regress_merge_no_privs;
|
||||||
SELECT 1 FROM master_remove_node('localhost', :master_port);
|
|
||||||
|
|
Loading…
Reference in New Issue