From d449c1857c90db8c2ff079aa71955998c6117a96 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Fri, 10 Jan 2020 02:16:58 -0800 Subject: [PATCH 01/22] INSERT/SELECT: Use ExecutePlan* instead of ExecuteSelect* --- .../executor/insert_select_executor.c | 71 +++++++++---------- 1 file changed, 35 insertions(+), 36 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 85de59949..1d5eb2197 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -52,14 +52,13 @@ 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); @@ -106,6 +105,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; @@ -134,6 +134,18 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) selectRte->subquery = selectQuery; ReorderInsertSelectTargetLists(insertSelectQuery, insertRte, selectRte); + /* + * 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 @@ -156,10 +168,10 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) ListCell *taskCell = NULL; List *prunedTaskList = NIL; - shardStateHash = ExecuteSelectIntoColocatedIntermediateResults( + shardStateHash = ExecutePlanIntoColocatedIntermediateResults( targetRelationId, insertTargetList, - selectQuery, + selectPlan, executorState, intermediateResultIdPrefix); @@ -209,8 +221,8 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) } else { - ExecuteSelectIntoRelation(targetRelationId, insertTargetList, selectQuery, - executorState); + ExecutePlanIntoRelation(targetRelationId, insertTargetList, selectPlan, + executorState); } scanState->finishedRemoteScan = true; @@ -438,17 +450,18 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery, /* - * 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 +486,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 +497,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 +527,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; From ced876358d6d00acecf0f554c96e7b278c33f2f2 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Fri, 10 Jan 2020 03:16:39 -0800 Subject: [PATCH 02/22] INSERT/SELECT: Refactor out AddInsertSelectCasts --- .../executor/insert_select_executor.c | 53 +++++++++++-------- 1 file changed, 32 insertions(+), 21 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 1d5eb2197..93d8f3aad 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -62,6 +62,7 @@ static HTAB * ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId, static List * BuildColumnNameListFromTargetList(Oid targetRelationId, List *insertTargetList); static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList); +static void AddInsertSelectCasts(List *targetList, TupleDesc destTupleDescriptor); /* @@ -354,8 +355,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); @@ -364,25 +363,7 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery, * 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; - } - } + AddInsertSelectCasts(insertSelectQuery->targetList, destTupleDescriptor); for (int shardOffset = 0; shardOffset < shardCount; shardOffset++) { @@ -594,3 +575,33 @@ ExecutingInsertSelect(void) { return insertSelectExecutorLevel > 0; } + + +/* + * AddInsertSelectCasts + */ +static void +AddInsertSelectCasts(List *targetList, TupleDesc destTupleDescriptor) +{ + ListCell *targetEntryCell = NULL; + + foreach(targetEntryCell, 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; + } + } +} From b4e5f4b10a24da49b2f66bac00de5a59a2672e9b Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Fri, 10 Jan 2020 15:42:18 -0800 Subject: [PATCH 03/22] Implement INSERT ... SELECT with repartitioning --- src/backend/distributed/commands/multi_copy.c | 25 ++ .../distributed_intermediate_results.c | 22 +- .../executor/insert_select_executor.c | 309 +++++++++++++++++- .../executor/intermediate_results.c | 2 +- .../distributed/planner/recursive_planning.c | 87 ++++- .../test/distributed_intermediate_results.c | 22 +- src/include/distributed/commands/multi_copy.h | 1 + .../distributed/intermediate_results.h | 2 + src/include/distributed/recursive_planning.h | 4 + .../expected/insert_select_repartition.out | 162 +++++++++ .../expected/intermediate_result_pruning.out | 6 +- .../regress/expected/multi_insert_select.out | 93 ++++-- .../expected/multi_insert_select_conflict.out | 18 +- ...lti_insert_select_non_pushable_queries.out | 2 - .../multi_mx_function_call_delegation.out | 2 +- .../expected/multi_reference_table.out | 4 +- src/test/regress/expected/with_dml.out | 3 +- src/test/regress/multi_schedule | 2 +- .../regress/sql/insert_select_repartition.sql | 77 +++++ 19 files changed, 764 insertions(+), 79 deletions(-) create mode 100644 src/test/regress/expected/insert_select_repartition.out create mode 100644 src/test/regress/sql/insert_select_repartition.sql diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index d41c737a4..07004df60 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -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 diff --git a/src/backend/distributed/executor/distributed_intermediate_results.c b/src/backend/distributed/executor/distributed_intermediate_results.c index d03b7e4db..095995856 100644 --- a/src/backend/distributed/executor/distributed_intermediate_results.c +++ b/src/backend/distributed/executor/distributed_intermediate_results.c @@ -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, diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 93d8f3aad..0a5f33bf7 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -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" @@ -63,6 +68,14 @@ static List * BuildColumnNameListFromTargetList(Oid targetRelationId, List *insertTargetList); static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList); static void AddInsertSelectCasts(List *targetList, TupleDesc destTupleDescriptor); +static bool IsSupportedRedistributionTarget(Oid targetRelationId); +static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery, + DistTableCacheEntry *targetRelation, + List **redistributedResults, + bool useBinaryFormat); +static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn); +static bool IsRedistributablePlan(Plan *selectPlan, bool hasReturning, bool + hasOnConflict); /* @@ -115,11 +128,9 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) Oid targetRelationId = insertRte->relid; char *intermediateResultIdPrefix = distributedPlan->intermediateResultIdPrefix; bool hasReturning = distributedPlan->hasReturning; + bool hasOnConflict = insertSelectQuery->onConflict != NULL; 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 @@ -157,8 +168,99 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) LockPartitionRelations(targetRelationId, RowExclusiveLock); } - if (insertSelectQuery->onConflict || hasReturning) + if (IsRedistributablePlan(selectPlan->planTree, hasReturning, hasOnConflict) && + 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. Jobs are executed + * sequentially, so we also don't need to include job id here. + */ + char *distResultPrefix = "repartitioned_results"; + + 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)))); + + 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); + + TransactionProperties xactProperties = { + .errorOnAnyFailure = true, + .useRemoteTransactionBlocks = TRANSACTION_BLOCKS_REQUIRED, + .requires2PC = false + }; + int64 rowsInserted = ExecuteTaskListExtended(ROW_MODIFY_COMMUTATIVE, taskList, + tupleDescriptor, + scanState->tuplestorestate, + hasReturning, + MaxAdaptiveExecutorPoolSize, + &xactProperties); + + 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 @@ -222,6 +324,9 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) } else { + ereport(DEBUG1, (errmsg( + "Collecting INSERT ... SELECT results on coordinator"))); + ExecutePlanIntoRelation(targetRelationId, insertTargetList, selectPlan, executorState); } @@ -578,7 +683,9 @@ ExecutingInsertSelect(void) /* - * AddInsertSelectCasts + * AddInsertSelectCasts makes sure that the types in columns in targetList + * have the same type as given tuple descriptor by adding necessary type + * casts. */ static void AddInsertSelectCasts(List *targetList, TupleDesc destTupleDescriptor) @@ -605,3 +712,193 @@ AddInsertSelectCasts(List *targetList, TupleDesc destTupleDescriptor) } } } + + +/* + * IsSupportedRedistributionTarget determines whether re-partitioning into the + * given target relation is supported. + */ +static 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; + + 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 + */ + AddInsertSelectCasts(insertSelectQuery->targetList, destTupleDescriptor); + + 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 = FinalizedShardPlacementList(shardId); + + RelationShard *relationShard = CitusMakeNode(RelationShard); + relationShard->relationId = targetShardInterval->relationId; + relationShard->shardId = targetShardInterval->shardId; + + Task *modifyTask = CreateBasicTask(jobId, taskIdIndex, MODIFY_TASK, + queryString->data); + modifyTask->anchorShardId = shardId; + modifyTask->taskPlacementList = insertShardPlacementList; + modifyTask->relationShardList = list_make1(relationShard); + modifyTask->replicationModel = targetRelation->replicationModel; + + taskList = lappend(taskList, modifyTask); + + taskIdIndex++; + } + + heap_close(distributedRelation, NoLock); + + 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. + */ +static bool +IsRedistributablePlan(Plan *selectPlan, bool hasReturning, bool hasOnConflict) +{ + if (hasReturning || hasOnConflict) + { + 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 case too. + */ + if (list_length(distSelectTaskList) <= 1) + { + return false; + } + + return true; +} diff --git a/src/backend/distributed/executor/intermediate_results.c b/src/backend/distributed/executor/intermediate_results.c index 9366e9dfc..4b7197363 100644 --- a/src/backend/distributed/executor/intermediate_results.c +++ b/src/backend/distributed/executor/intermediate_results.c @@ -894,7 +894,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); diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index a71953a42..18cd2f8f8 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -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 + * + * FROM + * read_intermediate_results(ARRAY['', ...]::text[], ') + * AS res (); + * + * 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; diff --git a/src/backend/distributed/test/distributed_intermediate_results.c b/src/backend/distributed/test/distributed_intermediate_results.c index 474a350df..1c449f69c 100644 --- a/src/backend/distributed/test/distributed_intermediate_results.c +++ b/src/backend/distributed/test/distributed_intermediate_results.c @@ -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; diff --git a/src/include/distributed/commands/multi_copy.h b/src/include/distributed/commands/multi_copy.h index a7a7708c5..e01cf9317 100644 --- a/src/include/distributed/commands/multi_copy.h +++ b/src/include/distributed/commands/multi_copy.h @@ -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, diff --git a/src/include/distributed/intermediate_results.h b/src/include/distributed/intermediate_results.h index 61e670fbc..028e25d54 100644 --- a/src/include/distributed/intermediate_results.h +++ b/src/include/distributed/intermediate_results.h @@ -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); diff --git a/src/include/distributed/recursive_planning.h b/src/include/distributed/recursive_planning.h index 99081c66f..006ee1c08 100644 --- a/src/include/distributed/recursive_planning.h +++ b/src/include/distributed/recursive_planning.h @@ -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 */ diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out new file mode 100644 index 000000000..68b889f1c --- /dev/null +++ b/src/test/regress/expected/insert_select_repartition.out @@ -0,0 +1,162 @@ +-- 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'; +-- Test 1 +-- 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 '?column?' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213585 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213583_to_0,repartitioned_results_from_4213584_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213586 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213582_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213587 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213581_to_2,repartitioned_results_from_4213582_to_2,repartitioned_results_from_4213584_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213588 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213581_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" 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; +-- +-- Test 2. +-- 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); +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 'mapped_key' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0,repartitioned_results_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_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, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_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 'mapped_key' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0,repartitioned_results_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_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, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_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_renamed' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0,repartitioned_results_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed 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_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed 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) + +-- 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; +SET client_min_messages TO WARNING; +DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/expected/intermediate_result_pruning.out b/src/test/regress/expected/intermediate_result_pruning.out index 3da52fee2..fb20c941b 100644 --- a/src/test/regress/expected/intermediate_result_pruning.out +++ b/src/test/regress/expected/intermediate_result_pruning.out @@ -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 diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index 1f504a725..6b5c6b2aa 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -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: 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_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_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_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_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] @@ -1083,6 +1086,7 @@ DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value 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: 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 'value_1' +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 '?column?' +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300000_to_0,repartitioned_results_from_13300001_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300000_to_1,repartitioned_results_from_13300001_to_1,repartitioned_results_from_13300003_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300001_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300000_to_3,repartitioned_results_from_13300002_to_3,repartitioned_results_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" 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_from_13300000_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) +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_from_13300001_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) +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_from_13300002_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) +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_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) 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 'avg' +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 'value_2' +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] @@ -1223,6 +1242,8 @@ 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: Router planner cannot handle multi-shard select queries +DEBUG: performing repartitioned INSERT ... SELECT +DEBUG: partitioning SELECT query by column index 0 with name '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] @@ -1268,6 +1288,8 @@ 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: Router planner cannot handle multi-shard select queries +DEBUG: performing repartitioned INSERT ... SELECT +DEBUG: partitioning SELECT query by column index 0 with name '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 'case' 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 'coalesce' 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 'greatest' 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 'least' 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 'nullif' 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 '?column?' 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 '?column?' 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 'val' 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 'val' 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 diff --git a/src/test/regress/expected/multi_insert_select_conflict.out b/src/test/regress/expected/multi_insert_select_conflict.out index f543ceb6f..b1d1803dd 100644 --- a/src/test/regress/expected/multi_insert_select_conflict.out +++ b/src/test/regress/expected/multi_insert_select_conflict.out @@ -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 --------------------------------------------------------------------- @@ -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. @@ -481,13 +481,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 +512,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 --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_insert_select_non_pushable_queries.out b/src/test/regress/expected/multi_insert_select_non_pushable_queries.out index 07280ec91..d37928073 100644 --- a/src/test/regress/expected/multi_insert_select_non_pushable_queries.out +++ b/src/test/regress/expected/multi_insert_select_non_pushable_queries.out @@ -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) diff --git a/src/test/regress/expected/multi_mx_function_call_delegation.out b/src/test/regress/expected/multi_mx_function_call_delegation.out index 23e6f5e33..935a01770 100644 --- a/src/test/regress/expected/multi_mx_function_call_delegation.out +++ b/src/test/regress/expected/multi_mx_function_call_delegation.out @@ -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) diff --git a/src/test/regress/expected/multi_reference_table.out b/src/test/regress/expected/multi_reference_table.out index 4a4f79b7f..8b15a8ff5 100644 --- a/src/test/regress/expected/multi_reference_table.out +++ b/src/test/regress/expected/multi_reference_table.out @@ -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 diff --git a/src/test/regress/expected/with_dml.out b/src/test/regress/expected/with_dml.out index 5936d623f..2367e602a 100644 --- a/src/test/regress/expected/with_dml.out +++ b/src/test/regress/expected/with_dml.out @@ -101,6 +101,7 @@ 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 +133,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 diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index 6990f6d27..4fc14f907 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -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 diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql new file mode 100644 index 000000000..be0686ef7 --- /dev/null +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -0,0 +1,77 @@ +-- 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'; + +-- Test 1 +-- 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; + +-- +-- Test 2. +-- 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); +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; + +-- missing value for distribution column +INSERT INTO target_table(value) SELECT value FROM source_table; + +DROP TABLE source_table, target_table; + +SET client_min_messages TO WARNING; +DROP SCHEMA insert_select_repartition CASCADE; From d67a384350495c8ea0b377bc4f0fafc2d7f4a136 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Sat, 11 Jan 2020 13:05:59 -0800 Subject: [PATCH 04/22] Enable repartitioned INSERT/SELECT ON CONFLICT. --- .../executor/insert_select_executor.c | 2 +- .../expected/insert_select_repartition.out | 28 +++++++++++++++- .../regress/sql/insert_select_repartition.sql | 32 ++++++++++++++++++- 3 files changed, 59 insertions(+), 3 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 0a5f33bf7..311567469 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -874,7 +874,7 @@ PartitionColumnIndex(List *insertTargetList, Var *partitionColumn) static bool IsRedistributablePlan(Plan *selectPlan, bool hasReturning, bool hasOnConflict) { - if (hasReturning || hasOnConflict) + if (hasReturning) { return false; } diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 68b889f1c..f135429b6 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -68,7 +68,7 @@ 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); +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. @@ -154,6 +154,32 @@ SELECT * FROM target_table ORDER BY key; 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_renamed' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed 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_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed 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 diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index be0686ef7..da07796f5 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -43,7 +43,7 @@ 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); +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)"}'); @@ -68,8 +68,38 @@ 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 +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),(2,2,2),(3,3,3),(4,4,4),(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; From 89463f9760cafb79b354b68530e15adf55a22904 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Sat, 11 Jan 2020 16:14:32 -0800 Subject: [PATCH 05/22] Repartitioned INSERT/SELECT: cast columns in SELECT targets --- .../executor/insert_select_executor.c | 234 ++++++++++++++---- .../expected/insert_select_repartition.out | 89 ++++++- .../regress/expected/multi_insert_select.out | 24 +- .../expected/multi_insert_select_conflict.out | 6 +- .../multi_router_planner_fast_path.out | 12 +- src/test/regress/expected/with_dml.out | 1 - .../regress/sql/insert_select_repartition.sql | 30 ++- 7 files changed, 319 insertions(+), 77 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 311567469..49c0db137 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -67,15 +67,17 @@ static HTAB * ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId, static List * BuildColumnNameListFromTargetList(Oid targetRelationId, List *insertTargetList); static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList); -static void AddInsertSelectCasts(List *targetList, TupleDesc destTupleDescriptor); +static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, + Oid targetRelationId); static bool IsSupportedRedistributionTarget(Oid targetRelationId); static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery, DistTableCacheEntry *targetRelation, List **redistributedResults, bool useBinaryFormat); static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn); -static bool IsRedistributablePlan(Plan *selectPlan, bool hasReturning, bool - hasOnConflict); +static bool IsRedistributablePlan(Plan *selectPlan, bool hasReturning); +static Expr * CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation, + int targetTypeMod); /* @@ -128,7 +130,6 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) Oid targetRelationId = insertRte->relid; char *intermediateResultIdPrefix = distributedPlan->intermediateResultIdPrefix; bool hasReturning = distributedPlan->hasReturning; - bool hasOnConflict = insertSelectQuery->onConflict != NULL; HTAB *shardStateHash = NULL; /* @@ -146,6 +147,16 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) selectRte->subquery = selectQuery; ReorderInsertSelectTargetLists(insertSelectQuery, insertRte, selectRte); + /* + * 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 + */ + selectQuery->targetList = + AddInsertSelectCasts(insertSelectQuery->targetList, + selectQuery->targetList, + targetRelationId); + /* * 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 @@ -168,7 +179,7 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) LockPartitionRelations(targetRelationId, RowExclusiveLock); } - if (IsRedistributablePlan(selectPlan->planTree, hasReturning, hasOnConflict) && + if (IsRedistributablePlan(selectPlan->planTree, hasReturning) && IsSupportedRedistributionTarget(targetRelationId)) { ereport(DEBUG1, (errmsg("performing repartitioned INSERT ... SELECT"))); @@ -460,16 +471,6 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery, uint32 taskIdIndex = 1; uint64 jobId = INVALID_JOB_ID; - 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 - */ - AddInsertSelectCasts(insertSelectQuery->targetList, destTupleDescriptor); - for (int shardOffset = 0; shardOffset < shardCount; shardOffset++) { ShardInterval *targetShardInterval = @@ -529,8 +530,6 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery, taskIdIndex++; } - heap_close(distributedRelation, NoLock); - return taskList; } @@ -683,33 +682,182 @@ ExecutingInsertSelect(void) /* - * AddInsertSelectCasts makes sure that the types in columns in targetList - * have the same type as given tuple descriptor by adding necessary type - * casts. + * 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 void -AddInsertSelectCasts(List *targetList, TupleDesc destTupleDescriptor) +static List * +AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, + Oid targetRelationId) { - ListCell *targetEntryCell = NULL; + ListCell *insertEntryCell = NULL; + ListCell *selectEntryCell = NULL; + List *projectedEntries = NIL; + List *nonProjectedEntries = NIL; - foreach(targetEntryCell, targetList) + /* + * 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 *targetEntry = (TargetEntry *) lfirst(targetEntryCell); - Var *insertColumn = (Var *) targetEntry->expr; - Form_pg_attribute attr = TupleDescAttr(destTupleDescriptor, targetEntry->resno - - 1); + TargetEntry *insertEntry = (TargetEntry *) lfirst(insertEntryCell); + TargetEntry *selectEntry = (TargetEntry *) lfirst(selectEntryCell); + Var *insertColumn = (Var *) insertEntry->expr; + Form_pg_attribute attr = TupleDescAttr(destTupleDescriptor, + insertEntry->resno - 1); - if (insertColumn->vartype != attr->atttypid) + Oid sourceType = insertColumn->vartype; + Oid targetType = attr->atttypid; + if (sourceType != targetType) { - 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; + insertEntry->expr = CastExpr((Expr *) insertColumn, sourceType, targetType, + attr->attcollation, attr->atttypmod); - targetEntry->expr = (Expr *) coerceExpr; + /* + * 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))); } } @@ -765,16 +913,6 @@ RedistributedInsertSelectTaskList(Query *insertSelectQuery, uint32 taskIdIndex = 1; uint64 jobId = INVALID_JOB_ID; - 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 - */ - AddInsertSelectCasts(insertSelectQuery->targetList, destTupleDescriptor); - for (shardOffset = 0; shardOffset < shardCount; shardOffset++) { ShardInterval *targetShardInterval = @@ -839,8 +977,6 @@ RedistributedInsertSelectTaskList(Query *insertSelectQuery, taskIdIndex++; } - heap_close(distributedRelation, NoLock); - return taskList; } @@ -872,7 +1008,7 @@ PartitionColumnIndex(List *insertTargetList, Var *partitionColumn) * IsRedistributablePlan returns true if the given plan is a redistrituable plan. */ static bool -IsRedistributablePlan(Plan *selectPlan, bool hasReturning, bool hasOnConflict) +IsRedistributablePlan(Plan *selectPlan, bool hasReturning) { if (hasReturning) { diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index f135429b6..e33e7a527 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -4,7 +4,6 @@ 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'; --- Test 1 -- 4 shards, hash distributed. -- Negate distribution column value. SET citus.shard_count TO 4; @@ -46,8 +45,8 @@ SELECT * FROM target_table WHERE a=-1 OR a=-3 OR a=-7 ORDER BY a; DROP TABLE source_table, target_table; -- --- Test 2. -- 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); @@ -183,6 +182,92 @@ SELECT * FROM target_table ORDER BY key; -- 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 'auto_coerced_by_citus_0' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213593 AS citus_table_alias (col_1, col_2) SELECT auto_coerced_by_citus_0, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213597_to_0,repartitioned_results_from_4213600_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer, auto_coerced_by_citus_1 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 auto_coerced_by_citus_0, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213599_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer, auto_coerced_by_citus_1 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 DEBUG2; +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: Router planner cannot handle multi-shard select queries +DEBUG: performing repartitioned INSERT ... SELECT +DEBUG: partitioning SELECT query by column index 0 with name 'mapped_key' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213604 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213602_to_0,repartitioned_results_from_4213603_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213605 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213601_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) +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; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index 6b5c6b2aa..b50514ba9 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -1083,7 +1083,7 @@ 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 auto_coerced_by_citus_0, int4(v1) AS auto_coerced_by_citus_1, int8(v4) AS auto_coerced_by_citus_2 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 @@ -1142,11 +1142,11 @@ DETAIL: Subquery contains an explicit cast in the same position as the target t 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 '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_from_13300000_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) -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_from_13300001_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) -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_from_13300002_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) -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_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id bigint) +DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300000_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300001_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300002_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) INSERT INTO agg_events (value_3_agg, value_4_agg, @@ -1165,7 +1165,7 @@ DETAIL: Subquery contains an aggregation in the same position as the target tab 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 'avg' +DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' ERROR: the partition column value cannot be NULL CONTEXT: while executing command on localhost:xxxxx INSERT INTO agg_events @@ -1240,10 +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 auto_coerced_by_citus_0 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 'id' +DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' -- 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) @@ -2149,7 +2149,7 @@ DETAIL: Subquery contains an expression that is not a simple column reference i 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 '?column?' +DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' 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. @@ -2163,13 +2163,13 @@ DETAIL: Subquery contains an expression that is not a simple column reference i 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 '?column?' +DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' 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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'val' +DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' 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. diff --git a/src/test/regress/expected/multi_insert_select_conflict.out b/src/test/regress/expected/multi_insert_select_conflict.out index b1d1803dd..4cbed96ce 100644 --- a/src/test/regress/expected/multi_insert_select_conflict.out +++ b/src/test/regress/expected/multi_insert_select_conflict.out @@ -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. @@ -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 --------------------------------------------------------------------- @@ -395,7 +395,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 --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_router_planner_fast_path.out b/src/test/regress/expected/multi_router_planner_fast_path.out index 4a9ad5e73..e89122ec7 100644 --- a/src/test/regress/expected/multi_router_planner_fast_path.out +++ b/src/test/regress/expected/multi_router_planner_fast_path.out @@ -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 diff --git a/src/test/regress/expected/with_dml.out b/src/test/regress/expected/with_dml.out index 2367e602a..766d4cece 100644 --- a/src/test/regress/expected/with_dml.out +++ b/src/test/regress/expected/with_dml.out @@ -96,7 +96,6 @@ 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) diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index da07796f5..72fc4d956 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -6,7 +6,6 @@ SET citus.next_shard_id TO 4213581; SET citus.shard_replication_factor TO 1; SET citus.replication_model TO 'streaming'; --- Test 1 -- 4 shards, hash distributed. -- Negate distribution column value. SET citus.shard_count TO 4; @@ -25,8 +24,8 @@ SELECT * FROM target_table WHERE a=-1 OR a=-3 OR a=-7 ORDER BY a; DROP TABLE source_table, target_table; -- --- Test 2. -- range partitioning, composite distribution column +-- CREATE TYPE composite_key_type AS (f1 int, f2 text); -- source @@ -82,13 +81,15 @@ 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); +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),(2,2,2),(3,3,3),(4,4,4),(5,5,5); +INSERT INTO source_table VALUES (1,1,1), (3,3,3), (5,5,5); SET client_min_messages TO DEBUG2; INSERT INTO target_table @@ -103,5 +104,26 @@ 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 DEBUG2; +INSERT INTO target_table SELECT mapped_key, c FROM source_table; +RESET client_min_messages; + +SELECT * FROM target_table ORDER BY a; + +DROP TABLE source_table, target_table; + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; From 42c3c03b85653eb579ec2cbc749ad9d595e9205e Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Sun, 12 Jan 2020 17:56:49 -0800 Subject: [PATCH 06/22] Handle extra columns added in ExpandWorkerTargetEntry() in repartitioned INSERT/SELECT --- .../executor/insert_select_executor.c | 52 +++++++++++++++++++ src/backend/distributed/utils/listutils.c | 26 ++++++++++ src/include/distributed/listutils.h | 1 + .../expected/insert_select_repartition.out | 36 +++++++++++++ .../regress/sql/insert_select_repartition.sql | 21 ++++++++ 5 files changed, 136 insertions(+) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 49c0db137..84ddca0d0 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -78,6 +78,7 @@ static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn); static bool IsRedistributablePlan(Plan *selectPlan, bool hasReturning); static Expr * CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation, int targetTypeMod); +static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries); /* @@ -233,6 +234,19 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) 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, @@ -1038,3 +1052,41 @@ IsRedistributablePlan(Plan *selectPlan, bool hasReturning) return true; } + + +/* + * WrapForProjection 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; + } +} diff --git a/src/backend/distributed/utils/listutils.c b/src/backend/distributed/utils/listutils.c index 6fffe1c4f..a94beab91 100644 --- a/src/backend/distributed/utils/listutils.c +++ b/src/backend/distributed/utils/listutils.c @@ -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; +} diff --git a/src/include/distributed/listutils.h b/src/include/distributed/listutils.h index 0190a7bea..891be1fe4 100644 --- a/src/include/distributed/listutils.h +++ b/src/include/distributed/listutils.h @@ -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 */ diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index e33e7a527..af45576b5 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -268,6 +268,42 @@ SELECT * FROM target_table ORDER BY a; -1 | {1,2,3} (4 rows) +-- +-- worker queries 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 +LOG: 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 +LOG: 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 +LOG: 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 DEBUG2; +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: Router planner cannot handle multi-shard select queries +DEBUG: performing repartitioned INSERT ... SELECT +DEBUG: partitioning SELECT query by column index 0 with name 'max' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213604 AS citus_table_alias (a, b) SELECT max, array_agg FROM read_intermediate_results('{repartitioned_results_from_4213602_to_0,repartitioned_results_from_4213603_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(max integer, array_agg integer[]) +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213605 AS citus_table_alias (a, b) SELECT max, array_agg FROM read_intermediate_results('{repartitioned_results_from_4213601_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(max integer, array_agg integer[]) +RESET client_min_messages; +SELECT * FROM target_table ORDER BY a; + a | b +--------------------------------------------------------------------- + -4 | {-4} + -3 | {-3} + -2 | {-2} + -1 | {-1} +(4 rows) + DROP TABLE source_table, target_table; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 72fc4d956..ef60350a9 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -123,6 +123,27 @@ RESET client_min_messages; SELECT * FROM target_table ORDER BY a; +-- +-- worker queries 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 DEBUG2; +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; + DROP TABLE source_table, target_table; SET client_min_messages TO WARNING; From 44a2aede16ee762263d626b3cb374f7a0e6223d5 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Mon, 13 Jan 2020 16:52:15 -0800 Subject: [PATCH 07/22] Don't start a coordinated transaction on workers. Otherwise transaction hooks of Citus kick in and might cause unwanted errors. --- .../executor/insert_select_executor.c | 19 ++-- .../executor/intermediate_results.c | 11 +- .../partitioned_intermediate_results.c | 2 +- .../transaction/remote_transaction.c | 61 ++++++---- .../transaction/transaction_management.c | 27 ++--- .../distributed/transaction_management.h | 9 +- .../expected/insert_select_repartition.out | 107 +++++++++++++++++- .../regress/sql/insert_select_repartition.sql | 44 ++++++- 8 files changed, 214 insertions(+), 66 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 84ddca0d0..97b31c9cf 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -267,17 +267,11 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) scanState->tuplestorestate = tuplestore_begin_heap(randomAccess, interTransactions, work_mem); - TransactionProperties xactProperties = { - .errorOnAnyFailure = true, - .useRemoteTransactionBlocks = TRANSACTION_BLOCKS_REQUIRED, - .requires2PC = false - }; - int64 rowsInserted = ExecuteTaskListExtended(ROW_MODIFY_COMMUTATIVE, taskList, - tupleDescriptor, - scanState->tuplestorestate, - hasReturning, - MaxAdaptiveExecutorPoolSize, - &xactProperties); + uint64 rowsInserted = ExecuteTaskListIntoTupleStore(ROW_MODIFY_COMMUTATIVE, + taskList, + tupleDescriptor, + scanState->tuplestorestate, + hasReturning); executorState->es_processed = rowsInserted; } @@ -981,6 +975,7 @@ RedistributedInsertSelectTaskList(Query *insertSelectQuery, Task *modifyTask = CreateBasicTask(jobId, taskIdIndex, MODIFY_TASK, queryString->data); + modifyTask->dependentTaskList = NIL; modifyTask->anchorShardId = shardId; modifyTask->taskPlacementList = insertShardPlacementList; modifyTask->relationShardList = list_make1(relationShard); @@ -1043,7 +1038,7 @@ IsRedistributablePlan(Plan *selectPlan, bool hasReturning) /* * 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 case too. + * call. It doesn't hurt much in other cases too. */ if (list_length(distSelectTaskList) <= 1) { diff --git a/src/backend/distributed/executor/intermediate_results.c b/src/backend/distributed/executor/intermediate_results.c index 4b7197363..c1b06e30b 100644 --- a/src/backend/distributed/executor/intermediate_results.c +++ b/src/backend/distributed/executor/intermediate_results.c @@ -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); } diff --git a/src/backend/distributed/executor/partitioned_intermediate_results.c b/src/backend/distributed/executor/partitioned_intermediate_results.c index a111a68fb..d1083f8fa 100644 --- a/src/backend/distributed/executor/partitioned_intermediate_results.c +++ b/src/backend/distributed/executor/partitioned_intermediate_results.c @@ -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(); diff --git a/src/backend/distributed/transaction/remote_transaction.c b/src/backend/distributed/transaction/remote_transaction.c index 8ca78791d..34e1cb15f 100644 --- a/src/backend/distributed/transaction/remote_transaction.c +++ b/src/backend/distributed/transaction/remote_transaction.c @@ -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). diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c index e86e5ac55..b94057da6 100644 --- a/src/backend/distributed/transaction/transaction_management.c +++ b/src/backend/distributed/transaction/transaction_management.c @@ -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(); } } diff --git a/src/include/distributed/transaction_management.h b/src/include/distributed/transaction_management.h index f1e9ece90..43c1e4d02 100644 --- a/src/include/distributed/transaction_management.h +++ b/src/include/distributed/transaction_management.h @@ -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 */ diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index af45576b5..68d1f9271 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -269,8 +269,8 @@ SELECT * FROM target_table ORDER BY a; (4 rows) -- --- worker queries have more columns than necessary. ExpandWorkerTargetEntry() might --- add additional columns to the target list. +-- worker queries can have more columns than necessary. ExpandWorkerTargetEntry() +-- might add additional columns to the target list. -- TRUNCATE target_table; \set VERBOSITY TERSE @@ -304,6 +304,107 @@ SELECT * FROM target_table ORDER BY a; -1 | {-1} (4 rows) -DROP TABLE source_table, target_table; +-- +-- repartitioned INSERT/SELECT followed by other DML in stame 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) + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index ef60350a9..db6cee7dd 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -124,8 +124,8 @@ RESET client_min_messages; SELECT * FROM target_table ORDER BY a; -- --- worker queries have more columns than necessary. ExpandWorkerTargetEntry() might --- add additional columns to the target list. +-- worker queries can have more columns than necessary. ExpandWorkerTargetEntry() +-- might add additional columns to the target list. -- TRUNCATE target_table; \set VERBOSITY TERSE @@ -144,7 +144,45 @@ RESET client_min_messages; SELECT * FROM target_table ORDER BY a; -DROP TABLE source_table, target_table; +-- +-- repartitioned INSERT/SELECT followed by other DML in stame 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; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; From 4b14347fc321cf51f8dc774e0fa78d0c04998a50 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Mon, 13 Jan 2020 17:17:44 -0800 Subject: [PATCH 08/22] Tests for DML followed by insert/select repartition --- .../expected/insert_select_repartition.out | 74 ++++++++++++++++++- .../regress/sql/insert_select_repartition.sql | 35 ++++++++- 2 files changed, 107 insertions(+), 2 deletions(-) diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 68d1f9271..1107aa8a3 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -305,7 +305,7 @@ SELECT * FROM target_table ORDER BY a; (4 rows) -- --- repartitioned INSERT/SELECT followed by other DML in stame transaction +-- repartitioned INSERT/SELECT followed/preceded by other DML in same transaction -- -- case 1. followed by DELETE TRUNCATE target_table; @@ -406,5 +406,77 @@ SELECT * FROM target_table ORDER BY a; -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) + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index db6cee7dd..6d296cf68 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -145,7 +145,7 @@ RESET client_min_messages; SELECT * FROM target_table ORDER BY a; -- --- repartitioned INSERT/SELECT followed by other DML in stame transaction +-- repartitioned INSERT/SELECT followed/preceded by other DML in same transaction -- -- case 1. followed by DELETE @@ -184,5 +184,38 @@ 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; + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; From 494cc383ccb09600b7cd564e9b1c2448e26417f4 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Mon, 13 Jan 2020 17:54:40 -0800 Subject: [PATCH 09/22] Repartitioned INSERT/SELECT: Enable RETURNING --- .../executor/insert_select_executor.c | 11 ++----- .../expected/insert_select_repartition.out | 30 +++++++++++++++++++ .../expected/multi_insert_select_conflict.out | 6 ++-- .../regress/sql/insert_select_repartition.sql | 12 ++++++++ 4 files changed, 48 insertions(+), 11 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 97b31c9cf..dc685afa4 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -75,7 +75,7 @@ static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery, List **redistributedResults, bool useBinaryFormat); static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn); -static bool IsRedistributablePlan(Plan *selectPlan, bool hasReturning); +static bool IsRedistributablePlan(Plan *selectPlan); static Expr * CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation, int targetTypeMod); static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries); @@ -180,7 +180,7 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) LockPartitionRelations(targetRelationId, RowExclusiveLock); } - if (IsRedistributablePlan(selectPlan->planTree, hasReturning) && + if (IsRedistributablePlan(selectPlan->planTree) && IsSupportedRedistributionTarget(targetRelationId)) { ereport(DEBUG1, (errmsg("performing repartitioned INSERT ... SELECT"))); @@ -1017,13 +1017,8 @@ PartitionColumnIndex(List *insertTargetList, Var *partitionColumn) * IsRedistributablePlan returns true if the given plan is a redistrituable plan. */ static bool -IsRedistributablePlan(Plan *selectPlan, bool hasReturning) +IsRedistributablePlan(Plan *selectPlan) { - if (hasReturning) - { - return false; - } - /* don't redistribute if query is not distributed or requires merge on coordinator */ if (!IsCitusCustomScan(selectPlan)) { diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 1107aa8a3..ab69137f7 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -478,5 +478,35 @@ SELECT * FROM target_table ORDER BY a; -1 | {1,2,3} (12 rows) +-- +-- repartitioned INSERT/SELECT with RETURNING +-- +TRUNCATE target_table; +SET client_min_messages TO DEBUG2; +WITH c AS ( + INSERT INTO target_table + SELECT mapped_key, c FROM source_table + RETURNING *) +SELECT * FROM c ORDER by a; +DEBUG: data-modifying statements are not supported in the WITH clauses of distributed queries +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: Creating router plan +DEBUG: Plan is router executable +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: performing repartitioned INSERT ... SELECT +DEBUG: partitioning SELECT query by column index 0 with name 'mapped_key' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213604 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213602_to_0,repartitioned_results_from_4213603_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) RETURNING citus_table_alias.a, citus_table_alias.b +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213605 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213601_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) RETURNING citus_table_alias.a, citus_table_alias.b + a | b +--------------------------------------------------------------------- + -4 | {3} + -3 | {} + -2 | {4,6} + -1 | {1,2,3} +(4 rows) + +RESET client_min_messages; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/expected/multi_insert_select_conflict.out b/src/test/regress/expected/multi_insert_select_conflict.out index 4cbed96ce..329763333 100644 --- a/src/test/regress/expected/multi_insert_select_conflict.out +++ b/src/test/regress/expected/multi_insert_select_conflict.out @@ -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 @@ -324,11 +324,11 @@ BEGIN; FROM source_table_1 ON CONFLICT (col_1) DO UPDATE SET col_2 = 1 RETURNING *; col_1 | col_2 --------------------------------------------------------------------- - 1 | 1 - 2 | 1 3 | 1 4 | 1 + 1 | 1 5 | 1 + 2 | 1 (5 rows) ROLLBACK; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 6d296cf68..7e0bc0cb2 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -217,5 +217,17 @@ 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 DEBUG2; +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; + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; From fe548b762fa70b0015e0db5e4e06b460fd228295 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Mon, 13 Jan 2020 18:19:46 -0800 Subject: [PATCH 10/22] Repartitioned INSERT/SELECT: Test CTEs --- .../executor/insert_select_executor.c | 9 ++-- .../expected/insert_select_repartition.out | 49 ++++++++++++------- .../regress/sql/insert_select_repartition.sql | 24 +++++++-- 3 files changed, 56 insertions(+), 26 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index dc685afa4..b52847df0 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -149,9 +149,8 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) ReorderInsertSelectTargetLists(insertSelectQuery, insertRte, selectRte); /* - * 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 + * Cast types of insert target list and select projection list to + * match the column types of the target relation. */ selectQuery->targetList = AddInsertSelectCasts(insertSelectQuery->targetList, @@ -1045,8 +1044,8 @@ IsRedistributablePlan(Plan *selectPlan) /* - * WrapForProjection wraps task->queryString to only select given projected - * columns. It modifies the taskList. + * WrapTaskListForProjection wraps task->queryString to only select given + * projected columns. It modifies the taskList. */ static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries) diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index ab69137f7..9a2fd0396 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -249,15 +249,11 @@ SELECT create_distributed_table('target_table', 'a'); (1 row) -SET client_min_messages TO DEBUG2; +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'mapped_key' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213604 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213602_to_0,repartitioned_results_from_4213603_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213605 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213601_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) RESET client_min_messages; SELECT * FROM target_table ORDER BY a; a | b @@ -286,14 +282,10 @@ 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 DEBUG2; +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'max' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213604 AS citus_table_alias (a, b) SELECT max, array_agg FROM read_intermediate_results('{repartitioned_results_from_4213602_to_0,repartitioned_results_from_4213603_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(max integer, array_agg integer[]) -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213605 AS citus_table_alias (a, b) SELECT max, array_agg FROM read_intermediate_results('{repartitioned_results_from_4213601_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(max integer, array_agg integer[]) RESET client_min_messages; SELECT * FROM target_table ORDER BY a; a | b @@ -482,23 +474,16 @@ SELECT * FROM target_table ORDER BY a; -- repartitioned INSERT/SELECT with RETURNING -- TRUNCATE target_table; -SET client_min_messages TO DEBUG2; +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: data-modifying statements are not supported in the WITH clauses of distributed queries 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: Creating router plan -DEBUG: Plan is router executable -DEBUG: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'mapped_key' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213604 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213602_to_0,repartitioned_results_from_4213603_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) RETURNING citus_table_alias.a, citus_table_alias.b -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213605 AS citus_table_alias (a, b) SELECT mapped_key, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213601_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(mapped_key integer, auto_coerced_by_citus_1 integer[]) RETURNING citus_table_alias.a, citus_table_alias.b a | b --------------------------------------------------------------------- -4 | {3} @@ -508,5 +493,33 @@ DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_tabl (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, (c)::integer[] AS auto_coerced_by_citus_1 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) + +-- +-- The case where select query has a GROUP BY ... +-- SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 7e0bc0cb2..7dbfd0f76 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -117,7 +117,7 @@ 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 DEBUG2; +SET client_min_messages TO DEBUG1; INSERT INTO target_table SELECT mapped_key, c FROM source_table; RESET client_min_messages; @@ -138,7 +138,7 @@ 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 DEBUG2; +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; @@ -221,7 +221,7 @@ SELECT * FROM target_table ORDER BY a; -- repartitioned INSERT/SELECT with RETURNING -- TRUNCATE target_table; -SET client_min_messages TO DEBUG2; +SET client_min_messages TO DEBUG1; WITH c AS ( INSERT INTO target_table SELECT mapped_key, c FROM source_table @@ -229,5 +229,23 @@ WITH c AS ( 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; + +-- +-- The case where select query has a GROUP BY ... +-- + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; From b143d9588a6b9c55d6b13d5addc08c4d1767994d Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Mon, 13 Jan 2020 19:24:06 -0800 Subject: [PATCH 11/22] Repartitioned INSERT/SELECT: Test GROUP BY --- .../expected/insert_select_repartition.out | 35 +++++++++++++++++++ .../expected/multi_insert_select_conflict.out | 18 +++++----- .../regress/sql/insert_select_repartition.sql | 20 +++++++++++ .../sql/multi_insert_select_conflict.sql | 14 ++++---- 4 files changed, 73 insertions(+), 14 deletions(-) diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 9a2fd0396..6cae63c01 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -518,8 +518,43 @@ SELECT * FROM target_table ORDER BY a; -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) + +DROP TABLE source_table, target_table; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/expected/multi_insert_select_conflict.out b/src/test/regress/expected/multi_insert_select_conflict.out index 329763333..400f60669 100644 --- a/src/test/regress/expected/multi_insert_select_conflict.out +++ b/src/test/regress/expected/multi_insert_select_conflict.out @@ -316,19 +316,21 @@ 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 + 2 | 1 3 | 1 4 | 1 - 1 | 1 5 | 1 - 2 | 1 (5 rows) ROLLBACK; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 7dbfd0f76..ad432162b 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -243,9 +243,29 @@ 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; + +DROP TABLE source_table, target_table; + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/multi_insert_select_conflict.sql b/src/test/regress/sql/multi_insert_select_conflict.sql index 08619d9e1..9e5e3cb34 100644 --- a/src/test/regress/sql/multi_insert_select_conflict.sql +++ b/src/test/regress/sql/multi_insert_select_conflict.sql @@ -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 From 97072c9eb18bdc131d0eb1c7ddef3a138e5611ed Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Tue, 14 Jan 2020 11:01:28 -0800 Subject: [PATCH 12/22] INSERT/SELECT: show method in EXPLAIN output --- .../distributed/executor/citus_custom_scan.c | 2 +- .../executor/insert_select_executor.c | 8 +++---- .../distributed/planner/multi_explain.c | 15 ++++++++++++ .../distributed/insert_select_executor.h | 2 ++ .../expected/insert_select_repartition.out | 24 ++++++++++++++++--- src/test/regress/expected/multi_explain.out | 15 ++++++++---- .../regress/sql/insert_select_repartition.sql | 7 ++++++ 7 files changed, 59 insertions(+), 14 deletions(-) diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 3f46161ae..4510e83ee 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -62,7 +62,7 @@ CustomScanMethods TaskTrackerCustomScanMethods = { }; CustomScanMethods CoordinatorInsertSelectCustomScanMethods = { - "Citus INSERT ... SELECT via coordinator", + "Citus INSERT ... SELECT", CoordinatorInsertSelectCreateScan }; diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index b52847df0..f4b1ac718 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -69,13 +69,11 @@ static List * BuildColumnNameListFromTargetList(Oid targetRelationId, static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList); static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, Oid targetRelationId); -static bool IsSupportedRedistributionTarget(Oid targetRelationId); static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery, DistTableCacheEntry *targetRelation, List **redistributedResults, bool useBinaryFormat); static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn); -static bool IsRedistributablePlan(Plan *selectPlan); static Expr * CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation, int targetTypeMod); static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries); @@ -873,7 +871,7 @@ CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation, * IsSupportedRedistributionTarget determines whether re-partitioning into the * given target relation is supported. */ -static bool +bool IsSupportedRedistributionTarget(Oid targetRelationId) { DistTableCacheEntry *tableEntry = DistributedTableCacheEntry(targetRelationId); @@ -966,7 +964,7 @@ RedistributedInsertSelectTaskList(Query *insertSelectQuery, ereport(DEBUG2, (errmsg("distributed statement: %s", queryString->data))); LockShardDistributionMetadata(shardId, ShareLock); - List *insertShardPlacementList = FinalizedShardPlacementList(shardId); + List *insertShardPlacementList = ActiveShardPlacementList(shardId); RelationShard *relationShard = CitusMakeNode(RelationShard); relationShard->relationId = targetShardInterval->relationId; @@ -1015,7 +1013,7 @@ PartitionColumnIndex(List *insertTargetList, Var *partitionColumn) /* * IsRedistributablePlan returns true if the given plan is a redistrituable plan. */ -static bool +bool IsRedistributablePlan(Plan *selectPlan) { /* don't redistribute if query is not distributed or requires merge on coordinator */ diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index 7a527269a..ab86b4aa2 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -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,17 @@ CoordinatorInsertSelectExplainScan(CustomScanState *node, List *ancestors, "... SELECT commands via the coordinator"))); } + PlannedStmt *selectPlan = pg_plan_query(query, 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 */ diff --git a/src/include/distributed/insert_select_executor.h b/src/include/distributed/insert_select_executor.h index eda3d6679..a2101ee1a 100644 --- a/src/include/distributed/insert_select_executor.h +++ b/src/include/distributed/insert_select_executor.h @@ -20,6 +20,8 @@ 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 */ diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 6cae63c01..792f66dbf 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -275,9 +275,9 @@ 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 -LOG: 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 -LOG: 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 -LOG: 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 +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 @@ -555,6 +555,24 @@ SELECT * FROM target_table ORDER BY a; 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) + DROP TABLE source_table, target_table; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/expected/multi_explain.out b/src/test/regress/expected/multi_explain.out index 9da8e24e0..ad633c38f 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -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 diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index ad432162b..1c1632fb5 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -265,6 +265,13 @@ 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; + + DROP TABLE source_table, target_table; SET client_min_messages TO WARNING; From af2349f21f91fddf9c3f3c05168a65ca63a69eed Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Tue, 14 Jan 2020 11:42:22 -0800 Subject: [PATCH 13/22] Repartitioned INSERT/SELECT: Add a prepared statement test --- .../expected/insert_select_repartition.out | 56 +++++++++++++++++++ .../regress/sql/insert_select_repartition.sql | 27 +++++++++ 2 files changed, 83 insertions(+) diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 792f66dbf..93b978009 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -573,6 +573,62 @@ EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a; -> Seq Scan on source_table_4213606 source_table (cost=0.00..32.60 rows=2260 width=8) (10 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) + DROP TABLE source_table, target_table; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 1c1632fb5..7c02c4f47 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -271,6 +271,33 @@ SELECT * FROM target_table ORDER BY a; EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP 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; + DROP TABLE source_table, target_table; From 665b33dca1c093640e43fe9535c68935c4eea066 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Tue, 14 Jan 2020 13:58:26 -0800 Subject: [PATCH 14/22] MX tests for INSERT/SELECT repartition --- .../multi_mx_insert_select_repartition.out | 96 +++++++++++++++++++ src/test/regress/multi_mx_schedule | 1 + .../multi_mx_insert_select_repartition.sql | 50 ++++++++++ 3 files changed, 147 insertions(+) create mode 100644 src/test/regress/expected/multi_mx_insert_select_repartition.out create mode 100644 src/test/regress/sql/multi_mx_insert_select_repartition.sql diff --git a/src/test/regress/expected/multi_mx_insert_select_repartition.out b/src/test/regress/expected/multi_mx_insert_select_repartition.out new file mode 100644 index 000000000..5cbf44293 --- /dev/null +++ b/src/test/regress/expected/multi_mx_insert_select_repartition.out @@ -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 diff --git a/src/test/regress/multi_mx_schedule b/src/test/regress/multi_mx_schedule index 74406ec49..8ee00809f 100644 --- a/src/test/regress/multi_mx_schedule +++ b/src/test/regress/multi_mx_schedule @@ -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 diff --git a/src/test/regress/sql/multi_mx_insert_select_repartition.sql b/src/test/regress/sql/multi_mx_insert_select_repartition.sql new file mode 100644 index 000000000..845b8b0c3 --- /dev/null +++ b/src/test/regress/sql/multi_mx_insert_select_repartition.sql @@ -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; From 43218eebf62612bbfbe250851b3222a200227f5c Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Tue, 14 Jan 2020 14:55:05 -0800 Subject: [PATCH 15/22] Failure tests for INSERT/SELECT repartition --- .../failure_insert_select_repartition.out | 200 ++++++++++++++++++ src/test/regress/failure_schedule | 1 + .../sql/failure_insert_select_repartition.sql | 96 +++++++++ 3 files changed, 297 insertions(+) create mode 100644 src/test/regress/expected/failure_insert_select_repartition.out create mode 100644 src/test/regress/sql/failure_insert_select_repartition.sql diff --git a/src/test/regress/expected/failure_insert_select_repartition.out b/src/test/regress/expected/failure_insert_select_repartition.out new file mode 100644 index 000000000..a0d91e0ef --- /dev/null +++ b/src/test/regress/expected/failure_insert_select_repartition.out @@ -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 diff --git a/src/test/regress/failure_schedule b/src/test/regress/failure_schedule index ddaff3a8c..0e2143370 100644 --- a/src/test/regress/failure_schedule +++ b/src/test/regress/failure_schedule @@ -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 diff --git a/src/test/regress/sql/failure_insert_select_repartition.sql b/src/test/regress/sql/failure_insert_select_repartition.sql new file mode 100644 index 000000000..1f6eb1812 --- /dev/null +++ b/src/test/regress/sql/failure_insert_select_repartition.sql @@ -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; From 8635396ceab30edd03cf904473760bb50e698289 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Wed, 15 Jan 2020 12:14:36 -0800 Subject: [PATCH 16/22] Repartitioned INSERT/SELECT: Test rollback behaviour --- .../distributed/planner/multi_explain.c | 7 +- .../expected/insert_select_repartition.out | 127 ++++++++++++++++++ .../regress/sql/insert_select_repartition.sql | 60 +++++++++ 3 files changed, 193 insertions(+), 1 deletion(-) diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index ab86b4aa2..11ba065ac 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -154,7 +154,12 @@ CoordinatorInsertSelectExplainScan(CustomScanState *node, List *ancestors, "... SELECT commands via the coordinator"))); } - PlannedStmt *selectPlan = pg_plan_query(query, cursorOptions, params); + /* + * 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)) { diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 93b978009..0c246286e 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -629,6 +629,133 @@ SELECT a, count(*), count(distinct b) distinct_values FROM target_table GROUP BY 4 | 6 | 1 (5 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 'auto_coerced_by_citus_0' +DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213617 AS citus_table_alias (a, b) SELECT auto_coerced_by_citus_0, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213613_to_0,repartitioned_results_from_4213614_to_0,repartitioned_results_from_4213615_to_0,repartitioned_results_from_4213616_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer, auto_coerced_by_citus_1 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; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 7c02c4f47..342d3c07f 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -299,6 +299,66 @@ RESET client_min_messages; SELECT a, count(*), count(distinct b) distinct_values FROM target_table GROUP BY a ORDER BY a; +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; SET client_min_messages TO WARNING; From 8b27a9a195e42882d6ff1eeee7b252542a5d81cf Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Wed, 15 Jan 2020 15:18:20 -0800 Subject: [PATCH 17/22] More range partitioned tests --- .../expected/insert_select_repartition.out | 55 +++++++++++++++++++ .../regress/sql/insert_select_repartition.sql | 23 ++++++++ 2 files changed, 78 insertions(+) diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 0c246286e..bf906e656 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -756,6 +756,61 @@ SELECT max(result) FROM run_command_on_placements('target_table', 'select count( 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; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 342d3c07f..7d8273444 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -357,7 +357,30 @@ 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; From 3258d87f3e2f8e69006b97d54ecfad0d9bdb932a Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Wed, 15 Jan 2020 17:55:44 -0800 Subject: [PATCH 18/22] Isolation tests for INSERT/SELECT repartition --- .../isolation_insert_select_repartition.out | 249 ++++++++++++++++++ src/test/regress/isolation_schedule | 1 + .../isolation_insert_select_repartition.spec | 54 ++++ 3 files changed, 304 insertions(+) create mode 100644 src/test/regress/expected/isolation_insert_select_repartition.out create mode 100644 src/test/regress/spec/isolation_insert_select_repartition.spec diff --git a/src/test/regress/expected/isolation_insert_select_repartition.out b/src/test/regress/expected/isolation_insert_select_repartition.out new file mode 100644 index 000000000..ab2579021 --- /dev/null +++ b/src/test/regress/expected/isolation_insert_select_repartition.out @@ -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; +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; +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; +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; +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 diff --git a/src/test/regress/isolation_schedule b/src/test/regress/isolation_schedule index a4bb53818..91f0a8065 100644 --- a/src/test/regress/isolation_schedule +++ b/src/test/regress/isolation_schedule @@ -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 diff --git a/src/test/regress/spec/isolation_insert_select_repartition.spec b/src/test/regress/spec/isolation_insert_select_repartition.spec new file mode 100644 index 000000000..3e2eeeb30 --- /dev/null +++ b/src/test/regress/spec/isolation_insert_select_repartition.spec @@ -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" From ce5eea488503ed686222891a2ee63d1e74595dbf Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Thu, 16 Jan 2020 13:11:31 -0800 Subject: [PATCH 19/22] INSERT/SELECT: make SELECT column names unique --- .../executor/insert_select_executor.c | 35 ++++++ src/test/regress/expected/cte_inline.out | 3 +- .../expected/insert_select_repartition.out | 110 ++++++++++++++---- .../regress/expected/multi_insert_select.out | 56 ++++----- .../regress/sql/insert_select_repartition.sql | 36 ++++++ 5 files changed, 188 insertions(+), 52 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index f4b1ac718..bc9dfdc9f 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -77,6 +77,7 @@ 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); /* @@ -155,6 +156,14 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) 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 @@ -753,6 +762,12 @@ AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, if (selectEntry->ressortgroupref != 0) { + /* make sure that the name doesn't match any insert target list entries */ + resnameString = makeStringInfo(); + appendStringInfo(resnameString, "auto_resjunked_by_citus_%d", + targetEntryIndex); + + selectEntry->resname = resnameString->data; selectEntry->resjunk = true; nonProjectedEntries = lappend(nonProjectedEntries, selectEntry); } @@ -1077,3 +1092,23 @@ WrapTaskListForProjection(List *taskList, List *projectedTargetEntries) 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; + } +} diff --git a/src/test/regress/expected/cte_inline.out b/src/test/regress/expected/cte_inline.out index e6d2145e3..a170fa927 100644 --- a/src/test/regress/expected/cte_inline.out +++ b/src/test/regress/expected/cte_inline.out @@ -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 diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index bf906e656..72202661e 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -29,11 +29,11 @@ DETAIL: Subquery contains an operator in the same position as the target table' 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 '?column?' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213585 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213583_to_0,repartitioned_results_from_4213584_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213586 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213582_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213587 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213581_to_2,repartitioned_results_from_4213582_to_2,repartitioned_results_from_4213584_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213588 AS citus_table_alias (a) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_4213581_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +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_from_4213583_to_0,repartitioned_results_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_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_from_4213581_to_2,repartitioned_results_from_4213582_to_2,repartitioned_results_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_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 @@ -83,9 +83,9 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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 'mapped_key' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0,repartitioned_results_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_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, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_key insert_select_repartition.composite_key_type) +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_from_4213589_to_0,repartitioned_results_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_from_4213589_to_1,repartitioned_results_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 @@ -113,9 +113,9 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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 'mapped_key' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0,repartitioned_results_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_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, mapped_key FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, mapped_key insert_select_repartition.composite_key_type) +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_from_4213589_to_0,repartitioned_results_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_from_4213589_to_1,repartitioned_results_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 @@ -137,9 +137,9 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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_renamed' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0,repartitioned_results_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed 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_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed insert_select_repartition.composite_key_type) +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_from_4213589_to_0,repartitioned_results_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_from_4213589_to_1,repartitioned_results_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 @@ -163,9 +163,9 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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_renamed' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed 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_renamed FROM read_intermediate_results('{repartitioned_results_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key_renamed insert_select_repartition.composite_key_type) ON CONFLICT(key) DO UPDATE SET f1 = 1 +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_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_from_4213589_to_1,repartitioned_results_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 @@ -213,9 +213,9 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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 'auto_coerced_by_citus_0' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213593 AS citus_table_alias (col_1, col_2) SELECT auto_coerced_by_citus_0, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213597_to_0,repartitioned_results_from_4213600_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer, auto_coerced_by_citus_1 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 auto_coerced_by_citus_0, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213599_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer, auto_coerced_by_citus_1 integer) ON CONFLICT(col_1) DO UPDATE SET col_2 = excluded.col_2 +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_from_4213597_to_0,repartitioned_results_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_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 @@ -506,7 +506,7 @@ 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, (c)::integer[] AS auto_coerced_by_citus_1 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: 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; @@ -573,6 +573,32 @@ EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY 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_from_4213606_to_0,repartitioned_results_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_from_4213607_to_1,repartitioned_results_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_from_4213606_to_2,repartitioned_results_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 -- @@ -726,8 +752,8 @@ INSERT INTO target_table SELECT 1.12, b::bigint FROM source_table WHERE b IS NOT 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 'auto_coerced_by_citus_0' -DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213617 AS citus_table_alias (a, b) SELECT auto_coerced_by_citus_0, auto_coerced_by_citus_1 FROM read_intermediate_results('{repartitioned_results_from_4213613_to_0,repartitioned_results_from_4213614_to_0,repartitioned_results_from_4213615_to_0,repartitioned_results_from_4213616_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer, auto_coerced_by_citus_1 integer) +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_from_4213613_to_0,repartitioned_results_from_4213614_to_0,repartitioned_results_from_4213615_to_0,repartitioned_results_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 @@ -811,6 +837,44 @@ SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s 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_from_4213629_to_0,repartitioned_results_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_from_4213630_to_1,repartitioned_results_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_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) + DROP TABLE source_table, target_table; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index b50514ba9..49758c713 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -696,7 +696,7 @@ DEBUG: distributed INSERT ... SELECT can only select from distributed tables 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 @@ -1083,7 +1083,7 @@ 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 int4(id) AS auto_coerced_by_citus_0, int4(v1) AS auto_coerced_by_citus_1, int8(v4) AS auto_coerced_by_citus_2 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 @@ -1116,7 +1116,7 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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 0 with name 'value_1' +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 @@ -1128,11 +1128,11 @@ DETAIL: Subquery contains an operator in the same position as the target table' 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 '?column?' -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300000_to_0,repartitioned_results_from_13300001_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300000_to_1,repartitioned_results_from_13300001_to_1,repartitioned_results_from_13300003_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300001_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT "?column?" FROM read_intermediate_results('{repartitioned_results_from_13300000_to_3,repartitioned_results_from_13300002_to_3,repartitioned_results_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result("?column?" integer) +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_from_13300000_to_0,repartitioned_results_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_from_13300000_to_1,repartitioned_results_from_13300001_to_1,repartitioned_results_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_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_from_13300000_to_3,repartitioned_results_from_13300002_to_3,repartitioned_results_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 @@ -1142,11 +1142,11 @@ DETAIL: Subquery contains an explicit cast in the same position as the target t 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 'auto_coerced_by_citus_0' -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300000_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300001_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300002_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) -DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT auto_coerced_by_citus_0 FROM read_intermediate_results('{repartitioned_results_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(auto_coerced_by_citus_0 integer) +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_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_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_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_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer) INSERT INTO agg_events (value_3_agg, value_4_agg, @@ -1165,7 +1165,7 @@ DETAIL: Subquery contains an aggregation in the same position as the target tab 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 'auto_coerced_by_citus_0' +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 @@ -1186,7 +1186,7 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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 0 with name 'value_2' +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 @@ -1240,10 +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 int4(f2.id) AS auto_coerced_by_citus_0 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 'auto_coerced_by_citus_0' +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) @@ -1286,10 +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 'id' +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) @@ -2114,42 +2114,42 @@ DETAIL: Subquery contains a case expression in the same position as the target 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 'case' +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'coalesce' +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'greatest' +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'least' +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'nullif' +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' +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. @@ -2163,20 +2163,20 @@ DETAIL: Subquery contains an expression that is not a simple column reference i 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 'auto_coerced_by_citus_0' +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'auto_coerced_by_citus_0' +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: Router planner cannot handle multi-shard select queries DEBUG: performing repartitioned INSERT ... SELECT -DEBUG: partitioning SELECT query by column index 0 with name 'val' +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 diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 7d8273444..4a44ed31b 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -271,6 +271,19 @@ SELECT * FROM target_table ORDER BY a; 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 -- @@ -384,5 +397,28 @@ SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s 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; + +DROP TABLE source_table, target_table; + SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; From a079278b0cd4f42bf73487f2e8a8ccd3d0b01eca Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Thu, 16 Jan 2020 13:56:14 -0800 Subject: [PATCH 20/22] Repartitioned INSERT/SELECT: Add a GUC to enable/disable it --- .../executor/insert_select_executor.c | 7 +++ src/backend/distributed/shared_library_init.c | 11 +++++ .../distributed/insert_select_executor.h | 1 + src/test/regress/expected/cte_inline_0.out | 2 +- .../expected/insert_select_repartition.out | 44 +++++++++++++++++++ .../regress/sql/insert_select_repartition.sql | 18 ++++++++ 6 files changed, 82 insertions(+), 1 deletion(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index bc9dfdc9f..179f00287 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -48,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; @@ -1031,6 +1033,11 @@ PartitionColumnIndex(List *insertTargetList, Var *partitionColumn) bool IsRedistributablePlan(Plan *selectPlan) { + if (!EnableRepartitionedInsertSelect) + { + return false; + } + /* don't redistribute if query is not distributed or requires merge on coordinator */ if (!IsCitusCustomScan(selectPlan)) { diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 11cc2c565..0e9d1a712 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -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"), diff --git a/src/include/distributed/insert_select_executor.h b/src/include/distributed/insert_select_executor.h index a2101ee1a..e94c7fe9c 100644 --- a/src/include/distributed/insert_select_executor.h +++ b/src/include/distributed/insert_select_executor.h @@ -16,6 +16,7 @@ #include "executor/execdesc.h" +extern bool EnableRepartitionedInsertSelect; extern TupleTableSlot * CoordinatorInsertSelectExecScan(CustomScanState *node); extern bool ExecutingInsertSelect(void); diff --git a/src/test/regress/expected/cte_inline_0.out b/src/test/regress/expected/cte_inline_0.out index 0ec39e590..66bd359e7 100644 --- a/src/test/regress/expected/cte_inline_0.out +++ b/src/test/regress/expected/cte_inline_0.out @@ -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 diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 72202661e..6eb070560 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -875,6 +875,50 @@ SELECT count(*) FROM target_table; 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; SET client_min_messages TO WARNING; DROP SCHEMA insert_select_repartition CASCADE; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 4a44ed31b..fb80de3b1 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -418,6 +418,24 @@ 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; SET client_min_messages TO WARNING; From 5eeb07124fcd1936caa0db5a95afde58c5a1f32c Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Thu, 16 Jan 2020 15:05:17 -0800 Subject: [PATCH 21/22] Repartitioned INSERT/SELECT: include job id in result id prefix --- .../executor/insert_select_executor.c | 16 ++- src/test/regress/bin/normalize.sed | 4 + .../expected/insert_select_repartition.out | 100 ++++++++++++++---- .../regress/expected/multi_insert_select.out | 24 ++--- .../regress/sql/insert_select_repartition.sql | 14 +++ 5 files changed, 116 insertions(+), 42 deletions(-) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 179f00287..8547c21c2 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -209,10 +209,14 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node) * 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. Jobs are executed - * sequentially, so we also don't need to include job id here. + * need to include them in the filename. We include job id here for + * the case "INSERT/SELECTs" are executed recursively. */ - char *distResultPrefix = "repartitioned_results"; + StringInfo distResultPrefixString = makeStringInfo(); + appendStringInfo(distResultPrefixString, + "repartitioned_results_" UINT64_FORMAT, + distSelectJob->jobId); + char *distResultPrefix = distResultPrefixString->data; DistTableCacheEntry *targetRelation = DistributedTableCacheEntry(targetRelationId); @@ -764,12 +768,6 @@ AddInsertSelectCasts(List *insertTargetList, List *selectTargetList, if (selectEntry->ressortgroupref != 0) { - /* make sure that the name doesn't match any insert target list entries */ - resnameString = makeStringInfo(); - appendStringInfo(resnameString, "auto_resjunked_by_citus_%d", - targetEntryIndex); - - selectEntry->resname = resnameString->data; selectEntry->resjunk = true; nonProjectedEntries = lappend(nonProjectedEntries, selectEntry); } diff --git a/src/test/regress/bin/normalize.sed b/src/test/regress/bin/normalize.sed index 0f9985a73..55ad1a72c 100644 --- a/src/test/regress/bin/normalize.sed +++ b/src/test/regress/bin/normalize.sed @@ -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 + diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index 6eb070560..bc50ab93c 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -30,10 +30,10 @@ HINT: Ensure the target table's partition column has a corresponding simple col 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_from_4213583_to_0,repartitioned_results_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_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_from_4213581_to_2,repartitioned_results_from_4213582_to_2,repartitioned_results_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_from_4213581_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer) +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 @@ -84,8 +84,8 @@ DETAIL: The target table's partition column should correspond to a partition co 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_from_4213589_to_0,repartitioned_results_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_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::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_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 @@ -114,8 +114,8 @@ DETAIL: The target table's partition column should correspond to a partition co 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_from_4213589_to_0,repartitioned_results_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_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::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_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 @@ -138,8 +138,8 @@ DETAIL: The target table's partition column should correspond to a partition co 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_from_4213589_to_0,repartitioned_results_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_from_4213589_to_1,repartitioned_results_from_4213590_to_1}'::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_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 @@ -164,8 +164,8 @@ DETAIL: The target table's partition column should correspond to a partition co 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_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_from_4213589_to_1,repartitioned_results_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 +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 @@ -214,8 +214,8 @@ DETAIL: The data type of the target table's partition column should exactly mat 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_from_4213597_to_0,repartitioned_results_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_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 +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 @@ -584,9 +584,9 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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_from_4213606_to_0,repartitioned_results_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_from_4213607_to_1,repartitioned_results_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_from_4213606_to_2,repartitioned_results_from_4213607_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) +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 @@ -655,6 +655,64 @@ SELECT a, count(*), count(distinct b) distinct_values FROM target_table GROUP BY 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 @@ -753,7 +811,7 @@ DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition 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_from_4213613_to_0,repartitioned_results_from_4213614_to_0,repartitioned_results_from_4213615_to_0,repartitioned_results_from_4213616_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) +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 @@ -865,9 +923,9 @@ DEBUG: INSERT target table and the source relation of the SELECT partition colu 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_from_4213629_to_0,repartitioned_results_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_from_4213630_to_1,repartitioned_results_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_from_4213632_to_2}'::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_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 diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index 49758c713..c8398ccbb 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -726,10 +726,10 @@ DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries 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_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_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_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_from_13300007_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer) +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; @@ -1129,10 +1129,10 @@ HINT: Ensure the target table's partition column has a corresponding simple col 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_from_13300000_to_0,repartitioned_results_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_from_13300000_to_1,repartitioned_results_from_13300001_to_1,repartitioned_results_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_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_from_13300000_to_3,repartitioned_results_from_13300002_to_3,repartitioned_results_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer) +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 @@ -1143,10 +1143,10 @@ HINT: Ensure the target table's partition column has a corresponding simple col 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_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_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_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_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer) +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, diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index fb80de3b1..127980686 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -311,6 +311,20 @@ 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; From 6cf1c0166001ae8255528ea10934292e7b219050 Mon Sep 17 00:00:00 2001 From: Hadi Moshayedi Date: Thu, 16 Jan 2020 23:35:34 -0800 Subject: [PATCH 22/22] Don't use repartitioned INSERT/SELECT for repartition joins --- .../executor/insert_select_executor.c | 6 ++++++ .../expected/insert_select_repartition.out | 16 ++++++++++++++++ .../regress/sql/insert_select_repartition.sql | 12 ++++++++++++ 3 files changed, 34 insertions(+) diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 8547c21c2..14e781be5 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -1057,6 +1057,12 @@ IsRedistributablePlan(Plan *selectPlan) return false; } + /* don't use redistribution for repartition joins for now */ + if (distSelectJob->dependentJobList != NIL) + { + return false; + } + return true; } diff --git a/src/test/regress/expected/insert_select_repartition.out b/src/test/regress/expected/insert_select_repartition.out index bc50ab93c..65d374e70 100644 --- a/src/test/regress/expected/insert_select_repartition.out +++ b/src/test/regress/expected/insert_select_repartition.out @@ -978,5 +978,21 @@ EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 (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; diff --git a/src/test/regress/sql/insert_select_repartition.sql b/src/test/regress/sql/insert_select_repartition.sql index 127980686..e6f4bfa87 100644 --- a/src/test/regress/sql/insert_select_repartition.sql +++ b/src/test/regress/sql/insert_select_repartition.sql @@ -452,5 +452,17 @@ EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 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;