Merge pull request #3376 from citusdata/insert_select

INSERT...SELECT with re-partitioning
pull/3350/head
Hadi Moshayedi 2020-01-17 01:36:36 -08:00 committed by GitHub
commit 60a2bc5ec2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
44 changed files with 3208 additions and 229 deletions

View File

@ -88,6 +88,7 @@
#include "foreign/foreign.h"
#include "libpq/pqformat.h"
#include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h"
#include "tsearch/ts_locale.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
@ -832,6 +833,30 @@ CanUseBinaryCopyFormat(TupleDesc tupleDescription)
}
/*
* CanUseBinaryCopyFormatForTargetList returns true if we can use binary
* copy format for all columns of the given target list.
*/
bool
CanUseBinaryCopyFormatForTargetList(List *targetEntryList)
{
ListCell *targetEntryCell = NULL;
foreach(targetEntryCell, targetEntryList)
{
TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell);
Node *targetExpr = (Node *) targetEntry->expr;
Oid columnType = exprType(targetExpr);
if (!CanUseBinaryCopyFormatForType(columnType))
{
return false;
}
}
return true;
}
/*
* CanUseBinaryCopyFormatForType determines whether it is safe to use the
* binary copy format for the given type. The binary copy format cannot

View File

@ -62,7 +62,7 @@ CustomScanMethods TaskTrackerCustomScanMethods = {
};
CustomScanMethods CoordinatorInsertSelectCustomScanMethods = {
"Citus INSERT ... SELECT via coordinator",
"Citus INSERT ... SELECT",
CoordinatorInsertSelectCreateScan
};

View File

@ -54,6 +54,7 @@ typedef struct NodeToNodeFragmentsTransfer
/* forward declarations of local functions */
static void WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList,
int partitionColumnIndex,
DistTableCacheEntry *targetRelation,
bool binaryFormat);
static List * ExecutePartitionTaskList(List *partitionTaskList,
@ -89,9 +90,13 @@ static void ExecuteFetchTaskList(List *fetchTaskList);
*
* returnValue[shardIndex] is list of cstrings each of which is a resultId which
* correspond to targetRelation->sortedShardIntervalArray[shardIndex].
*
* partitionColumnIndex determines the column in the selectTaskList to use for
* partitioning.
*/
List **
RedistributeTaskListResults(char *resultIdPrefix, List *selectTaskList,
int partitionColumnIndex,
DistTableCacheEntry *targetRelation,
bool binaryFormat)
{
@ -104,6 +109,7 @@ RedistributeTaskListResults(char *resultIdPrefix, List *selectTaskList,
UseCoordinatedTransaction();
List *fragmentList = PartitionTasklistResults(resultIdPrefix, selectTaskList,
partitionColumnIndex,
targetRelation, binaryFormat);
return ColocateFragmentsWithRelation(fragmentList, targetRelation);
}
@ -119,9 +125,13 @@ RedistributeTaskListResults(char *resultIdPrefix, List *selectTaskList,
* partition of results. Empty results are omitted. Therefore, if we have N tasks
* and target relation has M shards, we will have NxM-(number of empty results)
* fragments.
*
* partitionColumnIndex determines the column in the selectTaskList to use for
* partitioning.
*/
List *
PartitionTasklistResults(char *resultIdPrefix, List *selectTaskList,
int partitionColumnIndex,
DistTableCacheEntry *targetRelation,
bool binaryFormat)
{
@ -141,7 +151,8 @@ PartitionTasklistResults(char *resultIdPrefix, List *selectTaskList,
*/
UseCoordinatedTransaction();
WrapTasksForPartitioning(resultIdPrefix, selectTaskList, targetRelation,
WrapTasksForPartitioning(resultIdPrefix, selectTaskList,
partitionColumnIndex, targetRelation,
binaryFormat);
return ExecutePartitionTaskList(selectTaskList, targetRelation);
}
@ -154,6 +165,7 @@ PartitionTasklistResults(char *resultIdPrefix, List *selectTaskList,
*/
static void
WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList,
int partitionColumnIndex,
DistTableCacheEntry *targetRelation,
bool binaryFormat)
{
@ -164,11 +176,13 @@ WrapTasksForPartitioning(char *resultIdPrefix, List *selectTaskList,
ArrayType *minValueArray = NULL;
ArrayType *maxValueArray = NULL;
Var *partitionColumn = targetRelation->partitionColumn;
int partitionColumnIndex = partitionColumn->varoattno - 1;
Oid intervalTypeId = partitionColumn->vartype;
int32 intervalTypeMod = partitionColumn->vartypmod;
Oid intervalTypeId = InvalidOid;
int32 intervalTypeMod = 0;
Oid intervalTypeOutFunc = InvalidOid;
bool intervalTypeVarlena = false;
GetIntervalTypeInfo(targetRelation->partitionMethod, partitionColumn,
&intervalTypeId, &intervalTypeMod);
getTypeOutputInfo(intervalTypeId, &intervalTypeOutFunc, &intervalTypeVarlena);
ShardMinMaxValueArrays(shardIntervalArray, shardCount, intervalTypeOutFunc,

View File

@ -13,19 +13,24 @@
#include "distributed/citus_ruleutils.h"
#include "distributed/commands/multi_copy.h"
#include "distributed/adaptive_executor.h"
#include "distributed/distributed_execution_locks.h"
#include "distributed/insert_select_executor.h"
#include "distributed/insert_select_planner.h"
#include "distributed/intermediate_results.h"
#include "distributed/local_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/adaptive_executor.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_router_planner.h"
#include "distributed/distributed_planner.h"
#include "distributed/recursive_planning.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/resource_lock.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/subplan_execution.h"
#include "distributed/transaction_management.h"
#include "executor/executor.h"
#include "nodes/execnodes.h"
@ -43,6 +48,8 @@
#include "utils/rel.h"
#include "utils/snapmgr.h"
/* Config variables managed via guc.c */
bool EnableRepartitionedInsertSelect = true;
/* depth of current insert/select executor. */
static int insertSelectExecutorLevel = 0;
@ -52,17 +59,27 @@ static TupleTableSlot * CoordinatorInsertSelectExecScanInternal(CustomScanState
static Query * WrapSubquery(Query *subquery);
static List * TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery,
char *resultIdPrefix);
static void ExecuteSelectIntoRelation(Oid targetRelationId, List *insertTargetList,
Query *selectQuery, EState *executorState);
static HTAB * ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
List *insertTargetList,
Query *selectQuery,
EState *executorState,
char *
intermediateResultIdPrefix);
static void ExecutePlanIntoRelation(Oid targetRelationId, List *insertTargetList,
PlannedStmt *selectPlan, EState *executorState);
static HTAB * ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId,
List *insertTargetList,
PlannedStmt *selectPlan,
EState *executorState,
char *intermediateResultIdPrefix);
static List * BuildColumnNameListFromTargetList(Oid targetRelationId,
List *insertTargetList);
static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList);
static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
Oid targetRelationId);
static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery,
DistTableCacheEntry *targetRelation,
List **redistributedResults,
bool useBinaryFormat);
static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn);
static Expr * CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation,
int targetTypeMod);
static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries);
static void RelableTargetEntryList(List *selectTargetList, List *insertTargetList);
/*
@ -106,6 +123,7 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
if (!scanState->finishedRemoteScan)
{
EState *executorState = ScanStateGetExecutorState(scanState);
ParamListInfo paramListInfo = executorState->es_param_list_info;
DistributedPlan *distributedPlan = scanState->distributedPlan;
Query *insertSelectQuery = copyObject(distributedPlan->insertSelectQuery);
List *insertTargetList = insertSelectQuery->targetList;
@ -116,9 +134,6 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
bool hasReturning = distributedPlan->hasReturning;
HTAB *shardStateHash = NULL;
ereport(DEBUG1, (errmsg("Collecting INSERT ... SELECT results on coordinator")));
/*
* INSERT .. SELECT via coordinator consists of two steps, a SELECT is
* followd by a COPY. If the SELECT is executed locally, then the COPY
@ -134,6 +149,35 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
selectRte->subquery = selectQuery;
ReorderInsertSelectTargetLists(insertSelectQuery, insertRte, selectRte);
/*
* Cast types of insert target list and select projection list to
* match the column types of the target relation.
*/
selectQuery->targetList =
AddInsertSelectCasts(insertSelectQuery->targetList,
selectQuery->targetList,
targetRelationId);
/*
* Later we might need to call WrapTaskListForProjection(), which requires
* that select target list has unique names, otherwise the outer query
* cannot select columns unambiguously. So we relabel select columns to
* match target columns.
*/
RelableTargetEntryList(selectQuery->targetList, insertTargetList);
/*
* Make a copy of the query, since pg_plan_query may scribble on it and we
* want it to be replanned every time if it is stored in a prepared
* statement.
*/
selectQuery = copyObject(selectQuery);
/* plan the subquery, this may be another distributed query */
int cursorOptions = CURSOR_OPT_PARALLEL_OK;
PlannedStmt *selectPlan = pg_plan_query(selectQuery, cursorOptions,
paramListInfo);
/*
* If we are dealing with partitioned table, we also need to lock its
* partitions. Here we only lock targetRelation, we acquire necessary
@ -144,8 +188,110 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
LockPartitionRelations(targetRelationId, RowExclusiveLock);
}
if (insertSelectQuery->onConflict || hasReturning)
if (IsRedistributablePlan(selectPlan->planTree) &&
IsSupportedRedistributionTarget(targetRelationId))
{
ereport(DEBUG1, (errmsg("performing repartitioned INSERT ... SELECT")));
DistributedPlan *distSelectPlan =
GetDistributedPlan((CustomScan *) selectPlan->planTree);
Job *distSelectJob = distSelectPlan->workerJob;
List *distSelectTaskList = distSelectJob->taskList;
TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(scanState);
bool randomAccess = true;
bool interTransactions = false;
bool binaryFormat =
CanUseBinaryCopyFormatForTargetList(selectQuery->targetList);
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 "INSERT/SELECTs" are executed recursively.
*/
StringInfo distResultPrefixString = makeStringInfo();
appendStringInfo(distResultPrefixString,
"repartitioned_results_" UINT64_FORMAT,
distSelectJob->jobId);
char *distResultPrefix = distResultPrefixString->data;
DistTableCacheEntry *targetRelation =
DistributedTableCacheEntry(targetRelationId);
int partitionColumnIndex =
PartitionColumnIndex(insertTargetList, targetRelation->partitionColumn);
if (partitionColumnIndex == -1)
{
char *relationName = get_rel_name(targetRelationId);
Oid schemaOid = get_rel_namespace(targetRelationId);
char *schemaName = get_namespace_name(schemaOid);
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg(
"the partition column of table %s should have a value",
quote_qualified_identifier(schemaName,
relationName))));
}
TargetEntry *selectPartitionTE = list_nth(selectQuery->targetList,
partitionColumnIndex);
const char *partitionColumnName = selectPartitionTE->resname ?
selectPartitionTE->resname : "(none)";
ereport(DEBUG2, (errmsg(
"partitioning SELECT query by column index %d with name %s",
partitionColumnIndex, quote_literal_cstr(
partitionColumnName))));
/*
* ExpandWorkerTargetEntry() can add additional columns to the worker
* query. Modify the task queries to only select columns we need.
*/
int requiredColumnCount = list_length(insertTargetList);
List *jobTargetList = distSelectJob->jobQuery->targetList;
if (list_length(jobTargetList) > requiredColumnCount)
{
List *projectedTargetEntries = ListTake(jobTargetList,
requiredColumnCount);
WrapTaskListForProjection(distSelectTaskList, projectedTargetEntries);
}
List **redistributedResults = RedistributeTaskListResults(distResultPrefix,
distSelectTaskList,
partitionColumnIndex,
targetRelation,
binaryFormat);
/*
* At this point select query has been executed on workers and results
* have been fetched in such a way that they are colocated with corresponding
* target shard. Create and execute a list of tasks of form
* INSERT INTO ... SELECT * FROM read_intermediate_results(...);
*/
List *taskList = RedistributedInsertSelectTaskList(insertSelectQuery,
targetRelation,
redistributedResults,
binaryFormat);
scanState->tuplestorestate =
tuplestore_begin_heap(randomAccess, interTransactions, work_mem);
uint64 rowsInserted = ExecuteTaskListIntoTupleStore(ROW_MODIFY_COMMUTATIVE,
taskList,
tupleDescriptor,
scanState->tuplestorestate,
hasReturning);
executorState->es_processed = rowsInserted;
}
else if (insertSelectQuery->onConflict || hasReturning)
{
ereport(DEBUG1, (errmsg(
"Collecting INSERT ... SELECT results on coordinator")));
/*
* If we also have a workerJob that means there is a second step
* to the INSERT...SELECT. This happens when there is a RETURNING
@ -156,10 +302,10 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
ListCell *taskCell = NULL;
List *prunedTaskList = NIL;
shardStateHash = ExecuteSelectIntoColocatedIntermediateResults(
shardStateHash = ExecutePlanIntoColocatedIntermediateResults(
targetRelationId,
insertTargetList,
selectQuery,
selectPlan,
executorState,
intermediateResultIdPrefix);
@ -209,8 +355,11 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
}
else
{
ExecuteSelectIntoRelation(targetRelationId, insertTargetList, selectQuery,
executorState);
ereport(DEBUG1, (errmsg(
"Collecting INSERT ... SELECT results on coordinator")));
ExecutePlanIntoRelation(targetRelationId, insertTargetList, selectPlan,
executorState);
}
scanState->finishedRemoteScan = true;
@ -342,36 +491,6 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery,
uint32 taskIdIndex = 1;
uint64 jobId = INVALID_JOB_ID;
ListCell *targetEntryCell = NULL;
Relation distributedRelation = heap_open(targetRelationId, RowExclusiveLock);
TupleDesc destTupleDescriptor = RelationGetDescr(distributedRelation);
/*
* If the type of insert column and target table's column type is
* different from each other. Cast insert column't type to target
* table's column
*/
foreach(targetEntryCell, insertSelectQuery->targetList)
{
TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell);
Var *insertColumn = (Var *) targetEntry->expr;
Form_pg_attribute attr = TupleDescAttr(destTupleDescriptor, targetEntry->resno -
1);
if (insertColumn->vartype != attr->atttypid)
{
CoerceViaIO *coerceExpr = makeNode(CoerceViaIO);
coerceExpr->arg = (Expr *) copyObject(insertColumn);
coerceExpr->resulttype = attr->atttypid;
coerceExpr->resultcollid = attr->attcollation;
coerceExpr->coerceformat = COERCE_IMPLICIT_CAST;
coerceExpr->location = -1;
targetEntry->expr = (Expr *) coerceExpr;
}
}
for (int shardOffset = 0; shardOffset < shardCount; shardOffset++)
{
ShardInterval *targetShardInterval =
@ -431,24 +550,23 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery,
taskIdIndex++;
}
heap_close(distributedRelation, NoLock);
return taskList;
}
/*
* ExecuteSelectIntoColocatedIntermediateResults executes the given select query
* ExecutePlanIntoColocatedIntermediateResults executes the given PlannedStmt
* and inserts tuples into a set of intermediate results that are colocated with
* the target table for further processing of ON CONFLICT or RETURNING. It also
* returns the hash of shard states that were used to insert tuplesinto the target
* relation.
*/
static HTAB *
ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
List *insertTargetList,
Query *selectQuery, EState *executorState,
char *intermediateResultIdPrefix)
ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId,
List *insertTargetList,
PlannedStmt *selectPlan,
EState *executorState,
char *intermediateResultIdPrefix)
{
ParamListInfo paramListInfo = executorState->es_param_list_info;
bool stopOnFailure = false;
@ -473,14 +591,7 @@ ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
stopOnFailure,
intermediateResultIdPrefix);
/*
* Make a copy of the query, since ExecuteQueryIntoDestReceiver may scribble on it
* and we want it to be replanned every time if it is stored in a prepared
* statement.
*/
Query *queryCopy = copyObject(selectQuery);
ExecuteQueryIntoDestReceiver(queryCopy, paramListInfo, (DestReceiver *) copyDest);
ExecutePlanIntoDestReceiver(selectPlan, paramListInfo, (DestReceiver *) copyDest);
executorState->es_processed = copyDest->tuplesSent;
@ -491,13 +602,13 @@ ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
/*
* ExecuteSelectIntoRelation executes given SELECT query and inserts the
* ExecutePlanIntoRelation executes the given plan and inserts the
* results into the target relation, which is assumed to be a distributed
* table.
*/
static void
ExecuteSelectIntoRelation(Oid targetRelationId, List *insertTargetList,
Query *selectQuery, EState *executorState)
ExecutePlanIntoRelation(Oid targetRelationId, List *insertTargetList,
PlannedStmt *selectPlan, EState *executorState)
{
ParamListInfo paramListInfo = executorState->es_param_list_info;
bool stopOnFailure = false;
@ -521,14 +632,7 @@ ExecuteSelectIntoRelation(Oid targetRelationId, List *insertTargetList,
executorState,
stopOnFailure, NULL);
/*
* Make a copy of the query, since ExecuteQueryIntoDestReceiver may scribble on it
* and we want it to be replanned every time if it is stored in a prepared
* statement.
*/
Query *queryCopy = copyObject(selectQuery);
ExecuteQueryIntoDestReceiver(queryCopy, paramListInfo, (DestReceiver *) copyDest);
ExecutePlanIntoDestReceiver(selectPlan, paramListInfo, (DestReceiver *) copyDest);
executorState->es_processed = copyDest->tuplesSent;
@ -595,3 +699,427 @@ ExecutingInsertSelect(void)
{
return insertSelectExecutorLevel > 0;
}
/*
* AddInsertSelectCasts makes sure that the types in columns in the given
* target lists have the same type as the columns of the given relation.
* It might add casts to ensure that.
*
* It returns the updated selectTargetList.
*/
static List *
AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
Oid targetRelationId)
{
ListCell *insertEntryCell = NULL;
ListCell *selectEntryCell = NULL;
List *projectedEntries = NIL;
List *nonProjectedEntries = NIL;
/*
* ReorderInsertSelectTargetLists() makes sure that first few columns of
* the SELECT query match the insert targets. It might contain additional
* items for GROUP BY, etc.
*/
Assert(list_length(insertTargetList) <= list_length(selectTargetList));
Relation distributedRelation = heap_open(targetRelationId, RowExclusiveLock);
TupleDesc destTupleDescriptor = RelationGetDescr(distributedRelation);
int targetEntryIndex = 0;
forboth(insertEntryCell, insertTargetList, selectEntryCell, selectTargetList)
{
TargetEntry *insertEntry = (TargetEntry *) lfirst(insertEntryCell);
TargetEntry *selectEntry = (TargetEntry *) lfirst(selectEntryCell);
Var *insertColumn = (Var *) insertEntry->expr;
Form_pg_attribute attr = TupleDescAttr(destTupleDescriptor,
insertEntry->resno - 1);
Oid sourceType = insertColumn->vartype;
Oid targetType = attr->atttypid;
if (sourceType != targetType)
{
insertEntry->expr = CastExpr((Expr *) insertColumn, sourceType, targetType,
attr->attcollation, attr->atttypmod);
/*
* We cannot modify the selectEntry in-place, because ORDER BY or
* GROUP BY clauses might be pointing to it with comparison types
* of the source type. So instead we keep the original one as a
* non-projected entry, so GROUP BY and ORDER BY are happy, and
* create a duplicated projected entry with the coerced expression.
*/
TargetEntry *coercedEntry = copyObject(selectEntry);
coercedEntry->expr = CastExpr((Expr *) selectEntry->expr, sourceType,
targetType, attr->attcollation,
attr->atttypmod);
coercedEntry->ressortgroupref = 0;
/*
* The only requirement is that users don't use this name in ORDER BY
* or GROUP BY, and it should be unique across the same query.
*/
StringInfo resnameString = makeStringInfo();
appendStringInfo(resnameString, "auto_coerced_by_citus_%d", targetEntryIndex);
coercedEntry->resname = resnameString->data;
projectedEntries = lappend(projectedEntries, coercedEntry);
if (selectEntry->ressortgroupref != 0)
{
selectEntry->resjunk = true;
nonProjectedEntries = lappend(nonProjectedEntries, selectEntry);
}
}
else
{
projectedEntries = lappend(projectedEntries, selectEntry);
}
targetEntryIndex++;
}
for (int entryIndex = list_length(insertTargetList);
entryIndex < list_length(selectTargetList);
entryIndex++)
{
nonProjectedEntries = lappend(nonProjectedEntries, list_nth(selectTargetList,
entryIndex));
}
/* selectEntry->resno must be the ordinal number of the entry */
selectTargetList = list_concat(projectedEntries, nonProjectedEntries);
int entryResNo = 1;
foreach(selectEntryCell, selectTargetList)
{
TargetEntry *selectEntry = lfirst(selectEntryCell);
selectEntry->resno = entryResNo++;
}
heap_close(distributedRelation, NoLock);
return selectTargetList;
}
/*
* CastExpr returns an expression which casts the given expr from sourceType to
* the given targetType.
*/
static Expr *
CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation,
int targetTypeMod)
{
Oid coercionFuncId = InvalidOid;
CoercionPathType coercionType = find_coercion_pathway(targetType, sourceType,
COERCION_EXPLICIT,
&coercionFuncId);
if (coercionType == COERCION_PATH_FUNC)
{
FuncExpr *coerceExpr = makeNode(FuncExpr);
coerceExpr->funcid = coercionFuncId;
coerceExpr->args = list_make1(copyObject(expr));
coerceExpr->funccollid = targetCollation;
coerceExpr->funcresulttype = targetType;
return (Expr *) coerceExpr;
}
else if (coercionType == COERCION_PATH_RELABELTYPE)
{
RelabelType *coerceExpr = makeNode(RelabelType);
coerceExpr->arg = copyObject(expr);
coerceExpr->resulttype = targetType;
coerceExpr->resulttypmod = targetTypeMod;
coerceExpr->resultcollid = targetCollation;
coerceExpr->relabelformat = COERCE_IMPLICIT_CAST;
coerceExpr->location = -1;
return (Expr *) coerceExpr;
}
else if (coercionType == COERCION_PATH_ARRAYCOERCE)
{
Oid sourceBaseType = get_base_element_type(sourceType);
Oid targetBaseType = get_base_element_type(targetType);
CaseTestExpr *elemExpr = makeNode(CaseTestExpr);
elemExpr->collation = targetCollation;
elemExpr->typeId = sourceBaseType;
elemExpr->typeMod = -1;
Expr *elemCastExpr = CastExpr((Expr *) elemExpr, sourceBaseType,
targetBaseType, targetCollation,
targetTypeMod);
ArrayCoerceExpr *coerceExpr = makeNode(ArrayCoerceExpr);
coerceExpr->arg = copyObject(expr);
coerceExpr->elemexpr = elemCastExpr;
coerceExpr->resultcollid = targetCollation;
coerceExpr->resulttype = targetType;
coerceExpr->resulttypmod = targetTypeMod;
coerceExpr->location = -1;
coerceExpr->coerceformat = COERCE_IMPLICIT_CAST;
return (Expr *) coerceExpr;
}
else if (coercionType == COERCION_PATH_COERCEVIAIO)
{
CoerceViaIO *coerceExpr = makeNode(CoerceViaIO);
coerceExpr->arg = (Expr *) copyObject(expr);
coerceExpr->resulttype = targetType;
coerceExpr->resultcollid = targetCollation;
coerceExpr->coerceformat = COERCE_IMPLICIT_CAST;
coerceExpr->location = -1;
return (Expr *) coerceExpr;
}
else
{
ereport(ERROR, (errmsg("could not find a conversion path from type %d to %d",
sourceType, targetType)));
}
}
/*
* IsSupportedRedistributionTarget determines whether re-partitioning into the
* given target relation is supported.
*/
bool
IsSupportedRedistributionTarget(Oid targetRelationId)
{
DistTableCacheEntry *tableEntry = DistributedTableCacheEntry(targetRelationId);
/* only range and hash-distributed tables are currently supported */
if (tableEntry->partitionMethod != DISTRIBUTE_BY_HASH &&
tableEntry->partitionMethod != DISTRIBUTE_BY_RANGE)
{
return false;
}
return true;
}
/*
* RedistributedInsertSelectTaskList returns a task list to insert given
* redistributedResults into the given target relation.
* redistributedResults[shardIndex] is list of cstrings each of which is
* a result name which should be inserted into
* targetRelation->sortedShardIntervalArray[shardIndex].
*/
static List *
RedistributedInsertSelectTaskList(Query *insertSelectQuery,
DistTableCacheEntry *targetRelation,
List **redistributedResults,
bool useBinaryFormat)
{
List *taskList = NIL;
/*
* Make a copy of the INSERT ... SELECT. We'll repeatedly replace the
* subquery of insertResultQuery for different intermediate results and
* then deparse it.
*/
Query *insertResultQuery = copyObject(insertSelectQuery);
RangeTblEntry *insertRte = ExtractResultRelationRTE(insertResultQuery);
RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(insertResultQuery);
List *selectTargetList = selectRte->subquery->targetList;
Oid targetRelationId = targetRelation->relationId;
int shardCount = targetRelation->shardIntervalArrayLength;
int shardOffset = 0;
uint32 taskIdIndex = 1;
uint64 jobId = INVALID_JOB_ID;
for (shardOffset = 0; shardOffset < shardCount; shardOffset++)
{
ShardInterval *targetShardInterval =
targetRelation->sortedShardIntervalArray[shardOffset];
List *resultIdList = redistributedResults[targetShardInterval->shardIndex];
uint64 shardId = targetShardInterval->shardId;
StringInfo queryString = makeStringInfo();
/* skip empty tasks */
if (resultIdList == NIL)
{
continue;
}
/* sort result ids for consistent test output */
List *sortedResultIds = SortList(resultIdList, pg_qsort_strcmp);
/* generate the query on the intermediate result */
Query *fragmentSetQuery = BuildReadIntermediateResultsArrayQuery(selectTargetList,
NIL,
sortedResultIds,
useBinaryFormat);
/* put the intermediate result query in the INSERT..SELECT */
selectRte->subquery = fragmentSetQuery;
/* setting an alias simplifies deparsing of RETURNING */
if (insertRte->alias == NULL)
{
Alias *alias = makeAlias(CITUS_TABLE_ALIAS, NIL);
insertRte->alias = alias;
}
/*
* Generate a query string for the query that inserts into a shard and reads
* from an intermediate result.
*
* Since CTEs have already been converted to intermediate results, they need
* to removed from the query. Otherwise, worker queries include both
* intermediate results and CTEs in the query.
*/
insertResultQuery->cteList = NIL;
deparse_shard_query(insertResultQuery, targetRelationId, shardId, queryString);
ereport(DEBUG2, (errmsg("distributed statement: %s", queryString->data)));
LockShardDistributionMetadata(shardId, ShareLock);
List *insertShardPlacementList = ActiveShardPlacementList(shardId);
RelationShard *relationShard = CitusMakeNode(RelationShard);
relationShard->relationId = targetShardInterval->relationId;
relationShard->shardId = targetShardInterval->shardId;
Task *modifyTask = CreateBasicTask(jobId, taskIdIndex, MODIFY_TASK,
queryString->data);
modifyTask->dependentTaskList = NIL;
modifyTask->anchorShardId = shardId;
modifyTask->taskPlacementList = insertShardPlacementList;
modifyTask->relationShardList = list_make1(relationShard);
modifyTask->replicationModel = targetRelation->replicationModel;
taskList = lappend(taskList, modifyTask);
taskIdIndex++;
}
return taskList;
}
/*
* PartitionColumnIndex finds the index of given partition column in the
* given target list.
*/
static int
PartitionColumnIndex(List *insertTargetList, Var *partitionColumn)
{
TargetEntry *insertTargetEntry = NULL;
int targetEntryIndex = 0;
foreach_ptr(insertTargetEntry, insertTargetList)
{
if (insertTargetEntry->resno == partitionColumn->varattno)
{
return targetEntryIndex;
}
targetEntryIndex++;
}
return -1;
}
/*
* IsRedistributablePlan returns true if the given plan is a redistrituable plan.
*/
bool
IsRedistributablePlan(Plan *selectPlan)
{
if (!EnableRepartitionedInsertSelect)
{
return false;
}
/* don't redistribute if query is not distributed or requires merge on coordinator */
if (!IsCitusCustomScan(selectPlan))
{
return false;
}
DistributedPlan *distSelectPlan =
GetDistributedPlan((CustomScan *) selectPlan);
Job *distSelectJob = distSelectPlan->workerJob;
List *distSelectTaskList = distSelectJob->taskList;
/*
* Don't use redistribution if only one task. This is to keep the existing
* behaviour for CTEs that the last step is a read_intermediate_result()
* call. It doesn't hurt much in other cases too.
*/
if (list_length(distSelectTaskList) <= 1)
{
return false;
}
/* don't use redistribution for repartition joins for now */
if (distSelectJob->dependentJobList != NIL)
{
return false;
}
return true;
}
/*
* WrapTaskListForProjection wraps task->queryString to only select given
* projected columns. It modifies the taskList.
*/
static void
WrapTaskListForProjection(List *taskList, List *projectedTargetEntries)
{
StringInfo projectedColumnsString = makeStringInfo();
int entryIndex = 0;
TargetEntry *targetEntry = NULL;
foreach_ptr(targetEntry, projectedTargetEntries)
{
if (entryIndex != 0)
{
appendStringInfoChar(projectedColumnsString, ',');
}
char *columnName = targetEntry->resname;
Assert(columnName != NULL);
appendStringInfoString(projectedColumnsString, quote_identifier(columnName));
entryIndex++;
}
Task *task = NULL;
foreach_ptr(task, taskList)
{
Assert(task->queryString != NULL);
StringInfo wrappedQuery = makeStringInfo();
appendStringInfo(wrappedQuery, "SELECT %s FROM (%s) subquery",
projectedColumnsString->data,
task->queryString);
task->queryString = wrappedQuery->data;
}
}
/*
* RelableTargetEntryList relabels select target list to have matching names with
* insert target list.
*/
static void
RelableTargetEntryList(List *selectTargetList, List *insertTargetList)
{
ListCell *selectTargetCell = NULL;
ListCell *insertTargetCell = NULL;
forboth(selectTargetCell, selectTargetList, insertTargetCell, insertTargetList)
{
TargetEntry *selectTargetEntry = lfirst(selectTargetCell);
TargetEntry *insertTargetEntry = lfirst(insertTargetCell);
selectTargetEntry->resname = insertTargetEntry->resname;
}
}

View File

@ -804,7 +804,7 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
char *remoteHost = text_to_cstring(remoteHostText);
int remotePort = PG_GETARG_INT32(2);
int connectionFlags = 0;
int connectionFlags = FORCE_NEW_CONNECTION;
int resultIndex = 0;
int64 totalBytesWritten = 0L;
@ -827,7 +827,7 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
* Intermediate results will be stored in a directory that is derived
* from the distributed transaction ID.
*/
UseCoordinatedTransaction();
EnsureDistributedTransactionId();
MultiConnection *connection = GetNodeConnection(connectionFlags, remoteHost,
remotePort);
@ -838,7 +838,8 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
remoteHost, remotePort)));
}
RemoteTransactionBeginIfNecessary(connection);
StringInfo beginAndSetXactId = BeginAndSetDistributedTransactionIdCommand();
ExecuteCriticalRemoteCommand(connection, beginAndSetXactId->data);
for (resultIndex = 0; resultIndex < resultCount; resultIndex++)
{
@ -847,7 +848,9 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
totalBytesWritten += FetchRemoteIntermediateResult(connection, resultId);
}
UnclaimConnection(connection);
ExecuteCriticalRemoteCommand(connection, "END");
CloseConnection(connection);
PG_RETURN_INT64(totalBytesWritten);
}
@ -894,7 +897,7 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId)
while (true)
{
int waitFlags = WL_SOCKET_READABLE;
int waitFlags = WL_SOCKET_READABLE | WL_POSTMASTER_DEATH;
CopyStatus copyStatus = CopyDataFromConnection(connection, &fileCompat,
&totalBytesWritten);

View File

@ -149,7 +149,7 @@ worker_partition_query_result(PG_FUNCTION_ARGS)
* Intermediate results will be stored in a directory that is derived
* from the distributed transaction ID.
*/
UseCoordinatedTransaction();
EnsureDistributedTransactionId();
CreateIntermediateResultsDirectory();

View File

@ -32,6 +32,7 @@
#include "distributed/multi_logical_planner.h"
#include "distributed/multi_master_planner.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/multi_router_planner.h"
#include "distributed/distributed_planner.h"
#include "distributed/multi_server_executor.h"
#include "distributed/remote_commands.h"
@ -139,9 +140,12 @@ CoordinatorInsertSelectExplainScan(CustomScanState *node, List *ancestors,
DistributedPlan *distributedPlan = scanState->distributedPlan;
Query *insertSelectQuery = distributedPlan->insertSelectQuery;
Query *query = BuildSelectForInsertSelect(insertSelectQuery);
RangeTblEntry *insertRte = ExtractResultRelationRTE(insertSelectQuery);
Oid targetRelationId = insertRte->relid;
IntoClause *into = NULL;
ParamListInfo params = NULL;
char *queryString = NULL;
int cursorOptions = CURSOR_OPT_PARALLEL_OK;
if (es->analyze)
{
@ -150,6 +154,22 @@ CoordinatorInsertSelectExplainScan(CustomScanState *node, List *ancestors,
"... SELECT commands via the coordinator")));
}
/*
* Make a copy of the query, since pg_plan_query may scribble on it and later
* stages of EXPLAIN require it.
*/
Query *queryCopy = copyObject(query);
PlannedStmt *selectPlan = pg_plan_query(queryCopy, cursorOptions, params);
if (IsRedistributablePlan(selectPlan->planTree) &&
IsSupportedRedistributionTarget(targetRelationId))
{
ExplainPropertyText("INSERT/SELECT method", "repartition", es);
}
else
{
ExplainPropertyText("INSERT/SELECT method", "pull to coordinator", es);
}
ExplainOpenGroup("Select Query", "Select Query", false, es);
/* explain the inner SELECT query */

View File

@ -176,6 +176,10 @@ static bool ContainsReferencesToOuterQueryWalker(Node *node,
static void WrapFunctionsInSubqueries(Query *query);
static void TransformFunctionRTE(RangeTblEntry *rangeTblEntry);
static bool ShouldTransformRTE(RangeTblEntry *rangeTableEntry);
static Query * BuildReadIntermediateResultsQuery(List *targetEntryList,
List *columnAliasList,
Const *resultIdConst, Oid functionOid,
bool useBinaryCopyFormat);
/*
* GenerateSubplansForSubqueriesAndCTEs is a wrapper around RecursivelyPlanSubqueriesAndCTEs.
@ -1541,6 +1545,72 @@ ShouldTransformRTE(RangeTblEntry *rangeTableEntry)
*/
Query *
BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resultId)
{
Oid functionOid = CitusReadIntermediateResultFuncId();
bool useBinaryCopyFormat = CanUseBinaryCopyFormatForTargetList(targetEntryList);
Const *resultIdConst = makeNode(Const);
resultIdConst->consttype = TEXTOID;
resultIdConst->consttypmod = -1;
resultIdConst->constlen = -1;
resultIdConst->constvalue = CStringGetTextDatum(resultId);
resultIdConst->constbyval = false;
resultIdConst->constisnull = false;
resultIdConst->location = -1;
return BuildReadIntermediateResultsQuery(targetEntryList, columnAliasList,
resultIdConst, functionOid,
useBinaryCopyFormat);
}
/*
* BuildSubPlanResultQuery returns a query of the form:
*
* SELECT
* <target list>
* FROM
* read_intermediate_results(ARRAY['<resultId>', ...]::text[], '<copy format'>)
* AS res (<column definition list>);
*
* The caller can optionally supply a columnAliasList, which is useful for
* CTEs that have column aliases.
*
* If useBinaryCopyFormat is true, then 'binary' format is used. Otherwise,
* 'text' format is used.
*/
Query *
BuildReadIntermediateResultsArrayQuery(List *targetEntryList,
List *columnAliasList,
List *resultIdList,
bool useBinaryCopyFormat)
{
Oid functionOid = CitusReadIntermediateResultArrayFuncId();
Const *resultIdConst = makeNode(Const);
resultIdConst->consttype = TEXTARRAYOID;
resultIdConst->consttypmod = -1;
resultIdConst->constlen = -1;
resultIdConst->constvalue = PointerGetDatum(strlist_to_textarray(resultIdList));
resultIdConst->constbyval = false;
resultIdConst->constisnull = false;
resultIdConst->location = -1;
return BuildReadIntermediateResultsQuery(targetEntryList, columnAliasList,
resultIdConst, functionOid,
useBinaryCopyFormat);
}
/*
* BuildReadIntermediateResultsQuery is the common code for generating
* queries to read from result files. It is used by
* BuildReadIntermediateResultsArrayQuery and BuildSubPlanResultQuery.
*/
static Query *
BuildReadIntermediateResultsQuery(List *targetEntryList, List *columnAliasList,
Const *resultIdConst, Oid functionOid,
bool useBinaryCopyFormat)
{
List *funcColNames = NIL;
List *funcColTypes = NIL;
@ -1549,7 +1619,6 @@ BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resu
ListCell *targetEntryCell = NULL;
List *targetList = NIL;
int columnNumber = 1;
bool useBinaryCopyFormat = true;
Oid copyFormatId = BinaryCopyFormatId();
int columnAliasCount = list_length(columnAliasList);
@ -1608,23 +1677,9 @@ BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resu
targetList = lappend(targetList, newTargetEntry);
if (useBinaryCopyFormat && !CanUseBinaryCopyFormatForType(columnType))
{
useBinaryCopyFormat = false;
}
columnNumber++;
}
Const *resultIdConst = makeNode(Const);
resultIdConst->consttype = TEXTOID;
resultIdConst->consttypmod = -1;
resultIdConst->constlen = -1;
resultIdConst->constvalue = CStringGetTextDatum(resultId);
resultIdConst->constbyval = false;
resultIdConst->constisnull = false;
resultIdConst->location = -1;
/* build the citus_copy_format parameter for the call to read_intermediate_result */
if (!useBinaryCopyFormat)
{
@ -1642,7 +1697,7 @@ BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resu
/* build the call to read_intermediate_result */
FuncExpr *funcExpr = makeNode(FuncExpr);
funcExpr->funcid = CitusReadIntermediateResultFuncId();
funcExpr->funcid = functionOid;
funcExpr->funcretset = true;
funcExpr->funcvariadic = false;
funcExpr->funcformat = 0;

View File

@ -32,6 +32,7 @@
#include "distributed/connection_management.h"
#include "distributed/cte_inline.h"
#include "distributed/distributed_deadlock_detection.h"
#include "distributed/insert_select_executor.h"
#include "distributed/intermediate_result_pruning.h"
#include "distributed/local_executor.h"
#include "distributed/maintenanced.h"
@ -457,6 +458,16 @@ RegisterCitusConfigVariables(void)
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_repartitioned_insert_select",
gettext_noop("Enables repartitioned INSERT/SELECTs"),
NULL,
&EnableRepartitionedInsertSelect,
true,
PGC_USERSET,
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_fast_path_router_planner",
gettext_noop("Enables fast path router planner"),

View File

@ -63,9 +63,18 @@ partition_task_list_results(PG_FUNCTION_ARGS)
Job *job = distributedPlan->workerJob;
List *taskList = job->taskList;
DistTableCacheEntry *distTableCacheEntry = DistributedTableCacheEntry(relationId);
DistTableCacheEntry *targetRelation = DistributedTableCacheEntry(relationId);
/*
* Here SELECT query's target list should match column list of target relation,
* so their partition column indexes are equal.
*/
int partitionColumnIndex = targetRelation->partitionMethod != DISTRIBUTE_BY_NONE ?
targetRelation->partitionColumn->varattno - 1 : 0;
List *fragmentList = PartitionTasklistResults(resultIdPrefix, taskList,
distTableCacheEntry, binaryFormat);
partitionColumnIndex,
targetRelation, binaryFormat);
TupleDesc tupleDescriptor = NULL;
Tuplestorestate *tupleStore = SetupTuplestore(fcinfo, &tupleDescriptor);
@ -126,7 +135,16 @@ redistribute_task_list_results(PG_FUNCTION_ARGS)
List *taskList = job->taskList;
DistTableCacheEntry *targetRelation = DistributedTableCacheEntry(relationId);
/*
* Here SELECT query's target list should match column list of target relation,
* so their partition column indexes are equal.
*/
int partitionColumnIndex = targetRelation->partitionMethod != DISTRIBUTE_BY_NONE ?
targetRelation->partitionColumn->varattno - 1 : 0;
List **shardResultIds = RedistributeTaskListResults(resultIdPrefix, taskList,
partitionColumnIndex,
targetRelation, binaryFormat);
TupleDesc tupleDescriptor = NULL;

View File

@ -58,7 +58,6 @@ void
StartRemoteTransactionBegin(struct MultiConnection *connection)
{
RemoteTransaction *transaction = &connection->remoteTransaction;
StringInfo beginAndSetDistributedTransactionId = makeStringInfo();
ListCell *subIdCell = NULL;
Assert(transaction->transactionState == REMOTE_TRANS_NOT_STARTED);
@ -68,28 +67,8 @@ StartRemoteTransactionBegin(struct MultiConnection *connection)
transaction->transactionState = REMOTE_TRANS_STARTING;
/*
* Explicitly specify READ COMMITTED, the default on the remote
* side might have been changed, and that would cause problematic
* behaviour.
*/
appendStringInfoString(beginAndSetDistributedTransactionId,
"BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;");
/*
* Append BEGIN and assign_distributed_transaction_id() statements into a single command
* and send both in one step. The reason is purely performance, we don't want
* seperate roundtrips for these two statements.
*/
DistributedTransactionId *distributedTransactionId =
GetCurrentDistributedTransactionId();
const char *timestamp = timestamptz_to_str(distributedTransactionId->timestamp);
appendStringInfo(beginAndSetDistributedTransactionId,
"SELECT assign_distributed_transaction_id(%d, " UINT64_FORMAT
", '%s');",
distributedTransactionId->initiatorNodeIdentifier,
distributedTransactionId->transactionNumber,
timestamp);
StringInfo beginAndSetDistributedTransactionId =
BeginAndSetDistributedTransactionIdCommand();
/* append context for in-progress SAVEPOINTs for this transaction */
List *activeSubXacts = ActiveSubXactContexts();
@ -129,6 +108,42 @@ StartRemoteTransactionBegin(struct MultiConnection *connection)
}
/*
* BeginAndSetDistributedTransactionIdCommand returns a command which starts
* a transaction and assigns the current distributed transaction id.
*/
StringInfo
BeginAndSetDistributedTransactionIdCommand(void)
{
StringInfo beginAndSetDistributedTransactionId = makeStringInfo();
/*
* Explicitly specify READ COMMITTED, the default on the remote
* side might have been changed, and that would cause problematic
* behaviour.
*/
appendStringInfoString(beginAndSetDistributedTransactionId,
"BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;");
/*
* Append BEGIN and assign_distributed_transaction_id() statements into a single command
* and send both in one step. The reason is purely performance, we don't want
* seperate roundtrips for these two statements.
*/
DistributedTransactionId *distributedTransactionId =
GetCurrentDistributedTransactionId();
const char *timestamp = timestamptz_to_str(distributedTransactionId->timestamp);
appendStringInfo(beginAndSetDistributedTransactionId,
"SELECT assign_distributed_transaction_id(%d, " UINT64_FORMAT
", '%s');",
distributedTransactionId->initiatorNodeIdentifier,
distributedTransactionId->transactionNumber,
timestamp);
return beginAndSetDistributedTransactionId;
}
/*
* FinishRemoteTransactionBegin finishes the work StartRemoteTransactionBegin
* initiated. It blocks if necessary (i.e. if PQisBusy() would return true).

View File

@ -117,8 +117,7 @@ static bool MaybeExecutingUDF(void);
void
UseCoordinatedTransaction(void)
{
if (CurrentCoordinatedTransactionState == COORD_TRANS_STARTED ||
CurrentCoordinatedTransactionState == COORD_TRANS_STARTED_ON_WORKER)
if (CurrentCoordinatedTransactionState == COORD_TRANS_STARTED)
{
return;
}
@ -130,21 +129,23 @@ UseCoordinatedTransaction(void)
}
CurrentCoordinatedTransactionState = COORD_TRANS_STARTED;
AssignDistributedTransactionId();
}
/*
* This might be part of bigger distributed transaction originating from
* another node, in which case transaction id has already been assigned
* by a assign_distributed_transaction_id() call.
*/
/*
* EnsureDistributedTransactionId makes sure that the current transaction
* has a distributed transaction id. It is either assigned by a previous
* call of assign_distributed_transaction_id(), or by starting a coordinated
* transaction.
*/
void
EnsureDistributedTransactionId(void)
{
DistributedTransactionId *transactionId = GetCurrentDistributedTransactionId();
if (transactionId->transactionNumber == 0)
{
CurrentCoordinatedTransactionState = COORD_TRANS_STARTED_ON_WORKER;
AssignDistributedTransactionId();
}
else
{
CurrentCoordinatedTransactionState = COORD_TRANS_STARTED;
UseCoordinatedTransaction();
}
}

View File

@ -176,3 +176,29 @@ StringJoin(List *stringList, char delimiter)
return joinedString->data;
}
/*
* ListTake returns the first size elements of given list. If size is greater
* than list's length, it returns all elements of list. This is modeled after
* the "take" function used in some Scheme implementations.
*/
List *
ListTake(List *pointerList, int size)
{
List *result = NIL;
int listIndex = 0;
ListCell *pointerCell = NULL;
foreach(pointerCell, pointerList)
{
result = lappend(result, lfirst(pointerCell));
listIndex++;
if (listIndex >= size)
{
break;
}
}
return result;
}

View File

@ -130,6 +130,7 @@ extern CitusCopyDestReceiver * CreateCitusCopyDestReceiver(Oid relationId,
char *intermediateResultPrefix);
extern FmgrInfo * ColumnOutputFunctions(TupleDesc rowDescriptor, bool binaryFormat);
extern bool CanUseBinaryCopyFormat(TupleDesc tupleDescription);
extern bool CanUseBinaryCopyFormatForTargetList(List *targetEntryList);
extern bool CanUseBinaryCopyFormatForType(Oid typeId);
extern void AppendCopyRowData(Datum *valueArray, bool *isNullArray,
TupleDesc rowDescriptor,

View File

@ -16,10 +16,13 @@
#include "executor/execdesc.h"
extern bool EnableRepartitionedInsertSelect;
extern TupleTableSlot * CoordinatorInsertSelectExecScan(CustomScanState *node);
extern bool ExecutingInsertSelect(void);
extern Query * BuildSelectForInsertSelect(Query *insertSelectQuery);
extern bool IsSupportedRedistributionTarget(Oid targetRelationId);
extern bool IsRedistributablePlan(Plan *selectPlan);
#endif /* INSERT_SELECT_EXECUTOR_H */

View File

@ -62,9 +62,11 @@ extern char * CreateIntermediateResultsDirectory(void);
/* distributed_intermediate_results.c */
extern List ** RedistributeTaskListResults(char *resultIdPrefix,
List *selectTaskList,
int partitionColumnIndex,
DistTableCacheEntry *targetRelation,
bool binaryFormat);
extern List * PartitionTasklistResults(char *resultIdPrefix, List *selectTaskList,
int partitionColumnIndex,
DistTableCacheEntry *distributionScheme,
bool binaryFormat);

View File

@ -73,5 +73,6 @@ extern ArrayType * DatumArrayToArrayType(Datum *datumArray, int datumCount,
Oid datumTypeId);
extern HTAB * ListToHashSet(List *pointerList, Size keySize, bool isStringList);
extern char * StringJoin(List *stringList, char delimiter);
extern List * ListTake(List *pointerList, int size);
#endif /* CITUS_LISTUTILS_H */

View File

@ -28,6 +28,10 @@ extern List * GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *origina
extern char * GenerateResultId(uint64 planId, uint32 subPlanId);
extern Query * BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList,
char *resultId);
extern Query * BuildReadIntermediateResultsArrayQuery(List *targetEntryList,
List *columnAliasList,
List *resultIdList,
bool useBinaryCopyFormat);
extern bool GeneratingSubplans(void);
#endif /* RECURSIVE_PLANNING_H */

View File

@ -35,13 +35,6 @@ typedef enum CoordinatedTransactionState
/* no coordinated transaction in progress, but connections established */
COORD_TRANS_IDLE,
/*
* Coordinated transaction was initiated by coordinator, but the worker also
* needs to start a coordinated transaction to be able to send commands to
* other workers.
*/
COORD_TRANS_STARTED_ON_WORKER,
/* coordinated transaction in progress */
COORD_TRANS_STARTED,
@ -117,6 +110,7 @@ extern void UseCoordinatedTransaction(void);
extern bool InCoordinatedTransaction(void);
extern void CoordinatedTransactionUse2PC(void);
extern bool IsMultiStatementTransaction(void);
extern void EnsureDistributedTransactionId(void);
/* initialization function(s) */
extern void InitializeTransactionManagement(void);
@ -124,6 +118,7 @@ extern void InitializeTransactionManagement(void);
/* other functions */
extern List * ActiveSubXacts(void);
extern List * ActiveSubXactContexts(void);
extern StringInfo BeginAndSetDistributedTransactionIdCommand(void);
#endif /* TRANSACTION_MANAGMENT_H */

View File

@ -88,3 +88,7 @@ s/Subplan [0-9]+\_/Subplan XXX\_/g
# Plan numbers in insert select
s/read_intermediate_result\('insert_select_[0-9]+_/read_intermediate_result('insert_select_XXX_/g
# ignore job id in repartitioned insert/select
s/repartitioned_results_[0-9]+/repartitioned_results_xxxxx/g

View File

@ -825,9 +825,10 @@ INSERT INTO test_table
FROM
fist_table_cte;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: CTE fist_table_cte is going to be inlined via distributed planning
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'key'
-- the following INSERT..SELECT is even more interesting
-- the CTE becomes pushdownable
INSERT INTO test_table

View File

@ -791,7 +791,6 @@ INSERT INTO test_table
FROM
fist_table_cte;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: CTE fist_table_cte is going to be inlined via distributed planning
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Router planner cannot handle multi-shard select queries
@ -800,6 +799,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value FROM (SELECT fist_table_cte.key, fist_table_cte.value FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.other_value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, other_value jsonb)) fist_table_cte) citus_insert_select_subquery
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- the following INSERT..SELECT is even more interesting
-- the CTE becomes pushdownable
INSERT INTO test_table

View File

@ -0,0 +1,200 @@
--
-- failure_insert_select_repartition
--
-- performs failure/cancellation test for repartitioned insert/select.
--
CREATE SCHEMA repartitioned_insert_select;
SET SEARCH_PATH=repartitioned_insert_select;
SELECT pg_backend_pid() as pid \gset
SET citus.next_shard_id TO 4213581;
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 2;
CREATE TABLE replicated_source_table(a int, b int);
SELECT create_distributed_table('replicated_source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO replicated_source_table SELECT i, i*i FROM generate_series(1, 10) i;
SET citus.shard_count TO 3;
CREATE TABLE replicated_target_table(a int, b int);
SELECT create_distributed_table('replicated_target_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table SELECT i, i*i FROM generate_series(1, 10) i;
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int);
SELECT create_distributed_table('target_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
--
-- kill worker_partition_query_result
-- this fails the query on source table, so replicated case should succeed
--
SELECT citus.mitmproxy('conn.onQuery(query="worker_partition_query_result").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
INSERT INTO target_table SELECT * FROM source_table;
ERROR: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
(0 rows)
SELECT citus.mitmproxy('conn.onQuery(query="worker_partition_query_result").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
INSERT INTO target_table SELECT * FROM replicated_source_table;
WARNING: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
WARNING: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
1 | 1
2 | 4
3 | 9
4 | 16
5 | 25
6 | 36
7 | 49
8 | 64
9 | 81
10 | 100
(10 rows)
--
-- kill fetch_intermediate_results
-- this fails the fetch into target, so source replication doesn't matter
-- and both should fail
--
TRUNCATE target_table;
SELECT citus.mitmproxy('conn.onQuery(query="fetch_intermediate_results").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
INSERT INTO target_table SELECT * FROM source_table;
ERROR: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
(0 rows)
SELECT citus.mitmproxy('conn.onQuery(query="fetch_intermediate_results").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
INSERT INTO target_table SELECT * FROM replicated_source_table;
ERROR: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
(0 rows)
--
-- kill read_intermediate_results
-- again, both should fail
--
TRUNCATE target_table;
SELECT citus.mitmproxy('conn.onQuery(query="read_intermediate_results").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
INSERT INTO target_table SELECT * FROM source_table;
ERROR: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
(0 rows)
SELECT citus.mitmproxy('conn.onQuery(query="read_intermediate_results").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
INSERT INTO target_table SELECT * FROM replicated_source_table;
ERROR: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
(0 rows)
--
-- We error out even if table is replicated and only one of the replicas
-- fail.
--
SELECT citus.mitmproxy('conn.onQuery(query="read_intermediate_results").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
INSERT INTO replicated_target_table SELECT * FROM source_table;
ERROR: connection error: localhost:xxxxx
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT * FROM replicated_target_table;
a | b
---------------------------------------------------------------------
(0 rows)
RESET SEARCH_PATH;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
\set VERBOSITY TERSE
DROP SCHEMA repartitioned_insert_select CASCADE;
NOTICE: drop cascades to 4 other objects

View File

@ -0,0 +1,998 @@
-- tests behaviour of INSERT INTO ... SELECT with repartitioning
CREATE SCHEMA insert_select_repartition;
SET search_path TO 'insert_select_repartition';
SET citus.next_shard_id TO 4213581;
SET citus.shard_replication_factor TO 1;
SET citus.replication_model TO 'streaming';
-- 4 shards, hash distributed.
-- Negate distribution column value.
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table SELECT * FROM generate_series(1, 10);
CREATE TABLE target_table(a int);
SELECT create_distributed_table('target_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT -a FROM source_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an operator in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'a'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213585 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213583_to_0,repartitioned_results_xxxxx_from_4213584_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213586 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213582_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213587 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213581_to_2,repartitioned_results_xxxxx_from_4213582_to_2,repartitioned_results_xxxxx_from_4213584_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213588 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213581_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
RESET client_min_messages;
SELECT * FROM target_table WHERE a=-1 OR a=-3 OR a=-7 ORDER BY a;
a
---------------------------------------------------------------------
-7
-3
-1
(3 rows)
DROP TABLE source_table, target_table;
--
-- range partitioning, composite distribution column
--
CREATE TYPE composite_key_type AS (f1 int, f2 text);
-- source
CREATE TABLE source_table(f1 int, key composite_key_type, value int, mapped_key composite_key_type);
SELECT create_distributed_table('source_table', 'key', 'range');
NOTICE: using statement-based replication
DETAIL: Streaming replication is supported only for hash-distributed tables.
create_distributed_table
---------------------------------------------------------------------
(1 row)
CALL public.create_range_partitioned_shards('source_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
INSERT INTO source_table VALUES (0, (0, 'a'), 1, (0, 'a'));
INSERT INTO source_table VALUES (1, (1, 'b'), 2, (26, 'b'));
INSERT INTO source_table VALUES (2, (2, 'c'), 3, (3, 'c'));
INSERT INTO source_table VALUES (3, (4, 'd'), 4, (27, 'd'));
INSERT INTO source_table VALUES (4, (30, 'e'), 5, (30, 'e'));
INSERT INTO source_table VALUES (5, (31, 'f'), 6, (31, 'f'));
INSERT INTO source_table VALUES (6, (32, 'g'), 50, (8, 'g'));
-- target
CREATE TABLE target_table(f1 int DEFAULT 0, value int, key composite_key_type PRIMARY KEY);
SELECT create_distributed_table('target_table', 'key', 'range');
NOTICE: using statement-based replication
DETAIL: Streaming replication is supported only for hash-distributed tables.
create_distributed_table
---------------------------------------------------------------------
(1 row)
CALL public.create_range_partitioned_shards('target_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT f1, value, mapped_key FROM source_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 2 with name 'key'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0,repartitioned_results_xxxxx_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
f1 | value | key
---------------------------------------------------------------------
0 | 1 | (0,a)
2 | 3 | (3,c)
6 | 50 | (8,g)
1 | 2 | (26,b)
3 | 4 | (27,d)
4 | 5 | (30,e)
5 | 6 | (31,f)
(7 rows)
SELECT * FROM target_table WHERE key = (26, 'b')::composite_key_type;
f1 | value | key
---------------------------------------------------------------------
1 | 2 | (26,b)
(1 row)
-- with explicit column names
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table(value, key) SELECT value, mapped_key FROM source_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 2 with name 'key'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0,repartitioned_results_xxxxx_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
f1 | value | key
---------------------------------------------------------------------
0 | 1 | (0,a)
0 | 3 | (3,c)
0 | 50 | (8,g)
0 | 2 | (26,b)
0 | 4 | (27,d)
0 | 5 | (30,e)
0 | 6 | (31,f)
(7 rows)
-- missing value for a column
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table(key) SELECT mapped_key AS key_renamed FROM source_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 1 with name 'key'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0,repartitioned_results_xxxxx_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type)
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
f1 | value | key
---------------------------------------------------------------------
0 | | (0,a)
0 | | (3,c)
0 | | (8,g)
0 | | (26,b)
0 | | (27,d)
0 | | (30,e)
0 | | (31,f)
(7 rows)
-- ON CONFLICT
SET client_min_messages TO DEBUG2;
INSERT INTO target_table(key)
SELECT mapped_key AS key_renamed FROM source_table
WHERE (mapped_key).f1 % 2 = 1
ON CONFLICT (key) DO UPDATE SET f1=1;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 1 with name 'key'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type) ON CONFLICT(key) DO UPDATE SET f1 = 1
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type) ON CONFLICT(key) DO UPDATE SET f1 = 1
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
f1 | value | key
---------------------------------------------------------------------
0 | | (0,a)
1 | | (3,c)
0 | | (8,g)
0 | | (26,b)
1 | | (27,d)
0 | | (30,e)
1 | | (31,f)
(7 rows)
-- missing value for distribution column
INSERT INTO target_table(value) SELECT value FROM source_table;
ERROR: the partition column of table insert_select_repartition.target_table should have a value
DROP TABLE source_table, target_table;
-- different column types
-- verifies that we add necessary casts, otherwise even shard routing won't
-- work correctly and we will see 2 values for the same primary key.
CREATE TABLE target_table(col_1 int primary key, col_2 int);
SELECT create_distributed_table('target_table','col_1');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO target_table VALUES (1,2), (2,3), (3,4), (4,5), (5,6);
CREATE TABLE source_table(col_1 numeric, col_2 numeric, col_3 numeric);
SELECT create_distributed_table('source_table','col_1');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table VALUES (1,1,1), (3,3,3), (5,5,5);
SET client_min_messages TO DEBUG2;
INSERT INTO target_table
SELECT
col_1, col_2
FROM
source_table
ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery.
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'col_1'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213593 AS citus_table_alias (col_1, col_2) SELECT col_1, col_2 FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213597_to_0,repartitioned_results_xxxxx_from_4213600_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer) ON CONFLICT(col_1) DO UPDATE SET col_2 = excluded.col_2
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213594 AS citus_table_alias (col_1, col_2) SELECT col_1, col_2 FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213599_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer) ON CONFLICT(col_1) DO UPDATE SET col_2 = excluded.col_2
RESET client_min_messages;
SELECT * FROM target_table ORDER BY 1;
col_1 | col_2
---------------------------------------------------------------------
1 | 1
2 | 3
3 | 3
4 | 5
5 | 5
(5 rows)
DROP TABLE source_table, target_table;
--
-- array coercion
--
SET citus.shard_count TO 3;
CREATE TABLE source_table(a int, mapped_key int, c float[]);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table VALUES (1, -1, ARRAY[1.1, 2.2, 3.3]), (2, -2, ARRAY[4.5, 5.8]),
(3, -3, ARRAY[]::float[]), (4, -4, ARRAY[3.3]);
SET citus.shard_count TO 2;
CREATE TABLE target_table(a int, b int[]);
SELECT create_distributed_table('target_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET client_min_messages TO DEBUG1;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: performing repartitioned INSERT ... SELECT
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
--
-- worker queries can have more columns than necessary. ExpandWorkerTargetEntry()
-- might add additional columns to the target list.
--
TRUNCATE target_table;
\set VERBOSITY TERSE
-- first verify that the SELECT query below fetches 3 projected columns from workers
SET citus.log_remote_commands TO true; SET client_min_messages TO DEBUG;
CREATE TABLE results AS SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: building index "pg_toast_xxxxx_index" on table "pg_toast_xxxxx" serially
NOTICE: issuing SELECT max((OPERATOR(pg_catalog.-) a)) AS max, array_agg(mapped_key) AS array_agg, a AS worker_column_3 FROM insert_select_repartition.source_table_4213601 source_table WHERE true GROUP BY a
NOTICE: issuing SELECT max((OPERATOR(pg_catalog.-) a)) AS max, array_agg(mapped_key) AS array_agg, a AS worker_column_3 FROM insert_select_repartition.source_table_4213602 source_table WHERE true GROUP BY a
NOTICE: issuing SELECT max((OPERATOR(pg_catalog.-) a)) AS max, array_agg(mapped_key) AS array_agg, a AS worker_column_3 FROM insert_select_repartition.source_table_4213603 source_table WHERE true GROUP BY a
RESET citus.log_remote_commands; RESET client_min_messages;
DROP TABLE results;
-- now verify that we don't write the extra columns to the intermediate result files and
-- insertion to the target works fine.
SET client_min_messages TO DEBUG1;
INSERT INTO target_table SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DEBUG: performing repartitioned INSERT ... SELECT
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {-4}
-3 | {-3}
-2 | {-2}
-1 | {-1}
(4 rows)
--
-- repartitioned INSERT/SELECT followed/preceded by other DML in same transaction
--
-- case 1. followed by DELETE
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
DELETE FROM target_table;
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
(0 rows)
-- case 2. followed by UPDATE
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
UPDATE target_table SET b=array_append(b, a);
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3,-4}
-3 | {-3}
-2 | {4,6,-2}
-1 | {1,2,3,-1}
(4 rows)
-- case 3. followed by multi-row INSERT
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-7 | {999}
-6 | {11,12}
-5 | {10,11}
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(7 rows)
-- case 4. followed by distributed INSERT/SELECT
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
INSERT INTO target_table SELECT * FROM target_table;
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-4 | {3}
-3 | {}
-3 | {}
-2 | {4,6}
-2 | {4,6}
-1 | {1,2,3}
-1 | {1,2,3}
(8 rows)
-- case 5. preceded by DELETE
TRUNCATE target_table;
BEGIN;
DELETE FROM target_table;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
-- case 6. preceded by UPDATE
TRUNCATE target_table;
BEGIN;
UPDATE target_table SET b=array_append(b, a);
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
-- case 7. preceded by multi-row INSERT
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-7 | {999}
-6 | {11,12}
-5 | {10,11}
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(7 rows)
-- case 8. preceded by distributed INSERT/SELECT
TRUNCATE target_table;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
BEGIN;
INSERT INTO target_table SELECT * FROM target_table;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-4 | {3}
-4 | {3}
-3 | {}
-3 | {}
-3 | {}
-2 | {4,6}
-2 | {4,6}
-2 | {4,6}
-1 | {1,2,3}
-1 | {1,2,3}
-1 | {1,2,3}
(12 rows)
--
-- repartitioned INSERT/SELECT with RETURNING
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG1;
WITH c AS (
INSERT INTO target_table
SELECT mapped_key, c FROM source_table
RETURNING *)
SELECT * FROM c ORDER by a;
DEBUG: generating subplan XXX_1 for CTE c: INSERT INTO insert_select_repartition.target_table (a, b) SELECT mapped_key, c FROM insert_select_repartition.source_table RETURNING target_table.a, target_table.b
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer[])) c ORDER BY a
DEBUG: performing repartitioned INSERT ... SELECT
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
RESET client_min_messages;
--
-- in combination with CTEs
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG1;
WITH t AS (
SELECT mapped_key, a, c FROM source_table
WHERE a > floor(random())
)
INSERT INTO target_table
SELECT mapped_key, c FROM t NATURAL JOIN source_table;
DEBUG: volatile functions are not allowed in distributed INSERT ... SELECT queries
DEBUG: generating subplan XXX_1 for CTE t: SELECT mapped_key, a, c FROM insert_select_repartition.source_table WHERE ((a)::double precision OPERATOR(pg_catalog.>) floor(random()))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT mapped_key AS a, (c)::integer[] AS b FROM (SELECT t.mapped_key, t.c FROM ((SELECT intermediate_result.mapped_key, intermediate_result.a, intermediate_result.c FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(mapped_key integer, a integer, c double precision[])) t JOIN insert_select_repartition.source_table USING (mapped_key, a, c))) citus_insert_select_subquery
DEBUG: performing repartitioned INSERT ... SELECT
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
-4 | {3}
-3 | {}
-2 | {4,6}
-1 | {1,2,3}
(4 rows)
DROP TABLE source_table, target_table;
--
-- The case where select query has a GROUP BY ...
--
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int);
SELECT create_distributed_table('target_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table SELECT floor(i/4), i*i FROM generate_series(1, 20) i;
SET client_min_messages TO DEBUG1;
INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
0 | 9
1 | 49
2 | 121
3 | 225
4 | 361
5 | 400
(6 rows)
--
-- EXPLAIN output should specify repartitioned INSERT/SELECT
--
EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
QUERY PLAN
---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT) (cost=0.00..0.00 rows=0 width=0)
INSERT/SELECT method: repartition
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=xxxxx dbname=regression
-> HashAggregate (cost=43.90..45.90 rows=200 width=8)
Group Key: a
-> Seq Scan on source_table_4213606 source_table (cost=0.00..32.60 rows=2260 width=8)
(10 rows)
--
-- Duplicate names in target list
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table
SELECT max(b), max(b) FROM source_table GROUP BY a;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'a'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213610 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_0,repartitioned_results_xxxxx_from_4213607_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213611 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213607_to_1,repartitioned_results_xxxxx_from_4213609_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213612 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_2,repartitioned_results_xxxxx_from_4213607_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
9 | 9
49 | 49
121 | 121
225 | 225
361 | 361
400 | 400
(6 rows)
--
-- Prepared INSERT/SELECT
--
TRUNCATE target_table;
PREPARE insert_plan(int, int) AS
INSERT INTO target_table
SELECT a, max(b) FROM source_table
WHERE a BETWEEN $1 AND $2 GROUP BY a;
SET client_min_messages TO DEBUG1;
EXECUTE insert_plan(0, 2);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(0, 2);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(0, 2);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(0, 2);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(0, 2);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(0, 2);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(2, 4);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(2, 4);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(2, 4);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(2, 4);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(2, 4);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
EXECUTE insert_plan(2, 4);
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: performing repartitioned INSERT ... SELECT
RESET client_min_messages;
SELECT a, count(*), count(distinct b) distinct_values FROM target_table GROUP BY a ORDER BY a;
a | count | distinct_values
---------------------------------------------------------------------
0 | 6 | 1
1 | 6 | 1
2 | 12 | 1
3 | 6 | 1
4 | 6 | 1
(5 rows)
--
-- INSERT/SELECT in CTE
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
WITH r AS (
INSERT INTO target_table SELECT * FROM source_table RETURNING *
)
INSERT INTO target_table SELECT source_table.a, max(source_table.b) FROM source_table NATURAL JOIN r GROUP BY source_table.a;
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: data-modifying statements are not supported in the WITH clauses of distributed queries
DEBUG: generating subplan XXX_1 for CTE r: INSERT INTO insert_select_repartition.target_table (a, b) SELECT a, b FROM insert_select_repartition.source_table RETURNING target_table.a, target_table.b
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, max AS b FROM (SELECT source_table.a, max(source_table.b) AS max FROM (insert_select_repartition.source_table JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) r USING (a, b)) GROUP BY source_table.a) citus_insert_select_subquery
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'a'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213610 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_0,repartitioned_results_xxxxx_from_4213607_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) RETURNING citus_table_alias.a, citus_table_alias.b
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213611 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213607_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) RETURNING citus_table_alias.a, citus_table_alias.b
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213612 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213609_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) RETURNING citus_table_alias.a, citus_table_alias.b
DEBUG: partitioning SELECT query by column index 0 with name 'a'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213610 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_0,repartitioned_results_xxxxx_from_4213607_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213611 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213607_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213612 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213609_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a, b;
a | b
---------------------------------------------------------------------
0 | 1
0 | 4
0 | 9
0 | 9
1 | 16
1 | 25
1 | 36
1 | 49
1 | 49
2 | 64
2 | 81
2 | 100
2 | 121
2 | 121
3 | 144
3 | 169
3 | 196
3 | 225
3 | 225
4 | 256
4 | 289
4 | 324
4 | 361
4 | 361
5 | 400
5 | 400
(26 rows)
DROP TABLE source_table, target_table;
--
-- Constraint failure and rollback
--
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
UPDATE source_table SET b = NULL where b IN (9, 4);
SET citus.shard_replication_factor TO 2;
CREATE TABLE target_table(a int, b int not null);
SELECT create_distributed_table('target_table', 'a', 'range');
NOTICE: using statement-based replication
create_distributed_table
---------------------------------------------------------------------
(1 row)
CALL public.create_range_partitioned_shards('target_table', '{0,3,6,9}','{2,5,8,50}');
INSERT INTO target_table VALUES (11,9), (22,4);
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table;
QUERY PLAN
---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: repartition
-> Custom Scan (Citus Adaptive)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=xxxxx dbname=regression
-> Seq Scan on source_table_4213613 source_table
(8 rows)
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
QUERY PLAN
---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: repartition
-> Custom Scan (Citus Adaptive)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=xxxxx dbname=regression
-> Seq Scan on source_table_4213613 source_table
Filter: (b IS NOT NULL)
(9 rows)
BEGIN;
SAVEPOINT s1;
INSERT INTO target_table SELECT * FROM source_table;
ERROR: null value in column "b" violates not-null constraint
ROLLBACK TO SAVEPOINT s1;
INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
END;
SELECT * FROM target_table ORDER BY b;
a | b
---------------------------------------------------------------------
1 | 1
22 | 4
11 | 9
4 | 16
5 | 25
6 | 36
7 | 49
8 | 64
9 | 81
10 | 100
(10 rows)
-- verify that values have been replicated to both replicas
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
nodename | nodeport | shardid | success | result
---------------------------------------------------------------------
localhost | 57637 | 4213617 | t | 1
localhost | 57638 | 4213617 | t | 1
localhost | 57637 | 4213618 | t | 2
localhost | 57638 | 4213618 | t | 2
localhost | 57637 | 4213619 | t | 3
localhost | 57638 | 4213619 | t | 3
localhost | 57637 | 4213620 | t | 4
localhost | 57638 | 4213620 | t | 4
(8 rows)
--
-- Multiple casts in the SELECT query
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT 1.12, b::bigint FROM source_table WHERE b IS NOT NULL;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'a'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213617 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213613_to_0,repartitioned_results_xxxxx_from_4213614_to_0,repartitioned_results_xxxxx_from_4213615_to_0,repartitioned_results_xxxxx_from_4213616_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a, b;
a | b
---------------------------------------------------------------------
1 | 1
1 | 16
1 | 25
1 | 36
1 | 49
1 | 64
1 | 81
1 | 100
(8 rows)
--
-- ROLLBACK after out of range error
--
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT a * 10, b FROM source_table WHERE b IS NOT NULL;
ERROR: could not find shard for partition column value
END;
SELECT max(result) FROM run_command_on_placements('target_table', 'select count(*) from %s');
max
---------------------------------------------------------------------
0
(1 row)
DROP TABLE source_table, target_table;
--
-- Range partitioned target's ranges doesn't cover the whole range
--
SET citus.shard_replication_factor TO 2;
SET citus.replication_model TO 'statement';
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
SET citus.shard_replication_factor TO 2;
CREATE TABLE target_table(b int not null, a float);
SELECT create_distributed_table('target_table', 'a', 'range');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CALL public.create_range_partitioned_shards('target_table', '{0.0,3.5,6.5,9.5}','{2.9,5.9,8.9,50.0}');
INSERT INTO target_table SELECT b, a+0.6 FROM source_table;
SELECT * FROM target_table ORDER BY a;
b | a
---------------------------------------------------------------------
1 | 1.6
4 | 2.6
9 | 3.6
16 | 4.6
25 | 5.6
36 | 6.6
49 | 7.6
64 | 8.6
81 | 9.6
100 | 10.6
(10 rows)
-- verify that values have been replicated to both replicas, and that each
-- replica has received correct number of rows
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
nodename | nodeport | shardid | success | result
---------------------------------------------------------------------
localhost | 57637 | 4213625 | t | 2
localhost | 57638 | 4213625 | t | 2
localhost | 57637 | 4213626 | t | 3
localhost | 57638 | 4213626 | t | 3
localhost | 57637 | 4213627 | t | 3
localhost | 57638 | 4213627 | t | 3
localhost | 57637 | 4213628 | t | 2
localhost | 57638 | 4213628 | t | 2
(8 rows)
DROP TABLE source_table, target_table;
--
-- Select column names should be unique
--
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int, c int, d int, e int, f int);
SELECT create_distributed_table('target_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'a'
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213633 AS citus_table_alias (a, b, c, d) SELECT a, b, c, d FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213629_to_0,repartitioned_results_xxxxx_from_4213630_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer, c integer, d integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213634 AS citus_table_alias (a, b, c, d) SELECT a, b, c, d FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213630_to_1,repartitioned_results_xxxxx_from_4213631_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer, c integer, d integer)
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213635 AS citus_table_alias (a, b, c, d) SELECT a, b, c, d FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213632_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer, c integer, d integer)
RESET client_min_messages;
SELECT count(*) FROM target_table;
count
---------------------------------------------------------------------
10
(1 row)
--
-- Disable repartitioned insert/select
--
TRUNCATE target_table;
SET citus.enable_repartitioned_insert_select TO OFF;
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
QUERY PLAN
---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator
-> Custom Scan (Citus Adaptive)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=xxxxx dbname=regression
-> Seq Scan on source_table_4213629 source_table
(8 rows)
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
RESET client_min_messages;
SELECT count(*) FROM target_table;
count
---------------------------------------------------------------------
10
(1 row)
SET citus.enable_repartitioned_insert_select TO ON;
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
QUERY PLAN
---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: repartition
-> Custom Scan (Citus Adaptive)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=xxxxx dbname=regression
-> Seq Scan on source_table_4213629 source_table
(8 rows)
DROP TABLE source_table, target_table;
--
-- Don't use INSERT/SELECT repartition with repartition joins
--
create table test(x int, y int);
select create_distributed_table('test', 'x');
create_distributed_table
---------------------------------------------------------------------
(1 row)
set citus.enable_repartition_joins to true;
SET client_min_messages TO DEBUG1;
insert into test(y, x) select a.x, b.y from test a JOIN test b USING (y);
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DEBUG: Collecting INSERT ... SELECT results on coordinator
RESET client_min_messages;
SET client_min_messages TO WARNING;
DROP SCHEMA insert_select_repartition CASCADE;

View File

@ -757,9 +757,9 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO table_1
SELECT * FROM table_2 where value IN (SELECT value FROM table_1 WHERE random() > 1) AND key = 1;
DEBUG: volatile functions are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for subquery SELECT value FROM intermediate_result_pruning.table_1 WHERE (random() OPERATOR(pg_catalog.>) (1)::double precision)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value FROM intermediate_result_pruning.table_2 WHERE ((value OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text))) AND (key OPERATOR(pg_catalog.=) 1))
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx
-- a similar query, with more complex subquery
INSERT INTO table_1
@ -781,7 +781,6 @@ INSERT INTO table_1
UNION
SELECT * FROM cte_2);
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 1) INTERSECT SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 2)
DEBUG: generating subplan XXX_1 for subquery SELECT key FROM intermediate_result_pruning.table_1 WHERE (key OPERATOR(pg_catalog.=) 1)
DEBUG: generating subplan XXX_2 for subquery SELECT key FROM intermediate_result_pruning.table_1 WHERE (key OPERATOR(pg_catalog.=) 2)
@ -789,6 +788,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_
DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 3) INTERSECT SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 4)
DEBUG: generating subplan XXX_3 for subquery SELECT cte_1.key FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) cte_1 UNION SELECT cte_2.key FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) cte_2
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, value FROM intermediate_result_pruning.table_2 WHERE ((key OPERATOR(pg_catalog.=) 1) AND ((value)::integer OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.key FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer))))
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx
DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx
DEBUG: Subplan XXX_2 will be sent to localhost:xxxxx
@ -818,7 +818,6 @@ INSERT INTO table_1
where table_2.key != 1 AND
foo.key = table_2.value::int;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 1) INTERSECT SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 2)
DEBUG: generating subplan XXX_1 for subquery SELECT key FROM intermediate_result_pruning.table_1 WHERE (key OPERATOR(pg_catalog.=) 1)
DEBUG: generating subplan XXX_2 for subquery SELECT key FROM intermediate_result_pruning.table_1 WHERE (key OPERATOR(pg_catalog.=) 2)
@ -826,6 +825,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_
DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 3) INTERSECT SELECT table_1.key FROM intermediate_result_pruning.table_1 WHERE (table_1.key OPERATOR(pg_catalog.=) 4)
DEBUG: generating subplan XXX_3 for subquery SELECT cte_1.key FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) cte_1 UNION SELECT cte_2.key FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) cte_2
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table_2.key, table_2.value FROM intermediate_result_pruning.table_2, (SELECT intermediate_result.key FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) foo WHERE ((table_2.key OPERATOR(pg_catalog.<>) 1) AND (foo.key OPERATOR(pg_catalog.=) (table_2.value)::integer))
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx
DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx
DEBUG: Subplan XXX_2 will be sent to localhost:xxxxx

View File

@ -0,0 +1,249 @@
Parsed test spec with 2 sessions
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-delete-from-source s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s2-delete-from-source: DELETE FROM source_table;
step s1-end: END;
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-update-source s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s2-update-source: UPDATE source_table SET b = 50 - b;
step s1-end: END;
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-insert-into-source s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s2-insert-into-source: INSERT INTO source_table VALUES (0, 0);
step s1-end: END;
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s2-delete-from-source s1-repartitioned-insert-select s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s2-delete-from-source: DELETE FROM source_table;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s1-end: END;
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s2-update-source s1-repartitioned-insert-select s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s2-update-source: UPDATE source_table SET b = 50 - b;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s1-end: END;
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s2-insert-into-source s1-repartitioned-insert-select s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s2-insert-into-source: INSERT INTO source_table VALUES (0, 0);
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s1-end: END;
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-delete-from-target s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s2-delete-from-target: DELETE FROM target_table; <waiting ...>
step s1-end: END;
step s2-delete-from-target: <... completed>
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-update-target s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s2-update-target: UPDATE target_table SET b = 50 - b; <waiting ...>
step s1-end: END;
step s2-update-target: <... completed>
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 49
2 46
3 41
4 34
5 25
6 14
7 1
8 -14
9 -31
10 -50
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-insert-into-target s1-end s2-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s2-insert-into-target: INSERT INTO target_table VALUES (0, 0);
step s1-end: END;
step s2-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
0 0
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s2-delete-from-target s1-repartitioned-insert-select s2-end s1-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s2-delete-from-target: DELETE FROM target_table;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table; <waiting ...>
step s2-end: END;
step s1-repartitioned-insert-select: <... completed>
step s1-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s2-update-target s1-repartitioned-insert-select s2-end s1-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s2-update-target: UPDATE target_table SET b = 50 - b;
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table; <waiting ...>
step s2-end: END;
step s1-repartitioned-insert-select: <... completed>
step s1-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100
starting permutation: s1-begin s2-begin s2-insert-into-target s1-repartitioned-insert-select s2-end s1-end s1-select-target
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s2-insert-into-target: INSERT INTO target_table VALUES (0, 0);
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
step s2-end: END;
step s1-end: END;
step s1-select-target: SELECT * FROM target_table ORDER BY a;
a b
0 0
1 1
2 4
3 9
4 16
5 25
6 36
7 49
8 64
9 81
10 100

View File

@ -1188,7 +1188,8 @@ ROLLBACK;
EXPLAIN (COSTS OFF)
INSERT INTO lineitem_hash_part
SELECT o_orderkey FROM orders_hash_part LIMIT 3;
Custom Scan (Citus INSERT ... SELECT via coordinator)
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator
-> Limit
-> Custom Scan (Citus Adaptive)
Task Count: 4
@ -1205,7 +1206,8 @@ t
EXPLAIN (COSTS OFF)
INSERT INTO lineitem_hash_part (l_orderkey, l_quantity)
SELECT o_orderkey, 5 FROM orders_hash_part LIMIT 3;
Custom Scan (Citus INSERT ... SELECT via coordinator)
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator
-> Limit
-> Custom Scan (Citus Adaptive)
Task Count: 4
@ -1217,7 +1219,8 @@ Custom Scan (Citus INSERT ... SELECT via coordinator)
EXPLAIN (COSTS OFF)
INSERT INTO lineitem_hash_part (l_orderkey)
SELECT s FROM generate_series(1,5) s;
Custom Scan (Citus INSERT ... SELECT via coordinator)
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator
-> Function Scan on generate_series s
-- WHERE EXISTS forces pg12 to materialize cte
EXPLAIN (COSTS OFF)
@ -1225,7 +1228,8 @@ WITH cte1 AS (SELECT s FROM generate_series(1,10) s)
INSERT INTO lineitem_hash_part
WITH cte1 AS (SELECT * FROM cte1 WHERE EXISTS (SELECT * FROM cte1) LIMIT 5)
SELECT s FROM cte1 WHERE EXISTS (SELECT * FROM cte1);
Custom Scan (Citus INSERT ... SELECT via coordinator)
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator
-> Result
One-Time Filter: $3
CTE cte1
@ -1244,7 +1248,8 @@ EXPLAIN (COSTS OFF)
INSERT INTO lineitem_hash_part
( SELECT s FROM generate_series(1,5) s) UNION
( SELECT s FROM generate_series(5,10) s);
Custom Scan (Citus INSERT ... SELECT via coordinator)
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator
-> HashAggregate
Group Key: s.s
-> Append

View File

@ -631,8 +631,8 @@ INSERT INTO agg_events (value_1_agg, user_id)
FROM
raw_events_first;
DEBUG: DISTINCT ON (non-partition column) clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
SELECT user_id, value_1_agg FROM agg_events ORDER BY 1,2;
DEBUG: Router planner cannot handle multi-shard select queries
user_id | value_1_agg
@ -693,13 +693,13 @@ INSERT INTO agg_events
FROM
fist_table_agg;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: generating subplan XXX_1 for CTE fist_table_agg: SELECT (max(value_1) OPERATOR(pg_catalog.+) 1) AS v1_agg, user_id FROM public.raw_events_first GROUP BY user_id
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, v1_agg FROM (SELECT fist_table_agg.v1_agg, fist_table_agg.user_id FROM (SELECT intermediate_result.v1_agg, intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v1_agg integer, user_id integer)) fist_table_agg) citus_insert_select_subquery
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, v1_agg AS value_1_agg FROM (SELECT fist_table_agg.v1_agg, fist_table_agg.user_id FROM (SELECT intermediate_result.v1_agg, intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v1_agg integer, user_id integer)) fist_table_agg) citus_insert_select_subquery
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
ROLLBACK;
-- We don't support CTEs that are referenced in the target list
INSERT INTO agg_events
@ -710,7 +710,6 @@ INSERT INTO agg_events
raw_events_first;
DEBUG: CTE sub_cte is going to be inlined via distributed planning
DEBUG: Subqueries without relations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
ERROR: could not run distributed query with subquery outside the FROM, WHERE and HAVING clauses
HINT: Consider using an equality filter on the distributed table's partition column.
@ -724,8 +723,13 @@ FROM
((SELECT user_id FROM raw_events_first) UNION
(SELECT user_id FROM raw_events_second)) as foo;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300004_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300005_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300006_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300007_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
ROLLBACK;
-- We do support set operations through recursive planning
BEGIN;
@ -735,7 +739,6 @@ INSERT INTO
(SELECT user_id FROM raw_events_first) INTERSECT
(SELECT user_id FROM raw_events_first);
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.raw_events_first
@ -747,6 +750,7 @@ DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.user_id
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) citus_insert_select_subquery
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
ROLLBACK;
-- If the query is router plannable then it is executed via the coordinator
INSERT INTO
@ -757,9 +761,9 @@ FROM
((SELECT user_id FROM raw_events_first WHERE user_id = 15) EXCEPT
(SELECT user_id FROM raw_events_second where user_id = 17)) as foo;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- some supported LEFT joins
INSERT INTO agg_events (user_id)
SELECT
@ -1064,7 +1068,6 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
WHERE raw_events_first.user_id = raw_events_second.user_id
GROUP BY raw_events_second.value_3) AS foo;
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
@ -1080,9 +1083,10 @@ DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-10
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.value_3 AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_3
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id, v1, v4 FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 bigint, id double precision)) foo
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT int4(id) AS user_id, int4(v1) AS value_1_agg, int8(v4) AS value_4_agg FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 bigint, id double precision)) foo
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
ERROR: the partition column of table public.agg_events cannot be NULL
-- error cases
-- no part column at all
@ -1092,8 +1096,8 @@ SELECT value_1
FROM raw_events_first;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: the query doesn't include the target table's partition column
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
ERROR: the partition column of table public.raw_events_second should have a value
INSERT INTO raw_events_second
(value_1)
@ -1101,8 +1105,8 @@ SELECT user_id
FROM raw_events_first;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: the query doesn't include the target table's partition column
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
ERROR: the partition column of table public.raw_events_second should have a value
INSERT INTO raw_events_second
(user_id)
@ -1110,9 +1114,11 @@ SELECT value_1
FROM raw_events_first;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
ERROR: the partition column of table public.raw_events_second cannot be NULL
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
ERROR: the partition column value cannot be NULL
CONTEXT: while executing command on localhost:xxxxx
INSERT INTO raw_events_second
(user_id)
SELECT user_id * 2
@ -1120,8 +1126,13 @@ FROM raw_events_first;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an operator in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_0,repartitioned_results_xxxxx_from_13300001_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_1,repartitioned_results_xxxxx_from_13300001_to_1,repartitioned_results_xxxxx_from_13300003_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300001_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_3,repartitioned_results_xxxxx_from_13300002_to_3,repartitioned_results_xxxxx_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
INSERT INTO raw_events_second
(user_id)
SELECT user_id :: bigint
@ -1129,8 +1140,13 @@ FROM raw_events_first;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an explicit cast in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300001_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300002_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
INSERT INTO agg_events
(value_3_agg,
value_4_agg,
@ -1147,9 +1163,11 @@ GROUP BY user_id;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an aggregation in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
ERROR: the partition column of table public.agg_events cannot be NULL
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
ERROR: the partition column value cannot be NULL
CONTEXT: while executing command on localhost:xxxxx
INSERT INTO agg_events
(value_3_agg,
value_4_agg,
@ -1166,9 +1184,11 @@ GROUP BY user_id,
value_2;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
ERROR: the partition column of table public.agg_events cannot be NULL
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
ERROR: the partition column value cannot be NULL
CONTEXT: while executing command on localhost:xxxxx
-- tables should be co-located
INSERT INTO agg_events (user_id)
SELECT
@ -1177,10 +1197,10 @@ FROM
reference_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- foo2 is recursively planned and INSERT...SELECT is done via coordinator
INSERT INTO agg_events
(user_id)
@ -1205,7 +1225,6 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2
ON (f.id = f2.id);
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
@ -1221,8 +1240,10 @@ DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-10
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT int4(f2.id) AS user_id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
-- the second part of the query is not routable since
-- GROUP BY not on the partition column (i.e., value_1) and thus join
-- on f.id = f2.id is not on the partition key (instead on the sum of partition key)
@ -1250,7 +1271,6 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2
ON (f.id = f2.id);
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
@ -1266,8 +1286,10 @@ DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-10
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.id FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.id AS user_id FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
-- cannot pushdown the query since the JOIN is not equi JOIN
INSERT INTO agg_events
(user_id, value_4_agg)
@ -1676,7 +1698,6 @@ SELECT user_id,
FROM raw_events_second
GROUP BY grouping sets ( ( user_id ), ( value_1 ), ( user_id, value_1 ), ( ) );
DEBUG: grouping sets are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP
HINT: Consider using an equality filter on the distributed table's partition column.
@ -2091,61 +2112,71 @@ FROM text_table ;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains a case expression in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT COALESCE(part_col, 'onder') FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains a coalesce expression in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT GREATEST(part_col, 'jason') FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains a min/max expression in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT LEAST(part_col, 'andres') FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains a min/max expression in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT NULLIF(part_col, 'metin') FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT part_col isnull FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT part_col::text from char_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an explicit coercion in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT (part_col = 'burak') is true FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT val FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
INSERT INTO text_table (part_col) SELECT val::text FROM text_table;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an explicit coercion in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: performing repartitioned INSERT ... SELECT
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
RESET client_min_messages;
insert into table_with_starts_with_defaults (b,c) select b,c FROM table_with_starts_with_defaults;
-- Test on partition column without native hash function

View File

@ -54,7 +54,7 @@ FROM
ON CONFLICT DO NOTHING;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: performing repartitioned INSERT ... SELECT
-- Since partition columns do not match, pull the data to the coordinator
-- and update the non-partition column. Query is wrapped by CTE to return
-- ordered result.
@ -70,7 +70,7 @@ DEBUG: generating subplan XXX_1 for CTE inserted_table: INSERT INTO on_conflict
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: performing repartitioned INSERT ... SELECT
col_1 | col_2
---------------------------------------------------------------------
1 | 1
@ -93,10 +93,10 @@ FROM (
) as foo
ON CONFLICT DO NOTHING;
DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Subquery should be recursively planned due to the limit and update on conflict
-- Query is wrapped by CTE to return ordered result.
WITH inserted_table AS (
@ -115,10 +115,10 @@ WITH inserted_table AS (
DEBUG: generating subplan XXX_1 for CTE inserted_table: INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM (SELECT source_table_1.col_1, source_table_1.col_2, source_table_1.col_3 FROM on_conflict.source_table_1 LIMIT 5) foo ON CONFLICT(col_1) DO UPDATE SET col_2 = excluded.col_2 RETURNING target_table.col_1, target_table.col_2
DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
DEBUG: Collecting INSERT ... SELECT results on coordinator
col_1 | col_2
---------------------------------------------------------------------
1 | 1
@ -151,13 +151,13 @@ WITH inserted_table AS (
DEBUG: generating subplan XXX_1 for CTE inserted_table: INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM ((SELECT source_table_1.col_1, source_table_1.col_2, source_table_1.col_3 FROM on_conflict.source_table_1 LIMIT 5) UNION (SELECT source_table_2.col_1, source_table_2.col_2, source_table_2.col_3 FROM on_conflict.source_table_2 LIMIT 5)) foo ON CONFLICT(col_1) DO UPDATE SET col_2 = 0 RETURNING target_table.col_1, target_table.col_2
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_2 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_2 LIMIT 5
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer) UNION SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
DEBUG: Collecting INSERT ... SELECT results on coordinator
col_1 | col_2
---------------------------------------------------------------------
1 | 0
@ -180,18 +180,18 @@ WITH cte AS(
)
INSERT INTO target_table SELECT * FROM cte ON CONFLICT DO NOTHING;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2 FROM on_conflict.source_table_1
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte.col_1, cte.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte) citus_insert_select_subquery
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Get the select part from cte and update on conflict
WITH cte AS(
SELECT col_1, col_2 FROM source_table_1
)
INSERT INTO target_table SELECT * FROM cte ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 + 1;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2 FROM on_conflict.source_table_1
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte.col_1, cte.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte) citus_insert_select_subquery
DEBUG: Collecting INSERT ... SELECT results on coordinator
SELECT * FROM target_table ORDER BY 1;
col_1 | col_2
---------------------------------------------------------------------
@ -216,9 +216,9 @@ WITH cte AS(
)
INSERT INTO target_table ((SELECT * FROM cte) UNION (SELECT * FROM cte_2)) ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 + 1;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: CTE cte is going to be inlined via distributed planning
DEBUG: CTE cte_2 is going to be inlined via distributed planning
DEBUG: performing repartitioned INSERT ... SELECT
SELECT * FROM target_table ORDER BY 1;
col_1 | col_2
---------------------------------------------------------------------
@ -247,10 +247,10 @@ WITH inserted_table AS (
DEBUG: generating subplan XXX_1 for CTE inserted_table: WITH cte AS (SELECT source_table_1.col_1, source_table_1.col_2, source_table_1.col_3 FROM on_conflict.source_table_1), cte_2 AS (SELECT cte.col_1, cte.col_2 FROM cte) INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM cte_2 ON CONFLICT(col_1) DO UPDATE SET col_2 = (excluded.col_2 OPERATOR(pg_catalog.+) 1) RETURNING target_table.col_1, target_table.col_2
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1
DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) cte
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte_2.col_1, cte_2.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte_2) citus_insert_select_subquery
DEBUG: Collecting INSERT ... SELECT results on coordinator
col_1 | col_2
---------------------------------------------------------------------
1 | 2
@ -270,9 +270,9 @@ UPDATE target_table SET col_2 = 4 WHERE col_1 IN (SELECT col_1 FROM cte);
DEBUG: generating subplan XXX_1 for CTE cte: WITH basic AS (SELECT source_table_1.col_1, source_table_1.col_2 FROM on_conflict.source_table_1) INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM basic ON CONFLICT DO NOTHING RETURNING target_table.col_1, target_table.col_2
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE on_conflict.target_table SET col_2 = 4 WHERE (col_1 OPERATOR(pg_catalog.=) ANY (SELECT cte.col_1 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte))
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE basic: SELECT col_1, col_2 FROM on_conflict.source_table_1
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT basic.col_1, basic.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) basic) citus_insert_select_subquery
DEBUG: Collecting INSERT ... SELECT results on coordinator
RESET client_min_messages;
-- Following query is not supported since error checks of the subquery pushdown planner
-- and insert select planner have not been unified. It should work after unifying them.
@ -316,12 +316,14 @@ CONTEXT: while executing command on localhost:xxxxx
ROLLBACK;
BEGIN;
DELETE FROM test_ref_table WHERE key > 10;
INSERT INTO
target_table
SELECT
col_2,
col_1
FROM source_table_1 ON CONFLICT (col_1) DO UPDATE SET col_2 = 1 RETURNING *;
WITH r AS (
INSERT INTO
target_table
SELECT
col_2,
col_1
FROM source_table_1 ON CONFLICT (col_1) DO UPDATE SET col_2 = 1 RETURNING *)
SELECT * FROM r ORDER BY col_1;
col_1 | col_2
---------------------------------------------------------------------
1 | 1
@ -395,7 +397,7 @@ FROM
ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: performing repartitioned INSERT ... SELECT
SELECT * FROM target_table ORDER BY 1;
col_1 | col_2
---------------------------------------------------------------------
@ -481,13 +483,13 @@ FROM (
) as foo
ON CONFLICT(col_1) DO UPDATE SET col_2 = 0;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
DEBUG: push down of limit count: 5
DEBUG: generating subplan XXX_2 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_2 LIMIT 5
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer) UNION SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
DEBUG: Collecting INSERT ... SELECT results on coordinator
SELECT * FROM target_table ORDER BY 1;
col_1 | col_2
---------------------------------------------------------------------
@ -512,10 +514,10 @@ WITH cte AS(
)
INSERT INTO target_table SELECT * FROM cte_2 ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 + 1;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1
DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) cte
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte_2.col_1, cte_2.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte_2) citus_insert_select_subquery
DEBUG: Collecting INSERT ... SELECT results on coordinator
SELECT * FROM target_table ORDER BY 1;
col_1 | col_2
---------------------------------------------------------------------

View File

@ -159,7 +159,6 @@ FROM (
GROUP BY t1.user_id, hasdone_event
) t GROUP BY user_id, hasdone_event;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for subquery SELECT u.user_id, 'step=>1'::text AS event, e."time" FROM public.users_table u, public.events_table e WHERE ((u.user_id OPERATOR(pg_catalog.=) e.user_id) AND (u.user_id OPERATOR(pg_catalog.>=) 10) AND (u.user_id OPERATOR(pg_catalog.<=) 25) AND (e.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[100, 101, 102])))
DEBUG: generating subplan XXX_2 for subquery SELECT u.user_id, 'step=>2'::text AS event, e."time" FROM public.users_table u, public.events_table e WHERE ((u.user_id OPERATOR(pg_catalog.=) e.user_id) AND (u.user_id OPERATOR(pg_catalog.>=) 10) AND (u.user_id OPERATOR(pg_catalog.<=) 25) AND (e.event_type OPERATOR(pg_catalog.=) ANY (ARRAY[103, 104, 105])))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id, intermediate_result.event, intermediate_result."time" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event text, "time" timestamp without time zone) UNION SELECT intermediate_result.user_id, intermediate_result.event, intermediate_result."time" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event text, "time" timestamp without time zone)
@ -310,7 +309,6 @@ GROUP BY
ORDER BY
count_pay;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for subquery SELECT users_table.user_id, 'action=>1'::text AS event, events_table."time" FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (users_table.user_id OPERATOR(pg_catalog.>=) 10) AND (users_table.user_id OPERATOR(pg_catalog.<=) 70) AND (events_table.event_type OPERATOR(pg_catalog.>) 10) AND (events_table.event_type OPERATOR(pg_catalog.<) 12))
DEBUG: generating subplan XXX_2 for subquery SELECT users_table.user_id, 'action=>2'::text AS event, events_table."time" FROM public.users_table, public.events_table WHERE ((users_table.user_id OPERATOR(pg_catalog.=) events_table.user_id) AND (users_table.user_id OPERATOR(pg_catalog.>=) 10) AND (users_table.user_id OPERATOR(pg_catalog.<=) 70) AND (events_table.event_type OPERATOR(pg_catalog.>) 12) AND (events_table.event_type OPERATOR(pg_catalog.<) 14))
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.user_id, intermediate_result.event, intermediate_result."time" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event text, "time" timestamp without time zone) UNION SELECT intermediate_result.user_id, intermediate_result.event, intermediate_result."time" FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event text, "time" timestamp without time zone)

View File

@ -438,8 +438,8 @@ SELECT create_distributed_function('delegated_function(int)', 'a');
SET client_min_messages TO DEBUG1;
INSERT INTO test SELECT delegated_function(1);
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: not pushing down function calls in INSERT ... SELECT
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Don't push down in subqueries or CTEs.
SELECT * FROM test WHERE not exists(
SELECT delegated_function(4)

View File

@ -0,0 +1,96 @@
-- Test behaviour of repartitioned INSERT ... SELECT in MX setup
CREATE SCHEMA multi_mx_insert_select_repartition;
SET search_path TO multi_mx_insert_select_repartition;
SET citus.next_shard_id TO 4213581;
SET citus.replication_model TO 'streaming';
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO source_table SELECT floor(i/4), i*i FROM generate_series(1, 20) i;
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int);
SELECT create_distributed_table('target_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE FUNCTION square(int) RETURNS INT
AS $$ SELECT $1 * $1 $$
LANGUAGE SQL;
select create_distributed_function('square(int)');
create_distributed_function
---------------------------------------------------------------------
(1 row)
select public.colocate_proc_with_table('square', 'source_table'::regclass, 0);
colocate_proc_with_table
---------------------------------------------------------------------
(1 row)
-- Test along with function delegation
-- function delegation only happens for "SELECT f()", and we don't use
-- repartitioned INSERT/SELECT when task count is 1, so the following
-- should go via coordinator
EXPLAIN (costs off) INSERT INTO target_table(a) SELECT square(4);
QUERY PLAN
---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator
-> Result
(3 rows)
INSERT INTO target_table(a) SELECT square(4);
SELECT * FROM target_table;
a | b
---------------------------------------------------------------------
16 |
(1 row)
TRUNCATE target_table;
--
-- Test repartitioned INSERT/SELECT from MX worker
--
\c - - - :worker_1_port
SET search_path TO multi_mx_insert_select_repartition;
EXPLAIN (costs off) INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
QUERY PLAN
---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: repartition
-> Custom Scan (Citus Adaptive)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=xxxxx dbname=regression
-> HashAggregate
Group Key: a
-> Seq Scan on source_table_4213581 source_table
(10 rows)
INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
\c - - - :master_port
SET search_path TO multi_mx_insert_select_repartition;
SELECT * FROM target_table ORDER BY a;
a | b
---------------------------------------------------------------------
0 | 9
1 | 49
2 | 121
3 | 225
4 | 361
5 | 400
(6 rows)
RESET client_min_messages;
\set VERBOSITY terse
DROP SCHEMA multi_mx_insert_select_repartition CASCADE;
NOTICE: drop cascades to 3 other objects

View File

@ -1170,7 +1170,7 @@ WHERE
colocated_table_test_2.value_4 = reference_table_test.value_4;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: performing repartitioned INSERT ... SELECT
INSERT INTO
colocated_table_test (value_1, value_2)
SELECT
@ -1181,7 +1181,7 @@ WHERE
colocated_table_test_2.value_4 = reference_table_test.value_4;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: performing repartitioned INSERT ... SELECT
RESET client_min_messages;
-- some tests for mark_tables_colocated
-- should error out

View File

@ -1926,40 +1926,40 @@ INSERT INTO articles_hash
SELECT * FROM articles_hash WHERE author_id = $2 AND word_count = $1 OFFSET 0;
EXECUTE insert_sel(1,1);
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
EXECUTE insert_sel(1,1);
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
EXECUTE insert_sel(1,1);
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
EXECUTE insert_sel(1,1);
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
EXECUTE insert_sel(1,1);
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
EXECUTE insert_sel(1,1);
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Plan is router executable
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- one final interesting preperad statement
-- where one of the filters is on the target list
PREPARE fast_path_agg_filter(int, int) AS

View File

@ -96,11 +96,11 @@ INSERT INTO distributed_table
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: CTE ids_to_insert is going to be inlined via distributed planning
DEBUG: generating subplan XXX_1 for subquery SELECT (((tenant_id)::integer OPERATOR(pg_catalog.*) 100))::text AS tenant_id FROM with_dml.distributed_table WHERE (dept OPERATOR(pg_catalog.>) 7)
DEBUG: generating subplan XXX_2 for subquery SELECT DISTINCT ids_to_insert.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text)) ids_to_insert, with_dml.distributed_table WHERE (distributed_table.tenant_id OPERATOR(pg_catalog.<) ids_to_insert.tenant_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text)) citus_insert_select_subquery
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- not a very meaningful query
-- but has two modifying CTEs along with another
-- modify statement
@ -132,11 +132,11 @@ INSERT INTO second_distributed_table
SELECT *
FROM copy_to_other_table;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
DEBUG: generating subplan XXX_1 for CTE copy_to_other_table: INSERT INTO with_dml.distributed_table (tenant_id, dept) SELECT tenant_id, dept FROM with_dml.second_distributed_table WHERE (dept OPERATOR(pg_catalog.=) 3) ON CONFLICT(tenant_id) DO UPDATE SET dept = 4 RETURNING distributed_table.tenant_id, distributed_table.dept
DEBUG: generating subplan XXX_2 for CTE main_table_deleted: DELETE FROM with_dml.distributed_table WHERE ((dept OPERATOR(pg_catalog.<) 10) AND (NOT (EXISTS (SELECT 1 FROM with_dml.second_distributed_table WHERE ((second_distributed_table.dept OPERATOR(pg_catalog.=) 1) AND (second_distributed_table.tenant_id OPERATOR(pg_catalog.=) distributed_table.tenant_id)))))) RETURNING tenant_id, dept
DEBUG: generating subplan XXX_3 for subquery SELECT main_table_deleted.tenant_id, main_table_deleted.dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) main_table_deleted EXCEPT SELECT copy_to_other_table.tenant_id, copy_to_other_table.dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) copy_to_other_table
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT tenant_id, dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) citus_insert_select_subquery
DEBUG: Collecting INSERT ... SELECT results on coordinator
SET citus.force_max_query_parallelization TO off;
-- CTE inside the UPDATE statement
UPDATE

View File

@ -6,6 +6,7 @@ test: failure_setup
test: multi_test_helpers
test: failure_replicated_partitions
test: multi_test_catalog_views
test: failure_insert_select_repartition
test: failure_distributed_results
test: failure_ddl
test: failure_truncate

View File

@ -21,6 +21,7 @@ test: isolation_distributed_transaction_id
test: isolation_dump_global_wait_edges
test: isolation_citus_dist_activity
test: isolation_insert_select_repartition
test: isolation_dml_vs_repair isolation_copy_placement_vs_copy_placement
test: isolation_concurrent_dml isolation_data_migration

View File

@ -43,6 +43,7 @@ test: multi_mx_transaction_recovery
test: multi_mx_modifying_xacts
test: multi_mx_explain
test: multi_mx_reference_table
test: multi_mx_insert_select_repartition
# test that no tests leaked intermediate results. This should always be last
test: ensure_no_intermediate_data_leak

View File

@ -42,7 +42,7 @@ test: multi_read_from_secondaries
test: multi_create_table
test: multi_create_table_constraints multi_master_protocol multi_load_data multi_behavioral_analytics_create_table
test: multi_behavioral_analytics_basics multi_behavioral_analytics_single_shard_queries multi_insert_select_non_pushable_queries multi_insert_select
test: multi_insert_select_window multi_shard_update_delete window_functions dml_recursive recursive_dml_with_different_planners_executors
test: multi_insert_select_window multi_shard_update_delete window_functions dml_recursive recursive_dml_with_different_planners_executors insert_select_repartition
test: multi_insert_select_conflict
test: multi_row_insert

View File

@ -0,0 +1,54 @@
setup
{
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int);
SELECT create_distributed_table('target_table', 'a');
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
}
teardown
{
DROP TABLE IF EXISTS source_table;
DROP TABLE IF EXISTS target_table;
}
session "s1"
step "s1-begin" { BEGIN; }
step "s1-end" { END; }
step "s1-repartitioned-insert-select" { INSERT INTO target_table SELECT * FROM source_table; }
step "s1-select-target" { SELECT * FROM target_table ORDER BY a; }
session "s2"
step "s2-begin" { BEGIN; }
step "s2-end" { END; }
step "s2-delete-from-source" { DELETE FROM source_table; }
step "s2-update-source" { UPDATE source_table SET b = 50 - b; }
step "s2-insert-into-source" { INSERT INTO source_table VALUES (0, 0); }
step "s2-delete-from-target" { DELETE FROM target_table; }
step "s2-update-target" { UPDATE target_table SET b = 50 - b; }
step "s2-insert-into-target" { INSERT INTO target_table VALUES (0, 0); }
// INSERT/INTO shouldn't block DML on source_table
permutation "s1-begin" "s2-begin" "s1-repartitioned-insert-select" "s2-delete-from-source" "s1-end" "s2-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s1-repartitioned-insert-select" "s2-update-source" "s1-end" "s2-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s1-repartitioned-insert-select" "s2-insert-into-source" "s1-end" "s2-end" "s1-select-target"
// INSERT/INTO shouldn't be blocked by DML on source_table
permutation "s1-begin" "s2-begin" "s2-delete-from-source" "s1-repartitioned-insert-select" "s1-end" "s2-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s2-update-source" "s1-repartitioned-insert-select" "s1-end" "s2-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s2-insert-into-source" "s1-repartitioned-insert-select" "s1-end" "s2-end" "s1-select-target"
// INSERT/INTO should block UPDATE/DELETE on target_table, but not INSERT
permutation "s1-begin" "s2-begin" "s1-repartitioned-insert-select" "s2-delete-from-target" "s1-end" "s2-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s1-repartitioned-insert-select" "s2-update-target" "s1-end" "s2-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s1-repartitioned-insert-select" "s2-insert-into-target" "s1-end" "s2-end" "s1-select-target"
// INSERT/INTO should be blocked by UPDATE/DELETe on target_table, but not INSERT
permutation "s1-begin" "s2-begin" "s2-delete-from-target" "s1-repartitioned-insert-select" "s2-end" "s1-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s2-update-target" "s1-repartitioned-insert-select" "s2-end" "s1-end" "s1-select-target"
permutation "s1-begin" "s2-begin" "s2-insert-into-target" "s1-repartitioned-insert-select" "s2-end" "s1-end" "s1-select-target"

View File

@ -0,0 +1,96 @@
--
-- failure_insert_select_repartition
--
-- performs failure/cancellation test for repartitioned insert/select.
--
CREATE SCHEMA repartitioned_insert_select;
SET SEARCH_PATH=repartitioned_insert_select;
SELECT pg_backend_pid() as pid \gset
SET citus.next_shard_id TO 4213581;
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 2;
CREATE TABLE replicated_source_table(a int, b int);
SELECT create_distributed_table('replicated_source_table', 'a');
INSERT INTO replicated_source_table SELECT i, i*i FROM generate_series(1, 10) i;
SET citus.shard_count TO 3;
CREATE TABLE replicated_target_table(a int, b int);
SELECT create_distributed_table('replicated_target_table', 'a');
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
INSERT INTO source_table SELECT i, i*i FROM generate_series(1, 10) i;
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int);
SELECT create_distributed_table('target_table', 'a');
--
-- kill worker_partition_query_result
-- this fails the query on source table, so replicated case should succeed
--
SELECT citus.mitmproxy('conn.onQuery(query="worker_partition_query_result").kill()');
INSERT INTO target_table SELECT * FROM source_table;
SELECT * FROM target_table ORDER BY a;
SELECT citus.mitmproxy('conn.onQuery(query="worker_partition_query_result").kill()');
INSERT INTO target_table SELECT * FROM replicated_source_table;
SELECT * FROM target_table ORDER BY a;
--
-- kill fetch_intermediate_results
-- this fails the fetch into target, so source replication doesn't matter
-- and both should fail
--
TRUNCATE target_table;
SELECT citus.mitmproxy('conn.onQuery(query="fetch_intermediate_results").kill()');
INSERT INTO target_table SELECT * FROM source_table;
SELECT * FROM target_table ORDER BY a;
SELECT citus.mitmproxy('conn.onQuery(query="fetch_intermediate_results").kill()');
INSERT INTO target_table SELECT * FROM replicated_source_table;
SELECT * FROM target_table ORDER BY a;
--
-- kill read_intermediate_results
-- again, both should fail
--
TRUNCATE target_table;
SELECT citus.mitmproxy('conn.onQuery(query="read_intermediate_results").kill()');
INSERT INTO target_table SELECT * FROM source_table;
SELECT * FROM target_table ORDER BY a;
SELECT citus.mitmproxy('conn.onQuery(query="read_intermediate_results").kill()');
INSERT INTO target_table SELECT * FROM replicated_source_table;
SELECT * FROM target_table ORDER BY a;
--
-- We error out even if table is replicated and only one of the replicas
-- fail.
--
SELECT citus.mitmproxy('conn.onQuery(query="read_intermediate_results").kill()');
INSERT INTO replicated_target_table SELECT * FROM source_table;
SELECT * FROM replicated_target_table;
RESET SEARCH_PATH;
SELECT citus.mitmproxy('conn.allow()');
\set VERBOSITY TERSE
DROP SCHEMA repartitioned_insert_select CASCADE;

View File

@ -0,0 +1,468 @@
-- tests behaviour of INSERT INTO ... SELECT with repartitioning
CREATE SCHEMA insert_select_repartition;
SET search_path TO 'insert_select_repartition';
SET citus.next_shard_id TO 4213581;
SET citus.shard_replication_factor TO 1;
SET citus.replication_model TO 'streaming';
-- 4 shards, hash distributed.
-- Negate distribution column value.
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int);
SELECT create_distributed_table('source_table', 'a');
INSERT INTO source_table SELECT * FROM generate_series(1, 10);
CREATE TABLE target_table(a int);
SELECT create_distributed_table('target_table', 'a');
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT -a FROM source_table;
RESET client_min_messages;
SELECT * FROM target_table WHERE a=-1 OR a=-3 OR a=-7 ORDER BY a;
DROP TABLE source_table, target_table;
--
-- range partitioning, composite distribution column
--
CREATE TYPE composite_key_type AS (f1 int, f2 text);
-- source
CREATE TABLE source_table(f1 int, key composite_key_type, value int, mapped_key composite_key_type);
SELECT create_distributed_table('source_table', 'key', 'range');
CALL public.create_range_partitioned_shards('source_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
INSERT INTO source_table VALUES (0, (0, 'a'), 1, (0, 'a'));
INSERT INTO source_table VALUES (1, (1, 'b'), 2, (26, 'b'));
INSERT INTO source_table VALUES (2, (2, 'c'), 3, (3, 'c'));
INSERT INTO source_table VALUES (3, (4, 'd'), 4, (27, 'd'));
INSERT INTO source_table VALUES (4, (30, 'e'), 5, (30, 'e'));
INSERT INTO source_table VALUES (5, (31, 'f'), 6, (31, 'f'));
INSERT INTO source_table VALUES (6, (32, 'g'), 50, (8, 'g'));
-- target
CREATE TABLE target_table(f1 int DEFAULT 0, value int, key composite_key_type PRIMARY KEY);
SELECT create_distributed_table('target_table', 'key', 'range');
CALL public.create_range_partitioned_shards('target_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT f1, value, mapped_key FROM source_table;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
SELECT * FROM target_table WHERE key = (26, 'b')::composite_key_type;
-- with explicit column names
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table(value, key) SELECT value, mapped_key FROM source_table;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
-- missing value for a column
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table(key) SELECT mapped_key AS key_renamed FROM source_table;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
-- ON CONFLICT
SET client_min_messages TO DEBUG2;
INSERT INTO target_table(key)
SELECT mapped_key AS key_renamed FROM source_table
WHERE (mapped_key).f1 % 2 = 1
ON CONFLICT (key) DO UPDATE SET f1=1;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY key;
-- missing value for distribution column
INSERT INTO target_table(value) SELECT value FROM source_table;
DROP TABLE source_table, target_table;
-- different column types
-- verifies that we add necessary casts, otherwise even shard routing won't
-- work correctly and we will see 2 values for the same primary key.
CREATE TABLE target_table(col_1 int primary key, col_2 int);
SELECT create_distributed_table('target_table','col_1');
INSERT INTO target_table VALUES (1,2), (2,3), (3,4), (4,5), (5,6);
CREATE TABLE source_table(col_1 numeric, col_2 numeric, col_3 numeric);
SELECT create_distributed_table('source_table','col_1');
INSERT INTO source_table VALUES (1,1,1), (3,3,3), (5,5,5);
SET client_min_messages TO DEBUG2;
INSERT INTO target_table
SELECT
col_1, col_2
FROM
source_table
ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY 1;
DROP TABLE source_table, target_table;
--
-- array coercion
--
SET citus.shard_count TO 3;
CREATE TABLE source_table(a int, mapped_key int, c float[]);
SELECT create_distributed_table('source_table', 'a');
INSERT INTO source_table VALUES (1, -1, ARRAY[1.1, 2.2, 3.3]), (2, -2, ARRAY[4.5, 5.8]),
(3, -3, ARRAY[]::float[]), (4, -4, ARRAY[3.3]);
SET citus.shard_count TO 2;
CREATE TABLE target_table(a int, b int[]);
SELECT create_distributed_table('target_table', 'a');
SET client_min_messages TO DEBUG1;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
--
-- worker queries can have more columns than necessary. ExpandWorkerTargetEntry()
-- might add additional columns to the target list.
--
TRUNCATE target_table;
\set VERBOSITY TERSE
-- first verify that the SELECT query below fetches 3 projected columns from workers
SET citus.log_remote_commands TO true; SET client_min_messages TO DEBUG;
CREATE TABLE results AS SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
RESET citus.log_remote_commands; RESET client_min_messages;
DROP TABLE results;
-- now verify that we don't write the extra columns to the intermediate result files and
-- insertion to the target works fine.
SET client_min_messages TO DEBUG1;
INSERT INTO target_table SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
--
-- repartitioned INSERT/SELECT followed/preceded by other DML in same transaction
--
-- case 1. followed by DELETE
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
DELETE FROM target_table;
END;
SELECT * FROM target_table ORDER BY a;
-- case 2. followed by UPDATE
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
UPDATE target_table SET b=array_append(b, a);
END;
SELECT * FROM target_table ORDER BY a;
-- case 3. followed by multi-row INSERT
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
END;
SELECT * FROM target_table ORDER BY a;
-- case 4. followed by distributed INSERT/SELECT
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
SELECT * FROM target_table ORDER BY a;
INSERT INTO target_table SELECT * FROM target_table;
END;
SELECT * FROM target_table ORDER BY a;
-- case 5. preceded by DELETE
TRUNCATE target_table;
BEGIN;
DELETE FROM target_table;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
-- case 6. preceded by UPDATE
TRUNCATE target_table;
BEGIN;
UPDATE target_table SET b=array_append(b, a);
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
-- case 7. preceded by multi-row INSERT
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
-- case 8. preceded by distributed INSERT/SELECT
TRUNCATE target_table;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
BEGIN;
INSERT INTO target_table SELECT * FROM target_table;
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
END;
SELECT * FROM target_table ORDER BY a;
--
-- repartitioned INSERT/SELECT with RETURNING
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG1;
WITH c AS (
INSERT INTO target_table
SELECT mapped_key, c FROM source_table
RETURNING *)
SELECT * FROM c ORDER by a;
RESET client_min_messages;
--
-- in combination with CTEs
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG1;
WITH t AS (
SELECT mapped_key, a, c FROM source_table
WHERE a > floor(random())
)
INSERT INTO target_table
SELECT mapped_key, c FROM t NATURAL JOIN source_table;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
DROP TABLE source_table, target_table;
--
-- The case where select query has a GROUP BY ...
--
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int);
SELECT create_distributed_table('target_table', 'a');
INSERT INTO source_table SELECT floor(i/4), i*i FROM generate_series(1, 20) i;
SET client_min_messages TO DEBUG1;
INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
--
-- EXPLAIN output should specify repartitioned INSERT/SELECT
--
EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
--
-- Duplicate names in target list
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table
SELECT max(b), max(b) FROM source_table GROUP BY a;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a;
--
-- Prepared INSERT/SELECT
--
TRUNCATE target_table;
PREPARE insert_plan(int, int) AS
INSERT INTO target_table
SELECT a, max(b) FROM source_table
WHERE a BETWEEN $1 AND $2 GROUP BY a;
SET client_min_messages TO DEBUG1;
EXECUTE insert_plan(0, 2);
EXECUTE insert_plan(0, 2);
EXECUTE insert_plan(0, 2);
EXECUTE insert_plan(0, 2);
EXECUTE insert_plan(0, 2);
EXECUTE insert_plan(0, 2);
EXECUTE insert_plan(2, 4);
EXECUTE insert_plan(2, 4);
EXECUTE insert_plan(2, 4);
EXECUTE insert_plan(2, 4);
EXECUTE insert_plan(2, 4);
EXECUTE insert_plan(2, 4);
RESET client_min_messages;
SELECT a, count(*), count(distinct b) distinct_values FROM target_table GROUP BY a ORDER BY a;
--
-- INSERT/SELECT in CTE
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
WITH r AS (
INSERT INTO target_table SELECT * FROM source_table RETURNING *
)
INSERT INTO target_table SELECT source_table.a, max(source_table.b) FROM source_table NATURAL JOIN r GROUP BY source_table.a;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a, b;
DROP TABLE source_table, target_table;
--
-- Constraint failure and rollback
--
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
UPDATE source_table SET b = NULL where b IN (9, 4);
SET citus.shard_replication_factor TO 2;
CREATE TABLE target_table(a int, b int not null);
SELECT create_distributed_table('target_table', 'a', 'range');
CALL public.create_range_partitioned_shards('target_table', '{0,3,6,9}','{2,5,8,50}');
INSERT INTO target_table VALUES (11,9), (22,4);
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table;
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
BEGIN;
SAVEPOINT s1;
INSERT INTO target_table SELECT * FROM source_table;
ROLLBACK TO SAVEPOINT s1;
INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
END;
SELECT * FROM target_table ORDER BY b;
-- verify that values have been replicated to both replicas
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
--
-- Multiple casts in the SELECT query
--
TRUNCATE target_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT 1.12, b::bigint FROM source_table WHERE b IS NOT NULL;
RESET client_min_messages;
SELECT * FROM target_table ORDER BY a, b;
--
-- ROLLBACK after out of range error
--
TRUNCATE target_table;
BEGIN;
INSERT INTO target_table SELECT a * 10, b FROM source_table WHERE b IS NOT NULL;
END;
SELECT max(result) FROM run_command_on_placements('target_table', 'select count(*) from %s');
DROP TABLE source_table, target_table;
--
-- Range partitioned target's ranges doesn't cover the whole range
--
SET citus.shard_replication_factor TO 2;
SET citus.replication_model TO 'statement';
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
SET citus.shard_replication_factor TO 2;
CREATE TABLE target_table(b int not null, a float);
SELECT create_distributed_table('target_table', 'a', 'range');
CALL public.create_range_partitioned_shards('target_table', '{0.0,3.5,6.5,9.5}','{2.9,5.9,8.9,50.0}');
INSERT INTO target_table SELECT b, a+0.6 FROM source_table;
SELECT * FROM target_table ORDER BY a;
-- verify that values have been replicated to both replicas, and that each
-- replica has received correct number of rows
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
DROP TABLE source_table, target_table;
--
-- Select column names should be unique
--
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int, c int, d int, e int, f int);
SELECT create_distributed_table('target_table', 'a');
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
RESET client_min_messages;
SELECT count(*) FROM target_table;
--
-- Disable repartitioned insert/select
--
TRUNCATE target_table;
SET citus.enable_repartitioned_insert_select TO OFF;
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
SET client_min_messages TO DEBUG2;
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
RESET client_min_messages;
SELECT count(*) FROM target_table;
SET citus.enable_repartitioned_insert_select TO ON;
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
DROP TABLE source_table, target_table;
--
-- Don't use INSERT/SELECT repartition with repartition joins
--
create table test(x int, y int);
select create_distributed_table('test', 'x');
set citus.enable_repartition_joins to true;
SET client_min_messages TO DEBUG1;
insert into test(y, x) select a.x, b.y from test a JOIN test b USING (y);
RESET client_min_messages;
SET client_min_messages TO WARNING;
DROP SCHEMA insert_select_repartition CASCADE;

View File

@ -188,12 +188,14 @@ ROLLBACK;
BEGIN;
DELETE FROM test_ref_table WHERE key > 10;
INSERT INTO
target_table
SELECT
col_2,
col_1
FROM source_table_1 ON CONFLICT (col_1) DO UPDATE SET col_2 = 1 RETURNING *;
WITH r AS (
INSERT INTO
target_table
SELECT
col_2,
col_1
FROM source_table_1 ON CONFLICT (col_1) DO UPDATE SET col_2 = 1 RETURNING *)
SELECT * FROM r ORDER BY col_1;
ROLLBACK;
-- Following two queries are supported since we no not modify but only select from

View File

@ -0,0 +1,50 @@
-- Test behaviour of repartitioned INSERT ... SELECT in MX setup
CREATE SCHEMA multi_mx_insert_select_repartition;
SET search_path TO multi_mx_insert_select_repartition;
SET citus.next_shard_id TO 4213581;
SET citus.replication_model TO 'streaming';
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4;
CREATE TABLE source_table(a int, b int);
SELECT create_distributed_table('source_table', 'a');
INSERT INTO source_table SELECT floor(i/4), i*i FROM generate_series(1, 20) i;
SET citus.shard_count TO 3;
CREATE TABLE target_table(a int, b int);
SELECT create_distributed_table('target_table', 'a');
CREATE FUNCTION square(int) RETURNS INT
AS $$ SELECT $1 * $1 $$
LANGUAGE SQL;
select create_distributed_function('square(int)');
select public.colocate_proc_with_table('square', 'source_table'::regclass, 0);
-- Test along with function delegation
-- function delegation only happens for "SELECT f()", and we don't use
-- repartitioned INSERT/SELECT when task count is 1, so the following
-- should go via coordinator
EXPLAIN (costs off) INSERT INTO target_table(a) SELECT square(4);
INSERT INTO target_table(a) SELECT square(4);
SELECT * FROM target_table;
TRUNCATE target_table;
--
-- Test repartitioned INSERT/SELECT from MX worker
--
\c - - - :worker_1_port
SET search_path TO multi_mx_insert_select_repartition;
EXPLAIN (costs off) INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
\c - - - :master_port
SET search_path TO multi_mx_insert_select_repartition;
SELECT * FROM target_table ORDER BY a;
RESET client_min_messages;
\set VERBOSITY terse
DROP SCHEMA multi_mx_insert_select_repartition CASCADE;