mirror of https://github.com/citusdata/citus.git
Merge pull request #3376 from citusdata/insert_select
INSERT...SELECT with re-partitioningpull/3350/head
commit
60a2bc5ec2
|
@ -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
|
||||
|
|
|
@ -62,7 +62,7 @@ CustomScanMethods TaskTrackerCustomScanMethods = {
|
|||
};
|
||||
|
||||
CustomScanMethods CoordinatorInsertSelectCustomScanMethods = {
|
||||
"Citus INSERT ... SELECT via coordinator",
|
||||
"Citus INSERT ... SELECT",
|
||||
CoordinatorInsertSelectCreateScan
|
||||
};
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -13,19 +13,24 @@
|
|||
|
||||
#include "distributed/citus_ruleutils.h"
|
||||
#include "distributed/commands/multi_copy.h"
|
||||
#include "distributed/adaptive_executor.h"
|
||||
#include "distributed/distributed_execution_locks.h"
|
||||
#include "distributed/insert_select_executor.h"
|
||||
#include "distributed/insert_select_planner.h"
|
||||
#include "distributed/intermediate_results.h"
|
||||
#include "distributed/local_executor.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/adaptive_executor.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/multi_router_planner.h"
|
||||
#include "distributed/distributed_planner.h"
|
||||
#include "distributed/recursive_planning.h"
|
||||
#include "distributed/relation_access_tracking.h"
|
||||
#include "distributed/resource_lock.h"
|
||||
#include "distributed/shardinterval_utils.h"
|
||||
#include "distributed/subplan_execution.h"
|
||||
#include "distributed/transaction_management.h"
|
||||
#include "executor/executor.h"
|
||||
#include "nodes/execnodes.h"
|
||||
|
@ -43,6 +48,8 @@
|
|||
#include "utils/rel.h"
|
||||
#include "utils/snapmgr.h"
|
||||
|
||||
/* Config variables managed via guc.c */
|
||||
bool EnableRepartitionedInsertSelect = true;
|
||||
|
||||
/* depth of current insert/select executor. */
|
||||
static int insertSelectExecutorLevel = 0;
|
||||
|
@ -52,17 +59,27 @@ static TupleTableSlot * CoordinatorInsertSelectExecScanInternal(CustomScanState
|
|||
static Query * WrapSubquery(Query *subquery);
|
||||
static List * TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery,
|
||||
char *resultIdPrefix);
|
||||
static void ExecuteSelectIntoRelation(Oid targetRelationId, List *insertTargetList,
|
||||
Query *selectQuery, EState *executorState);
|
||||
static HTAB * ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
|
||||
List *insertTargetList,
|
||||
Query *selectQuery,
|
||||
EState *executorState,
|
||||
char *
|
||||
intermediateResultIdPrefix);
|
||||
static void ExecutePlanIntoRelation(Oid targetRelationId, List *insertTargetList,
|
||||
PlannedStmt *selectPlan, EState *executorState);
|
||||
static HTAB * ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId,
|
||||
List *insertTargetList,
|
||||
PlannedStmt *selectPlan,
|
||||
EState *executorState,
|
||||
char *intermediateResultIdPrefix);
|
||||
static List * BuildColumnNameListFromTargetList(Oid targetRelationId,
|
||||
List *insertTargetList);
|
||||
static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameList);
|
||||
static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
|
||||
Oid targetRelationId);
|
||||
static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
List **redistributedResults,
|
||||
bool useBinaryFormat);
|
||||
static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn);
|
||||
static Expr * CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation,
|
||||
int targetTypeMod);
|
||||
static void WrapTaskListForProjection(List *taskList, List *projectedTargetEntries);
|
||||
static void RelableTargetEntryList(List *selectTargetList, List *insertTargetList);
|
||||
|
||||
|
||||
/*
|
||||
|
@ -106,6 +123,7 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
|
|||
if (!scanState->finishedRemoteScan)
|
||||
{
|
||||
EState *executorState = ScanStateGetExecutorState(scanState);
|
||||
ParamListInfo paramListInfo = executorState->es_param_list_info;
|
||||
DistributedPlan *distributedPlan = scanState->distributedPlan;
|
||||
Query *insertSelectQuery = copyObject(distributedPlan->insertSelectQuery);
|
||||
List *insertTargetList = insertSelectQuery->targetList;
|
||||
|
@ -116,9 +134,6 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
|
|||
bool hasReturning = distributedPlan->hasReturning;
|
||||
HTAB *shardStateHash = NULL;
|
||||
|
||||
ereport(DEBUG1, (errmsg("Collecting INSERT ... SELECT results on coordinator")));
|
||||
|
||||
|
||||
/*
|
||||
* INSERT .. SELECT via coordinator consists of two steps, a SELECT is
|
||||
* followd by a COPY. If the SELECT is executed locally, then the COPY
|
||||
|
@ -134,6 +149,35 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
|
|||
selectRte->subquery = selectQuery;
|
||||
ReorderInsertSelectTargetLists(insertSelectQuery, insertRte, selectRte);
|
||||
|
||||
/*
|
||||
* Cast types of insert target list and select projection list to
|
||||
* match the column types of the target relation.
|
||||
*/
|
||||
selectQuery->targetList =
|
||||
AddInsertSelectCasts(insertSelectQuery->targetList,
|
||||
selectQuery->targetList,
|
||||
targetRelationId);
|
||||
|
||||
/*
|
||||
* Later we might need to call WrapTaskListForProjection(), which requires
|
||||
* that select target list has unique names, otherwise the outer query
|
||||
* cannot select columns unambiguously. So we relabel select columns to
|
||||
* match target columns.
|
||||
*/
|
||||
RelableTargetEntryList(selectQuery->targetList, insertTargetList);
|
||||
|
||||
/*
|
||||
* Make a copy of the query, since pg_plan_query may scribble on it and we
|
||||
* want it to be replanned every time if it is stored in a prepared
|
||||
* statement.
|
||||
*/
|
||||
selectQuery = copyObject(selectQuery);
|
||||
|
||||
/* plan the subquery, this may be another distributed query */
|
||||
int cursorOptions = CURSOR_OPT_PARALLEL_OK;
|
||||
PlannedStmt *selectPlan = pg_plan_query(selectQuery, cursorOptions,
|
||||
paramListInfo);
|
||||
|
||||
/*
|
||||
* If we are dealing with partitioned table, we also need to lock its
|
||||
* partitions. Here we only lock targetRelation, we acquire necessary
|
||||
|
@ -144,8 +188,110 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
|
|||
LockPartitionRelations(targetRelationId, RowExclusiveLock);
|
||||
}
|
||||
|
||||
if (insertSelectQuery->onConflict || hasReturning)
|
||||
if (IsRedistributablePlan(selectPlan->planTree) &&
|
||||
IsSupportedRedistributionTarget(targetRelationId))
|
||||
{
|
||||
ereport(DEBUG1, (errmsg("performing repartitioned INSERT ... SELECT")));
|
||||
|
||||
DistributedPlan *distSelectPlan =
|
||||
GetDistributedPlan((CustomScan *) selectPlan->planTree);
|
||||
Job *distSelectJob = distSelectPlan->workerJob;
|
||||
List *distSelectTaskList = distSelectJob->taskList;
|
||||
TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(scanState);
|
||||
bool randomAccess = true;
|
||||
bool interTransactions = false;
|
||||
bool binaryFormat =
|
||||
CanUseBinaryCopyFormatForTargetList(selectQuery->targetList);
|
||||
|
||||
ExecuteSubPlans(distSelectPlan);
|
||||
|
||||
/*
|
||||
* We have a separate directory for each transaction, so choosing
|
||||
* the same result prefix won't cause filename conflicts. Results
|
||||
* directory name also includes node id and database id, so we don't
|
||||
* need to include them in the filename. We include job id here for
|
||||
* the case "INSERT/SELECTs" are executed recursively.
|
||||
*/
|
||||
StringInfo distResultPrefixString = makeStringInfo();
|
||||
appendStringInfo(distResultPrefixString,
|
||||
"repartitioned_results_" UINT64_FORMAT,
|
||||
distSelectJob->jobId);
|
||||
char *distResultPrefix = distResultPrefixString->data;
|
||||
|
||||
DistTableCacheEntry *targetRelation =
|
||||
DistributedTableCacheEntry(targetRelationId);
|
||||
|
||||
int partitionColumnIndex =
|
||||
PartitionColumnIndex(insertTargetList, targetRelation->partitionColumn);
|
||||
if (partitionColumnIndex == -1)
|
||||
{
|
||||
char *relationName = get_rel_name(targetRelationId);
|
||||
Oid schemaOid = get_rel_namespace(targetRelationId);
|
||||
char *schemaName = get_namespace_name(schemaOid);
|
||||
|
||||
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
|
||||
errmsg(
|
||||
"the partition column of table %s should have a value",
|
||||
quote_qualified_identifier(schemaName,
|
||||
relationName))));
|
||||
}
|
||||
|
||||
TargetEntry *selectPartitionTE = list_nth(selectQuery->targetList,
|
||||
partitionColumnIndex);
|
||||
const char *partitionColumnName = selectPartitionTE->resname ?
|
||||
selectPartitionTE->resname : "(none)";
|
||||
|
||||
ereport(DEBUG2, (errmsg(
|
||||
"partitioning SELECT query by column index %d with name %s",
|
||||
partitionColumnIndex, quote_literal_cstr(
|
||||
partitionColumnName))));
|
||||
|
||||
/*
|
||||
* ExpandWorkerTargetEntry() can add additional columns to the worker
|
||||
* query. Modify the task queries to only select columns we need.
|
||||
*/
|
||||
int requiredColumnCount = list_length(insertTargetList);
|
||||
List *jobTargetList = distSelectJob->jobQuery->targetList;
|
||||
if (list_length(jobTargetList) > requiredColumnCount)
|
||||
{
|
||||
List *projectedTargetEntries = ListTake(jobTargetList,
|
||||
requiredColumnCount);
|
||||
WrapTaskListForProjection(distSelectTaskList, projectedTargetEntries);
|
||||
}
|
||||
|
||||
List **redistributedResults = RedistributeTaskListResults(distResultPrefix,
|
||||
distSelectTaskList,
|
||||
partitionColumnIndex,
|
||||
targetRelation,
|
||||
binaryFormat);
|
||||
|
||||
/*
|
||||
* At this point select query has been executed on workers and results
|
||||
* have been fetched in such a way that they are colocated with corresponding
|
||||
* target shard. Create and execute a list of tasks of form
|
||||
* INSERT INTO ... SELECT * FROM read_intermediate_results(...);
|
||||
*/
|
||||
List *taskList = RedistributedInsertSelectTaskList(insertSelectQuery,
|
||||
targetRelation,
|
||||
redistributedResults,
|
||||
binaryFormat);
|
||||
|
||||
scanState->tuplestorestate =
|
||||
tuplestore_begin_heap(randomAccess, interTransactions, work_mem);
|
||||
|
||||
uint64 rowsInserted = ExecuteTaskListIntoTupleStore(ROW_MODIFY_COMMUTATIVE,
|
||||
taskList,
|
||||
tupleDescriptor,
|
||||
scanState->tuplestorestate,
|
||||
hasReturning);
|
||||
|
||||
executorState->es_processed = rowsInserted;
|
||||
}
|
||||
else if (insertSelectQuery->onConflict || hasReturning)
|
||||
{
|
||||
ereport(DEBUG1, (errmsg(
|
||||
"Collecting INSERT ... SELECT results on coordinator")));
|
||||
|
||||
/*
|
||||
* If we also have a workerJob that means there is a second step
|
||||
* to the INSERT...SELECT. This happens when there is a RETURNING
|
||||
|
@ -156,10 +302,10 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
|
|||
ListCell *taskCell = NULL;
|
||||
List *prunedTaskList = NIL;
|
||||
|
||||
shardStateHash = ExecuteSelectIntoColocatedIntermediateResults(
|
||||
shardStateHash = ExecutePlanIntoColocatedIntermediateResults(
|
||||
targetRelationId,
|
||||
insertTargetList,
|
||||
selectQuery,
|
||||
selectPlan,
|
||||
executorState,
|
||||
intermediateResultIdPrefix);
|
||||
|
||||
|
@ -209,8 +355,11 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
|
|||
}
|
||||
else
|
||||
{
|
||||
ExecuteSelectIntoRelation(targetRelationId, insertTargetList, selectQuery,
|
||||
executorState);
|
||||
ereport(DEBUG1, (errmsg(
|
||||
"Collecting INSERT ... SELECT results on coordinator")));
|
||||
|
||||
ExecutePlanIntoRelation(targetRelationId, insertTargetList, selectPlan,
|
||||
executorState);
|
||||
}
|
||||
|
||||
scanState->finishedRemoteScan = true;
|
||||
|
@ -342,36 +491,6 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery,
|
|||
uint32 taskIdIndex = 1;
|
||||
uint64 jobId = INVALID_JOB_ID;
|
||||
|
||||
ListCell *targetEntryCell = NULL;
|
||||
|
||||
Relation distributedRelation = heap_open(targetRelationId, RowExclusiveLock);
|
||||
TupleDesc destTupleDescriptor = RelationGetDescr(distributedRelation);
|
||||
|
||||
/*
|
||||
* If the type of insert column and target table's column type is
|
||||
* different from each other. Cast insert column't type to target
|
||||
* table's column
|
||||
*/
|
||||
foreach(targetEntryCell, insertSelectQuery->targetList)
|
||||
{
|
||||
TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell);
|
||||
Var *insertColumn = (Var *) targetEntry->expr;
|
||||
Form_pg_attribute attr = TupleDescAttr(destTupleDescriptor, targetEntry->resno -
|
||||
1);
|
||||
|
||||
if (insertColumn->vartype != attr->atttypid)
|
||||
{
|
||||
CoerceViaIO *coerceExpr = makeNode(CoerceViaIO);
|
||||
coerceExpr->arg = (Expr *) copyObject(insertColumn);
|
||||
coerceExpr->resulttype = attr->atttypid;
|
||||
coerceExpr->resultcollid = attr->attcollation;
|
||||
coerceExpr->coerceformat = COERCE_IMPLICIT_CAST;
|
||||
coerceExpr->location = -1;
|
||||
|
||||
targetEntry->expr = (Expr *) coerceExpr;
|
||||
}
|
||||
}
|
||||
|
||||
for (int shardOffset = 0; shardOffset < shardCount; shardOffset++)
|
||||
{
|
||||
ShardInterval *targetShardInterval =
|
||||
|
@ -431,24 +550,23 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery,
|
|||
taskIdIndex++;
|
||||
}
|
||||
|
||||
heap_close(distributedRelation, NoLock);
|
||||
|
||||
return taskList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ExecuteSelectIntoColocatedIntermediateResults executes the given select query
|
||||
* ExecutePlanIntoColocatedIntermediateResults executes the given PlannedStmt
|
||||
* and inserts tuples into a set of intermediate results that are colocated with
|
||||
* the target table for further processing of ON CONFLICT or RETURNING. It also
|
||||
* returns the hash of shard states that were used to insert tuplesinto the target
|
||||
* relation.
|
||||
*/
|
||||
static HTAB *
|
||||
ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
|
||||
List *insertTargetList,
|
||||
Query *selectQuery, EState *executorState,
|
||||
char *intermediateResultIdPrefix)
|
||||
ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId,
|
||||
List *insertTargetList,
|
||||
PlannedStmt *selectPlan,
|
||||
EState *executorState,
|
||||
char *intermediateResultIdPrefix)
|
||||
{
|
||||
ParamListInfo paramListInfo = executorState->es_param_list_info;
|
||||
bool stopOnFailure = false;
|
||||
|
@ -473,14 +591,7 @@ ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
|
|||
stopOnFailure,
|
||||
intermediateResultIdPrefix);
|
||||
|
||||
/*
|
||||
* Make a copy of the query, since ExecuteQueryIntoDestReceiver may scribble on it
|
||||
* and we want it to be replanned every time if it is stored in a prepared
|
||||
* statement.
|
||||
*/
|
||||
Query *queryCopy = copyObject(selectQuery);
|
||||
|
||||
ExecuteQueryIntoDestReceiver(queryCopy, paramListInfo, (DestReceiver *) copyDest);
|
||||
ExecutePlanIntoDestReceiver(selectPlan, paramListInfo, (DestReceiver *) copyDest);
|
||||
|
||||
executorState->es_processed = copyDest->tuplesSent;
|
||||
|
||||
|
@ -491,13 +602,13 @@ ExecuteSelectIntoColocatedIntermediateResults(Oid targetRelationId,
|
|||
|
||||
|
||||
/*
|
||||
* ExecuteSelectIntoRelation executes given SELECT query and inserts the
|
||||
* ExecutePlanIntoRelation executes the given plan and inserts the
|
||||
* results into the target relation, which is assumed to be a distributed
|
||||
* table.
|
||||
*/
|
||||
static void
|
||||
ExecuteSelectIntoRelation(Oid targetRelationId, List *insertTargetList,
|
||||
Query *selectQuery, EState *executorState)
|
||||
ExecutePlanIntoRelation(Oid targetRelationId, List *insertTargetList,
|
||||
PlannedStmt *selectPlan, EState *executorState)
|
||||
{
|
||||
ParamListInfo paramListInfo = executorState->es_param_list_info;
|
||||
bool stopOnFailure = false;
|
||||
|
@ -521,14 +632,7 @@ ExecuteSelectIntoRelation(Oid targetRelationId, List *insertTargetList,
|
|||
executorState,
|
||||
stopOnFailure, NULL);
|
||||
|
||||
/*
|
||||
* Make a copy of the query, since ExecuteQueryIntoDestReceiver may scribble on it
|
||||
* and we want it to be replanned every time if it is stored in a prepared
|
||||
* statement.
|
||||
*/
|
||||
Query *queryCopy = copyObject(selectQuery);
|
||||
|
||||
ExecuteQueryIntoDestReceiver(queryCopy, paramListInfo, (DestReceiver *) copyDest);
|
||||
ExecutePlanIntoDestReceiver(selectPlan, paramListInfo, (DestReceiver *) copyDest);
|
||||
|
||||
executorState->es_processed = copyDest->tuplesSent;
|
||||
|
||||
|
@ -595,3 +699,427 @@ ExecutingInsertSelect(void)
|
|||
{
|
||||
return insertSelectExecutorLevel > 0;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* AddInsertSelectCasts makes sure that the types in columns in the given
|
||||
* target lists have the same type as the columns of the given relation.
|
||||
* It might add casts to ensure that.
|
||||
*
|
||||
* It returns the updated selectTargetList.
|
||||
*/
|
||||
static List *
|
||||
AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
|
||||
Oid targetRelationId)
|
||||
{
|
||||
ListCell *insertEntryCell = NULL;
|
||||
ListCell *selectEntryCell = NULL;
|
||||
List *projectedEntries = NIL;
|
||||
List *nonProjectedEntries = NIL;
|
||||
|
||||
/*
|
||||
* ReorderInsertSelectTargetLists() makes sure that first few columns of
|
||||
* the SELECT query match the insert targets. It might contain additional
|
||||
* items for GROUP BY, etc.
|
||||
*/
|
||||
Assert(list_length(insertTargetList) <= list_length(selectTargetList));
|
||||
|
||||
Relation distributedRelation = heap_open(targetRelationId, RowExclusiveLock);
|
||||
TupleDesc destTupleDescriptor = RelationGetDescr(distributedRelation);
|
||||
|
||||
int targetEntryIndex = 0;
|
||||
forboth(insertEntryCell, insertTargetList, selectEntryCell, selectTargetList)
|
||||
{
|
||||
TargetEntry *insertEntry = (TargetEntry *) lfirst(insertEntryCell);
|
||||
TargetEntry *selectEntry = (TargetEntry *) lfirst(selectEntryCell);
|
||||
Var *insertColumn = (Var *) insertEntry->expr;
|
||||
Form_pg_attribute attr = TupleDescAttr(destTupleDescriptor,
|
||||
insertEntry->resno - 1);
|
||||
|
||||
Oid sourceType = insertColumn->vartype;
|
||||
Oid targetType = attr->atttypid;
|
||||
if (sourceType != targetType)
|
||||
{
|
||||
insertEntry->expr = CastExpr((Expr *) insertColumn, sourceType, targetType,
|
||||
attr->attcollation, attr->atttypmod);
|
||||
|
||||
/*
|
||||
* We cannot modify the selectEntry in-place, because ORDER BY or
|
||||
* GROUP BY clauses might be pointing to it with comparison types
|
||||
* of the source type. So instead we keep the original one as a
|
||||
* non-projected entry, so GROUP BY and ORDER BY are happy, and
|
||||
* create a duplicated projected entry with the coerced expression.
|
||||
*/
|
||||
TargetEntry *coercedEntry = copyObject(selectEntry);
|
||||
coercedEntry->expr = CastExpr((Expr *) selectEntry->expr, sourceType,
|
||||
targetType, attr->attcollation,
|
||||
attr->atttypmod);
|
||||
coercedEntry->ressortgroupref = 0;
|
||||
|
||||
/*
|
||||
* The only requirement is that users don't use this name in ORDER BY
|
||||
* or GROUP BY, and it should be unique across the same query.
|
||||
*/
|
||||
StringInfo resnameString = makeStringInfo();
|
||||
appendStringInfo(resnameString, "auto_coerced_by_citus_%d", targetEntryIndex);
|
||||
coercedEntry->resname = resnameString->data;
|
||||
|
||||
projectedEntries = lappend(projectedEntries, coercedEntry);
|
||||
|
||||
if (selectEntry->ressortgroupref != 0)
|
||||
{
|
||||
selectEntry->resjunk = true;
|
||||
nonProjectedEntries = lappend(nonProjectedEntries, selectEntry);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
projectedEntries = lappend(projectedEntries, selectEntry);
|
||||
}
|
||||
|
||||
targetEntryIndex++;
|
||||
}
|
||||
|
||||
for (int entryIndex = list_length(insertTargetList);
|
||||
entryIndex < list_length(selectTargetList);
|
||||
entryIndex++)
|
||||
{
|
||||
nonProjectedEntries = lappend(nonProjectedEntries, list_nth(selectTargetList,
|
||||
entryIndex));
|
||||
}
|
||||
|
||||
/* selectEntry->resno must be the ordinal number of the entry */
|
||||
selectTargetList = list_concat(projectedEntries, nonProjectedEntries);
|
||||
int entryResNo = 1;
|
||||
foreach(selectEntryCell, selectTargetList)
|
||||
{
|
||||
TargetEntry *selectEntry = lfirst(selectEntryCell);
|
||||
selectEntry->resno = entryResNo++;
|
||||
}
|
||||
|
||||
heap_close(distributedRelation, NoLock);
|
||||
|
||||
return selectTargetList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CastExpr returns an expression which casts the given expr from sourceType to
|
||||
* the given targetType.
|
||||
*/
|
||||
static Expr *
|
||||
CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation,
|
||||
int targetTypeMod)
|
||||
{
|
||||
Oid coercionFuncId = InvalidOid;
|
||||
CoercionPathType coercionType = find_coercion_pathway(targetType, sourceType,
|
||||
COERCION_EXPLICIT,
|
||||
&coercionFuncId);
|
||||
|
||||
if (coercionType == COERCION_PATH_FUNC)
|
||||
{
|
||||
FuncExpr *coerceExpr = makeNode(FuncExpr);
|
||||
coerceExpr->funcid = coercionFuncId;
|
||||
coerceExpr->args = list_make1(copyObject(expr));
|
||||
coerceExpr->funccollid = targetCollation;
|
||||
coerceExpr->funcresulttype = targetType;
|
||||
|
||||
return (Expr *) coerceExpr;
|
||||
}
|
||||
else if (coercionType == COERCION_PATH_RELABELTYPE)
|
||||
{
|
||||
RelabelType *coerceExpr = makeNode(RelabelType);
|
||||
coerceExpr->arg = copyObject(expr);
|
||||
coerceExpr->resulttype = targetType;
|
||||
coerceExpr->resulttypmod = targetTypeMod;
|
||||
coerceExpr->resultcollid = targetCollation;
|
||||
coerceExpr->relabelformat = COERCE_IMPLICIT_CAST;
|
||||
coerceExpr->location = -1;
|
||||
|
||||
return (Expr *) coerceExpr;
|
||||
}
|
||||
else if (coercionType == COERCION_PATH_ARRAYCOERCE)
|
||||
{
|
||||
Oid sourceBaseType = get_base_element_type(sourceType);
|
||||
Oid targetBaseType = get_base_element_type(targetType);
|
||||
|
||||
CaseTestExpr *elemExpr = makeNode(CaseTestExpr);
|
||||
elemExpr->collation = targetCollation;
|
||||
elemExpr->typeId = sourceBaseType;
|
||||
elemExpr->typeMod = -1;
|
||||
|
||||
Expr *elemCastExpr = CastExpr((Expr *) elemExpr, sourceBaseType,
|
||||
targetBaseType, targetCollation,
|
||||
targetTypeMod);
|
||||
|
||||
ArrayCoerceExpr *coerceExpr = makeNode(ArrayCoerceExpr);
|
||||
coerceExpr->arg = copyObject(expr);
|
||||
coerceExpr->elemexpr = elemCastExpr;
|
||||
coerceExpr->resultcollid = targetCollation;
|
||||
coerceExpr->resulttype = targetType;
|
||||
coerceExpr->resulttypmod = targetTypeMod;
|
||||
coerceExpr->location = -1;
|
||||
coerceExpr->coerceformat = COERCE_IMPLICIT_CAST;
|
||||
|
||||
return (Expr *) coerceExpr;
|
||||
}
|
||||
else if (coercionType == COERCION_PATH_COERCEVIAIO)
|
||||
{
|
||||
CoerceViaIO *coerceExpr = makeNode(CoerceViaIO);
|
||||
coerceExpr->arg = (Expr *) copyObject(expr);
|
||||
coerceExpr->resulttype = targetType;
|
||||
coerceExpr->resultcollid = targetCollation;
|
||||
coerceExpr->coerceformat = COERCE_IMPLICIT_CAST;
|
||||
coerceExpr->location = -1;
|
||||
|
||||
return (Expr *) coerceExpr;
|
||||
}
|
||||
else
|
||||
{
|
||||
ereport(ERROR, (errmsg("could not find a conversion path from type %d to %d",
|
||||
sourceType, targetType)));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* IsSupportedRedistributionTarget determines whether re-partitioning into the
|
||||
* given target relation is supported.
|
||||
*/
|
||||
bool
|
||||
IsSupportedRedistributionTarget(Oid targetRelationId)
|
||||
{
|
||||
DistTableCacheEntry *tableEntry = DistributedTableCacheEntry(targetRelationId);
|
||||
|
||||
/* only range and hash-distributed tables are currently supported */
|
||||
if (tableEntry->partitionMethod != DISTRIBUTE_BY_HASH &&
|
||||
tableEntry->partitionMethod != DISTRIBUTE_BY_RANGE)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* RedistributedInsertSelectTaskList returns a task list to insert given
|
||||
* redistributedResults into the given target relation.
|
||||
* redistributedResults[shardIndex] is list of cstrings each of which is
|
||||
* a result name which should be inserted into
|
||||
* targetRelation->sortedShardIntervalArray[shardIndex].
|
||||
*/
|
||||
static List *
|
||||
RedistributedInsertSelectTaskList(Query *insertSelectQuery,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
List **redistributedResults,
|
||||
bool useBinaryFormat)
|
||||
{
|
||||
List *taskList = NIL;
|
||||
|
||||
/*
|
||||
* Make a copy of the INSERT ... SELECT. We'll repeatedly replace the
|
||||
* subquery of insertResultQuery for different intermediate results and
|
||||
* then deparse it.
|
||||
*/
|
||||
Query *insertResultQuery = copyObject(insertSelectQuery);
|
||||
RangeTblEntry *insertRte = ExtractResultRelationRTE(insertResultQuery);
|
||||
RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(insertResultQuery);
|
||||
List *selectTargetList = selectRte->subquery->targetList;
|
||||
Oid targetRelationId = targetRelation->relationId;
|
||||
|
||||
int shardCount = targetRelation->shardIntervalArrayLength;
|
||||
int shardOffset = 0;
|
||||
uint32 taskIdIndex = 1;
|
||||
uint64 jobId = INVALID_JOB_ID;
|
||||
|
||||
for (shardOffset = 0; shardOffset < shardCount; shardOffset++)
|
||||
{
|
||||
ShardInterval *targetShardInterval =
|
||||
targetRelation->sortedShardIntervalArray[shardOffset];
|
||||
List *resultIdList = redistributedResults[targetShardInterval->shardIndex];
|
||||
uint64 shardId = targetShardInterval->shardId;
|
||||
StringInfo queryString = makeStringInfo();
|
||||
|
||||
/* skip empty tasks */
|
||||
if (resultIdList == NIL)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
/* sort result ids for consistent test output */
|
||||
List *sortedResultIds = SortList(resultIdList, pg_qsort_strcmp);
|
||||
|
||||
/* generate the query on the intermediate result */
|
||||
Query *fragmentSetQuery = BuildReadIntermediateResultsArrayQuery(selectTargetList,
|
||||
NIL,
|
||||
sortedResultIds,
|
||||
useBinaryFormat);
|
||||
|
||||
/* put the intermediate result query in the INSERT..SELECT */
|
||||
selectRte->subquery = fragmentSetQuery;
|
||||
|
||||
/* setting an alias simplifies deparsing of RETURNING */
|
||||
if (insertRte->alias == NULL)
|
||||
{
|
||||
Alias *alias = makeAlias(CITUS_TABLE_ALIAS, NIL);
|
||||
insertRte->alias = alias;
|
||||
}
|
||||
|
||||
/*
|
||||
* Generate a query string for the query that inserts into a shard and reads
|
||||
* from an intermediate result.
|
||||
*
|
||||
* Since CTEs have already been converted to intermediate results, they need
|
||||
* to removed from the query. Otherwise, worker queries include both
|
||||
* intermediate results and CTEs in the query.
|
||||
*/
|
||||
insertResultQuery->cteList = NIL;
|
||||
deparse_shard_query(insertResultQuery, targetRelationId, shardId, queryString);
|
||||
ereport(DEBUG2, (errmsg("distributed statement: %s", queryString->data)));
|
||||
|
||||
LockShardDistributionMetadata(shardId, ShareLock);
|
||||
List *insertShardPlacementList = ActiveShardPlacementList(shardId);
|
||||
|
||||
RelationShard *relationShard = CitusMakeNode(RelationShard);
|
||||
relationShard->relationId = targetShardInterval->relationId;
|
||||
relationShard->shardId = targetShardInterval->shardId;
|
||||
|
||||
Task *modifyTask = CreateBasicTask(jobId, taskIdIndex, MODIFY_TASK,
|
||||
queryString->data);
|
||||
modifyTask->dependentTaskList = NIL;
|
||||
modifyTask->anchorShardId = shardId;
|
||||
modifyTask->taskPlacementList = insertShardPlacementList;
|
||||
modifyTask->relationShardList = list_make1(relationShard);
|
||||
modifyTask->replicationModel = targetRelation->replicationModel;
|
||||
|
||||
taskList = lappend(taskList, modifyTask);
|
||||
|
||||
taskIdIndex++;
|
||||
}
|
||||
|
||||
return taskList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* PartitionColumnIndex finds the index of given partition column in the
|
||||
* given target list.
|
||||
*/
|
||||
static int
|
||||
PartitionColumnIndex(List *insertTargetList, Var *partitionColumn)
|
||||
{
|
||||
TargetEntry *insertTargetEntry = NULL;
|
||||
int targetEntryIndex = 0;
|
||||
foreach_ptr(insertTargetEntry, insertTargetList)
|
||||
{
|
||||
if (insertTargetEntry->resno == partitionColumn->varattno)
|
||||
{
|
||||
return targetEntryIndex;
|
||||
}
|
||||
|
||||
targetEntryIndex++;
|
||||
}
|
||||
|
||||
return -1;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* IsRedistributablePlan returns true if the given plan is a redistrituable plan.
|
||||
*/
|
||||
bool
|
||||
IsRedistributablePlan(Plan *selectPlan)
|
||||
{
|
||||
if (!EnableRepartitionedInsertSelect)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/* don't redistribute if query is not distributed or requires merge on coordinator */
|
||||
if (!IsCitusCustomScan(selectPlan))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
DistributedPlan *distSelectPlan =
|
||||
GetDistributedPlan((CustomScan *) selectPlan);
|
||||
Job *distSelectJob = distSelectPlan->workerJob;
|
||||
List *distSelectTaskList = distSelectJob->taskList;
|
||||
|
||||
/*
|
||||
* Don't use redistribution if only one task. This is to keep the existing
|
||||
* behaviour for CTEs that the last step is a read_intermediate_result()
|
||||
* call. It doesn't hurt much in other cases too.
|
||||
*/
|
||||
if (list_length(distSelectTaskList) <= 1)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/* don't use redistribution for repartition joins for now */
|
||||
if (distSelectJob->dependentJobList != NIL)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* WrapTaskListForProjection wraps task->queryString to only select given
|
||||
* projected columns. It modifies the taskList.
|
||||
*/
|
||||
static void
|
||||
WrapTaskListForProjection(List *taskList, List *projectedTargetEntries)
|
||||
{
|
||||
StringInfo projectedColumnsString = makeStringInfo();
|
||||
int entryIndex = 0;
|
||||
TargetEntry *targetEntry = NULL;
|
||||
foreach_ptr(targetEntry, projectedTargetEntries)
|
||||
{
|
||||
if (entryIndex != 0)
|
||||
{
|
||||
appendStringInfoChar(projectedColumnsString, ',');
|
||||
}
|
||||
|
||||
char *columnName = targetEntry->resname;
|
||||
Assert(columnName != NULL);
|
||||
appendStringInfoString(projectedColumnsString, quote_identifier(columnName));
|
||||
|
||||
entryIndex++;
|
||||
}
|
||||
|
||||
Task *task = NULL;
|
||||
foreach_ptr(task, taskList)
|
||||
{
|
||||
Assert(task->queryString != NULL);
|
||||
|
||||
StringInfo wrappedQuery = makeStringInfo();
|
||||
appendStringInfo(wrappedQuery, "SELECT %s FROM (%s) subquery",
|
||||
projectedColumnsString->data,
|
||||
task->queryString);
|
||||
task->queryString = wrappedQuery->data;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* RelableTargetEntryList relabels select target list to have matching names with
|
||||
* insert target list.
|
||||
*/
|
||||
static void
|
||||
RelableTargetEntryList(List *selectTargetList, List *insertTargetList)
|
||||
{
|
||||
ListCell *selectTargetCell = NULL;
|
||||
ListCell *insertTargetCell = NULL;
|
||||
|
||||
forboth(selectTargetCell, selectTargetList, insertTargetCell, insertTargetList)
|
||||
{
|
||||
TargetEntry *selectTargetEntry = lfirst(selectTargetCell);
|
||||
TargetEntry *insertTargetEntry = lfirst(insertTargetCell);
|
||||
|
||||
selectTargetEntry->resname = insertTargetEntry->resname;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -804,7 +804,7 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
|
|||
char *remoteHost = text_to_cstring(remoteHostText);
|
||||
int remotePort = PG_GETARG_INT32(2);
|
||||
|
||||
int connectionFlags = 0;
|
||||
int connectionFlags = FORCE_NEW_CONNECTION;
|
||||
int resultIndex = 0;
|
||||
int64 totalBytesWritten = 0L;
|
||||
|
||||
|
@ -827,7 +827,7 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
|
|||
* Intermediate results will be stored in a directory that is derived
|
||||
* from the distributed transaction ID.
|
||||
*/
|
||||
UseCoordinatedTransaction();
|
||||
EnsureDistributedTransactionId();
|
||||
|
||||
MultiConnection *connection = GetNodeConnection(connectionFlags, remoteHost,
|
||||
remotePort);
|
||||
|
@ -838,7 +838,8 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
|
|||
remoteHost, remotePort)));
|
||||
}
|
||||
|
||||
RemoteTransactionBeginIfNecessary(connection);
|
||||
StringInfo beginAndSetXactId = BeginAndSetDistributedTransactionIdCommand();
|
||||
ExecuteCriticalRemoteCommand(connection, beginAndSetXactId->data);
|
||||
|
||||
for (resultIndex = 0; resultIndex < resultCount; resultIndex++)
|
||||
{
|
||||
|
@ -847,7 +848,9 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
|
|||
totalBytesWritten += FetchRemoteIntermediateResult(connection, resultId);
|
||||
}
|
||||
|
||||
UnclaimConnection(connection);
|
||||
ExecuteCriticalRemoteCommand(connection, "END");
|
||||
|
||||
CloseConnection(connection);
|
||||
|
||||
PG_RETURN_INT64(totalBytesWritten);
|
||||
}
|
||||
|
@ -894,7 +897,7 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId)
|
|||
|
||||
while (true)
|
||||
{
|
||||
int waitFlags = WL_SOCKET_READABLE;
|
||||
int waitFlags = WL_SOCKET_READABLE | WL_POSTMASTER_DEATH;
|
||||
|
||||
CopyStatus copyStatus = CopyDataFromConnection(connection, &fileCompat,
|
||||
&totalBytesWritten);
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -32,6 +32,7 @@
|
|||
#include "distributed/multi_logical_planner.h"
|
||||
#include "distributed/multi_master_planner.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/multi_router_planner.h"
|
||||
#include "distributed/distributed_planner.h"
|
||||
#include "distributed/multi_server_executor.h"
|
||||
#include "distributed/remote_commands.h"
|
||||
|
@ -139,9 +140,12 @@ CoordinatorInsertSelectExplainScan(CustomScanState *node, List *ancestors,
|
|||
DistributedPlan *distributedPlan = scanState->distributedPlan;
|
||||
Query *insertSelectQuery = distributedPlan->insertSelectQuery;
|
||||
Query *query = BuildSelectForInsertSelect(insertSelectQuery);
|
||||
RangeTblEntry *insertRte = ExtractResultRelationRTE(insertSelectQuery);
|
||||
Oid targetRelationId = insertRte->relid;
|
||||
IntoClause *into = NULL;
|
||||
ParamListInfo params = NULL;
|
||||
char *queryString = NULL;
|
||||
int cursorOptions = CURSOR_OPT_PARALLEL_OK;
|
||||
|
||||
if (es->analyze)
|
||||
{
|
||||
|
@ -150,6 +154,22 @@ CoordinatorInsertSelectExplainScan(CustomScanState *node, List *ancestors,
|
|||
"... SELECT commands via the coordinator")));
|
||||
}
|
||||
|
||||
/*
|
||||
* Make a copy of the query, since pg_plan_query may scribble on it and later
|
||||
* stages of EXPLAIN require it.
|
||||
*/
|
||||
Query *queryCopy = copyObject(query);
|
||||
PlannedStmt *selectPlan = pg_plan_query(queryCopy, cursorOptions, params);
|
||||
if (IsRedistributablePlan(selectPlan->planTree) &&
|
||||
IsSupportedRedistributionTarget(targetRelationId))
|
||||
{
|
||||
ExplainPropertyText("INSERT/SELECT method", "repartition", es);
|
||||
}
|
||||
else
|
||||
{
|
||||
ExplainPropertyText("INSERT/SELECT method", "pull to coordinator", es);
|
||||
}
|
||||
|
||||
ExplainOpenGroup("Select Query", "Select Query", false, es);
|
||||
|
||||
/* explain the inner SELECT query */
|
||||
|
|
|
@ -176,6 +176,10 @@ static bool ContainsReferencesToOuterQueryWalker(Node *node,
|
|||
static void WrapFunctionsInSubqueries(Query *query);
|
||||
static void TransformFunctionRTE(RangeTblEntry *rangeTblEntry);
|
||||
static bool ShouldTransformRTE(RangeTblEntry *rangeTableEntry);
|
||||
static Query * BuildReadIntermediateResultsQuery(List *targetEntryList,
|
||||
List *columnAliasList,
|
||||
Const *resultIdConst, Oid functionOid,
|
||||
bool useBinaryCopyFormat);
|
||||
|
||||
/*
|
||||
* GenerateSubplansForSubqueriesAndCTEs is a wrapper around RecursivelyPlanSubqueriesAndCTEs.
|
||||
|
@ -1541,6 +1545,72 @@ ShouldTransformRTE(RangeTblEntry *rangeTableEntry)
|
|||
*/
|
||||
Query *
|
||||
BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resultId)
|
||||
{
|
||||
Oid functionOid = CitusReadIntermediateResultFuncId();
|
||||
bool useBinaryCopyFormat = CanUseBinaryCopyFormatForTargetList(targetEntryList);
|
||||
|
||||
Const *resultIdConst = makeNode(Const);
|
||||
resultIdConst->consttype = TEXTOID;
|
||||
resultIdConst->consttypmod = -1;
|
||||
resultIdConst->constlen = -1;
|
||||
resultIdConst->constvalue = CStringGetTextDatum(resultId);
|
||||
resultIdConst->constbyval = false;
|
||||
resultIdConst->constisnull = false;
|
||||
resultIdConst->location = -1;
|
||||
|
||||
return BuildReadIntermediateResultsQuery(targetEntryList, columnAliasList,
|
||||
resultIdConst, functionOid,
|
||||
useBinaryCopyFormat);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* BuildSubPlanResultQuery returns a query of the form:
|
||||
*
|
||||
* SELECT
|
||||
* <target list>
|
||||
* FROM
|
||||
* read_intermediate_results(ARRAY['<resultId>', ...]::text[], '<copy format'>)
|
||||
* AS res (<column definition list>);
|
||||
*
|
||||
* The caller can optionally supply a columnAliasList, which is useful for
|
||||
* CTEs that have column aliases.
|
||||
*
|
||||
* If useBinaryCopyFormat is true, then 'binary' format is used. Otherwise,
|
||||
* 'text' format is used.
|
||||
*/
|
||||
Query *
|
||||
BuildReadIntermediateResultsArrayQuery(List *targetEntryList,
|
||||
List *columnAliasList,
|
||||
List *resultIdList,
|
||||
bool useBinaryCopyFormat)
|
||||
{
|
||||
Oid functionOid = CitusReadIntermediateResultArrayFuncId();
|
||||
|
||||
Const *resultIdConst = makeNode(Const);
|
||||
resultIdConst->consttype = TEXTARRAYOID;
|
||||
resultIdConst->consttypmod = -1;
|
||||
resultIdConst->constlen = -1;
|
||||
resultIdConst->constvalue = PointerGetDatum(strlist_to_textarray(resultIdList));
|
||||
resultIdConst->constbyval = false;
|
||||
resultIdConst->constisnull = false;
|
||||
resultIdConst->location = -1;
|
||||
|
||||
return BuildReadIntermediateResultsQuery(targetEntryList, columnAliasList,
|
||||
resultIdConst, functionOid,
|
||||
useBinaryCopyFormat);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* BuildReadIntermediateResultsQuery is the common code for generating
|
||||
* queries to read from result files. It is used by
|
||||
* BuildReadIntermediateResultsArrayQuery and BuildSubPlanResultQuery.
|
||||
*/
|
||||
static Query *
|
||||
BuildReadIntermediateResultsQuery(List *targetEntryList, List *columnAliasList,
|
||||
Const *resultIdConst, Oid functionOid,
|
||||
bool useBinaryCopyFormat)
|
||||
{
|
||||
List *funcColNames = NIL;
|
||||
List *funcColTypes = NIL;
|
||||
|
@ -1549,7 +1619,6 @@ BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resu
|
|||
ListCell *targetEntryCell = NULL;
|
||||
List *targetList = NIL;
|
||||
int columnNumber = 1;
|
||||
bool useBinaryCopyFormat = true;
|
||||
Oid copyFormatId = BinaryCopyFormatId();
|
||||
int columnAliasCount = list_length(columnAliasList);
|
||||
|
||||
|
@ -1608,23 +1677,9 @@ BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resu
|
|||
|
||||
targetList = lappend(targetList, newTargetEntry);
|
||||
|
||||
if (useBinaryCopyFormat && !CanUseBinaryCopyFormatForType(columnType))
|
||||
{
|
||||
useBinaryCopyFormat = false;
|
||||
}
|
||||
|
||||
columnNumber++;
|
||||
}
|
||||
|
||||
Const *resultIdConst = makeNode(Const);
|
||||
resultIdConst->consttype = TEXTOID;
|
||||
resultIdConst->consttypmod = -1;
|
||||
resultIdConst->constlen = -1;
|
||||
resultIdConst->constvalue = CStringGetTextDatum(resultId);
|
||||
resultIdConst->constbyval = false;
|
||||
resultIdConst->constisnull = false;
|
||||
resultIdConst->location = -1;
|
||||
|
||||
/* build the citus_copy_format parameter for the call to read_intermediate_result */
|
||||
if (!useBinaryCopyFormat)
|
||||
{
|
||||
|
@ -1642,7 +1697,7 @@ BuildSubPlanResultQuery(List *targetEntryList, List *columnAliasList, char *resu
|
|||
|
||||
/* build the call to read_intermediate_result */
|
||||
FuncExpr *funcExpr = makeNode(FuncExpr);
|
||||
funcExpr->funcid = CitusReadIntermediateResultFuncId();
|
||||
funcExpr->funcid = functionOid;
|
||||
funcExpr->funcretset = true;
|
||||
funcExpr->funcvariadic = false;
|
||||
funcExpr->funcformat = 0;
|
||||
|
|
|
@ -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"),
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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).
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -16,10 +16,13 @@
|
|||
|
||||
#include "executor/execdesc.h"
|
||||
|
||||
extern bool EnableRepartitionedInsertSelect;
|
||||
|
||||
extern TupleTableSlot * CoordinatorInsertSelectExecScan(CustomScanState *node);
|
||||
extern bool ExecutingInsertSelect(void);
|
||||
extern Query * BuildSelectForInsertSelect(Query *insertSelectQuery);
|
||||
extern bool IsSupportedRedistributionTarget(Oid targetRelationId);
|
||||
extern bool IsRedistributablePlan(Plan *selectPlan);
|
||||
|
||||
|
||||
#endif /* INSERT_SELECT_EXECUTOR_H */
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -0,0 +1,998 @@
|
|||
-- tests behaviour of INSERT INTO ... SELECT with repartitioning
|
||||
CREATE SCHEMA insert_select_repartition;
|
||||
SET search_path TO 'insert_select_repartition';
|
||||
SET citus.next_shard_id TO 4213581;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.replication_model TO 'streaming';
|
||||
-- 4 shards, hash distributed.
|
||||
-- Negate distribution column value.
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO source_table SELECT * FROM generate_series(1, 10);
|
||||
CREATE TABLE target_table(a int);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT -a FROM source_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an operator in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'a'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213585 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213583_to_0,repartitioned_results_xxxxx_from_4213584_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213586 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213582_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213587 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213581_to_2,repartitioned_results_xxxxx_from_4213582_to_2,repartitioned_results_xxxxx_from_4213584_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213588 AS citus_table_alias (a) SELECT a FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213581_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer)
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table WHERE a=-1 OR a=-3 OR a=-7 ORDER BY a;
|
||||
a
|
||||
---------------------------------------------------------------------
|
||||
-7
|
||||
-3
|
||||
-1
|
||||
(3 rows)
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
--
|
||||
-- range partitioning, composite distribution column
|
||||
--
|
||||
CREATE TYPE composite_key_type AS (f1 int, f2 text);
|
||||
-- source
|
||||
CREATE TABLE source_table(f1 int, key composite_key_type, value int, mapped_key composite_key_type);
|
||||
SELECT create_distributed_table('source_table', 'key', 'range');
|
||||
NOTICE: using statement-based replication
|
||||
DETAIL: Streaming replication is supported only for hash-distributed tables.
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CALL public.create_range_partitioned_shards('source_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
|
||||
INSERT INTO source_table VALUES (0, (0, 'a'), 1, (0, 'a'));
|
||||
INSERT INTO source_table VALUES (1, (1, 'b'), 2, (26, 'b'));
|
||||
INSERT INTO source_table VALUES (2, (2, 'c'), 3, (3, 'c'));
|
||||
INSERT INTO source_table VALUES (3, (4, 'd'), 4, (27, 'd'));
|
||||
INSERT INTO source_table VALUES (4, (30, 'e'), 5, (30, 'e'));
|
||||
INSERT INTO source_table VALUES (5, (31, 'f'), 6, (31, 'f'));
|
||||
INSERT INTO source_table VALUES (6, (32, 'g'), 50, (8, 'g'));
|
||||
-- target
|
||||
CREATE TABLE target_table(f1 int DEFAULT 0, value int, key composite_key_type PRIMARY KEY);
|
||||
SELECT create_distributed_table('target_table', 'key', 'range');
|
||||
NOTICE: using statement-based replication
|
||||
DETAIL: Streaming replication is supported only for hash-distributed tables.
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CALL public.create_range_partitioned_shards('target_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT f1, value, mapped_key FROM source_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 2 with name 'key'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0,repartitioned_results_xxxxx_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
f1 | value | key
|
||||
---------------------------------------------------------------------
|
||||
0 | 1 | (0,a)
|
||||
2 | 3 | (3,c)
|
||||
6 | 50 | (8,g)
|
||||
1 | 2 | (26,b)
|
||||
3 | 4 | (27,d)
|
||||
4 | 5 | (30,e)
|
||||
5 | 6 | (31,f)
|
||||
(7 rows)
|
||||
|
||||
SELECT * FROM target_table WHERE key = (26, 'b')::composite_key_type;
|
||||
f1 | value | key
|
||||
---------------------------------------------------------------------
|
||||
1 | 2 | (26,b)
|
||||
(1 row)
|
||||
|
||||
-- with explicit column names
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table(value, key) SELECT value, mapped_key FROM source_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 2 with name 'key'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0,repartitioned_results_xxxxx_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, value, key) SELECT f1, value, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, value integer, key insert_select_repartition.composite_key_type)
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
f1 | value | key
|
||||
---------------------------------------------------------------------
|
||||
0 | 1 | (0,a)
|
||||
0 | 3 | (3,c)
|
||||
0 | 50 | (8,g)
|
||||
0 | 2 | (26,b)
|
||||
0 | 4 | (27,d)
|
||||
0 | 5 | (30,e)
|
||||
0 | 6 | (31,f)
|
||||
(7 rows)
|
||||
|
||||
-- missing value for a column
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table(key) SELECT mapped_key AS key_renamed FROM source_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 1 with name 'key'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0,repartitioned_results_xxxxx_from_4213590_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type)
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
f1 | value | key
|
||||
---------------------------------------------------------------------
|
||||
0 | | (0,a)
|
||||
0 | | (3,c)
|
||||
0 | | (8,g)
|
||||
0 | | (26,b)
|
||||
0 | | (27,d)
|
||||
0 | | (30,e)
|
||||
0 | | (31,f)
|
||||
(7 rows)
|
||||
|
||||
-- ON CONFLICT
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table(key)
|
||||
SELECT mapped_key AS key_renamed FROM source_table
|
||||
WHERE (mapped_key).f1 % 2 = 1
|
||||
ON CONFLICT (key) DO UPDATE SET f1=1;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 1 with name 'key'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213591 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_0}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type) ON CONFLICT(key) DO UPDATE SET f1 = 1
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213592 AS citus_table_alias (f1, key) SELECT f1, key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213589_to_1,repartitioned_results_xxxxx_from_4213590_to_1}'::text[], 'text'::citus_copy_format) intermediate_result(f1 integer, key insert_select_repartition.composite_key_type) ON CONFLICT(key) DO UPDATE SET f1 = 1
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
f1 | value | key
|
||||
---------------------------------------------------------------------
|
||||
0 | | (0,a)
|
||||
1 | | (3,c)
|
||||
0 | | (8,g)
|
||||
0 | | (26,b)
|
||||
1 | | (27,d)
|
||||
0 | | (30,e)
|
||||
1 | | (31,f)
|
||||
(7 rows)
|
||||
|
||||
-- missing value for distribution column
|
||||
INSERT INTO target_table(value) SELECT value FROM source_table;
|
||||
ERROR: the partition column of table insert_select_repartition.target_table should have a value
|
||||
DROP TABLE source_table, target_table;
|
||||
-- different column types
|
||||
-- verifies that we add necessary casts, otherwise even shard routing won't
|
||||
-- work correctly and we will see 2 values for the same primary key.
|
||||
CREATE TABLE target_table(col_1 int primary key, col_2 int);
|
||||
SELECT create_distributed_table('target_table','col_1');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO target_table VALUES (1,2), (2,3), (3,4), (4,5), (5,6);
|
||||
CREATE TABLE source_table(col_1 numeric, col_2 numeric, col_3 numeric);
|
||||
SELECT create_distributed_table('source_table','col_1');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO source_table VALUES (1,1,1), (3,3,3), (5,5,5);
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table
|
||||
SELECT
|
||||
col_1, col_2
|
||||
FROM
|
||||
source_table
|
||||
ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery.
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'col_1'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213593 AS citus_table_alias (col_1, col_2) SELECT col_1, col_2 FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213597_to_0,repartitioned_results_xxxxx_from_4213600_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer) ON CONFLICT(col_1) DO UPDATE SET col_2 = excluded.col_2
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213594 AS citus_table_alias (col_1, col_2) SELECT col_1, col_2 FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213599_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer) ON CONFLICT(col_1) DO UPDATE SET col_2 = excluded.col_2
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY 1;
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
2 | 3
|
||||
3 | 3
|
||||
4 | 5
|
||||
5 | 5
|
||||
(5 rows)
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
--
|
||||
-- array coercion
|
||||
--
|
||||
SET citus.shard_count TO 3;
|
||||
CREATE TABLE source_table(a int, mapped_key int, c float[]);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO source_table VALUES (1, -1, ARRAY[1.1, 2.2, 3.3]), (2, -2, ARRAY[4.5, 5.8]),
|
||||
(3, -3, ARRAY[]::float[]), (4, -4, ARRAY[3.3]);
|
||||
SET citus.shard_count TO 2;
|
||||
CREATE TABLE target_table(a int, b int[]);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
--
|
||||
-- worker queries can have more columns than necessary. ExpandWorkerTargetEntry()
|
||||
-- might add additional columns to the target list.
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
\set VERBOSITY TERSE
|
||||
-- first verify that the SELECT query below fetches 3 projected columns from workers
|
||||
SET citus.log_remote_commands TO true; SET client_min_messages TO DEBUG;
|
||||
CREATE TABLE results AS SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: building index "pg_toast_xxxxx_index" on table "pg_toast_xxxxx" serially
|
||||
NOTICE: issuing SELECT max((OPERATOR(pg_catalog.-) a)) AS max, array_agg(mapped_key) AS array_agg, a AS worker_column_3 FROM insert_select_repartition.source_table_4213601 source_table WHERE true GROUP BY a
|
||||
NOTICE: issuing SELECT max((OPERATOR(pg_catalog.-) a)) AS max, array_agg(mapped_key) AS array_agg, a AS worker_column_3 FROM insert_select_repartition.source_table_4213602 source_table WHERE true GROUP BY a
|
||||
NOTICE: issuing SELECT max((OPERATOR(pg_catalog.-) a)) AS max, array_agg(mapped_key) AS array_agg, a AS worker_column_3 FROM insert_select_repartition.source_table_4213603 source_table WHERE true GROUP BY a
|
||||
RESET citus.log_remote_commands; RESET client_min_messages;
|
||||
DROP TABLE results;
|
||||
-- now verify that we don't write the extra columns to the intermediate result files and
|
||||
-- insertion to the target works fine.
|
||||
SET client_min_messages TO DEBUG1;
|
||||
INSERT INTO target_table SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {-4}
|
||||
-3 | {-3}
|
||||
-2 | {-2}
|
||||
-1 | {-1}
|
||||
(4 rows)
|
||||
|
||||
--
|
||||
-- repartitioned INSERT/SELECT followed/preceded by other DML in same transaction
|
||||
--
|
||||
-- case 1. followed by DELETE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
DELETE FROM target_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
-- case 2. followed by UPDATE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
UPDATE target_table SET b=array_append(b, a);
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3,-4}
|
||||
-3 | {-3}
|
||||
-2 | {4,6,-2}
|
||||
-1 | {1,2,3,-1}
|
||||
(4 rows)
|
||||
|
||||
-- case 3. followed by multi-row INSERT
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-7 | {999}
|
||||
-6 | {11,12}
|
||||
-5 | {10,11}
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(7 rows)
|
||||
|
||||
-- case 4. followed by distributed INSERT/SELECT
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
INSERT INTO target_table SELECT * FROM target_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
-1 | {1,2,3}
|
||||
(8 rows)
|
||||
|
||||
-- case 5. preceded by DELETE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
DELETE FROM target_table;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
-- case 6. preceded by UPDATE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
UPDATE target_table SET b=array_append(b, a);
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
-- case 7. preceded by multi-row INSERT
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-7 | {999}
|
||||
-6 | {11,12}
|
||||
-5 | {10,11}
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(7 rows)
|
||||
|
||||
-- case 8. preceded by distributed INSERT/SELECT
|
||||
TRUNCATE target_table;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT * FROM target_table;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-4 | {3}
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-3 | {}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-2 | {4,6}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
-1 | {1,2,3}
|
||||
-1 | {1,2,3}
|
||||
(12 rows)
|
||||
|
||||
--
|
||||
-- repartitioned INSERT/SELECT with RETURNING
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
WITH c AS (
|
||||
INSERT INTO target_table
|
||||
SELECT mapped_key, c FROM source_table
|
||||
RETURNING *)
|
||||
SELECT * FROM c ORDER by a;
|
||||
DEBUG: generating subplan XXX_1 for CTE c: INSERT INTO insert_select_repartition.target_table (a, b) SELECT mapped_key, c FROM insert_select_repartition.source_table RETURNING target_table.a, target_table.b
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer[])) c ORDER BY a
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
RESET client_min_messages;
|
||||
--
|
||||
-- in combination with CTEs
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
WITH t AS (
|
||||
SELECT mapped_key, a, c FROM source_table
|
||||
WHERE a > floor(random())
|
||||
)
|
||||
INSERT INTO target_table
|
||||
SELECT mapped_key, c FROM t NATURAL JOIN source_table;
|
||||
DEBUG: volatile functions are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: generating subplan XXX_1 for CTE t: SELECT mapped_key, a, c FROM insert_select_repartition.source_table WHERE ((a)::double precision OPERATOR(pg_catalog.>) floor(random()))
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT mapped_key AS a, (c)::integer[] AS b FROM (SELECT t.mapped_key, t.c FROM ((SELECT intermediate_result.mapped_key, intermediate_result.a, intermediate_result.c FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(mapped_key integer, a integer, c double precision[])) t JOIN insert_select_repartition.source_table USING (mapped_key, a, c))) citus_insert_select_subquery
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
-4 | {3}
|
||||
-3 | {}
|
||||
-2 | {4,6}
|
||||
-1 | {1,2,3}
|
||||
(4 rows)
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
--
|
||||
-- The case where select query has a GROUP BY ...
|
||||
--
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET citus.shard_count TO 3;
|
||||
CREATE TABLE target_table(a int, b int);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO source_table SELECT floor(i/4), i*i FROM generate_series(1, 20) i;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
0 | 9
|
||||
1 | 49
|
||||
2 | 121
|
||||
3 | 225
|
||||
4 | 361
|
||||
5 | 400
|
||||
(6 rows)
|
||||
|
||||
--
|
||||
-- EXPLAIN output should specify repartitioned INSERT/SELECT
|
||||
--
|
||||
EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus INSERT ... SELECT) (cost=0.00..0.00 rows=0 width=0)
|
||||
INSERT/SELECT method: repartition
|
||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> HashAggregate (cost=43.90..45.90 rows=200 width=8)
|
||||
Group Key: a
|
||||
-> Seq Scan on source_table_4213606 source_table (cost=0.00..32.60 rows=2260 width=8)
|
||||
(10 rows)
|
||||
|
||||
--
|
||||
-- Duplicate names in target list
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table
|
||||
SELECT max(b), max(b) FROM source_table GROUP BY a;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'a'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213610 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_0,repartitioned_results_xxxxx_from_4213607_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213611 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213607_to_1,repartitioned_results_xxxxx_from_4213609_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213612 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_2,repartitioned_results_xxxxx_from_4213607_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
9 | 9
|
||||
49 | 49
|
||||
121 | 121
|
||||
225 | 225
|
||||
361 | 361
|
||||
400 | 400
|
||||
(6 rows)
|
||||
|
||||
--
|
||||
-- Prepared INSERT/SELECT
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
PREPARE insert_plan(int, int) AS
|
||||
INSERT INTO target_table
|
||||
SELECT a, max(b) FROM source_table
|
||||
WHERE a BETWEEN $1 AND $2 GROUP BY a;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
EXECUTE insert_plan(0, 2);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(0, 2);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(0, 2);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(0, 2);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(0, 2);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(0, 2);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(2, 4);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(2, 4);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(2, 4);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(2, 4);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(2, 4);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
EXECUTE insert_plan(2, 4);
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
RESET client_min_messages;
|
||||
SELECT a, count(*), count(distinct b) distinct_values FROM target_table GROUP BY a ORDER BY a;
|
||||
a | count | distinct_values
|
||||
---------------------------------------------------------------------
|
||||
0 | 6 | 1
|
||||
1 | 6 | 1
|
||||
2 | 12 | 1
|
||||
3 | 6 | 1
|
||||
4 | 6 | 1
|
||||
(5 rows)
|
||||
|
||||
--
|
||||
-- INSERT/SELECT in CTE
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
WITH r AS (
|
||||
INSERT INTO target_table SELECT * FROM source_table RETURNING *
|
||||
)
|
||||
INSERT INTO target_table SELECT source_table.a, max(source_table.b) FROM source_table NATURAL JOIN r GROUP BY source_table.a;
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: data-modifying statements are not supported in the WITH clauses of distributed queries
|
||||
DEBUG: generating subplan XXX_1 for CTE r: INSERT INTO insert_select_repartition.target_table (a, b) SELECT a, b FROM insert_select_repartition.source_table RETURNING target_table.a, target_table.b
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, max AS b FROM (SELECT source_table.a, max(source_table.b) AS max FROM (insert_select_repartition.source_table JOIN (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) r USING (a, b)) GROUP BY source_table.a) citus_insert_select_subquery
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'a'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213610 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_0,repartitioned_results_xxxxx_from_4213607_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) RETURNING citus_table_alias.a, citus_table_alias.b
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213611 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213607_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) RETURNING citus_table_alias.a, citus_table_alias.b
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213612 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213609_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer) RETURNING citus_table_alias.a, citus_table_alias.b
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'a'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213610 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213606_to_0,repartitioned_results_xxxxx_from_4213607_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213611 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213607_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213612 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213609_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a, b;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
0 | 1
|
||||
0 | 4
|
||||
0 | 9
|
||||
0 | 9
|
||||
1 | 16
|
||||
1 | 25
|
||||
1 | 36
|
||||
1 | 49
|
||||
1 | 49
|
||||
2 | 64
|
||||
2 | 81
|
||||
2 | 100
|
||||
2 | 121
|
||||
2 | 121
|
||||
3 | 144
|
||||
3 | 169
|
||||
3 | 196
|
||||
3 | 225
|
||||
3 | 225
|
||||
4 | 256
|
||||
4 | 289
|
||||
4 | 324
|
||||
4 | 361
|
||||
4 | 361
|
||||
5 | 400
|
||||
5 | 400
|
||||
(26 rows)
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
--
|
||||
-- Constraint failure and rollback
|
||||
--
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
|
||||
UPDATE source_table SET b = NULL where b IN (9, 4);
|
||||
SET citus.shard_replication_factor TO 2;
|
||||
CREATE TABLE target_table(a int, b int not null);
|
||||
SELECT create_distributed_table('target_table', 'a', 'range');
|
||||
NOTICE: using statement-based replication
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CALL public.create_range_partitioned_shards('target_table', '{0,3,6,9}','{2,5,8,50}');
|
||||
INSERT INTO target_table VALUES (11,9), (22,4);
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table;
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus INSERT ... SELECT)
|
||||
INSERT/SELECT method: repartition
|
||||
-> Custom Scan (Citus Adaptive)
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Seq Scan on source_table_4213613 source_table
|
||||
(8 rows)
|
||||
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus INSERT ... SELECT)
|
||||
INSERT/SELECT method: repartition
|
||||
-> Custom Scan (Citus Adaptive)
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Seq Scan on source_table_4213613 source_table
|
||||
Filter: (b IS NOT NULL)
|
||||
(9 rows)
|
||||
|
||||
BEGIN;
|
||||
SAVEPOINT s1;
|
||||
INSERT INTO target_table SELECT * FROM source_table;
|
||||
ERROR: null value in column "b" violates not-null constraint
|
||||
ROLLBACK TO SAVEPOINT s1;
|
||||
INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY b;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
22 | 4
|
||||
11 | 9
|
||||
4 | 16
|
||||
5 | 25
|
||||
6 | 36
|
||||
7 | 49
|
||||
8 | 64
|
||||
9 | 81
|
||||
10 | 100
|
||||
(10 rows)
|
||||
|
||||
-- verify that values have been replicated to both replicas
|
||||
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
|
||||
nodename | nodeport | shardid | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | 4213617 | t | 1
|
||||
localhost | 57638 | 4213617 | t | 1
|
||||
localhost | 57637 | 4213618 | t | 2
|
||||
localhost | 57638 | 4213618 | t | 2
|
||||
localhost | 57637 | 4213619 | t | 3
|
||||
localhost | 57638 | 4213619 | t | 3
|
||||
localhost | 57637 | 4213620 | t | 4
|
||||
localhost | 57638 | 4213620 | t | 4
|
||||
(8 rows)
|
||||
|
||||
--
|
||||
-- Multiple casts in the SELECT query
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT 1.12, b::bigint FROM source_table WHERE b IS NOT NULL;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'a'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213617 AS citus_table_alias (a, b) SELECT a, b FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213613_to_0,repartitioned_results_xxxxx_from_4213614_to_0,repartitioned_results_xxxxx_from_4213615_to_0,repartitioned_results_xxxxx_from_4213616_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer)
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a, b;
|
||||
a | b
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
1 | 16
|
||||
1 | 25
|
||||
1 | 36
|
||||
1 | 49
|
||||
1 | 64
|
||||
1 | 81
|
||||
1 | 100
|
||||
(8 rows)
|
||||
|
||||
--
|
||||
-- ROLLBACK after out of range error
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT a * 10, b FROM source_table WHERE b IS NOT NULL;
|
||||
ERROR: could not find shard for partition column value
|
||||
END;
|
||||
SELECT max(result) FROM run_command_on_placements('target_table', 'select count(*) from %s');
|
||||
max
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
--
|
||||
-- Range partitioned target's ranges doesn't cover the whole range
|
||||
--
|
||||
SET citus.shard_replication_factor TO 2;
|
||||
SET citus.replication_model TO 'statement';
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
|
||||
SET citus.shard_replication_factor TO 2;
|
||||
CREATE TABLE target_table(b int not null, a float);
|
||||
SELECT create_distributed_table('target_table', 'a', 'range');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CALL public.create_range_partitioned_shards('target_table', '{0.0,3.5,6.5,9.5}','{2.9,5.9,8.9,50.0}');
|
||||
INSERT INTO target_table SELECT b, a+0.6 FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
b | a
|
||||
---------------------------------------------------------------------
|
||||
1 | 1.6
|
||||
4 | 2.6
|
||||
9 | 3.6
|
||||
16 | 4.6
|
||||
25 | 5.6
|
||||
36 | 6.6
|
||||
49 | 7.6
|
||||
64 | 8.6
|
||||
81 | 9.6
|
||||
100 | 10.6
|
||||
(10 rows)
|
||||
|
||||
-- verify that values have been replicated to both replicas, and that each
|
||||
-- replica has received correct number of rows
|
||||
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
|
||||
nodename | nodeport | shardid | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | 4213625 | t | 2
|
||||
localhost | 57638 | 4213625 | t | 2
|
||||
localhost | 57637 | 4213626 | t | 3
|
||||
localhost | 57638 | 4213626 | t | 3
|
||||
localhost | 57637 | 4213627 | t | 3
|
||||
localhost | 57638 | 4213627 | t | 3
|
||||
localhost | 57637 | 4213628 | t | 2
|
||||
localhost | 57638 | 4213628 | t | 2
|
||||
(8 rows)
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
--
|
||||
-- Select column names should be unique
|
||||
--
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET citus.shard_count TO 3;
|
||||
CREATE TABLE target_table(a int, b int, c int, d int, e int, f int);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'a'
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213633 AS citus_table_alias (a, b, c, d) SELECT a, b, c, d FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213629_to_0,repartitioned_results_xxxxx_from_4213630_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer, c integer, d integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213634 AS citus_table_alias (a, b, c, d) SELECT a, b, c, d FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213630_to_1,repartitioned_results_xxxxx_from_4213631_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer, c integer, d integer)
|
||||
DEBUG: distributed statement: INSERT INTO insert_select_repartition.target_table_4213635 AS citus_table_alias (a, b, c, d) SELECT a, b, c, d FROM read_intermediate_results('{repartitioned_results_xxxxx_from_4213632_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(a integer, b integer, c integer, d integer)
|
||||
RESET client_min_messages;
|
||||
SELECT count(*) FROM target_table;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10
|
||||
(1 row)
|
||||
|
||||
--
|
||||
-- Disable repartitioned insert/select
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET citus.enable_repartitioned_insert_select TO OFF;
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus INSERT ... SELECT)
|
||||
INSERT/SELECT method: pull to coordinator
|
||||
-> Custom Scan (Citus Adaptive)
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Seq Scan on source_table_4213629 source_table
|
||||
(8 rows)
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
DEBUG: INSERT target table and the source relation of the SELECT partition column value must be colocated in distributed INSERT ... SELECT
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
RESET client_min_messages;
|
||||
SELECT count(*) FROM target_table;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
10
|
||||
(1 row)
|
||||
|
||||
SET citus.enable_repartitioned_insert_select TO ON;
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------
|
||||
Custom Scan (Citus INSERT ... SELECT)
|
||||
INSERT/SELECT method: repartition
|
||||
-> Custom Scan (Citus Adaptive)
|
||||
Task Count: 4
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Seq Scan on source_table_4213629 source_table
|
||||
(8 rows)
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
--
|
||||
-- Don't use INSERT/SELECT repartition with repartition joins
|
||||
--
|
||||
create table test(x int, y int);
|
||||
select create_distributed_table('test', 'x');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
set citus.enable_repartition_joins to true;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
insert into test(y, x) select a.x, b.y from test a JOIN test b USING (y);
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
RESET client_min_messages;
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA insert_select_repartition CASCADE;
|
|
@ -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
|
||||
|
|
|
@ -0,0 +1,249 @@
|
|||
Parsed test spec with 2 sessions
|
||||
|
||||
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-delete-from-source s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s2-delete-from-source: DELETE FROM source_table;
|
||||
step s1-end: END;
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-update-source s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s2-update-source: UPDATE source_table SET b = 50 - b;
|
||||
step s1-end: END;
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-insert-into-source s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s2-insert-into-source: INSERT INTO source_table VALUES (0, 0);
|
||||
step s1-end: END;
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-delete-from-source s1-repartitioned-insert-select s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s2-delete-from-source: DELETE FROM source_table;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s1-end: END;
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-update-source s1-repartitioned-insert-select s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s2-update-source: UPDATE source_table SET b = 50 - b;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s1-end: END;
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-insert-into-source s1-repartitioned-insert-select s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s2-insert-into-source: INSERT INTO source_table VALUES (0, 0);
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s1-end: END;
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-delete-from-target s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s2-delete-from-target: DELETE FROM target_table; <waiting ...>
|
||||
step s1-end: END;
|
||||
step s2-delete-from-target: <... completed>
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
|
||||
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-update-target s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s2-update-target: UPDATE target_table SET b = 50 - b; <waiting ...>
|
||||
step s1-end: END;
|
||||
step s2-update-target: <... completed>
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 49
|
||||
2 46
|
||||
3 41
|
||||
4 34
|
||||
5 25
|
||||
6 14
|
||||
7 1
|
||||
8 -14
|
||||
9 -31
|
||||
10 -50
|
||||
|
||||
starting permutation: s1-begin s2-begin s1-repartitioned-insert-select s2-insert-into-target s1-end s2-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s2-insert-into-target: INSERT INTO target_table VALUES (0, 0);
|
||||
step s1-end: END;
|
||||
step s2-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
0 0
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-delete-from-target s1-repartitioned-insert-select s2-end s1-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s2-delete-from-target: DELETE FROM target_table;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table; <waiting ...>
|
||||
step s2-end: END;
|
||||
step s1-repartitioned-insert-select: <... completed>
|
||||
step s1-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-update-target s1-repartitioned-insert-select s2-end s1-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s2-update-target: UPDATE target_table SET b = 50 - b;
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table; <waiting ...>
|
||||
step s2-end: END;
|
||||
step s1-repartitioned-insert-select: <... completed>
|
||||
step s1-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-insert-into-target s1-repartitioned-insert-select s2-end s1-end s1-select-target
|
||||
step s1-begin: BEGIN;
|
||||
step s2-begin: BEGIN;
|
||||
step s2-insert-into-target: INSERT INTO target_table VALUES (0, 0);
|
||||
step s1-repartitioned-insert-select: INSERT INTO target_table SELECT * FROM source_table;
|
||||
step s2-end: END;
|
||||
step s1-end: END;
|
||||
step s1-select-target: SELECT * FROM target_table ORDER BY a;
|
||||
a b
|
||||
|
||||
0 0
|
||||
1 1
|
||||
2 4
|
||||
3 9
|
||||
4 16
|
||||
5 25
|
||||
6 36
|
||||
7 49
|
||||
8 64
|
||||
9 81
|
||||
10 100
|
|
@ -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
|
||||
|
|
|
@ -631,8 +631,8 @@ INSERT INTO agg_events (value_1_agg, user_id)
|
|||
FROM
|
||||
raw_events_first;
|
||||
DEBUG: DISTINCT ON (non-partition column) clauses are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
SELECT user_id, value_1_agg FROM agg_events ORDER BY 1,2;
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
user_id | value_1_agg
|
||||
|
@ -693,13 +693,13 @@ INSERT INTO agg_events
|
|||
FROM
|
||||
fist_table_agg;
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for CTE fist_table_agg: SELECT (max(value_1) OPERATOR(pg_catalog.+) 1) AS v1_agg, user_id FROM public.raw_events_first GROUP BY user_id
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, v1_agg FROM (SELECT fist_table_agg.v1_agg, fist_table_agg.user_id FROM (SELECT intermediate_result.v1_agg, intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v1_agg integer, user_id integer)) fist_table_agg) citus_insert_select_subquery
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id, v1_agg AS value_1_agg FROM (SELECT fist_table_agg.v1_agg, fist_table_agg.user_id FROM (SELECT intermediate_result.v1_agg, intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v1_agg integer, user_id integer)) fist_table_agg) citus_insert_select_subquery
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
ROLLBACK;
|
||||
-- We don't support CTEs that are referenced in the target list
|
||||
INSERT INTO agg_events
|
||||
|
@ -710,7 +710,6 @@ INSERT INTO agg_events
|
|||
raw_events_first;
|
||||
DEBUG: CTE sub_cte is going to be inlined via distributed planning
|
||||
DEBUG: Subqueries without relations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
ERROR: could not run distributed query with subquery outside the FROM, WHERE and HAVING clauses
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
|
@ -724,8 +723,13 @@ FROM
|
|||
((SELECT user_id FROM raw_events_first) UNION
|
||||
(SELECT user_id FROM raw_events_second)) as foo;
|
||||
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300004_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300005_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300006_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300007_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
ROLLBACK;
|
||||
-- We do support set operations through recursive planning
|
||||
BEGIN;
|
||||
|
@ -735,7 +739,6 @@ INSERT INTO
|
|||
(SELECT user_id FROM raw_events_first) INTERSECT
|
||||
(SELECT user_id FROM raw_events_first);
|
||||
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT user_id FROM public.raw_events_first
|
||||
|
@ -747,6 +750,7 @@ DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.user_id
|
|||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) citus_insert_select_subquery
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
ROLLBACK;
|
||||
-- If the query is router plannable then it is executed via the coordinator
|
||||
INSERT INTO
|
||||
|
@ -757,9 +761,9 @@ FROM
|
|||
((SELECT user_id FROM raw_events_first WHERE user_id = 15) EXCEPT
|
||||
(SELECT user_id FROM raw_events_second where user_id = 17)) as foo;
|
||||
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- some supported LEFT joins
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
|
@ -1064,7 +1068,6 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
WHERE raw_events_first.user_id = raw_events_second.user_id
|
||||
GROUP BY raw_events_second.value_3) AS foo;
|
||||
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
|
||||
|
@ -1080,9 +1083,10 @@ DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-10
|
|||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.value_3 AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_3
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT id, v1, v4 FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 bigint, id double precision)) foo
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT int4(id) AS user_id, int4(v1) AS value_1_agg, int8(v4) AS value_4_agg FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 bigint, id double precision)) foo
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
ERROR: the partition column of table public.agg_events cannot be NULL
|
||||
-- error cases
|
||||
-- no part column at all
|
||||
|
@ -1092,8 +1096,8 @@ SELECT value_1
|
|||
FROM raw_events_first;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: the query doesn't include the target table's partition column
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
ERROR: the partition column of table public.raw_events_second should have a value
|
||||
INSERT INTO raw_events_second
|
||||
(value_1)
|
||||
|
@ -1101,8 +1105,8 @@ SELECT user_id
|
|||
FROM raw_events_first;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: the query doesn't include the target table's partition column
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
ERROR: the partition column of table public.raw_events_second should have a value
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
|
@ -1110,9 +1114,11 @@ SELECT value_1
|
|||
FROM raw_events_first;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
ERROR: the partition column of table public.raw_events_second cannot be NULL
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
ERROR: the partition column value cannot be NULL
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
SELECT user_id * 2
|
||||
|
@ -1120,8 +1126,13 @@ FROM raw_events_first;
|
|||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an operator in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_0,repartitioned_results_xxxxx_from_13300001_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_1,repartitioned_results_xxxxx_from_13300001_to_1,repartitioned_results_xxxxx_from_13300003_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300001_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_3,repartitioned_results_xxxxx_from_13300002_to_3,repartitioned_results_xxxxx_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
INSERT INTO raw_events_second
|
||||
(user_id)
|
||||
SELECT user_id :: bigint
|
||||
|
@ -1129,8 +1140,13 @@ FROM raw_events_first;
|
|||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an explicit cast in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300000_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300001_to_1}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300002_to_2}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM read_intermediate_results('{repartitioned_results_xxxxx_from_13300003_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(user_id integer)
|
||||
INSERT INTO agg_events
|
||||
(value_3_agg,
|
||||
value_4_agg,
|
||||
|
@ -1147,9 +1163,11 @@ GROUP BY user_id;
|
|||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an aggregation in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
ERROR: the partition column of table public.agg_events cannot be NULL
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
ERROR: the partition column value cannot be NULL
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
INSERT INTO agg_events
|
||||
(value_3_agg,
|
||||
value_4_agg,
|
||||
|
@ -1166,9 +1184,11 @@ GROUP BY user_id,
|
|||
value_2;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
ERROR: the partition column of table public.agg_events cannot be NULL
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
ERROR: the partition column value cannot be NULL
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
-- tables should be co-located
|
||||
INSERT INTO agg_events (user_id)
|
||||
SELECT
|
||||
|
@ -1177,10 +1197,10 @@ FROM
|
|||
reference_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- foo2 is recursively planned and INSERT...SELECT is done via coordinator
|
||||
INSERT INTO agg_events
|
||||
(user_id)
|
||||
|
@ -1205,7 +1225,6 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2
|
||||
ON (f.id = f2.id);
|
||||
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
|
||||
|
@ -1221,8 +1240,10 @@ DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-10
|
|||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT int4(f2.id) AS user_id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
-- the second part of the query is not routable since
|
||||
-- GROUP BY not on the partition column (i.e., value_1) and thus join
|
||||
-- on f.id = f2.id is not on the partition key (instead on the sum of partition key)
|
||||
|
@ -1250,7 +1271,6 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2
|
||||
ON (f.id = f2.id);
|
||||
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
|
||||
|
@ -1266,8 +1286,10 @@ DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-10
|
|||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.id FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.id AS user_id FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'user_id'
|
||||
-- cannot pushdown the query since the JOIN is not equi JOIN
|
||||
INSERT INTO agg_events
|
||||
(user_id, value_4_agg)
|
||||
|
@ -1676,7 +1698,6 @@ SELECT user_id,
|
|||
FROM raw_events_second
|
||||
GROUP BY grouping sets ( ( user_id ), ( value_1 ), ( user_id, value_1 ), ( ) );
|
||||
DEBUG: grouping sets are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
|
@ -2091,61 +2112,71 @@ FROM text_table ;
|
|||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains a case expression in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT COALESCE(part_col, 'onder') FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains a coalesce expression in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT GREATEST(part_col, 'jason') FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains a min/max expression in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT LEAST(part_col, 'andres') FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains a min/max expression in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT NULLIF(part_col, 'metin') FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT part_col isnull FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT part_col::text from char_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an explicit coercion in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT (part_col = 'burak') is true FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT val FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
INSERT INTO text_table (part_col) SELECT val::text FROM text_table;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an explicit coercion in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
DEBUG: partitioning SELECT query by column index 0 with name 'part_col'
|
||||
RESET client_min_messages;
|
||||
insert into table_with_starts_with_defaults (b,c) select b,c FROM table_with_starts_with_defaults;
|
||||
-- Test on partition column without native hash function
|
||||
|
|
|
@ -54,7 +54,7 @@ FROM
|
|||
ON CONFLICT DO NOTHING;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
-- Since partition columns do not match, pull the data to the coordinator
|
||||
-- and update the non-partition column. Query is wrapped by CTE to return
|
||||
-- ordered result.
|
||||
|
@ -70,7 +70,7 @@ DEBUG: generating subplan XXX_1 for CTE inserted_table: INSERT INTO on_conflict
|
|||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -93,10 +93,10 @@ FROM (
|
|||
) as foo
|
||||
ON CONFLICT DO NOTHING;
|
||||
DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- Subquery should be recursively planned due to the limit and update on conflict
|
||||
-- Query is wrapped by CTE to return ordered result.
|
||||
WITH inserted_table AS (
|
||||
|
@ -115,10 +115,10 @@ WITH inserted_table AS (
|
|||
DEBUG: generating subplan XXX_1 for CTE inserted_table: INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM (SELECT source_table_1.col_1, source_table_1.col_2, source_table_1.col_3 FROM on_conflict.source_table_1 LIMIT 5) foo ON CONFLICT(col_1) DO UPDATE SET col_2 = excluded.col_2 RETURNING target_table.col_1, target_table.col_2
|
||||
DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -151,13 +151,13 @@ WITH inserted_table AS (
|
|||
DEBUG: generating subplan XXX_1 for CTE inserted_table: INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM ((SELECT source_table_1.col_1, source_table_1.col_2, source_table_1.col_3 FROM on_conflict.source_table_1 LIMIT 5) UNION (SELECT source_table_2.col_1, source_table_2.col_2, source_table_2.col_3 FROM on_conflict.source_table_2 LIMIT 5)) foo ON CONFLICT(col_1) DO UPDATE SET col_2 = 0 RETURNING target_table.col_1, target_table.col_2
|
||||
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_2 LIMIT 5
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer) UNION SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | 0
|
||||
|
@ -180,18 +180,18 @@ WITH cte AS(
|
|||
)
|
||||
INSERT INTO target_table SELECT * FROM cte ON CONFLICT DO NOTHING;
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2 FROM on_conflict.source_table_1
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte.col_1, cte.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte) citus_insert_select_subquery
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- Get the select part from cte and update on conflict
|
||||
WITH cte AS(
|
||||
SELECT col_1, col_2 FROM source_table_1
|
||||
)
|
||||
INSERT INTO target_table SELECT * FROM cte ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 + 1;
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2 FROM on_conflict.source_table_1
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte.col_1, cte.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte) citus_insert_select_subquery
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
SELECT * FROM target_table ORDER BY 1;
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
|
@ -216,9 +216,9 @@ WITH cte AS(
|
|||
)
|
||||
INSERT INTO target_table ((SELECT * FROM cte) UNION (SELECT * FROM cte_2)) ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 + 1;
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: CTE cte is going to be inlined via distributed planning
|
||||
DEBUG: CTE cte_2 is going to be inlined via distributed planning
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
SELECT * FROM target_table ORDER BY 1;
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
|
@ -247,10 +247,10 @@ WITH inserted_table AS (
|
|||
DEBUG: generating subplan XXX_1 for CTE inserted_table: WITH cte AS (SELECT source_table_1.col_1, source_table_1.col_2, source_table_1.col_3 FROM on_conflict.source_table_1), cte_2 AS (SELECT cte.col_1, cte.col_2 FROM cte) INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM cte_2 ON CONFLICT(col_1) DO UPDATE SET col_2 = (excluded.col_2 OPERATOR(pg_catalog.+) 1) RETURNING target_table.col_1, target_table.col_2
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) inserted_table ORDER BY col_1
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1
|
||||
DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) cte
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte_2.col_1, cte_2.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte_2) citus_insert_select_subquery
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | 2
|
||||
|
@ -270,9 +270,9 @@ UPDATE target_table SET col_2 = 4 WHERE col_1 IN (SELECT col_1 FROM cte);
|
|||
DEBUG: generating subplan XXX_1 for CTE cte: WITH basic AS (SELECT source_table_1.col_1, source_table_1.col_2 FROM on_conflict.source_table_1) INSERT INTO on_conflict.target_table (col_1, col_2) SELECT col_1, col_2 FROM basic ON CONFLICT DO NOTHING RETURNING target_table.col_1, target_table.col_2
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE on_conflict.target_table SET col_2 = 4 WHERE (col_1 OPERATOR(pg_catalog.=) ANY (SELECT cte.col_1 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte))
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: generating subplan XXX_1 for CTE basic: SELECT col_1, col_2 FROM on_conflict.source_table_1
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT basic.col_1, basic.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) basic) citus_insert_select_subquery
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
RESET client_min_messages;
|
||||
-- Following query is not supported since error checks of the subquery pushdown planner
|
||||
-- and insert select planner have not been unified. It should work after unifying them.
|
||||
|
@ -316,12 +316,14 @@ CONTEXT: while executing command on localhost:xxxxx
|
|||
ROLLBACK;
|
||||
BEGIN;
|
||||
DELETE FROM test_ref_table WHERE key > 10;
|
||||
INSERT INTO
|
||||
target_table
|
||||
SELECT
|
||||
col_2,
|
||||
col_1
|
||||
FROM source_table_1 ON CONFLICT (col_1) DO UPDATE SET col_2 = 1 RETURNING *;
|
||||
WITH r AS (
|
||||
INSERT INTO
|
||||
target_table
|
||||
SELECT
|
||||
col_2,
|
||||
col_1
|
||||
FROM source_table_1 ON CONFLICT (col_1) DO UPDATE SET col_2 = 1 RETURNING *)
|
||||
SELECT * FROM r ORDER BY col_1;
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | 1
|
||||
|
@ -395,7 +397,7 @@ FROM
|
|||
ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2;
|
||||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: performing repartitioned INSERT ... SELECT
|
||||
SELECT * FROM target_table ORDER BY 1;
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
|
@ -481,13 +483,13 @@ FROM (
|
|||
) as foo
|
||||
ON CONFLICT(col_1) DO UPDATE SET col_2 = 0;
|
||||
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1 LIMIT 5
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT col_1, col_2, col_3 FROM on_conflict.source_table_2 LIMIT 5
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer) UNION SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) foo
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
SELECT * FROM target_table ORDER BY 1;
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
|
@ -512,10 +514,10 @@ WITH cte AS(
|
|||
)
|
||||
INSERT INTO target_table SELECT * FROM cte_2 ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 + 1;
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: generating subplan XXX_1 for CTE cte: SELECT col_1, col_2, col_3 FROM on_conflict.source_table_1
|
||||
DEBUG: generating subplan XXX_2 for CTE cte_2: SELECT col_1, col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2, intermediate_result.col_3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer, col_3 integer)) cte
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT col_1, col_2 FROM (SELECT cte_2.col_1, cte_2.col_2 FROM (SELECT intermediate_result.col_1, intermediate_result.col_2 FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(col_1 integer, col_2 integer)) cte_2) citus_insert_select_subquery
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
SELECT * FROM target_table ORDER BY 1;
|
||||
col_1 | col_2
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -96,11 +96,11 @@ INSERT INTO distributed_table
|
|||
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
|
||||
DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column.
|
||||
HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery.
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: CTE ids_to_insert is going to be inlined via distributed planning
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT (((tenant_id)::integer OPERATOR(pg_catalog.*) 100))::text AS tenant_id FROM with_dml.distributed_table WHERE (dept OPERATOR(pg_catalog.>) 7)
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT DISTINCT ids_to_insert.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text)) ids_to_insert, with_dml.distributed_table WHERE (distributed_table.tenant_id OPERATOR(pg_catalog.<) ids_to_insert.tenant_id)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text)) citus_insert_select_subquery
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
-- not a very meaningful query
|
||||
-- but has two modifying CTEs along with another
|
||||
-- modify statement
|
||||
|
@ -132,11 +132,11 @@ INSERT INTO second_distributed_table
|
|||
SELECT *
|
||||
FROM copy_to_other_table;
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: generating subplan XXX_1 for CTE copy_to_other_table: INSERT INTO with_dml.distributed_table (tenant_id, dept) SELECT tenant_id, dept FROM with_dml.second_distributed_table WHERE (dept OPERATOR(pg_catalog.=) 3) ON CONFLICT(tenant_id) DO UPDATE SET dept = 4 RETURNING distributed_table.tenant_id, distributed_table.dept
|
||||
DEBUG: generating subplan XXX_2 for CTE main_table_deleted: DELETE FROM with_dml.distributed_table WHERE ((dept OPERATOR(pg_catalog.<) 10) AND (NOT (EXISTS (SELECT 1 FROM with_dml.second_distributed_table WHERE ((second_distributed_table.dept OPERATOR(pg_catalog.=) 1) AND (second_distributed_table.tenant_id OPERATOR(pg_catalog.=) distributed_table.tenant_id)))))) RETURNING tenant_id, dept
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT main_table_deleted.tenant_id, main_table_deleted.dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) main_table_deleted EXCEPT SELECT copy_to_other_table.tenant_id, copy_to_other_table.dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) copy_to_other_table
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT tenant_id, dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) citus_insert_select_subquery
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
SET citus.force_max_query_parallelization TO off;
|
||||
-- CTE inside the UPDATE statement
|
||||
UPDATE
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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"
|
|
@ -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;
|
|
@ -0,0 +1,468 @@
|
|||
-- tests behaviour of INSERT INTO ... SELECT with repartitioning
|
||||
CREATE SCHEMA insert_select_repartition;
|
||||
SET search_path TO 'insert_select_repartition';
|
||||
|
||||
SET citus.next_shard_id TO 4213581;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.replication_model TO 'streaming';
|
||||
|
||||
-- 4 shards, hash distributed.
|
||||
-- Negate distribution column value.
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
INSERT INTO source_table SELECT * FROM generate_series(1, 10);
|
||||
CREATE TABLE target_table(a int);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT -a FROM source_table;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table WHERE a=-1 OR a=-3 OR a=-7 ORDER BY a;
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
--
|
||||
-- range partitioning, composite distribution column
|
||||
--
|
||||
CREATE TYPE composite_key_type AS (f1 int, f2 text);
|
||||
|
||||
-- source
|
||||
CREATE TABLE source_table(f1 int, key composite_key_type, value int, mapped_key composite_key_type);
|
||||
SELECT create_distributed_table('source_table', 'key', 'range');
|
||||
CALL public.create_range_partitioned_shards('source_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
|
||||
|
||||
INSERT INTO source_table VALUES (0, (0, 'a'), 1, (0, 'a'));
|
||||
INSERT INTO source_table VALUES (1, (1, 'b'), 2, (26, 'b'));
|
||||
INSERT INTO source_table VALUES (2, (2, 'c'), 3, (3, 'c'));
|
||||
INSERT INTO source_table VALUES (3, (4, 'd'), 4, (27, 'd'));
|
||||
INSERT INTO source_table VALUES (4, (30, 'e'), 5, (30, 'e'));
|
||||
INSERT INTO source_table VALUES (5, (31, 'f'), 6, (31, 'f'));
|
||||
INSERT INTO source_table VALUES (6, (32, 'g'), 50, (8, 'g'));
|
||||
|
||||
-- target
|
||||
CREATE TABLE target_table(f1 int DEFAULT 0, value int, key composite_key_type PRIMARY KEY);
|
||||
SELECT create_distributed_table('target_table', 'key', 'range');
|
||||
CALL public.create_range_partitioned_shards('target_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}');
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT f1, value, mapped_key FROM source_table;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
SELECT * FROM target_table WHERE key = (26, 'b')::composite_key_type;
|
||||
|
||||
-- with explicit column names
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table(value, key) SELECT value, mapped_key FROM source_table;
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
|
||||
-- missing value for a column
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table(key) SELECT mapped_key AS key_renamed FROM source_table;
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
|
||||
-- ON CONFLICT
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table(key)
|
||||
SELECT mapped_key AS key_renamed FROM source_table
|
||||
WHERE (mapped_key).f1 % 2 = 1
|
||||
ON CONFLICT (key) DO UPDATE SET f1=1;
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY key;
|
||||
|
||||
-- missing value for distribution column
|
||||
INSERT INTO target_table(value) SELECT value FROM source_table;
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
-- different column types
|
||||
-- verifies that we add necessary casts, otherwise even shard routing won't
|
||||
-- work correctly and we will see 2 values for the same primary key.
|
||||
CREATE TABLE target_table(col_1 int primary key, col_2 int);
|
||||
SELECT create_distributed_table('target_table','col_1');
|
||||
INSERT INTO target_table VALUES (1,2), (2,3), (3,4), (4,5), (5,6);
|
||||
|
||||
CREATE TABLE source_table(col_1 numeric, col_2 numeric, col_3 numeric);
|
||||
SELECT create_distributed_table('source_table','col_1');
|
||||
INSERT INTO source_table VALUES (1,1,1), (3,3,3), (5,5,5);
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table
|
||||
SELECT
|
||||
col_1, col_2
|
||||
FROM
|
||||
source_table
|
||||
ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY 1;
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
--
|
||||
-- array coercion
|
||||
--
|
||||
SET citus.shard_count TO 3;
|
||||
CREATE TABLE source_table(a int, mapped_key int, c float[]);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
INSERT INTO source_table VALUES (1, -1, ARRAY[1.1, 2.2, 3.3]), (2, -2, ARRAY[4.5, 5.8]),
|
||||
(3, -3, ARRAY[]::float[]), (4, -4, ARRAY[3.3]);
|
||||
|
||||
SET citus.shard_count TO 2;
|
||||
CREATE TABLE target_table(a int, b int[]);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
--
|
||||
-- worker queries can have more columns than necessary. ExpandWorkerTargetEntry()
|
||||
-- might add additional columns to the target list.
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
\set VERBOSITY TERSE
|
||||
|
||||
-- first verify that the SELECT query below fetches 3 projected columns from workers
|
||||
SET citus.log_remote_commands TO true; SET client_min_messages TO DEBUG;
|
||||
CREATE TABLE results AS SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
|
||||
RESET citus.log_remote_commands; RESET client_min_messages;
|
||||
DROP TABLE results;
|
||||
|
||||
-- now verify that we don't write the extra columns to the intermediate result files and
|
||||
-- insertion to the target works fine.
|
||||
SET client_min_messages TO DEBUG1;
|
||||
INSERT INTO target_table SELECT max(-a), array_agg(mapped_key) FROM source_table GROUP BY a;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
--
|
||||
-- repartitioned INSERT/SELECT followed/preceded by other DML in same transaction
|
||||
--
|
||||
|
||||
-- case 1. followed by DELETE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
DELETE FROM target_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- case 2. followed by UPDATE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
UPDATE target_table SET b=array_append(b, a);
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- case 3. followed by multi-row INSERT
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- case 4. followed by distributed INSERT/SELECT
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
INSERT INTO target_table SELECT * FROM target_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- case 5. preceded by DELETE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
DELETE FROM target_table;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- case 6. preceded by UPDATE
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
UPDATE target_table SET b=array_append(b, a);
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- case 7. preceded by multi-row INSERT
|
||||
TRUNCATE target_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table VALUES (-5, ARRAY[10,11]), (-6, ARRAY[11,12]), (-7, ARRAY[999]);
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- case 8. preceded by distributed INSERT/SELECT
|
||||
TRUNCATE target_table;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT * FROM target_table;
|
||||
INSERT INTO target_table SELECT mapped_key, c FROM source_table;
|
||||
END;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
--
|
||||
-- repartitioned INSERT/SELECT with RETURNING
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
WITH c AS (
|
||||
INSERT INTO target_table
|
||||
SELECT mapped_key, c FROM source_table
|
||||
RETURNING *)
|
||||
SELECT * FROM c ORDER by a;
|
||||
RESET client_min_messages;
|
||||
|
||||
--
|
||||
-- in combination with CTEs
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
WITH t AS (
|
||||
SELECT mapped_key, a, c FROM source_table
|
||||
WHERE a > floor(random())
|
||||
)
|
||||
INSERT INTO target_table
|
||||
SELECT mapped_key, c FROM t NATURAL JOIN source_table;
|
||||
RESET client_min_messages;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
--
|
||||
-- The case where select query has a GROUP BY ...
|
||||
--
|
||||
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
|
||||
SET citus.shard_count TO 3;
|
||||
CREATE TABLE target_table(a int, b int);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
|
||||
INSERT INTO source_table SELECT floor(i/4), i*i FROM generate_series(1, 20) i;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
--
|
||||
-- EXPLAIN output should specify repartitioned INSERT/SELECT
|
||||
--
|
||||
|
||||
EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
|
||||
|
||||
--
|
||||
-- Duplicate names in target list
|
||||
--
|
||||
|
||||
TRUNCATE target_table;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table
|
||||
SELECT max(b), max(b) FROM source_table GROUP BY a;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
--
|
||||
-- Prepared INSERT/SELECT
|
||||
--
|
||||
TRUNCATE target_table;
|
||||
PREPARE insert_plan(int, int) AS
|
||||
INSERT INTO target_table
|
||||
SELECT a, max(b) FROM source_table
|
||||
WHERE a BETWEEN $1 AND $2 GROUP BY a;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
EXECUTE insert_plan(0, 2);
|
||||
EXECUTE insert_plan(0, 2);
|
||||
EXECUTE insert_plan(0, 2);
|
||||
EXECUTE insert_plan(0, 2);
|
||||
EXECUTE insert_plan(0, 2);
|
||||
EXECUTE insert_plan(0, 2);
|
||||
|
||||
EXECUTE insert_plan(2, 4);
|
||||
EXECUTE insert_plan(2, 4);
|
||||
EXECUTE insert_plan(2, 4);
|
||||
EXECUTE insert_plan(2, 4);
|
||||
EXECUTE insert_plan(2, 4);
|
||||
EXECUTE insert_plan(2, 4);
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT a, count(*), count(distinct b) distinct_values FROM target_table GROUP BY a ORDER BY a;
|
||||
|
||||
--
|
||||
-- INSERT/SELECT in CTE
|
||||
--
|
||||
|
||||
TRUNCATE target_table;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
WITH r AS (
|
||||
INSERT INTO target_table SELECT * FROM source_table RETURNING *
|
||||
)
|
||||
INSERT INTO target_table SELECT source_table.a, max(source_table.b) FROM source_table NATURAL JOIN r GROUP BY source_table.a;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY a, b;
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
--
|
||||
-- Constraint failure and rollback
|
||||
--
|
||||
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
|
||||
UPDATE source_table SET b = NULL where b IN (9, 4);
|
||||
|
||||
SET citus.shard_replication_factor TO 2;
|
||||
CREATE TABLE target_table(a int, b int not null);
|
||||
SELECT create_distributed_table('target_table', 'a', 'range');
|
||||
CALL public.create_range_partitioned_shards('target_table', '{0,3,6,9}','{2,5,8,50}');
|
||||
|
||||
INSERT INTO target_table VALUES (11,9), (22,4);
|
||||
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table;
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
|
||||
|
||||
BEGIN;
|
||||
SAVEPOINT s1;
|
||||
INSERT INTO target_table SELECT * FROM source_table;
|
||||
ROLLBACK TO SAVEPOINT s1;
|
||||
INSERT INTO target_table SELECT * FROM source_table WHERE b IS NOT NULL;
|
||||
END;
|
||||
|
||||
SELECT * FROM target_table ORDER BY b;
|
||||
|
||||
-- verify that values have been replicated to both replicas
|
||||
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
|
||||
|
||||
--
|
||||
-- Multiple casts in the SELECT query
|
||||
--
|
||||
|
||||
TRUNCATE target_table;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT 1.12, b::bigint FROM source_table WHERE b IS NOT NULL;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT * FROM target_table ORDER BY a, b;
|
||||
|
||||
--
|
||||
-- ROLLBACK after out of range error
|
||||
--
|
||||
|
||||
TRUNCATE target_table;
|
||||
|
||||
BEGIN;
|
||||
INSERT INTO target_table SELECT a * 10, b FROM source_table WHERE b IS NOT NULL;
|
||||
END;
|
||||
|
||||
SELECT max(result) FROM run_command_on_placements('target_table', 'select count(*) from %s');
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
--
|
||||
-- Range partitioned target's ranges doesn't cover the whole range
|
||||
--
|
||||
|
||||
SET citus.shard_replication_factor TO 2;
|
||||
SET citus.replication_model TO 'statement';
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
|
||||
|
||||
SET citus.shard_replication_factor TO 2;
|
||||
CREATE TABLE target_table(b int not null, a float);
|
||||
SELECT create_distributed_table('target_table', 'a', 'range');
|
||||
CALL public.create_range_partitioned_shards('target_table', '{0.0,3.5,6.5,9.5}','{2.9,5.9,8.9,50.0}');
|
||||
|
||||
INSERT INTO target_table SELECT b, a+0.6 FROM source_table;
|
||||
SELECT * FROM target_table ORDER BY a;
|
||||
|
||||
-- verify that values have been replicated to both replicas, and that each
|
||||
-- replica has received correct number of rows
|
||||
SELECT * FROM run_command_on_placements('target_table', 'select count(*) from %s') ORDER BY shardid, nodeport;
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
--
|
||||
-- Select column names should be unique
|
||||
--
|
||||
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.shard_count TO 4;
|
||||
CREATE TABLE source_table(a int, b int);
|
||||
SELECT create_distributed_table('source_table', 'a');
|
||||
|
||||
SET citus.shard_count TO 3;
|
||||
CREATE TABLE target_table(a int, b int, c int, d int, e int, f int);
|
||||
SELECT create_distributed_table('target_table', 'a');
|
||||
|
||||
INSERT INTO source_table SELECT i, i * i FROM generate_series(1, 10) i;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT count(*) FROM target_table;
|
||||
|
||||
--
|
||||
-- Disable repartitioned insert/select
|
||||
--
|
||||
|
||||
TRUNCATE target_table;
|
||||
SET citus.enable_repartitioned_insert_select TO OFF;
|
||||
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
RESET client_min_messages;
|
||||
|
||||
SELECT count(*) FROM target_table;
|
||||
|
||||
SET citus.enable_repartitioned_insert_select TO ON;
|
||||
EXPLAIN (costs off) INSERT INTO target_table SELECT a AS aa, b AS aa, 1 AS aa, 2 AS aa FROM source_table;
|
||||
|
||||
DROP TABLE source_table, target_table;
|
||||
|
||||
--
|
||||
-- Don't use INSERT/SELECT repartition with repartition joins
|
||||
--
|
||||
|
||||
create table test(x int, y int);
|
||||
select create_distributed_table('test', 'x');
|
||||
set citus.enable_repartition_joins to true;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
insert into test(y, x) select a.x, b.y from test a JOIN test b USING (y);
|
||||
RESET client_min_messages;
|
||||
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA insert_select_repartition CASCADE;
|
|
@ -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
|
||||
|
|
|
@ -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;
|
Loading…
Reference in New Issue