diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 54c3048f9..1cda964ba 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -37,7 +37,7 @@ * same connection since it may hold relevant locks or have uncommitted * writes. In that case we "assign" the task to a connection by adding * it to the task queue of specific connection (in - * AssignTasksToConnectionsOrWorkerPool ). Otherwise we consider the task + * AssignTasksToConnectionsOrWorkerPool). Otherwise we consider the task * unassigned and add it to the task queue of a worker pool, which means * that it can be executed over any connection in the pool. * diff --git a/src/backend/distributed/planner/extended_op_node_utils.c b/src/backend/distributed/planner/extended_op_node_utils.c index 882034e08..6078c0f17 100644 --- a/src/backend/distributed/planner/extended_op_node_utils.c +++ b/src/backend/distributed/planner/extended_op_node_utils.c @@ -33,7 +33,12 @@ static bool HasNonPartitionColumnDistinctAgg(List *targetEntryList, Node *having static bool PartitionColumnInTableList(Var *column, List *tableNodeList); static bool ShouldPullDistinctColumn(bool repartitionSubquery, bool groupedByDisjointPartitionColumn, - bool hasNonPartitionColumnDistinctAgg); + bool hasNonPartitionColumnDistinctAgg, + bool onlyPushableWindowFunctions); +static bool CanPushDownGroupingAndHaving(bool pullUpIntermediateRows, + bool groupedByDisjointPartitionColumn, + bool hasWindowFuncs, + bool onlyPushableWindowFunctions); /* @@ -42,8 +47,8 @@ static bool ShouldPullDistinctColumn(bool repartitionSubquery, * value should be used in a read-only manner. */ ExtendedOpNodeProperties -BuildExtendedOpNodeProperties(MultiExtendedOp *extendedOpNode, bool - pullUpIntermediateRows) +BuildExtendedOpNodeProperties(MultiExtendedOp *extendedOpNode, + bool hasNonDistributableAggregates) { ExtendedOpNodeProperties extendedOpNodeProperties; @@ -51,6 +56,9 @@ BuildExtendedOpNodeProperties(MultiExtendedOp *extendedOpNode, bool bool groupedByDisjointPartitionColumn = GroupedByPartitionColumn((MultiNode *) extendedOpNode, extendedOpNode); + bool pullUpIntermediateRows = !groupedByDisjointPartitionColumn && + hasNonDistributableAggregates; + bool repartitionSubquery = ExtendedOpNodeContainsRepartitionSubquery(extendedOpNode); List *targetList = extendedOpNode->targetList; @@ -58,16 +66,17 @@ BuildExtendedOpNodeProperties(MultiExtendedOp *extendedOpNode, bool bool hasNonPartitionColumnDistinctAgg = HasNonPartitionColumnDistinctAgg(targetList, havingQual, tableNodeList); - bool pullDistinctColumns = - ShouldPullDistinctColumn(repartitionSubquery, groupedByDisjointPartitionColumn, - hasNonPartitionColumnDistinctAgg); + bool pushDownGroupingAndHaving = + CanPushDownGroupingAndHaving(pullUpIntermediateRows, + groupedByDisjointPartitionColumn, + extendedOpNode->hasWindowFuncs, + extendedOpNode->onlyPushableWindowFunctions); - /* - * TODO: Only window functions that can be pushed down reach here, thus, - * using hasWindowFuncs is safe for now. However, this should be fixed - * when we support pull-to-master window functions. - */ - bool pushDownWindowFunctions = extendedOpNode->hasWindowFuncs; + bool pullDistinctColumns = + ShouldPullDistinctColumn(repartitionSubquery, + groupedByDisjointPartitionColumn, + hasNonPartitionColumnDistinctAgg, + extendedOpNode->onlyPushableWindowFunctions); extendedOpNodeProperties.groupedByDisjointPartitionColumn = groupedByDisjointPartitionColumn; @@ -75,9 +84,11 @@ BuildExtendedOpNodeProperties(MultiExtendedOp *extendedOpNode, bool extendedOpNodeProperties.hasNonPartitionColumnDistinctAgg = hasNonPartitionColumnDistinctAgg; extendedOpNodeProperties.pullDistinctColumns = pullDistinctColumns; - extendedOpNodeProperties.pushDownWindowFunctions = pushDownWindowFunctions; - extendedOpNodeProperties.pullUpIntermediateRows = - !groupedByDisjointPartitionColumn && pullUpIntermediateRows; + extendedOpNodeProperties.pullUpIntermediateRows = pullUpIntermediateRows; + extendedOpNodeProperties.hasWindowFuncs = extendedOpNode->hasWindowFuncs; + extendedOpNodeProperties.onlyPushableWindowFunctions = + extendedOpNode->onlyPushableWindowFunctions; + extendedOpNodeProperties.pushDownGroupingAndHaving = pushDownGroupingAndHaving; return extendedOpNodeProperties; } @@ -309,21 +320,52 @@ PartitionColumnInTableList(Var *column, List *tableNodeList) static bool ShouldPullDistinctColumn(bool repartitionSubquery, bool groupedByDisjointPartitionColumn, - bool hasNonPartitionColumnDistinctAgg) + bool hasNonPartitionColumnDistinctAgg, + bool onlyPushableWindowFunctions) { if (repartitionSubquery) { return true; } - if (groupedByDisjointPartitionColumn) + /* don't pull distinct columns when it can be pushed down */ + if (onlyPushableWindowFunctions && groupedByDisjointPartitionColumn) { return false; } - else if (!groupedByDisjointPartitionColumn && hasNonPartitionColumnDistinctAgg) + else if (hasNonPartitionColumnDistinctAgg) { return true; } return false; } + + +/* + * CanPushDownGroupingAndHaving returns whether GROUP BY & HAVING should be + * pushed down to worker. + */ +static bool +CanPushDownGroupingAndHaving(bool pullUpIntermediateRows, + bool groupedByDisjointPartitionColumn, + bool hasWindowFuncs, bool onlyPushableWindowFunctions) +{ + /* don't push down if we're pulling up */ + if (pullUpIntermediateRows) + { + return false; + } + + /* + * If grouped by a partition column we can push down the having qualifier. + * + * When a query with subquery is provided, we can't determine if + * groupedByDisjointPartitionColumn, therefore we also check if there is a + * window function too. If there is a window function we would know that it + * is safe to push down (i.e. it is partitioned on distribution column, and + * if there is a group by, it contains distribution column). + */ + return groupedByDisjointPartitionColumn || + (hasWindowFuncs && onlyPushableWindowFunctions); +} diff --git a/src/backend/distributed/planner/multi_logical_optimizer.c b/src/backend/distributed/planner/multi_logical_optimizer.c index 9281f8e48..e692c5bee 100644 --- a/src/backend/distributed/planner/multi_logical_optimizer.c +++ b/src/backend/distributed/planner/multi_logical_optimizer.c @@ -64,6 +64,9 @@ int LimitClauseRowFetchCount = -1; /* number of rows to fetch from each task */ double CountDistinctErrorRate = 0.0; /* precision of count(distinct) approximate */ int CoordinatorAggregationStrategy = COORDINATOR_AGGREGATION_ROW_GATHER; +/* Constant used throughout file */ +static const uint32 masterTableId = 1; /* first range table reference on the master node */ + typedef struct MasterAggregateWalkerContext { const ExtendedOpNodeProperties *extendedOpNodeProperties; @@ -156,6 +159,7 @@ typedef enum LimitPushdownable typedef struct OrderByLimitReference { bool groupedByDisjointPartitionColumn; + bool onlyPushableWindowFunctions; bool groupClauseIsEmpty; bool sortClauseIsEmpty; bool hasOrderByAggregate; @@ -203,8 +207,8 @@ static void ParentSetNewChild(MultiNode *parentNode, MultiNode *oldChildNode, static void ApplyExtendedOpNodes(MultiExtendedOp *originalNode, MultiExtendedOp *masterNode, MultiExtendedOp *workerNode); -static void TransformSubqueryNode(MultiTable *subqueryNode, bool - requiresIntermediateRowPullUp); +static void TransformSubqueryNode(MultiTable *subqueryNode, + bool subqueryHasNonDistributableAggregates); static MultiExtendedOp * MasterExtendedOpNode(MultiExtendedOp *originalOpNode, ExtendedOpNodeProperties * extendedOpNodeProperties); @@ -239,6 +243,8 @@ static void ProcessWindowFunctionsForWorkerQuery(List *windowClauseList, List *originalTargetEntryList, QueryWindowClause *queryWindowClause, QueryTargetList *queryTargetList); +static void ProcessWindowFunctionPullUpForWorkerQuery(MultiExtendedOp *originalOpNode, + QueryTargetList *queryTargetList); static void ProcessLimitOrderByForWorkerQuery(OrderByLimitReference orderByLimitReference, Node *originalLimitCount, Node *limitOffset, List *sortClauseList, List *groupClauseList, @@ -247,6 +253,7 @@ static void ProcessLimitOrderByForWorkerQuery(OrderByLimitReference orderByLimit QueryTargetList *queryTargetList); static OrderByLimitReference BuildOrderByLimitReference(bool hasDistinctOn, bool groupedByDisjointPartitionColumn, + bool onlyPushableWindowFunctions, List *groupClause, List *sortClauseList, List *targetList); @@ -282,8 +289,10 @@ static Const * MakeIntegerConst(int32 integerValue); static Const * MakeIntegerConstInt64(int64 integerValue); /* Local functions forward declarations for aggregate expression checks */ -static bool RequiresIntermediateRowPullUp(MultiNode *logicalPlanNode); -static DeferredErrorMessage * DeferErrorIfContainsNonPushdownableAggregate( +static bool HasNonDistributableAggregates(MultiNode *logicalPlanNode); +static bool CanPushDownExpression(Node *expression, + const ExtendedOpNodeProperties *extendedOpNodeProperties); +static DeferredErrorMessage * DeferErrorIfHasNonDistributableAggregates( MultiNode *logicalPlanNode); static DeferredErrorMessage * DeferErrorIfUnsupportedArrayAggregate( Aggref *arrayAggregateExpression); @@ -315,6 +324,13 @@ static bool HasOrderByAggregate(List *sortClauseList, List *targetList); static bool HasOrderByNonCommutativeAggregate(List *sortClauseList, List *targetList); static bool HasOrderByComplexExpression(List *sortClauseList, List *targetList); static bool HasOrderByHllType(List *sortClauseList, List *targetList); +static bool ShouldPushDownGroupingToWorker(MultiExtendedOp *opNode, + ExtendedOpNodeProperties * + extendedOpNodeProperties); +static bool ShouldProcessDistinctOrderAndLimitForWorker( + ExtendedOpNodeProperties *extendedOpNodeProperties, + bool pushingDownOriginalGrouping, + Node *havingQual); /* @@ -333,20 +349,19 @@ static bool HasOrderByHllType(List *sortClauseList, List *targetList); void MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan) { - ListCell *collectNodeCell = NULL; - ListCell *tableNodeCell = NULL; MultiNode *logicalPlanNode = (MultiNode *) multiLogicalPlan; - bool requiresIntermediateRowPullUp = RequiresIntermediateRowPullUp(logicalPlanNode); + bool hasNonDistributableAggregates = HasNonDistributableAggregates( + logicalPlanNode); List *extendedOpNodeList = FindNodesOfType(logicalPlanNode, T_MultiExtendedOp); MultiExtendedOp *extendedOpNode = (MultiExtendedOp *) linitial(extendedOpNodeList); ExtendedOpNodeProperties extendedOpNodeProperties = BuildExtendedOpNodeProperties( - extendedOpNode, requiresIntermediateRowPullUp); + extendedOpNode, hasNonDistributableAggregates); if (!extendedOpNodeProperties.groupedByDisjointPartitionColumn && !extendedOpNodeProperties.pullUpIntermediateRows) { DeferredErrorMessage *aggregatePushdownError = - DeferErrorIfContainsNonPushdownableAggregate(logicalPlanNode); + DeferErrorIfHasNonDistributableAggregates(logicalPlanNode); if (aggregatePushdownError != NULL) { @@ -357,6 +372,7 @@ MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan) else { extendedOpNodeProperties.pullUpIntermediateRows = true; + extendedOpNodeProperties.pushDownGroupingAndHaving = false; } } } @@ -402,9 +418,9 @@ MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan) /* pull up collect nodes and merge duplicate collects */ List *collectNodeList = FindNodesOfType(logicalPlanNode, T_MultiCollect); - foreach(collectNodeCell, collectNodeList) + MultiCollect *collectNode = NULL; + foreach_ptr(collectNode, collectNodeList) { - MultiCollect *collectNode = (MultiCollect *) lfirst(collectNodeCell); PullUpCollectLoop(collectNode); } @@ -424,14 +440,14 @@ MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan) ApplyExtendedOpNodes(extendedOpNode, masterExtendedOpNode, workerExtendedOpNode); List *tableNodeList = FindNodesOfType(logicalPlanNode, T_MultiTable); - foreach(tableNodeCell, tableNodeList) + MultiTable *tableNode = NULL; + foreach_ptr(tableNode, tableNodeList) { - MultiTable *tableNode = (MultiTable *) lfirst(tableNodeCell); if (tableNode->relationId == SUBQUERY_RELATION_ID) { DeferredErrorMessage *error = - DeferErrorIfContainsNonPushdownableAggregate((MultiNode *) tableNode); - bool subqueryRequiresIntermediateRowPullUp = false; + DeferErrorIfHasNonDistributableAggregates((MultiNode *) tableNode); + bool subqueryHasNonDistributableAggregates = false; if (error != NULL) { @@ -441,11 +457,11 @@ MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan) } else { - subqueryRequiresIntermediateRowPullUp = true; + subqueryHasNonDistributableAggregates = true; } } - TransformSubqueryNode(tableNode, subqueryRequiresIntermediateRowPullUp); + TransformSubqueryNode(tableNode, subqueryHasNonDistributableAggregates); } } @@ -520,11 +536,10 @@ static List * OrSelectClauseList(List *selectClauseList) { List *orSelectClauseList = NIL; - ListCell *selectClauseCell = NULL; - foreach(selectClauseCell, selectClauseList) + Node *selectClause = NULL; + foreach_ptr(selectClause, selectClauseList) { - Node *selectClause = (Node *) lfirst(selectClauseCell); bool orClause = or_clause(selectClause); if (orClause) { @@ -947,11 +962,10 @@ static List * SelectClauseTableIdList(List *selectClauseList) { List *tableIdList = NIL; - ListCell *selectClauseCell = NULL; - foreach(selectClauseCell, selectClauseList) + Node *selectClause = NULL; + foreach_ptr(selectClause, selectClauseList) { - Node *selectClause = (Node *) lfirst(selectClauseCell); List *selectColumnList = pull_var_clause_default(selectClause); if (list_length(selectColumnList) == 0) @@ -1057,11 +1071,10 @@ static List * TableIdListColumns(List *tableIdList, List *columnList) { List *tableColumnList = NIL; - ListCell *columnCell = NULL; - foreach(columnCell, columnList) + Var *column = NULL; + foreach_ptr(column, columnList) { - Var *column = (Var *) lfirst(columnCell); int columnTableId = (int) column->varno; bool tableListMember = list_member_int(tableIdList, columnTableId); @@ -1084,12 +1097,10 @@ static List * TableIdListSelectClauses(List *tableIdList, List *selectClauseList) { List *tableSelectClauseList = NIL; - ListCell *selectClauseCell = NULL; - foreach(selectClauseCell, selectClauseList) + Node *selectClause = NULL; + foreach_ptr(selectClause, selectClauseList) { - Node *selectClause = (Node *) lfirst(selectClauseCell); - List *selectColumnList = pull_var_clause_default(selectClause); if (list_length(selectColumnList) == 0) { @@ -1310,12 +1321,13 @@ ApplyExtendedOpNodes(MultiExtendedOp *originalNode, MultiExtendedOp *masterNode, * operator node. */ static void -TransformSubqueryNode(MultiTable *subqueryNode, bool requiresIntermediateRowPullUp) +TransformSubqueryNode(MultiTable *subqueryNode, + bool subqueryHasNonDistributableAggregates) { if (CoordinatorAggregationStrategy != COORDINATOR_AGGREGATION_DISABLED && - RequiresIntermediateRowPullUp((MultiNode *) subqueryNode)) + HasNonDistributableAggregates((MultiNode *) subqueryNode)) { - requiresIntermediateRowPullUp = true; + subqueryHasNonDistributableAggregates = true; } MultiExtendedOp *extendedOpNode = @@ -1324,7 +1336,8 @@ TransformSubqueryNode(MultiTable *subqueryNode, bool requiresIntermediateRowPull MultiNode *collectChildNode = ChildNode((MultiUnaryNode *) collectNode); ExtendedOpNodeProperties extendedOpNodeProperties = - BuildExtendedOpNodeProperties(extendedOpNode, requiresIntermediateRowPullUp); + BuildExtendedOpNodeProperties(extendedOpNode, + subqueryHasNonDistributableAggregates); MultiExtendedOp *masterExtendedOpNode = MasterExtendedOpNode(extendedOpNode, &extendedOpNodeProperties); @@ -1399,7 +1412,6 @@ MasterExtendedOpNode(MultiExtendedOp *originalOpNode, List *targetEntryList = originalOpNode->targetList; List *newTargetEntryList = NIL; List *newGroupClauseList = NIL; - ListCell *targetEntryCell = NULL; Node *originalHavingQual = originalOpNode->havingQual; Node *newHavingQual = NULL; MasterAggregateWalkerContext walkerContext = { @@ -1408,36 +1420,19 @@ MasterExtendedOpNode(MultiExtendedOp *originalOpNode, }; /* iterate over original target entries */ - foreach(targetEntryCell, targetEntryList) + TargetEntry *originalTargetEntry = NULL; + foreach_ptr(originalTargetEntry, targetEntryList) { - TargetEntry *originalTargetEntry = (TargetEntry *) lfirst(targetEntryCell); TargetEntry *newTargetEntry = flatCopyTargetEntry(originalTargetEntry); Expr *originalExpression = originalTargetEntry->expr; Expr *newExpression = NULL; - bool hasAggregates = contain_aggs_of_level((Node *) originalExpression, 0); - bool hasWindowFunction = contain_window_function((Node *) originalExpression); - - /* - * if the aggregate belongs to a window function, it is not mutated, but pushed - * down to worker as it is. Master query should treat that as a Var. - */ - if (hasAggregates && !hasWindowFunction && - !extendedOpNodeProperties->groupedByDisjointPartitionColumn) - { - Node *newNode = MasterAggregateMutator((Node *) originalExpression, - &walkerContext); - newExpression = (Expr *) newNode; - } - else + if (CanPushDownExpression((Node *) originalExpression, extendedOpNodeProperties)) { /* - * The expression does not have any aggregates or the group by - * is on the partition column. We simply make it reference the - * output generated by worker nodes. + * The expression was entirely pushed down to worker. + * We simply make it reference the output generated by worker nodes. */ - const uint32 masterTableId = 1; /* only one table on master node */ - Var *column = makeVarFromTargetEntry(masterTableId, originalTargetEntry); column->varattno = walkerContext.columnId; column->varoattno = walkerContext.columnId; @@ -1450,12 +1445,18 @@ MasterExtendedOpNode(MultiExtendedOp *originalOpNode, newExpression = (Expr *) column; } + else + { + Node *newNode = MasterAggregateMutator((Node *) originalExpression, + &walkerContext); + newExpression = (Expr *) newNode; + } newTargetEntry->expr = newExpression; newTargetEntryList = lappend(newTargetEntryList, newTargetEntry); } - if (!extendedOpNodeProperties->groupedByDisjointPartitionColumn) + if (!extendedOpNodeProperties->pushDownGroupingAndHaving) { /* * Not pushing down GROUP BY, need to regroup on coordinator @@ -1488,6 +1489,13 @@ MasterExtendedOpNode(MultiExtendedOp *originalOpNode, masterExtendedOpNode->limitOffset = originalOpNode->limitOffset; masterExtendedOpNode->havingQual = newHavingQual; + if (!extendedOpNodeProperties->onlyPushableWindowFunctions) + { + masterExtendedOpNode->hasWindowFuncs = originalOpNode->hasWindowFuncs; + masterExtendedOpNode->windowClause = originalOpNode->windowClause; + masterExtendedOpNode->onlyPushableWindowFunctions = false; + } + return masterExtendedOpNode; } @@ -1515,13 +1523,35 @@ MasterAggregateMutator(Node *originalNode, MasterAggregateWalkerContext *walkerC if (IsA(originalNode, Aggref)) { Aggref *originalAggregate = (Aggref *) originalNode; - Expr *newExpression = MasterAggregateExpression(originalAggregate, walkerContext); + if (CanPushDownExpression(originalNode, + walkerContext->extendedOpNodeProperties)) + { + /* + * The expression was entirely pushed down to worker. + * We simply make it reference the output generated by worker nodes. + */ + Var *column = makeVar(masterTableId, walkerContext->columnId, + originalAggregate->aggtype, + -1, originalAggregate->aggcollid, 0); + walkerContext->columnId++; - newNode = (Node *) newExpression; + if (column->vartype == RECORDOID || column->vartype == RECORDARRAYOID) + { + column->vartypmod = BlessRecordExpression((Expr *) originalNode); + } + + newNode = (Node *) column; + } + else + { + Expr *newExpression = MasterAggregateExpression(originalAggregate, + walkerContext); + + newNode = (Node *) newExpression; + } } else if (IsA(originalNode, Var)) { - uint32 masterTableId = 1; /* one table on the master node */ Var *newColumn = copyObject((Var *) originalNode); newColumn->varno = masterTableId; newColumn->varattno = walkerContext->columnId; @@ -1553,7 +1583,6 @@ static Expr * MasterAggregateExpression(Aggref *originalAggregate, MasterAggregateWalkerContext *walkerContext) { - const uint32 masterTableId = 1; /* one table on the master node */ const Index columnLevelsUp = 0; /* normal column */ const AttrNumber argumentId = 1; /* our aggregates have single arguments */ AggregateType aggregateType = GetAggregateType(originalAggregate); @@ -1612,14 +1641,13 @@ MasterAggregateExpression(Aggref *originalAggregate, { Aggref *aggregate = (Aggref *) copyObject(originalAggregate); List *varList = pull_var_clause_default((Node *) aggregate); - ListCell *varCell = NULL; List *uniqueVarList = NIL; int startColumnCount = walkerContext->columnId; /* determine unique vars that were placed in target list by worker */ - foreach(varCell, varList) + Var *column = NULL; + foreach_ptr(column, varList) { - Var *column = (Var *) lfirst(varCell); uniqueVarList = list_append_unique(uniqueVarList, copyObject(column)); } @@ -1627,15 +1655,14 @@ MasterAggregateExpression(Aggref *originalAggregate, * Go over each var inside aggregate and update their varattno's according to * worker query target entry column index. */ - foreach(varCell, varList) + Var *columnToUpdate = NULL; + foreach_ptr(columnToUpdate, varList) { - Var *columnToUpdate = (Var *) lfirst(varCell); - ListCell *uniqueVarCell = NULL; int columnIndex = 0; - foreach(uniqueVarCell, uniqueVarList) + Var *currentVar = NULL; + foreach_ptr(currentVar, uniqueVarList) { - Var *currentVar = (Var *) lfirst(uniqueVarCell); if (equal(columnToUpdate, currentVar)) { break; @@ -2002,7 +2029,7 @@ MasterAggregateExpression(Aggref *originalAggregate, Oid masterReturnType = get_func_rettype(aggregateFunctionId); /* - * If return type aggregate is anyelement, it's actual return type is + * If return type aggregate is anyelement, its actual return type is * determined on the type of its argument. So we replace it with the * argument type in that case. */ @@ -2063,7 +2090,6 @@ MasterAverageExpression(Oid sumAggregateType, Oid countAggregateType, AttrNumber *columnId) { const char *sumAggregateName = AggregateNames[AGGREGATE_SUM]; - const uint32 masterTableId = 1; const int32 defaultTypeMod = -1; const Index defaultLevelsUp = 0; const AttrNumber argumentId = 1; @@ -2156,8 +2182,6 @@ WorkerExtendedOpNode(MultiExtendedOp *originalOpNode, ExtendedOpNodeProperties *extendedOpNodeProperties) { bool distinctPreventsLimitPushdown = false; - bool groupedByDisjointPartitionColumn = - extendedOpNodeProperties->groupedByDisjointPartitionColumn; QueryTargetList queryTargetList; QueryGroupClause queryGroupClause; @@ -2177,7 +2201,6 @@ WorkerExtendedOpNode(MultiExtendedOp *originalOpNode, bool hasDistinctOn = originalOpNode->hasDistinctOn; int originalGroupClauseLength = list_length(originalGroupClauseList); - bool queryHasAggregates = TargetListHasAggregates(originalTargetEntryList); /* initialize to default values */ memset(&queryTargetList, 0, sizeof(queryTargetList)); @@ -2192,17 +2215,22 @@ WorkerExtendedOpNode(MultiExtendedOp *originalOpNode, /* targetProjectionNumber starts from 1 */ queryTargetList.targetProjectionNumber = 1; - /* - * only push down grouping to worker query when pushing down aggregates - */ - if (extendedOpNodeProperties->pullUpIntermediateRows) - { - queryGroupClause.groupClauseList = NIL; - } - else + if (ShouldPushDownGroupingToWorker(originalOpNode, extendedOpNodeProperties)) { queryGroupClause.groupClauseList = copyObject(originalGroupClauseList); } + else + { + queryGroupClause.groupClauseList = NIL; + } + + /* + * For the purpose of this variable, not pushing down when there are no groups + * is pushing down the original grouping, ie the worker's GROUP BY matches + * the master's GROUP BY. + */ + bool pushingDownOriginalGrouping = + list_length(queryGroupClause.groupClauseList) == originalGroupClauseLength; /* * nextSortGroupRefIndex is used by group by, window and order by clauses. @@ -2221,11 +2249,23 @@ WorkerExtendedOpNode(MultiExtendedOp *originalOpNode, &queryHavingQual, &queryTargetList, &queryGroupClause); - ProcessWindowFunctionsForWorkerQuery(originalWindowClause, originalTargetEntryList, - &queryWindowClause, &queryTargetList); - - if (!extendedOpNodeProperties->pullUpIntermediateRows) + if (extendedOpNodeProperties->onlyPushableWindowFunctions) { + ProcessWindowFunctionsForWorkerQuery(originalWindowClause, + originalTargetEntryList, + &queryWindowClause, &queryTargetList); + } + else + { + ProcessWindowFunctionPullUpForWorkerQuery(originalOpNode, &queryTargetList); + } + + if (ShouldProcessDistinctOrderAndLimitForWorker(extendedOpNodeProperties, + pushingDownOriginalGrouping, + originalHavingQual)) + { + bool queryHasAggregates = TargetListHasAggregates(originalTargetEntryList); + ProcessDistinctClauseForWorkerQuery(originalDistinctClause, hasDistinctOn, queryGroupClause.groupClauseList, queryHasAggregates, &queryDistinctClause, @@ -2242,12 +2282,16 @@ WorkerExtendedOpNode(MultiExtendedOp *originalOpNode, */ bool groupByExtended = list_length(queryGroupClause.groupClauseList) > originalGroupClauseLength; - if (!groupByExtended && !distinctPreventsLimitPushdown) + if (pushingDownOriginalGrouping && !groupByExtended && + !distinctPreventsLimitPushdown) { /* both sort and limit clauses rely on similar information */ OrderByLimitReference limitOrderByReference = BuildOrderByLimitReference(hasDistinctOn, + extendedOpNodeProperties-> groupedByDisjointPartitionColumn, + extendedOpNodeProperties-> + onlyPushableWindowFunctions, originalGroupClauseList, originalSortClauseList, originalTargetEntryList); @@ -2290,9 +2334,8 @@ WorkerExtendedOpNode(MultiExtendedOp *originalOpNode, * might end up with multiple expressions in the worker query. * * The function doesn't change the aggregates in the window functions and sends them - * as-is. The reason is that Citus currently only supports pushing down window - * functions as-is. As we implement pull-to-master window functions, we should - * revisit here as well. + * as-is. The reason is that Citus only supports pushing down window functions when + * this is safe to do. * * The function also handles count distinct operator if it is used in repartition * subqueries or on non-partition columns (e.g., cannot be pushed down). Each @@ -2309,43 +2352,35 @@ ProcessTargetListForWorkerQuery(List *targetEntryList, QueryTargetList *queryTargetList, QueryGroupClause *queryGroupClause) { - ListCell *targetEntryCell = NULL; WorkerAggregateWalkerContext workerAggContext = { .extendedOpNodeProperties = extendedOpNodeProperties, }; /* iterate over original target entries */ - foreach(targetEntryCell, targetEntryList) + TargetEntry *originalTargetEntry = NULL; + foreach_ptr(originalTargetEntry, targetEntryList) { - TargetEntry *originalTargetEntry = (TargetEntry *) lfirst(targetEntryCell); Expr *originalExpression = originalTargetEntry->expr; List *newExpressionList = NIL; - bool hasAggregates = contain_aggs_of_level((Node *) originalExpression, 0); - bool hasWindowFunction = contain_window_function((Node *) originalExpression); /* reset walker context */ workerAggContext.expressionList = NIL; workerAggContext.createGroupByClause = false; /* - * If the query has a window function, we currently assume it's safe to push - * down the target list. - * - * If there are aggregates without a GROUP BY on the distribution column - * then the results of those aggregates need to be combined on the coordinator. - * In that case we rewrite the expressions using WorkerAggregateWalker. + * If we can push down the expression we copy the expression to the targetlist of the worker query. + * Otherwise the expression is processed to be combined on the coordinator. */ - if (!hasWindowFunction && hasAggregates && - !extendedOpNodeProperties->groupedByDisjointPartitionColumn) + if (CanPushDownExpression((Node *) originalExpression, extendedOpNodeProperties)) + { + newExpressionList = list_make1(originalExpression); + } + else { WorkerAggregateWalker((Node *) originalExpression, &workerAggContext); newExpressionList = workerAggContext.expressionList; } - else - { - newExpressionList = list_make1(originalExpression); - } ExpandWorkerTargetEntry(newExpressionList, originalTargetEntry, workerAggContext.createGroupByClause, @@ -2374,45 +2409,14 @@ ProcessHavingClauseForWorkerQuery(Node *originalHavingQual, QueryTargetList *queryTargetList, QueryGroupClause *queryGroupClause) { + *workerHavingQual = NULL; + if (originalHavingQual == NULL) { return; } - *workerHavingQual = NULL; - - if (!extendedOpNodeProperties->groupedByDisjointPartitionColumn) - { - /* - * If the GROUP BY or PARTITION BY is not on the distribution column - * then we need to combine the aggregates in the HAVING across shards. - */ - WorkerAggregateWalkerContext workerAggContext = { - .extendedOpNodeProperties = extendedOpNodeProperties, - }; - - WorkerAggregateWalker(originalHavingQual, &workerAggContext); - List *newExpressionList = workerAggContext.expressionList; - TargetEntry *targetEntry = NULL; - - ExpandWorkerTargetEntry(newExpressionList, targetEntry, - workerAggContext.createGroupByClause, - queryTargetList, queryGroupClause); - } - - /* - * If grouped by a partition column whose values are shards have disjoint sets - * of partition values, we can push down the having qualifier. - * - * When a query with subquery is provided, we can't determine if - * groupedByDisjointPartitionColumn, therefore we also check if there is a - * window function too. If there is a window function we would know that it - * is safe to push down (i.e. it is partitioned on distribution column, and - * if there is a group by, it contains distribution column). - * - */ - if (extendedOpNodeProperties->groupedByDisjointPartitionColumn || - extendedOpNodeProperties->pushDownWindowFunctions) + if (extendedOpNodeProperties->pushDownGroupingAndHaving) { /* * We converted the having expression to a list in subquery pushdown @@ -2430,6 +2434,24 @@ ProcessHavingClauseForWorkerQuery(Node *originalHavingQual, *workerHavingQual = originalHavingQual; } } + else + { + /* + * If the GROUP BY or PARTITION BY is not on the distribution column + * then we need to combine the aggregates in the HAVING across shards. + */ + WorkerAggregateWalkerContext workerAggContext = { + .extendedOpNodeProperties = extendedOpNodeProperties, + }; + + WorkerAggregateWalker(originalHavingQual, &workerAggContext); + List *newExpressionList = workerAggContext.expressionList; + TargetEntry *targetEntry = NULL; + + ExpandWorkerTargetEntry(newExpressionList, targetEntry, + workerAggContext.createGroupByClause, + queryTargetList, queryGroupClause); + } } @@ -2506,11 +2528,6 @@ ProcessDistinctClauseForWorkerQuery(List *distinctClause, bool hasDistinctOn, * that worker query's workerWindowClauseList is set when the window clauses are safe to * pushdown. * - * TODO: Citus only supports pushing down window clauses as-is under certain circumstances. - * And, at this point in the planning, we are guaranteed to process a window function - * which is safe to pushdown as-is. It should also be possible to pull the relevant data - * to the coordinator and apply the window clauses for the remaining cases. - * * Note that even though Citus only pushes down the window functions, it may need to * modify the target list of the worker query when the window function refers to * an avg(). The reason is that any aggregate which is also referred by other @@ -2528,8 +2545,6 @@ ProcessWindowFunctionsForWorkerQuery(List *windowClauseList, QueryWindowClause *queryWindowClause, QueryTargetList *queryTargetList) { - ListCell *windowClauseCell = NULL; - if (windowClauseList == NIL) { queryWindowClause->hasWindowFunctions = false; @@ -2537,10 +2552,9 @@ ProcessWindowFunctionsForWorkerQuery(List *windowClauseList, return; } - foreach(windowClauseCell, windowClauseList) + WindowClause *windowClause = NULL; + foreach_ptr(windowClause, windowClauseList) { - WindowClause *windowClause = (WindowClause *) lfirst(windowClauseCell); - List *partitionClauseTargetList = GenerateNewTargetEntriesForSortClauses(originalTargetEntryList, windowClause->partitionClause, @@ -2575,6 +2589,37 @@ ProcessWindowFunctionsForWorkerQuery(List *windowClauseList, } +/* ProcessWindowFunctionPullUpForWorkerQuery pulls up inputs for window functions */ +static void +ProcessWindowFunctionPullUpForWorkerQuery(MultiExtendedOp *originalOpNode, + QueryTargetList *queryTargetList) +{ + if (originalOpNode->windowClause != NIL) + { + List *columnList = pull_var_clause_default((Node *) originalOpNode->windowClause); + + Expr *newExpression = NULL; + foreach_ptr(newExpression, columnList) + { + TargetEntry *newTargetEntry = makeNode(TargetEntry); + + newTargetEntry->expr = newExpression; + + newTargetEntry->resname = + WorkerColumnName(queryTargetList->targetProjectionNumber); + + /* force resjunk to false as we may need this on the master */ + newTargetEntry->resjunk = false; + newTargetEntry->resno = queryTargetList->targetProjectionNumber; + + queryTargetList->targetEntryList = + lappend(queryTargetList->targetEntryList, newTargetEntry); + queryTargetList->targetProjectionNumber++; + } + } +} + + /* * ProcessLimitOrderByForWorkerQuery gets the inputs and modifies the outputs * such that worker query's LIMIT and ORDER BY clauses are set accordingly. @@ -2627,12 +2672,15 @@ ProcessLimitOrderByForWorkerQuery(OrderByLimitReference orderByLimitReference, */ static OrderByLimitReference BuildOrderByLimitReference(bool hasDistinctOn, bool groupedByDisjointPartitionColumn, + bool onlyPushableWindowFunctions, List *groupClause, List *sortClauseList, List *targetList) { OrderByLimitReference limitOrderByReference; limitOrderByReference.groupedByDisjointPartitionColumn = groupedByDisjointPartitionColumn; + limitOrderByReference.onlyPushableWindowFunctions = + onlyPushableWindowFunctions; limitOrderByReference.hasDistinctOn = hasDistinctOn; limitOrderByReference.groupClauseIsEmpty = (groupClause == NIL); limitOrderByReference.sortClauseIsEmpty = (sortClauseList == NIL); @@ -2648,16 +2696,14 @@ BuildOrderByLimitReference(bool hasDistinctOn, bool groupedByDisjointPartitionCo /* * TargetListHasAggregates returns true if any of the elements in the * target list contain aggregates that are not inside the window functions. + * This function should not be called if window functions are being pulled up. */ static bool TargetListHasAggregates(List *targetEntryList) { - ListCell *targetEntryCell = NULL; - - /* iterate over original target entries */ - foreach(targetEntryCell, targetEntryList) + TargetEntry *targetEntry = NULL; + foreach_ptr(targetEntry, targetEntryList) { - TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); Expr *targetExpr = targetEntry->expr; bool hasAggregates = contain_aggs_of_level((Node *) targetExpr, 0); bool hasWindowFunction = contain_window_function((Node *) targetExpr); @@ -2698,13 +2744,10 @@ ExpandWorkerTargetEntry(List *expressionList, TargetEntry *originalTargetEntry, bool addToGroupByClause, QueryTargetList *queryTargetList, QueryGroupClause *queryGroupClause) { - ListCell *newExpressionCell = NULL; - /* now create target entries for each new expression */ - foreach(newExpressionCell, expressionList) + Expr *newExpression = NULL; + foreach_ptr(newExpression, expressionList) { - Expr *newExpression = (Expr *) lfirst(newExpressionCell); - /* generate and add the new target entry to the target list */ TargetEntry *newTargetEntry = GenerateWorkerTargetEntry(originalTargetEntry, newExpression, @@ -2736,14 +2779,12 @@ ExpandWorkerTargetEntry(List *expressionList, TargetEntry *originalTargetEntry, static Index GetNextSortGroupRef(List *targetEntryList) { - ListCell *targetEntryCell = NULL; Index nextSortGroupRefIndex = 0; /* find max of sort group ref index */ - foreach(targetEntryCell, targetEntryList) + TargetEntry *targetEntry = NULL; + foreach_ptr(targetEntry, targetEntryList) { - TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); - if (targetEntry->ressortgroupref > nextSortGroupRefIndex) { nextSortGroupRefIndex = targetEntry->ressortgroupref; @@ -2786,12 +2827,7 @@ GenerateWorkerTargetEntry(TargetEntry *targetEntry, Expr *workerExpression, if (newTargetEntry->resname == NULL) { - StringInfo columnNameString = makeStringInfo(); - - appendStringInfo(columnNameString, WORKER_COLUMN_FORMAT, - targetProjectionNumber); - - newTargetEntry->resname = columnNameString->data; + newTargetEntry->resname = WorkerColumnName(targetProjectionNumber); } /* we can't generate a target entry without an expression */ @@ -2856,12 +2892,20 @@ WorkerAggregateWalker(Node *node, WorkerAggregateWalkerContext *walkerContext) if (IsA(node, Aggref)) { - Aggref *originalAggregate = (Aggref *) node; - List *workerAggregateList = WorkerAggregateExpressionList(originalAggregate, - walkerContext); + if (CanPushDownExpression(node, walkerContext->extendedOpNodeProperties)) + { + walkerContext->expressionList = lappend(walkerContext->expressionList, + node); + } + else + { + Aggref *originalAggregate = (Aggref *) node; + List *workerAggregateList = WorkerAggregateExpressionList(originalAggregate, + walkerContext); - walkerContext->expressionList = list_concat(walkerContext->expressionList, - workerAggregateList); + walkerContext->expressionList = list_concat(walkerContext->expressionList, + workerAggregateList); + } } else if (IsA(node, Var)) { @@ -2927,10 +2971,10 @@ WorkerAggregateExpressionList(Aggref *originalAggregate, { Aggref *aggregate = (Aggref *) copyObject(originalAggregate); List *columnList = pull_var_clause_default((Node *) aggregate); - ListCell *columnCell = NULL; - foreach(columnCell, columnList) + + Var *column = NULL; + foreach_ptr(column, columnList) { - Var *column = (Var *) lfirst(columnCell); workerAggregateList = list_append_unique(workerAggregateList, column); } @@ -3054,7 +3098,6 @@ WorkerAggregateExpressionList(Aggref *originalAggregate, if (combine != InvalidOid) { - ListCell *originalAggArgCell; Oid workerPartialId = WorkerPartialAggOid(); Const *aggOidParam = makeConst(REGPROCEDUREOID, -1, InvalidOid, sizeof(Oid), @@ -3062,9 +3105,9 @@ WorkerAggregateExpressionList(Aggref *originalAggregate, false, true); List *aggArguments = list_make1(makeTargetEntry((Expr *) aggOidParam, 1, NULL, false)); - foreach(originalAggArgCell, originalAggregate->args) + TargetEntry *arg = NULL; + foreach_ptr(arg, originalAggregate->args) { - TargetEntry *arg = lfirst(originalAggArgCell); TargetEntry *newArg = copyObject(arg); newArg->resno++; aggArguments = lappend(aggArguments, newArg); @@ -3466,10 +3509,12 @@ MakeIntegerConstInt64(int64 integerValue) /* - * RequiresIntermediateRowPullUp checks for if any aggregates cannot be pushed down. + * HasNonDistributableAggregates checks for if any aggregates cannot be pushed down. + * This only checks with GetAggregateType. DeferErrorIfHasNonDistributableAggregates + * performs further checks which should be done if aggregates are not being pushed down. */ static bool -RequiresIntermediateRowPullUp(MultiNode *logicalPlanNode) +HasNonDistributableAggregates(MultiNode *logicalPlanNode) { if (CoordinatorAggregationStrategy == COORDINATOR_AGGREGATION_DISABLED) { @@ -3514,12 +3559,50 @@ RequiresIntermediateRowPullUp(MultiNode *logicalPlanNode) /* - * DeferErrorIfContainsNonPushdownableAggregate extracts aggregate expressions from + * CanPushDownExpression returns whether the expression can be pushed down to workers. + */ +static bool +CanPushDownExpression(Node *expression, + const ExtendedOpNodeProperties *extendedOpNodeProperties) +{ + bool hasAggregate = contain_aggs_of_level(expression, 0); + bool hasWindowFunction = contain_window_function(expression); + bool hasPushableWindowFunction = + hasWindowFunction && extendedOpNodeProperties->onlyPushableWindowFunctions; + + if (!hasAggregate && !hasWindowFunction) + { + return true; + } + + /* aggregates inside pushed down window functions can be pushed down */ + if (hasPushableWindowFunction) + { + return true; + } + + if (extendedOpNodeProperties->pushDownGroupingAndHaving && !hasWindowFunction) + { + return true; + } + + if (hasAggregate && !hasWindowFunction && + extendedOpNodeProperties->groupedByDisjointPartitionColumn) + { + return true; + } + + return false; +} + + +/* + * DeferErrorIfHasNonDistributableAggregates extracts aggregate expressions from * the logical plan, walks over them and uses helper functions to check if we * can transform these aggregate expressions and push them down to worker nodes. */ static DeferredErrorMessage * -DeferErrorIfContainsNonPushdownableAggregate(MultiNode *logicalPlanNode) +DeferErrorIfHasNonDistributableAggregates(MultiNode *logicalPlanNode) { DeferredErrorMessage *error = NULL; List *opNodeList = FindNodesOfType(logicalPlanNode, T_MultiExtendedOp); @@ -3537,11 +3620,9 @@ DeferErrorIfContainsNonPushdownableAggregate(MultiNode *logicalPlanNode) expressionList = list_concat(expressionList, pull_var_clause(havingQual, PVC_INCLUDE_AGGREGATES)); - ListCell *expressionCell = NULL; - foreach(expressionCell, expressionList) + Node *expression = NULL; + foreach_ptr(expression, expressionList) { - Node *expression = (Node *) lfirst(expressionCell); - /* only consider aggregate expressions */ if (!IsA(expression, Aggref)) { @@ -3681,10 +3762,10 @@ DeferErrorIfUnsupportedAggregateDistinct(Aggref *aggregateExpression, { Node *aggregateArgument = (Node *) linitial(aggregateExpression->args); List *columnList = pull_var_clause_default(aggregateArgument); - ListCell *columnCell = NULL; - foreach(columnCell, columnList) + + Var *column = NULL; + foreach_ptr(column, columnList) { - Var *column = (Var *) lfirst(columnCell); if (column->varattno <= 0) { return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, @@ -3697,10 +3778,10 @@ DeferErrorIfUnsupportedAggregateDistinct(Aggref *aggregateExpression, else { List *multiTableNodeList = FindNodesOfType(logicalPlanNode, T_MultiTable); - ListCell *multiTableNodeCell = NULL; - foreach(multiTableNodeCell, multiTableNodeList) + + MultiTable *multiTable = NULL; + foreach_ptr(multiTable, multiTableNodeList) { - MultiTable *multiTable = (MultiTable *) lfirst(multiTableNodeCell); if (multiTable->relationId == SUBQUERY_RELATION_ID || multiTable->relationId == SUBQUERY_PUSHDOWN_RELATION_ID) { @@ -3853,11 +3934,10 @@ TablePartitioningSupportsDistinct(List *tableNodeList, MultiExtendedOp *opNode, Var *distinctColumn, AggregateType aggregateType) { bool distinctSupported = true; - ListCell *tableNodeCell = NULL; - foreach(tableNodeCell, tableNodeList) + MultiTable *tableNode = NULL; + foreach_ptr(tableNode, tableNodeList) { - MultiTable *tableNode = (MultiTable *) lfirst(tableNodeCell); Oid relationId = tableNode->relationId; bool tableDistinctSupported = false; @@ -3927,11 +4007,10 @@ bool GroupedByColumn(List *groupClauseList, List *targetList, Var *column) { bool groupedByColumn = false; - ListCell *groupClauseCell = NULL; - foreach(groupClauseCell, groupClauseList) + SortGroupClause *groupClause = NULL; + foreach_ptr(groupClause, groupClauseList) { - SortGroupClause *groupClause = (SortGroupClause *) lfirst(groupClauseCell); TargetEntry *groupTargetEntry = get_sortgroupclause_tle(groupClause, targetList); Expr *groupExpression = (Expr *) groupTargetEntry->expr; @@ -3961,10 +4040,9 @@ SubqueryMultiTableList(MultiNode *multiNode) List *subqueryMultiTableList = NIL; List *multiTableNodeList = FindNodesOfType(multiNode, T_MultiTable); - ListCell *multiTableNodeCell = NULL; - foreach(multiTableNodeCell, multiTableNodeList) + MultiTable *multiTable = NULL; + foreach_ptr(multiTable, multiTableNodeList) { - MultiTable *multiTable = (MultiTable *) lfirst(multiTableNodeCell); Query *subquery = multiTable->subquery; if (subquery != NULL) @@ -3985,11 +4063,10 @@ List * GroupTargetEntryList(List *groupClauseList, List *targetEntryList) { List *groupTargetEntryList = NIL; - ListCell *groupClauseCell = NULL; - foreach(groupClauseCell, groupClauseList) + SortGroupClause *groupClause = NULL; + foreach_ptr(groupClause, groupClauseList) { - SortGroupClause *groupClause = (SortGroupClause *) lfirst(groupClauseCell); TargetEntry *groupTargetEntry = get_sortgroupclause_tle(groupClause, targetEntryList); groupTargetEntryList = lappend(groupTargetEntryList, groupTargetEntry); @@ -4037,7 +4114,7 @@ IsPartitionColumn(Expr *columnExpression, Query *query) /* * FindReferencedTableColumn recursively traverses query tree to find actual relation * id, and column that columnExpression refers to. If columnExpression is a - * non-relational or computed/derived expression, the function returns InvolidOid for + * non-relational or computed/derived expression, the function returns InvalidOid for * relationId and NULL for column. The caller should provide parent query list from * top of the tree to this particular Query's parent. This argument is used to look * into CTEs that may be present in the query. @@ -4076,7 +4153,7 @@ FindReferencedTableColumn(Expr *columnExpression, List *parentQueryList, Query * /* * We currently don't support finding partition keys in the subqueries - * that reference outer subqueries. For example, in corrolated + * that reference outer subqueries. For example, in correlated * subqueries in WHERE clause, we don't support use of partition keys * in the subquery that is referred from the outer query. */ @@ -4122,7 +4199,6 @@ FindReferencedTableColumn(Expr *columnExpression, List *parentQueryList, Query * rangeTableEntry->ctelevelsup - 1; Query *cteParentQuery = NULL; List *cteList = NIL; - ListCell *cteListCell = NULL; CommonTableExpr *cte = NULL; /* @@ -4136,9 +4212,9 @@ FindReferencedTableColumn(Expr *columnExpression, List *parentQueryList, Query * cteList = cteParentQuery->cteList; } - foreach(cteListCell, cteList) + CommonTableExpr *candidateCte = NULL; + foreach_ptr(candidateCte, cteList) { - CommonTableExpr *candidateCte = (CommonTableExpr *) lfirst(cteListCell); if (strcmp(candidateCte->ctename, rangeTableEntry->ctename) == 0) { cte = candidateCte; @@ -4233,13 +4309,18 @@ WorkerLimitCount(Node *limitCount, Node *limitOffset, OrderByLimitReference Assert(limitOffset == NULL || IsA(limitOffset, Const)); /* + * If window functions are computed on coordinator, we cannot push down LIMIT. * If we don't have group by clauses, or we have group by partition column, * or if we have order by clauses without aggregates, we can push down the * original limit. Else if we have order by clauses with commutative aggregates, * we can push down approximate limits. */ - if (orderByLimitReference.groupClauseIsEmpty || - orderByLimitReference.groupedByDisjointPartitionColumn) + if (!orderByLimitReference.onlyPushableWindowFunctions) + { + canPushDownLimit = LIMIT_CANNOT_PUSHDOWN; + } + else if (orderByLimitReference.groupClauseIsEmpty || + orderByLimitReference.groupedByDisjointPartitionColumn) { canPushDownLimit = LIMIT_CAN_PUSHDOWN; } @@ -4318,6 +4399,12 @@ WorkerSortClauseList(Node *limitCount, List *groupClauseList, List *sortClauseLi return NIL; } + /* If window functions are computed on coordinator, we cannot push down sorting. */ + if (!orderByLimitReference.onlyPushableWindowFunctions) + { + return NIL; + } + sortClauseList = copyObject(sortClauseList); /* @@ -4374,11 +4461,10 @@ GenerateNewTargetEntriesForSortClauses(List *originalTargetList, Index *nextSortGroupRefIndex) { List *createdTargetList = NIL; - ListCell *sortClauseCell = NULL; - foreach(sortClauseCell, sortClauseList) + SortGroupClause *sgClause = NULL; + foreach_ptr(sgClause, sortClauseList) { - SortGroupClause *sgClause = (SortGroupClause *) lfirst(sortClauseCell); TargetEntry *targetEntry = get_sortgroupclause_tle(sgClause, originalTargetList); Expr *targetExpr = targetEntry->expr; bool containsAggregate = contain_aggs_of_level((Node *) targetExpr, 0); @@ -4476,11 +4562,10 @@ static bool HasOrderByAggregate(List *sortClauseList, List *targetList) { bool hasOrderByAggregate = false; - ListCell *sortClauseCell = NULL; - foreach(sortClauseCell, sortClauseList) + SortGroupClause *sortClause = NULL; + foreach_ptr(sortClause, sortClauseList) { - SortGroupClause *sortClause = (SortGroupClause *) lfirst(sortClauseCell); Node *sortExpression = get_sortgroupclause_expr(sortClause, targetList); bool containsAggregate = contain_aggs_of_level(sortExpression, 0); @@ -4503,11 +4588,10 @@ static bool HasOrderByNonCommutativeAggregate(List *sortClauseList, List *targetList) { bool hasOrderByNonCommutativeAggregate = false; - ListCell *sortClauseCell = NULL; - foreach(sortClauseCell, sortClauseList) + SortGroupClause *sortClause = NULL; + foreach_ptr(sortClause, sortClauseList) { - SortGroupClause *sortClause = (SortGroupClause *) lfirst(sortClauseCell); Node *sortExpression = get_sortgroupclause_expr(sortClause, targetList); /* if sort expression is an aggregate, check its type */ @@ -4544,11 +4628,10 @@ static bool HasOrderByComplexExpression(List *sortClauseList, List *targetList) { bool hasOrderByComplexExpression = false; - ListCell *sortClauseCell = NULL; - foreach(sortClauseCell, sortClauseList) + SortGroupClause *sortClause = NULL; + foreach_ptr(sortClause, sortClauseList) { - SortGroupClause *sortClause = (SortGroupClause *) lfirst(sortClauseCell); Node *sortExpression = get_sortgroupclause_expr(sortClause, targetList); /* simple aggregate functions are ok */ @@ -4577,7 +4660,6 @@ static bool HasOrderByHllType(List *sortClauseList, List *targetList) { bool hasOrderByHllType = false; - ListCell *sortClauseCell = NULL; /* check whether HLL is loaded */ Oid hllId = get_extension_oid(HLL_EXTENSION_NAME, true); @@ -4589,9 +4671,9 @@ HasOrderByHllType(List *sortClauseList, List *targetList) Oid hllSchemaOid = get_extension_schema(hllId); Oid hllTypeId = TypeOid(hllSchemaOid, HLL_TYPE_NAME); - foreach(sortClauseCell, sortClauseList) + SortGroupClause *sortClause = NULL; + foreach_ptr(sortClause, sortClauseList) { - SortGroupClause *sortClause = (SortGroupClause *) lfirst(sortClauseCell); Node *sortExpression = get_sortgroupclause_expr(sortClause, targetList); Oid sortColumnTypeId = exprType(sortExpression); @@ -4606,6 +4688,99 @@ HasOrderByHllType(List *sortClauseList, List *targetList) } +/* + * ShouldPushDownGroupingToWorker returns whether we push down GROUP BY. + * This may return true even when GROUP BY is necessary on master. + */ +static bool +ShouldPushDownGroupingToWorker(MultiExtendedOp *opNode, + ExtendedOpNodeProperties *extendedOpNodeProperties) +{ + if (extendedOpNodeProperties->pushDownGroupingAndHaving) + { + return true; + } + + if (extendedOpNodeProperties->pullUpIntermediateRows) + { + return false; + } + + /* + * Duplicate grouping if we have LIMIT without HAVING, as this can + * often result in LIMIT being pushed down. + */ + if (opNode->havingQual == NULL && opNode->limitCount != NULL) + { + return true; + } + + /* + * If aggregates are being split across worker & master, so must grouping. + */ + if (contain_aggs_of_level(opNode->havingQual, 0) || + contain_aggs_of_level((Node *) opNode->targetList, 0)) + { + return true; + } + + return false; +} + + +/* + * ShouldProcessDistinctOrderAndLimitForWorker returns whether + * ProcessDistinctClauseForWorkerQuery should be called. If not, + * neither should ProcessLimitOrderByForWorkerQuery. + */ +static bool +ShouldProcessDistinctOrderAndLimitForWorker( + ExtendedOpNodeProperties *extendedOpNodeProperties, + bool pushingDownOriginalGrouping, + Node *havingQual) +{ + if (extendedOpNodeProperties->pullUpIntermediateRows) + { + return false; + } + + /* window functions must be evaluated beforehand */ + if (!extendedOpNodeProperties->onlyPushableWindowFunctions) + { + return false; + } + + if (extendedOpNodeProperties->pushDownGroupingAndHaving) + { + return true; + } + + /* If the same GROUP BY is being pushed down and there's no HAVING, + * then the push down logic will be able to handle this scenario. + */ + if (pushingDownOriginalGrouping && havingQual == NULL) + { + return true; + } + + return false; +} + + +/* + * WorkerColumnName returns a palloc'd string for being the resname of a TargetEntry. + */ +char * +WorkerColumnName(AttrNumber resno) +{ + StringInfoData name = { 0 }; + initStringInfo(&name); + appendStringInfo(&name, WORKER_COLUMN_FORMAT, resno); + + return name.data; +} + + /* * IsGroupBySubsetOfDistinct checks whether each clause in group clauses also * exists in the distinct clauses. Note that, empty group clause is not a subset @@ -4614,24 +4789,20 @@ HasOrderByHllType(List *sortClauseList, List *targetList) bool IsGroupBySubsetOfDistinct(List *groupClauses, List *distinctClauses) { - ListCell *distinctCell = NULL; - ListCell *groupCell = NULL; - /* There must be a group clause */ if (list_length(groupClauses) == 0) { return false; } - foreach(groupCell, groupClauses) + SortGroupClause *groupClause = NULL; + foreach_ptr(groupClause, groupClauses) { - SortGroupClause *groupClause = (SortGroupClause *) lfirst(groupCell); bool isFound = false; - foreach(distinctCell, distinctClauses) + SortGroupClause *distinctClause = NULL; + foreach_ptr(distinctClause, distinctClauses) { - SortGroupClause *distinctClause = (SortGroupClause *) lfirst(distinctCell); - if (groupClause->tleSortGroupRef == distinctClause->tleSortGroupRef) { isFound = true; diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index b4bd5f2de..d022dd0ef 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -206,7 +206,7 @@ FindNodeCheck(Node *node, bool (*check)(Node *)) * - Only a single RTE_RELATION exists, which means only a single table * name is specified on the whole query * - No sublinks exists in the subquery - * - No window functions in the subquery + * - No window functions exists in the subquery * * Note that the caller should still call DeferErrorIfUnsupportedSubqueryRepartition() * to ensure that Citus supports the subquery. Also, this function is designed to run @@ -777,7 +777,7 @@ MultiNodeTree(Query *queryTree) * distinguish between aggregates and expressions; and we address this later * in the logical optimizer. */ - MultiExtendedOp *extendedOpNode = MultiExtendedOpNode(queryTree); + MultiExtendedOp *extendedOpNode = MultiExtendedOpNode(queryTree, queryTree); SetChild((MultiUnaryNode *) extendedOpNode, currentTopNode); currentTopNode = (MultiNode *) extendedOpNode; @@ -923,7 +923,6 @@ DeferErrorIfQueryNotSupported(Query *queryTree) { char *errorMessage = NULL; bool preconditionsSatisfied = true; - StringInfo errorInfo = NULL; const char *errorHint = NULL; const char *joinHint = "Consider joining tables on partition column and have " "equal filter on joining columns."; @@ -942,18 +941,6 @@ DeferErrorIfQueryNotSupported(Query *queryTree) errorHint = filterHint; } - if (queryTree->hasWindowFuncs && - !SafeToPushdownWindowFunction(queryTree, &errorInfo)) - { - preconditionsSatisfied = false; - errorMessage = "could not run distributed query because the window " - "function that is used cannot be pushed down"; - errorHint = "Window functions are supported in two ways. Either add " - "an equality filter on the distributed tables' partition " - "column or use the window functions with a PARTITION BY " - "clause containing the distribution column"; - } - if (queryTree->setOperations) { preconditionsSatisfied = false; @@ -1826,7 +1813,7 @@ MultiProjectNode(List *targetEntryList) /* Builds the extended operator node using fields from the given query tree. */ MultiExtendedOp * -MultiExtendedOpNode(Query *queryTree) +MultiExtendedOpNode(Query *queryTree, Query *originalQuery) { MultiExtendedOp *extendedOpNode = CitusMakeNode(MultiExtendedOp); extendedOpNode->targetList = queryTree->targetList; @@ -1839,6 +1826,9 @@ MultiExtendedOpNode(Query *queryTree) extendedOpNode->hasDistinctOn = queryTree->hasDistinctOn; extendedOpNode->hasWindowFuncs = queryTree->hasWindowFuncs; extendedOpNode->windowClause = queryTree->windowClause; + extendedOpNode->onlyPushableWindowFunctions = + !queryTree->hasWindowFuncs || + SafeToPushdownWindowFunction(originalQuery, NULL); return extendedOpNode; } diff --git a/src/backend/distributed/planner/multi_master_planner.c b/src/backend/distributed/planner/multi_master_planner.c index 30702e79b..bd483f478 100644 --- a/src/backend/distributed/planner/multi_master_planner.c +++ b/src/backend/distributed/planner/multi_master_planner.c @@ -21,6 +21,7 @@ #include "distributed/multi_physical_planner.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" +#include "optimizer/clauses.h" #include "optimizer/planner.h" #include "rewrite/rewriteManip.h" @@ -206,10 +207,11 @@ BuildSelectStatementViaStdPlanner(Query *masterQuery, List *masterTargetList, /* probably want to do this where we add sublinks to the master plan */ masterQuery->hasSubLinks = checkExprHasSubLink((Node *) masterQuery); + Assert(masterQuery->hasWindowFuncs == contain_window_function((Node *) masterQuery)); /* * We will overwrite the alias of the rangetable which describes the custom scan. - * Idealy we would have set the correct column names and alias on the range table in + * Ideally we would have set the correct column names and alias on the range table in * the master query already when we inserted the extra data container. This could be * improved in the future. */ diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index c19cfa64c..d94eb1ab9 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -590,6 +590,8 @@ BuildJobQuery(MultiNode *multiNode, List *dependentJobList) bool hasDistinctOn = false; List *distinctClause = NIL; bool isRepartitionJoin = false; + bool hasWindowFuncs = false; + List *windowClause = NIL; /* we start building jobs from below the collect node */ Assert(!CitusIsA(multiNode, MultiCollect)); @@ -640,6 +642,8 @@ BuildJobQuery(MultiNode *multiNode, List *dependentJobList) targetList = copyObject(extendedOp->targetList); distinctClause = extendedOp->distinctClause; hasDistinctOn = extendedOp->hasDistinctOn; + hasWindowFuncs = extendedOp->hasWindowFuncs; + windowClause = extendedOp->windowClause; } else { @@ -725,6 +729,8 @@ BuildJobQuery(MultiNode *multiNode, List *dependentJobList) contain_aggs_of_level((Node *) havingQual, 0); jobQuery->distinctClause = distinctClause; jobQuery->hasDistinctOn = hasDistinctOn; + jobQuery->windowClause = windowClause; + jobQuery->hasWindowFuncs = hasWindowFuncs; return jobQuery; } diff --git a/src/backend/distributed/planner/query_pushdown_planning.c b/src/backend/distributed/planner/query_pushdown_planning.c index d1f031e1b..bda86407a 100644 --- a/src/backend/distributed/planner/query_pushdown_planning.c +++ b/src/backend/distributed/planner/query_pushdown_planning.c @@ -107,8 +107,6 @@ bool ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery, PlannerRestrictionContext *plannerRestrictionContext) { - StringInfo errorMessage = NULL; - /* * We check the existence of subqueries in FROM clause on the modified query * given that if postgres already flattened the subqueries, MultiNodeTree() @@ -190,7 +188,7 @@ ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery, /* check if the query has a window function and it is safe to pushdown */ if (originalQuery->hasWindowFuncs && - SafeToPushdownWindowFunction(originalQuery, &errorMessage)) + SafeToPushdownWindowFunction(originalQuery, NULL)) { return true; } @@ -393,7 +391,7 @@ IsOuterJoinExpr(Node *node) /* * SafeToPushdownWindowFunction checks if the query with window function is supported. - * It returns the result accordingly and modifies the error detail. + * Returns the result accordingly and modifies errorDetail if non null. */ bool SafeToPushdownWindowFunction(Query *query, StringInfo *errorDetail) @@ -411,20 +409,26 @@ SafeToPushdownWindowFunction(Query *query, StringInfo *errorDetail) if (!windowClause->partitionClause) { - *errorDetail = makeStringInfo(); - appendStringInfoString(*errorDetail, - "Window functions without PARTITION BY on distribution " - "column is currently unsupported"); + if (errorDetail) + { + *errorDetail = makeStringInfo(); + appendStringInfoString(*errorDetail, + "Window functions without PARTITION BY on distribution " + "column is currently unsupported"); + } return false; } } if (!WindowPartitionOnDistributionColumn(query)) { - *errorDetail = makeStringInfo(); - appendStringInfoString(*errorDetail, - "Window functions with PARTITION BY list missing distribution " - "column is currently unsupported"); + if (errorDetail) + { + *errorDetail = makeStringInfo(); + appendStringInfoString(*errorDetail, + "Window functions with PARTITION BY list missing distribution " + "column is currently unsupported"); + } return false; } @@ -1515,8 +1519,9 @@ HasRecurringTuples(Node *node, RecurringTuplesType *recurType) * down to workers without invoking join order planner. */ static MultiNode * -SubqueryPushdownMultiNodeTree(Query *queryTree) +SubqueryPushdownMultiNodeTree(Query *originalQuery) { + Query *queryTree = copyObject(originalQuery); List *targetEntryList = queryTree->targetList; MultiCollect *subqueryCollectNode = CitusMakeNode(MultiCollect); @@ -1616,7 +1621,7 @@ SubqueryPushdownMultiNodeTree(Query *queryTree) * distinguish between aggregates and expressions; and we address this later * in the logical optimizer. */ - MultiExtendedOp *extendedOpNode = MultiExtendedOpNode(queryTree); + MultiExtendedOp *extendedOpNode = MultiExtendedOpNode(queryTree, originalQuery); /* * Postgres standard planner converts having qual node to a list of and @@ -1763,11 +1768,9 @@ CreateSubqueryTargetEntryList(List *exprList) { Node *expr = (Node *) lfirst(exprCell); TargetEntry *newTargetEntry = makeNode(TargetEntry); - StringInfo exprNameString = makeStringInfo(); newTargetEntry->expr = (Expr *) copyObject(expr); - appendStringInfo(exprNameString, WORKER_COLUMN_FORMAT, resNo); - newTargetEntry->resname = exprNameString->data; + newTargetEntry->resname = WorkerColumnName(resNo); newTargetEntry->resjunk = false; newTargetEntry->resno = resNo; diff --git a/src/backend/distributed/planner/relation_restriction_equivalence.c b/src/backend/distributed/planner/relation_restriction_equivalence.c index 04cb6c778..4ce08c6f9 100644 --- a/src/backend/distributed/planner/relation_restriction_equivalence.c +++ b/src/backend/distributed/planner/relation_restriction_equivalence.c @@ -1651,8 +1651,8 @@ RelationRestrictionPartitionKeyIndex(RelationRestriction *relationRestriction) partitionKeyTargetAttrIndex++; if (!targetEntry->resjunk && - IsPartitionColumn(targetExpression, relationPlannerParseQuery) && - IsA(targetExpression, Var)) + IsA(targetExpression, Var) && + IsPartitionColumn(targetExpression, relationPlannerParseQuery)) { Var *targetColumn = (Var *) targetExpression; diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index 73df7fa23..16e376cf0 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -323,6 +323,9 @@ OutMultiExtendedOp(OUTFUNC_ARGS) WRITE_NODE_FIELD(havingQual); WRITE_BOOL_FIELD(hasDistinctOn); WRITE_NODE_FIELD(distinctClause); + WRITE_BOOL_FIELD(hasWindowFuncs); + WRITE_BOOL_FIELD(onlyPushableWindowFunctions); + WRITE_NODE_FIELD(windowClause); OutMultiUnaryNodeFields(str, (const MultiUnaryNode *) node); } diff --git a/src/include/distributed/extended_op_node_utils.h b/src/include/distributed/extended_op_node_utils.h index 72615f615..5d2c7cead 100644 --- a/src/include/distributed/extended_op_node_utils.h +++ b/src/include/distributed/extended_op_node_utils.h @@ -27,13 +27,15 @@ typedef struct ExtendedOpNodeProperties bool repartitionSubquery; bool hasNonPartitionColumnDistinctAgg; bool pullDistinctColumns; - bool pushDownWindowFunctions; + bool hasWindowFuncs; + bool onlyPushableWindowFunctions; bool pullUpIntermediateRows; + bool pushDownGroupingAndHaving; } ExtendedOpNodeProperties; extern ExtendedOpNodeProperties BuildExtendedOpNodeProperties( - MultiExtendedOp *extendedOpNode, bool pullUpIntermediateRows); + MultiExtendedOp *extendedOpNode, bool hasNonDistributableAggregates); #endif /* EXTENDED_OP_NODE_UTILS_H_ */ diff --git a/src/include/distributed/multi_logical_optimizer.h b/src/include/distributed/multi_logical_optimizer.h index f9997c4ce..76467fbca 100644 --- a/src/include/distributed/multi_logical_optimizer.h +++ b/src/include/distributed/multi_logical_optimizer.h @@ -162,7 +162,7 @@ extern bool ExtractQueryWalker(Node *node, List **queryList); extern bool IsPartitionColumn(Expr *columnExpression, Query *query); extern void FindReferencedTableColumn(Expr *columnExpression, List *parentQueryList, Query *query, Oid *relationId, Var **column); - +extern char * WorkerColumnName(AttrNumber resno); extern bool IsGroupBySubsetOfDistinct(List *groupClauses, List *distinctClauses); #endif /* MULTI_LOGICAL_OPTIMIZER_H */ diff --git a/src/include/distributed/multi_logical_planner.h b/src/include/distributed/multi_logical_planner.h index 8a2fa1a0e..4cafe1a40 100644 --- a/src/include/distributed/multi_logical_planner.h +++ b/src/include/distributed/multi_logical_planner.h @@ -176,9 +176,10 @@ typedef struct MultiExtendedOp Node *limitOffset; Node *havingQual; List *distinctClause; + List *windowClause; bool hasDistinctOn; bool hasWindowFuncs; - List *windowClause; + bool onlyPushableWindowFunctions; } MultiExtendedOp; @@ -219,7 +220,7 @@ extern List * pull_var_clause_default(Node *node); extern bool OperatorImplementsEquality(Oid opno); extern DeferredErrorMessage * DeferErrorIfUnsupportedClause(List *clauseList); extern MultiProject * MultiProjectNode(List *targetEntryList); -extern MultiExtendedOp * MultiExtendedOpNode(Query *queryTree); +extern MultiExtendedOp * MultiExtendedOpNode(Query *queryTree, Query *originalQuery); extern DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryRepartition(Query * subqueryTree); extern MultiNode * MultiNodeTree(Query *queryTree); diff --git a/src/test/regress/expected/custom_aggregate_support.out b/src/test/regress/expected/custom_aggregate_support.out index f3816363d..cb72ad31f 100644 --- a/src/test/regress/expected/custom_aggregate_support.out +++ b/src/test/regress/expected/custom_aggregate_support.out @@ -89,17 +89,118 @@ ORDER BY 1; 7 | 13 (2 rows) --- These are going to be supported after window function support SELECT day, hll_cardinality(hll_union_agg(unique_users) OVER seven_days) FROM daily_uniques -WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING); -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column +WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING) +ORDER BY 1; + day | hll_cardinality +--------------------------------------------------------------------- + 05-24-2018 | 19 + 05-25-2018 | 19 + 05-26-2018 | 19 + 05-27-2018 | 19 + 05-28-2018 | 19 + 05-29-2018 | 19 + 05-30-2018 | 19 + 05-31-2018 | 19 + 06-01-2018 | 19 + 06-02-2018 | 19 + 06-03-2018 | 19 + 06-04-2018 | 19 + 06-05-2018 | 19 + 06-06-2018 | 19 + 06-07-2018 | 19 + 06-08-2018 | 19 + 06-09-2018 | 19 + 06-10-2018 | 19 + 06-11-2018 | 19 + 06-12-2018 | 19 + 06-13-2018 | 19 + 06-14-2018 | 19 + 06-15-2018 | 19 + 06-16-2018 | 19 + 06-17-2018 | 19 + 06-18-2018 | 19 + 06-19-2018 | 19 + 06-20-2018 | 19 + 06-21-2018 | 19 + 06-22-2018 | 19 + 06-23-2018 | 19 + 06-24-2018 | 19 + 06-25-2018 | 19 + 06-26-2018 | 19 + 06-27-2018 | 19 + 06-28-2018 | 19 + 06-29-2018 | 19 + 06-30-2018 | 19 + 07-01-2018 | 13 + 07-02-2018 | 13 + 07-03-2018 | 13 + 07-04-2018 | 13 + 07-05-2018 | 13 + 07-06-2018 | 13 + 07-07-2018 | 13 + 07-08-2018 | 13 + 07-09-2018 | 13 + 07-10-2018 | 13 +(48 rows) + SELECT day, (hll_cardinality(hll_union_agg(unique_users) OVER two_days)) - hll_cardinality(unique_users) AS lost_uniques FROM daily_uniques -WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING); -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column +WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING) +ORDER BY 1; + day | lost_uniques +--------------------------------------------------------------------- + 05-24-2018 | 0 + 05-25-2018 | 0 + 05-26-2018 | 0 + 05-27-2018 | 0 + 05-28-2018 | 0 + 05-29-2018 | 0 + 05-30-2018 | 0 + 05-31-2018 | 0 + 06-01-2018 | 0 + 06-02-2018 | 0 + 06-03-2018 | 0 + 06-04-2018 | 0 + 06-05-2018 | 0 + 06-06-2018 | 0 + 06-07-2018 | 0 + 06-08-2018 | 0 + 06-09-2018 | 0 + 06-10-2018 | 0 + 06-11-2018 | 0 + 06-12-2018 | 0 + 06-13-2018 | 0 + 06-14-2018 | 0 + 06-15-2018 | 0 + 06-16-2018 | 0 + 06-17-2018 | 0 + 06-18-2018 | 0 + 06-19-2018 | 0 + 06-20-2018 | 0 + 06-21-2018 | 0 + 06-22-2018 | 0 + 06-23-2018 | 0 + 06-24-2018 | 0 + 06-25-2018 | 6 + 06-26-2018 | 0 + 06-27-2018 | 0 + 06-28-2018 | 0 + 06-29-2018 | 0 + 06-30-2018 | 0 + 07-01-2018 | 0 + 07-02-2018 | 0 + 07-03-2018 | 0 + 07-04-2018 | 0 + 07-05-2018 | 0 + 07-06-2018 | 0 + 07-07-2018 | 0 + 07-08-2018 | 0 + 07-09-2018 | 0 + 07-10-2018 | 0 +(48 rows) + -- Test disabling hash_agg on coordinator query SET citus.explain_all_tasks to true; SET hll.force_groupagg to OFF; @@ -514,7 +615,6 @@ SELECT (topn(topn_add_agg(user_id::text), 10)).* FROM customer_reviews ORDER BY 2 DESC, 1; ERROR: set-valued function called in context that cannot accept a set --- The following is going to be supported after window function support SELECT day, (topn(agg, 10)).* FROM ( SELECT day, topn_union_agg(reviewers) OVER seven_days AS agg @@ -523,14 +623,25 @@ FROM ( )a ORDER BY 3 DESC, 1, 2 LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + day | item | frequency +--------------------------------------------------------------------- + 06-16-2018 | 1 | 1736 + 06-16-2018 | 2 | 1736 + 06-17-2018 | 1 | 1736 + 06-17-2018 | 2 | 1736 + 06-18-2018 | 1 | 1736 + 06-18-2018 | 2 | 1736 + 06-19-2018 | 1 | 1736 + 06-19-2018 | 2 | 1736 + 06-20-2018 | 1 | 1736 + 06-20-2018 | 2 | 1736 +(10 rows) + SELECT day, (topn(topn_add_agg(user_id::text) OVER seven_days, 10)).* FROM customer_reviews WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING) ORDER BY 3 DESC, 1, 2 LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column +ERROR: set-valued function called in context that cannot accept a set DROP TABLE customer_reviews; DROP TABLE popular_reviewer; diff --git a/src/test/regress/expected/custom_aggregate_support_0.out b/src/test/regress/expected/custom_aggregate_support_0.out index 2e7c62bf4..deaba8e63 100644 --- a/src/test/regress/expected/custom_aggregate_support_0.out +++ b/src/test/regress/expected/custom_aggregate_support_0.out @@ -66,14 +66,15 @@ WHERE day >= '2018-06-23' AND day <= '2018-07-01' GROUP BY 1 ORDER BY 1; ERROR: relation "daily_uniques" does not exist --- These are going to be supported after window function support SELECT day, hll_cardinality(hll_union_agg(unique_users) OVER seven_days) FROM daily_uniques -WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING); +WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING) +ORDER BY 1; ERROR: relation "daily_uniques" does not exist SELECT day, (hll_cardinality(hll_union_agg(unique_users) OVER two_days)) - hll_cardinality(unique_users) AS lost_uniques FROM daily_uniques -WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING); +WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING) +ORDER BY 1; ERROR: relation "daily_uniques" does not exist -- Test disabling hash_agg on coordinator query SET citus.explain_all_tasks to true; @@ -242,7 +243,6 @@ FROM customer_reviews ORDER BY 2 DESC, 1; ERROR: function topn_add_agg(text) does not exist HINT: No function matches the given name and argument types. You might need to add explicit type casts. --- The following is going to be supported after window function support SELECT day, (topn(agg, 10)).* FROM ( SELECT day, topn_union_agg(reviewers) OVER seven_days AS agg diff --git a/src/test/regress/expected/custom_aggregate_support_1.out b/src/test/regress/expected/custom_aggregate_support_1.out index 21dedfaaa..5a27b9d82 100644 --- a/src/test/regress/expected/custom_aggregate_support_1.out +++ b/src/test/regress/expected/custom_aggregate_support_1.out @@ -9,7 +9,6 @@ AS create_cmd FROM pg_available_extensions() WHERE name = 'hll' \gset :create_cmd; -ERROR: extension "hll" already exists SET citus.shard_count TO 4; set citus.coordinator_aggregation_strategy to 'disabled'; CREATE TABLE raw_table (day date, user_id int); @@ -89,17 +88,118 @@ ORDER BY 1; 7 | 13 (2 rows) --- These are going to be supported after window function support SELECT day, hll_cardinality(hll_union_agg(unique_users) OVER seven_days) FROM daily_uniques -WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING); -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column +WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING) +ORDER BY 1; + day | hll_cardinality +--------------------------------------------------------------------- + 05-24-2018 | 19 + 05-25-2018 | 19 + 05-26-2018 | 19 + 05-27-2018 | 19 + 05-28-2018 | 19 + 05-29-2018 | 19 + 05-30-2018 | 19 + 05-31-2018 | 19 + 06-01-2018 | 19 + 06-02-2018 | 19 + 06-03-2018 | 19 + 06-04-2018 | 19 + 06-05-2018 | 19 + 06-06-2018 | 19 + 06-07-2018 | 19 + 06-08-2018 | 19 + 06-09-2018 | 19 + 06-10-2018 | 19 + 06-11-2018 | 19 + 06-12-2018 | 19 + 06-13-2018 | 19 + 06-14-2018 | 19 + 06-15-2018 | 19 + 06-16-2018 | 19 + 06-17-2018 | 19 + 06-18-2018 | 19 + 06-19-2018 | 19 + 06-20-2018 | 19 + 06-21-2018 | 19 + 06-22-2018 | 19 + 06-23-2018 | 19 + 06-24-2018 | 19 + 06-25-2018 | 19 + 06-26-2018 | 19 + 06-27-2018 | 19 + 06-28-2018 | 19 + 06-29-2018 | 19 + 06-30-2018 | 19 + 07-01-2018 | 13 + 07-02-2018 | 13 + 07-03-2018 | 13 + 07-04-2018 | 13 + 07-05-2018 | 13 + 07-06-2018 | 13 + 07-07-2018 | 13 + 07-08-2018 | 13 + 07-09-2018 | 13 + 07-10-2018 | 13 +(48 rows) + SELECT day, (hll_cardinality(hll_union_agg(unique_users) OVER two_days)) - hll_cardinality(unique_users) AS lost_uniques FROM daily_uniques -WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING); -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column +WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING) +ORDER BY 1; + day | lost_uniques +--------------------------------------------------------------------- + 05-24-2018 | 0 + 05-25-2018 | 0 + 05-26-2018 | 0 + 05-27-2018 | 0 + 05-28-2018 | 0 + 05-29-2018 | 0 + 05-30-2018 | 0 + 05-31-2018 | 0 + 06-01-2018 | 0 + 06-02-2018 | 0 + 06-03-2018 | 0 + 06-04-2018 | 0 + 06-05-2018 | 0 + 06-06-2018 | 0 + 06-07-2018 | 0 + 06-08-2018 | 0 + 06-09-2018 | 0 + 06-10-2018 | 0 + 06-11-2018 | 0 + 06-12-2018 | 0 + 06-13-2018 | 0 + 06-14-2018 | 0 + 06-15-2018 | 0 + 06-16-2018 | 0 + 06-17-2018 | 0 + 06-18-2018 | 0 + 06-19-2018 | 0 + 06-20-2018 | 0 + 06-21-2018 | 0 + 06-22-2018 | 0 + 06-23-2018 | 0 + 06-24-2018 | 0 + 06-25-2018 | 6 + 06-26-2018 | 0 + 06-27-2018 | 0 + 06-28-2018 | 0 + 06-29-2018 | 0 + 06-30-2018 | 0 + 07-01-2018 | 0 + 07-02-2018 | 0 + 07-03-2018 | 0 + 07-04-2018 | 0 + 07-05-2018 | 0 + 07-06-2018 | 0 + 07-07-2018 | 0 + 07-08-2018 | 0 + 07-09-2018 | 0 + 07-10-2018 | 0 +(48 rows) + -- Test disabling hash_agg on coordinator query SET citus.explain_all_tasks to true; SET hll.force_groupagg to OFF; @@ -109,34 +209,32 @@ SELECT FROM daily_uniques GROUP BY(1); - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - HashAggregate - Group Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(25 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(23 rows) SET hll.force_groupagg to ON; EXPLAIN(COSTS OFF) @@ -145,36 +243,32 @@ SELECT FROM daily_uniques GROUP BY(1); - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - GroupAggregate - Group Key: remote_scan.day - -> Sort - Sort Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(27 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(23 rows) -- Test disabling hash_agg with operator on coordinator query SET hll.force_groupagg to OFF; @@ -184,34 +278,32 @@ SELECT FROM daily_uniques GROUP BY(1); - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - HashAggregate - Group Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(25 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(23 rows) SET hll.force_groupagg to ON; EXPLAIN(COSTS OFF) @@ -220,36 +312,32 @@ SELECT FROM daily_uniques GROUP BY(1); - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - GroupAggregate - Group Key: remote_scan.day - -> Sort - Sort Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(27 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(23 rows) -- Test disabling hash_agg with expression on coordinator query SET hll.force_groupagg to OFF; @@ -259,34 +347,32 @@ SELECT FROM daily_uniques GROUP BY(1); - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - HashAggregate - Group Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(25 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(23 rows) SET hll.force_groupagg to ON; EXPLAIN(COSTS OFF) @@ -295,36 +381,32 @@ SELECT FROM daily_uniques GROUP BY(1); - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - GroupAggregate - Group Key: remote_scan.day - -> Sort - Sort Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(27 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(23 rows) -- Test disabling hash_agg with having SET hll.force_groupagg to OFF; @@ -334,34 +416,32 @@ SELECT FROM daily_uniques GROUP BY(1); - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - HashAggregate - Group Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(25 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(23 rows) SET hll.force_groupagg to ON; EXPLAIN(COSTS OFF) @@ -371,49 +451,36 @@ FROM daily_uniques GROUP BY(1) HAVING hll_cardinality(hll_union_agg(unique_users)) > 1; - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - GroupAggregate - Group Key: remote_scan.day - Filter: (hll_cardinality(hll_union_agg(remote_scan.worker_column_3)) > '1'::double precision) - -> Sort - Sort Key: remote_scan.day - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: All - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> GroupAggregate - Group Key: day - Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) - -> Sort - Sort Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> GroupAggregate - Group Key: day - Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) - -> Sort - Sort Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> GroupAggregate - Group Key: day - Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) - -> Sort - Sort Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> GroupAggregate - Group Key: day - Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) - -> Sort - Sort Key: day - -> Seq Scan on daily_uniques_xxxxxxx daily_uniques -(40 rows) + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> HashAggregate + Group Key: day + Filter: (hll_cardinality(hll_union_agg(unique_users)) > '1'::double precision) + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(27 rows) DROP TABLE raw_table; DROP TABLE daily_uniques; @@ -542,35 +609,11 @@ SELECT (topn(topn_union_agg(reviewers), 10)).* FROM popular_reviewer WHERE day >= '2018-05-24'::date AND day <= '2018-05-31'::date ORDER BY 2 DESC, 1; - item | frequency ---------------------------------------------------------------------- - 1 | 1240 - 2 | 1240 - 0 | 992 - 3 | 992 - 4 | 992 - 5 | 992 - 6 | 992 -(7 rows) - +ERROR: set-valued function called in context that cannot accept a set SELECT (topn(topn_add_agg(user_id::text), 10)).* FROM customer_reviews ORDER BY 2 DESC, 1; - item | frequency ---------------------------------------------------------------------- - 1 | 7843 - 2 | 7843 - 3 | 6851 - 4 | 6851 - 0 | 5890 - 5 | 5890 - 6 | 5890 - 7 | 1922 - 8 | 1922 - 9 | 1922 -(10 rows) - --- The following is going to be supported after window function support +ERROR: set-valued function called in context that cannot accept a set SELECT day, (topn(agg, 10)).* FROM ( SELECT day, topn_union_agg(reviewers) OVER seven_days AS agg @@ -579,14 +622,25 @@ FROM ( )a ORDER BY 3 DESC, 1, 2 LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + day | item | frequency +--------------------------------------------------------------------- + 06-16-2018 | 1 | 1736 + 06-16-2018 | 2 | 1736 + 06-17-2018 | 1 | 1736 + 06-17-2018 | 2 | 1736 + 06-18-2018 | 1 | 1736 + 06-18-2018 | 2 | 1736 + 06-19-2018 | 1 | 1736 + 06-19-2018 | 2 | 1736 + 06-20-2018 | 1 | 1736 + 06-20-2018 | 2 | 1736 +(10 rows) + SELECT day, (topn(topn_add_agg(user_id::text) OVER seven_days, 10)).* FROM customer_reviews WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING) ORDER BY 3 DESC, 1, 2 LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column +ERROR: set-valued function called in context that cannot accept a set DROP TABLE customer_reviews; DROP TABLE popular_reviewer; diff --git a/src/test/regress/expected/local_shard_execution.out b/src/test/regress/expected/local_shard_execution.out index 9dbf9975a..6ebad0dcd 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -133,16 +133,11 @@ FROM DEBUG: CTE joined_stats_cte_1 is going to be inlined via distributed planning DEBUG: CTE joined_stats_cte_2 is going to be inlined via distributed planning DEBUG: generating subplan XXX_1 for CTE accounts_cte: SELECT id AS account_id FROM local_shard_execution.accounts -DEBUG: generating subplan XXX_1 for CTE accounts_cte: SELECT id AS account_id FROM local_shard_execution.accounts -DEBUG: generating subplan XXX_2 for CTE joined_stats_cte_1: SELECT stats.spent, stats.account_id FROM (local_shard_execution.stats JOIN (SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte USING (account_id)) -DEBUG: generating subplan XXX_3 for CTE joined_stats_cte_2: SELECT joined_stats_cte_1.spent, joined_stats_cte_1.account_id FROM ((SELECT intermediate_result.spent, intermediate_result.account_id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(spent integer, account_id text)) joined_stats_cte_1 JOIN (SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte USING (account_id)) -DEBUG: generating subplan XXX_4 for subquery SELECT sum(joined_stats_cte_2.spent) OVER (PARTITION BY COALESCE(accounts_cte.account_id, NULL::text)) AS sum FROM ((SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte JOIN (SELECT intermediate_result.spent, intermediate_result.account_id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(spent integer, account_id text)) joined_stats_cte_2 USING (account_id)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum FROM (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)) inner_query -DEBUG: Subplan XXX_1 will be written to local file +DEBUG: generating subplan XXX_2 for subquery SELECT sum(joined_stats_cte_2.spent) OVER (PARTITION BY COALESCE(accounts_cte.account_id, NULL::text)) AS sum FROM ((SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte JOIN (SELECT joined_stats_cte_1.spent, joined_stats_cte_1.account_id FROM ((SELECT stats.spent, stats.account_id FROM (local_shard_execution.stats JOIN (SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte_2 USING (account_id))) joined_stats_cte_1 JOIN (SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte_1 USING (account_id))) joined_stats_cte_2 USING (account_id)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum FROM (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)) inner_query +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 written to local file -DEBUG: Subplan XXX_3 will be written to local file -DEBUG: Subplan XXX_4 will be written to local file sum --------------------------------------------------------------------- 100 diff --git a/src/test/regress/expected/multi_complex_expressions.out b/src/test/regress/expected/multi_complex_expressions.out index bb37e3ea3..b77a637c6 100644 --- a/src/test/regress/expected/multi_complex_expressions.out +++ b/src/test/regress/expected/multi_complex_expressions.out @@ -410,6 +410,56 @@ LIMIT 10 OFFSET 20; 1453 | 5 (10 rows) +SELECT + customer_keys.o_custkey, + SUM(order_count) AS total_order_count +FROM + (SELECT o_custkey, o_orderstatus, COUNT(*) over (partition by o_orderstatus) AS order_count + FROM orders GROUP BY o_custkey, o_orderstatus ) customer_keys +GROUP BY + customer_keys.o_custkey +ORDER BY + customer_keys.o_custkey DESC +LIMIT 10 OFFSET 20; + o_custkey | total_order_count +--------------------------------------------------------------------- + 1466 | 759 + 1465 | 759 + 1463 | 1499 + 1462 | 1499 + 1460 | 759 + 1459 | 1499 + 1457 | 740 + 1456 | 830 + 1454 | 1499 + 1453 | 1499 +(10 rows) + +SELECT + customer_keys.o_custkey, + SUM(order_count1 + order_count) AS total_order_count +FROM + (SELECT o_custkey, o_orderstatus, count(*) order_count1, COUNT(*) over (partition by o_orderstatus) AS order_count + FROM orders GROUP BY o_custkey, o_orderstatus ) customer_keys +GROUP BY + customer_keys.o_custkey +ORDER BY + customer_keys.o_custkey DESC +LIMIT 10 OFFSET 20; + o_custkey | total_order_count +--------------------------------------------------------------------- + 1466 | 760 + 1465 | 761 + 1463 | 1503 + 1462 | 1509 + 1460 | 760 + 1459 | 1505 + 1457 | 741 + 1456 | 833 + 1454 | 1501 + 1453 | 1504 +(10 rows) + RESET citus.task_executor_type; SET client_min_messages TO DEBUG1; -- Ensure that we push down LIMIT and OFFSET properly diff --git a/src/test/regress/expected/multi_complex_expressions_0.out b/src/test/regress/expected/multi_complex_expressions_0.out index 9a2418d41..b020b0055 100644 --- a/src/test/regress/expected/multi_complex_expressions_0.out +++ b/src/test/regress/expected/multi_complex_expressions_0.out @@ -396,6 +396,56 @@ LIMIT 10 OFFSET 20; 1453 | 5 (10 rows) +SELECT + customer_keys.o_custkey, + SUM(order_count) AS total_order_count +FROM + (SELECT o_custkey, o_orderstatus, COUNT(*) over (partition by o_orderstatus) AS order_count + FROM orders GROUP BY o_custkey, o_orderstatus ) customer_keys +GROUP BY + customer_keys.o_custkey +ORDER BY + customer_keys.o_custkey DESC +LIMIT 10 OFFSET 20; + o_custkey | total_order_count +--------------------------------------------------------------------- + 1466 | 759 + 1465 | 759 + 1463 | 1499 + 1462 | 1499 + 1460 | 759 + 1459 | 1499 + 1457 | 740 + 1456 | 830 + 1454 | 1499 + 1453 | 1499 +(10 rows) + +SELECT + customer_keys.o_custkey, + SUM(order_count1 + order_count) AS total_order_count +FROM + (SELECT o_custkey, o_orderstatus, count(*) order_count1, COUNT(*) over (partition by o_orderstatus) AS order_count + FROM orders GROUP BY o_custkey, o_orderstatus ) customer_keys +GROUP BY + customer_keys.o_custkey +ORDER BY + customer_keys.o_custkey DESC +LIMIT 10 OFFSET 20; + o_custkey | total_order_count +--------------------------------------------------------------------- + 1466 | 760 + 1465 | 761 + 1463 | 1503 + 1462 | 1509 + 1460 | 760 + 1459 | 1505 + 1457 | 741 + 1456 | 833 + 1454 | 1501 + 1453 | 1504 +(10 rows) + RESET citus.task_executor_type; SET client_min_messages TO DEBUG1; -- Ensure that we push down LIMIT and OFFSET properly diff --git a/src/test/regress/expected/multi_explain.out b/src/test/regress/expected/multi_explain.out index 7ca2f509f..48ef0d0f9 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -453,11 +453,8 @@ HashAggregate Tasks Shown: One of 2 -> Task Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate + -> Seq Scan on public.lineitem_290000 lineitem Output: l_quantity, l_quantity - Group Key: lineitem.l_quantity - -> Seq Scan on public.lineitem_290000 lineitem - Output: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment -- Subquery pushdown tests with explain EXPLAIN (COSTS OFF) SELECT diff --git a/src/test/regress/expected/multi_insert_select_window.out b/src/test/regress/expected/multi_insert_select_window.out index aa3dea909..2f522b127 100644 --- a/src/test/regress/expected/multi_insert_select_window.out +++ b/src/test/regress/expected/multi_insert_select_window.out @@ -637,7 +637,7 @@ SELECT count(*), count(DISTINCT user_id), avg(user_id) FROM agg_results_window; (1 row) TRUNCATE agg_results_window; --- lets have some queries that Citus shouldn't push down +-- test queries where the window function isn't pushed down INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, rnk @@ -653,10 +653,6 @@ ORDER BY 3 DESC, 1 DESC, 2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- user needs to supply partition by which should --- include the distribution key INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, rnk @@ -672,10 +668,6 @@ ORDER BY 3 DESC, 1 DESC, 2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- user needs to supply partition by which should --- include the distribution key INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, rnk @@ -691,9 +683,6 @@ ORDER BY 3 DESC, 1 DESC, 2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- w2 should not be pushed down INSERT INTO agg_results_window (user_id, value_1_agg, value_2_agg) SELECT * FROM ( @@ -709,9 +698,6 @@ SELECT * FROM ) as foo LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- GROUP BY includes the partition key, but not the WINDOW function INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, my_rank @@ -727,9 +713,6 @@ FROM ) as foo WHERE my_rank > 125; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- GROUP BY includes the partition key, but not the WINDOW function INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, my_rank @@ -745,9 +728,6 @@ FROM ) as foo WHERE my_rank > 125; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- w2 should not be allowed INSERT INTO agg_results_window (user_id, value_2_agg, value_3_agg) SELECT * FROM ( @@ -761,9 +741,6 @@ SELECT * FROM WINDOW w1 AS (PARTITION BY users_table.user_id, events_table.event_type ORDER BY events_table.time), w2 AS (ORDER BY events_table.time) ) as foo; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- unsupported window function with an override INSERT INTO agg_results_window(user_id, agg_time, value_2_agg) SELECT * FROM ( SELECT @@ -779,9 +756,6 @@ SELECT * FROM ( WINDOW w2 as (PARTITION BY user_id, time) ) a; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column - -- Subquery in where with unsupported window function INSERT INTO agg_results_window(user_id) SELECT user_id @@ -801,7 +775,6 @@ GROUP BY user_id; ERROR: cannot push down this subquery DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported --- Aggregate function on distribution column should error out INSERT INTO agg_results_window(user_id, value_2_agg) SELECT * FROM ( SELECT @@ -811,10 +784,6 @@ SELECT * FROM ( GROUP BY user_id ) a; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- UNION with only one subquery which has a partition on non-distribution column should --- error out INSERT INTO agg_results_window(user_id, value_1_agg) SELECT * FROM ( @@ -849,6 +818,4 @@ FROM ( user_id ) ) AS ftop; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column DROP VIEW view_with_window_func; diff --git a/src/test/regress/expected/multi_mx_router_planner.out b/src/test/regress/expected/multi_mx_router_planner.out index 7b28dedac..24c22d4c6 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -1194,19 +1194,44 @@ DETAIL: distribution column value: 1 11814 | 5 (5 rows) --- window functions are not supported for not router plannable queries SELECT id, MIN(id) over (order by word_count) FROM articles_hash_mx - WHERE author_id = 1 or author_id = 2; + WHERE author_id = 1 or author_id = 2 + ORDER BY 1; DEBUG: Router planner cannot handle multi-shard select queries -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + id | min +--------------------------------------------------------------------- + 1 | 1 + 2 | 1 + 11 | 11 + 12 | 1 + 21 | 11 + 22 | 11 + 31 | 11 + 32 | 1 + 41 | 1 + 42 | 1 +(10 rows) + SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count FROM articles_hash_mx - WHERE author_id = 5 or author_id = 2; + WHERE author_id = 5 or author_id = 2 + ORDER BY 2; DEBUG: Router planner cannot handle multi-shard select queries -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + prev | title | word_count +--------------------------------------------------------------------- + aruru | abducing | 13642 + antipope | adversa | 3164 + | afrasia | 864 + aminate | amazon | 11342 + antehall | aminate | 9089 + adversa | antehall | 7707 + afrasia | antipope | 2728 + ausable | archiblast | 18185 + amazon | aruru | 11389 + abducing | ausable | 15885 +(10 rows) + -- complex query hitting a single shard SELECT count(DISTINCT CASE diff --git a/src/test/regress/expected/multi_read_from_secondaries.out b/src/test/regress/expected/multi_read_from_secondaries.out index a7bc7c9a5..d7818ec72 100644 --- a/src/test/regress/expected/multi_read_from_secondaries.out +++ b/src/test/regress/expected/multi_read_from_secondaries.out @@ -103,17 +103,11 @@ FROM DEBUG: CTE joined_source_table_cte_1 is going to be inlined via distributed planning DEBUG: CTE joined_source_table_cte_2 is going to be inlined via distributed planning DEBUG: generating subplan XXX_1 for CTE dest_table_cte: SELECT a FROM public.dest_table -DEBUG: generating subplan XXX_1 for CTE dest_table_cte: SELECT a FROM public.dest_table -DEBUG: generating subplan XXX_2 for CTE joined_source_table_cte_1: SELECT source_table.b, source_table.a FROM (public.source_table JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte USING (a)) -DEBUG: generating subplan XXX_3 for CTE joined_source_table_cte_2: SELECT joined_source_table_cte_1.b, joined_source_table_cte_1.a FROM ((SELECT intermediate_result.b, intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(b integer, a integer)) joined_source_table_cte_1 JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte USING (a)) -DEBUG: generating subplan XXX_4 for subquery SELECT sum(joined_source_table_cte_2.b) OVER (PARTITION BY COALESCE(dest_table_cte.a, NULL::integer)) AS sum FROM ((SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte JOIN (SELECT intermediate_result.b, intermediate_result.a FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(b integer, a integer)) joined_source_table_cte_2 USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum FROM (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)) inner_query -DEBUG: Subplan XXX_1 will be written to local file +DEBUG: generating subplan XXX_2 for subquery SELECT sum(joined_source_table_cte_2.b) OVER (PARTITION BY COALESCE(dest_table_cte.a, NULL::integer)) AS sum FROM ((SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte JOIN (SELECT joined_source_table_cte_1.b, joined_source_table_cte_1.a FROM ((SELECT source_table.b, source_table.a FROM (public.source_table JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte_2 USING (a))) joined_source_table_cte_1 JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte_1 USING (a))) joined_source_table_cte_2 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum FROM (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)) inner_query 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 written to local file -DEBUG: Subplan XXX_3 will be written to local file -DEBUG: Subplan XXX_4 will be written to local file sum --------------------------------------------------------------------- 5 @@ -145,16 +139,11 @@ FROM DEBUG: CTE joined_source_table_cte_1 is going to be inlined via distributed planning DEBUG: CTE joined_source_table_cte_2 is going to be inlined via distributed planning DEBUG: generating subplan XXX_1 for CTE dest_table_cte: SELECT a FROM public.dest_table -DEBUG: generating subplan XXX_1 for CTE dest_table_cte: SELECT a FROM public.dest_table -DEBUG: generating subplan XXX_2 for CTE joined_source_table_cte_1: SELECT source_table.b, source_table.a FROM (public.source_table JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte USING (a)) -DEBUG: generating subplan XXX_3 for CTE joined_source_table_cte_2: SELECT joined_source_table_cte_1.b, joined_source_table_cte_1.a FROM ((SELECT intermediate_result.b, intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(b integer, a integer)) joined_source_table_cte_1 JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte USING (a)) -DEBUG: generating subplan XXX_4 for subquery SELECT sum(joined_source_table_cte_2.b) OVER (PARTITION BY COALESCE(dest_table_cte.a, NULL::integer)) AS sum FROM ((SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte JOIN (SELECT intermediate_result.b, intermediate_result.a FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(b integer, a integer)) joined_source_table_cte_2 USING (a)) -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum FROM (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)) inner_query +DEBUG: generating subplan XXX_2 for subquery SELECT sum(joined_source_table_cte_2.b) OVER (PARTITION BY COALESCE(dest_table_cte.a, NULL::integer)) AS sum FROM ((SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte JOIN (SELECT joined_source_table_cte_1.b, joined_source_table_cte_1.a FROM ((SELECT source_table.b, source_table.a FROM (public.source_table JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte_2 USING (a))) joined_source_table_cte_1 JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) dest_table_cte_1 USING (a))) joined_source_table_cte_2 USING (a)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum FROM (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)) inner_query 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 -DEBUG: Subplan XXX_3 will be sent to localhost:xxxxx -DEBUG: Subplan XXX_4 will be sent to localhost:xxxxx sum --------------------------------------------------------------------- 5 diff --git a/src/test/regress/expected/multi_router_planner.out b/src/test/regress/expected/multi_router_planner.out index d87250868..2333eefcb 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -1311,7 +1311,6 @@ DETAIL: distribution column value: 1 41 | 1 | aznavour | 11814 (3 rows) --- window functions are supported if query is router plannable SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count FROM articles_hash WHERE author_id = 5; @@ -1388,19 +1387,44 @@ DETAIL: distribution column value: 1 11814 | 5 (5 rows) --- window functions are not supported for not router plannable queries SELECT id, MIN(id) over (order by word_count) FROM articles_hash - WHERE author_id = 1 or author_id = 2; + WHERE author_id = 1 or author_id = 2 + ORDER BY 1; DEBUG: Router planner cannot handle multi-shard select queries -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + id | min +--------------------------------------------------------------------- + 1 | 1 + 2 | 1 + 11 | 11 + 12 | 1 + 21 | 11 + 22 | 11 + 31 | 11 + 32 | 1 + 41 | 1 + 42 | 1 +(10 rows) + SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count FROM articles_hash - WHERE author_id = 5 or author_id = 2; + WHERE author_id = 5 or author_id = 2 + ORDER BY 2; DEBUG: Router planner cannot handle multi-shard select queries -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + prev | title | word_count +--------------------------------------------------------------------- + aruru | abducing | 13642 + antipope | adversa | 3164 + | afrasia | 864 + aminate | amazon | 11342 + antehall | aminate | 9089 + adversa | antehall | 7707 + afrasia | antipope | 2728 + ausable | archiblast | 18185 + amazon | aruru | 11389 + abducing | ausable | 15885 +(10 rows) + -- where false queries are router plannable SELECT * FROM articles_hash diff --git a/src/test/regress/expected/multi_select_distinct.out b/src/test/regress/expected/multi_select_distinct.out index c16014c48..506ce215f 100644 --- a/src/test/regress/expected/multi_select_distinct.out +++ b/src/test/regress/expected/multi_select_distinct.out @@ -1116,18 +1116,18 @@ EXPLAIN (COSTS FALSE) SELECT DISTINCT ON (l_orderkey) l_orderkey, l_partkey, l_suppkey FROM lineitem_hash_part WHERE l_orderkey < 35 - ORDER BY 1; + ORDER BY 1, 2, 3; l_orderkey | l_partkey | l_suppkey --------------------------------------------------------------------- - 1 | 155190 | 7706 + 1 | 2132 | 4633 2 | 106170 | 1191 3 | 4297 | 1798 4 | 88035 | 5560 - 5 | 108570 | 8571 + 5 | 37531 | 35 6 | 139636 | 2150 - 7 | 182052 | 9607 - 32 | 82704 | 7721 - 33 | 61336 | 8855 + 7 | 79251 | 1759 + 32 | 2743 | 7744 + 33 | 33918 | 3919 34 | 88362 | 871 (10 rows) @@ -1135,12 +1135,12 @@ EXPLAIN (COSTS FALSE) SELECT DISTINCT ON (l_orderkey) l_orderkey, l_partkey, l_suppkey FROM lineitem_hash_part WHERE l_orderkey < 35 - ORDER BY 1; + ORDER BY 1, 2, 3; QUERY PLAN --------------------------------------------------------------------- Unique -> Sort - Sort Key: remote_scan.l_orderkey + Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey, remote_scan.l_suppkey -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 @@ -1148,7 +1148,7 @@ EXPLAIN (COSTS FALSE) Node: host=localhost port=xxxxx dbname=regression -> Unique -> Sort - Sort Key: l_orderkey + Sort Key: l_orderkey, l_partkey, l_suppkey -> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part Filter: (l_orderkey < 35) (13 rows) diff --git a/src/test/regress/expected/multi_subquery_window_functions.out b/src/test/regress/expected/multi_subquery_window_functions.out index aabfb9068..321ccdb59 100644 --- a/src/test/regress/expected/multi_subquery_window_functions.out +++ b/src/test/regress/expected/multi_subquery_window_functions.out @@ -775,7 +775,7 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) Output: events_table_1.user_id, events_table_1.value_2 (62 rows) --- lets have some queries that Citus shouldn't push down +-- test with window functions which aren't pushed down SELECT user_id, time, rnk FROM @@ -790,10 +790,20 @@ ORDER BY 3 DESC, 1 DESC, 2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- user needs to supply partition by which should --- include the distribution key + user_id | time | rnk +--------------------------------------------------------------------- + 1 | Wed Nov 22 19:07:03.846437 2017 | 24 + 5 | Wed Nov 22 20:45:35.99031 2017 | 23 + 1 | Wed Nov 22 18:49:42.327403 2017 | 23 + 3 | Wed Nov 22 21:12:24.542921 2017 | 22 + 3 | Wed Nov 22 20:23:46.906523 2017 | 22 + 6 | Wed Nov 22 20:36:09.106561 2017 | 21 + 3 | Wed Nov 22 21:26:21.185134 2017 | 21 + 1 | Wed Nov 22 19:03:01.772353 2017 | 21 + 6 | Wed Nov 22 22:44:48.458334 2017 | 20 + 3 | Wed Nov 22 22:05:38.409323 2017 | 20 +(10 rows) + SELECT user_id, time, rnk FROM @@ -808,10 +818,20 @@ ORDER BY 3 DESC, 1 DESC, 2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- user needs to supply partition by which should --- include the distribution key + user_id | time | rnk +--------------------------------------------------------------------- + 6 | Thu Nov 23 14:00:13.20013 2017 | 1 + 6 | Thu Nov 23 11:16:13.106691 2017 | 1 + 6 | Thu Nov 23 07:27:32.822068 2017 | 1 + 6 | Thu Nov 23 02:06:53.132461 2017 | 1 + 6 | Thu Nov 23 00:45:41.784391 2017 | 1 + 6 | Thu Nov 23 00:01:48.155345 2017 | 1 + 6 | Wed Nov 22 23:15:15.875499 2017 | 1 + 6 | Wed Nov 22 22:44:48.458334 2017 | 1 + 6 | Wed Nov 22 21:17:09.549341 2017 | 1 + 6 | Wed Nov 22 20:36:09.106561 2017 | 1 +(10 rows) + SELECT user_id, time, rnk FROM @@ -826,9 +846,20 @@ ORDER BY 3 DESC, 1 DESC, 2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- w2 should not be pushed down + user_id | time | rnk +--------------------------------------------------------------------- + 3 | Wed Nov 22 18:36:16.372893 2017 | 101 + 1 | Wed Nov 22 18:49:42.327403 2017 | 100 + 4 | Wed Nov 22 19:00:10.396739 2017 | 99 + 1 | Wed Nov 22 19:03:01.772353 2017 | 98 + 1 | Wed Nov 22 19:07:03.846437 2017 | 97 + 2 | Wed Nov 22 20:16:16.614779 2017 | 96 + 3 | Wed Nov 22 20:23:46.906523 2017 | 95 + 6 | Wed Nov 22 20:36:09.106561 2017 | 94 + 5 | Wed Nov 22 20:45:35.99031 2017 | 93 + 1 | Wed Nov 22 20:56:21.122638 2017 | 92 +(10 rows) + SELECT * FROM ( SELECT @@ -843,9 +874,20 @@ SELECT * FROM ) as foo ORDER BY 3 DESC, 1 DESC, 2 DESC NULLS LAST LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- w2 should not be pushed down + user_id | lag | rank +--------------------------------------------------------------------- + 2 | 2 | 73 + 4 | 4 | 70 + 3 | 3 | 69 + 2 | 2 | 55 + 5 | 5 | 53 + 5 | | 53 + 3 | 3 | 52 + 4 | 4 | 47 + 2 | 2 | 37 + 3 | 3 | 35 +(10 rows) + SELECT * FROM ( SELECT @@ -862,9 +904,20 @@ ORDER BY 3 DESC, 1 DESC, 2 DESC NULLS LAST LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- GROUP BY includes the partition key, but not the WINDOW function + user_id | lag | rank +--------------------------------------------------------------------- + 4 | 4 | 1262 + 3 | 3 | 1245 + 2 | 2 | 1227 + 4 | 4 | 1204 + 4 | | 1204 + 5 | 5 | 1178 + 5 | 5 | 1152 + 5 | 5 | 1126 + 4 | 4 | 1103 + 2 | 2 | 1085 +(10 rows) + SELECT user_id, time, my_rank FROM @@ -883,9 +936,20 @@ ORDER BY 3 DESC, 1 DESC,2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- GROUP BY includes the partition key, but not the WINDOW function + user_id | time | my_rank +--------------------------------------------------------------------- + 4 | Wed Nov 22 00:00:00 2017 | 12 + 6 | Wed Nov 22 00:00:00 2017 | 11 + 5 | Wed Nov 22 00:00:00 2017 | 10 + 6 | Thu Nov 23 00:00:00 2017 | 9 + 3 | Thu Nov 23 00:00:00 2017 | 8 + 1 | Thu Nov 23 00:00:00 2017 | 7 + 1 | Wed Nov 22 00:00:00 2017 | 6 + 2 | Thu Nov 23 00:00:00 2017 | 5 + 4 | Thu Nov 23 00:00:00 2017 | 4 + 2 | Wed Nov 22 00:00:00 2017 | 3 +(10 rows) + SELECT user_id, time, my_rank FROM @@ -904,9 +968,20 @@ ORDER BY 3 DESC, 1 DESC,2 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- Overriding window function but not supported + user_id | time | my_rank +--------------------------------------------------------------------- + 6 | Thu Nov 23 00:00:00 2017 | 6 + 4 | Wed Nov 22 00:00:00 2017 | 6 + 6 | Wed Nov 22 00:00:00 2017 | 5 + 3 | Thu Nov 23 00:00:00 2017 | 5 + 5 | Wed Nov 22 00:00:00 2017 | 4 + 1 | Thu Nov 23 00:00:00 2017 | 4 + 2 | Thu Nov 23 00:00:00 2017 | 3 + 1 | Wed Nov 22 00:00:00 2017 | 3 + 4 | Thu Nov 23 00:00:00 2017 | 2 + 2 | Wed Nov 22 00:00:00 2017 | 2 +(10 rows) + SELECT * FROM ( SELECT user_id, date_trunc('day', time) as time, sum(rank) OVER w2 @@ -923,9 +998,111 @@ SELECT * FROM ( ) a ORDER BY 1,2,3; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- Aggregate function on distribution column should error out + user_id | time | sum +--------------------------------------------------------------------- + 1 | Wed Nov 22 00:00:00 2017 | 1 + 1 | Thu Nov 23 00:00:00 2017 | 1 + 1 | Thu Nov 23 00:00:00 2017 | 1 + 1 | Thu Nov 23 00:00:00 2017 | 1 + 1 | Thu Nov 23 00:00:00 2017 | 1 + 1 | Thu Nov 23 00:00:00 2017 | 1 + 1 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Wed Nov 22 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 2 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Wed Nov 22 00:00:00 2017 | 1 + 3 | Wed Nov 22 00:00:00 2017 | 1 + 3 | Wed Nov 22 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 3 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Wed Nov 22 00:00:00 2017 | 1 + 4 | Wed Nov 22 00:00:00 2017 | 1 + 4 | Wed Nov 22 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 4 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Wed Nov 22 00:00:00 2017 | 1 + 5 | Wed Nov 22 00:00:00 2017 | 1 + 5 | Wed Nov 22 00:00:00 2017 | 1 + 5 | Wed Nov 22 00:00:00 2017 | 1 + 5 | Wed Nov 22 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 5 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Wed Nov 22 00:00:00 2017 | 1 + 6 | Wed Nov 22 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 + 6 | Thu Nov 23 00:00:00 2017 | 1 +(101 rows) + SELECT * FROM ( SELECT user_id, COUNT(*) OVER (PARTITION BY sum(user_id), MIN(value_2)) @@ -936,8 +1113,16 @@ SELECT * FROM ( ) a ORDER BY 1 DESC, 2 DESC; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + user_id | count +--------------------------------------------------------------------- + 6 | 1 + 5 | 1 + 4 | 1 + 3 | 1 + 2 | 1 + 1 | 1 +(6 rows) + -- test with reference table partitioned on only a column from reference table SELECT * FROM @@ -951,10 +1136,30 @@ ORDER BY 1, 2, 3 LIMIT 20; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- UNION ALL with only one of them is not partitioned over distribution column which --- should not be allowed. + user_id | it_name | count +--------------------------------------------------------------------- + 1 | User_1 | 101 + 1 | User_2 | 101 + 1 | User_3 | 101 + 1 | User_4 | 101 + 1 | User_5 | 101 + 1 | User_6 | 101 + 2 | User_1 | 101 + 2 | User_2 | 101 + 2 | User_3 | 101 + 2 | User_4 | 101 + 2 | User_5 | 101 + 2 | User_6 | 101 + 3 | User_1 | 101 + 3 | User_2 | 101 + 3 | User_3 | 101 + 3 | User_4 | 101 + 3 | User_5 | 101 + 3 | User_6 | 101 + 4 | User_1 | 101 + 4 | User_2 | 101 +(20 rows) + SELECT max(avg) FROM @@ -974,10 +1179,15 @@ FROM GROUP BY user_id ORDER BY 1 DESC LIMIT 5; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column --- UNION with only one subquery which has a partition on non-distribution column should --- error out + max +--------------------------------------------------------------------- + 5 + 3.09090909090909 + 3 + 3 + 2.875 +(5 rows) + SELECT * FROM ( ( SELECT user_id, @@ -1011,6 +1221,13 @@ FROM ( user_id)) AS ftop ORDER BY 2 DESC, 1 DESC LIMIT 5; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + user_id | sum +--------------------------------------------------------------------- + 5 | 298 + 6 | 244 + 1 | 244 + 4 | 235 + 2 | 235 +(5 rows) + DROP VIEW subq; diff --git a/src/test/regress/expected/multi_view.out b/src/test/regress/expected/multi_view.out index 82e20b8f3..5edd3a257 100644 --- a/src/test/regress/expected/multi_view.out +++ b/src/test/regress/expected/multi_view.out @@ -747,7 +747,7 @@ RESET citus.subquery_pushdown; VACUUM ANALYZE users_table; -- explain tests EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- Sort Sort Key: remote_scan.user_id @@ -758,19 +758,17 @@ EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER Tasks Shown: One of 4 -> Task Node: host=localhost port=xxxxx dbname=regression - -> HashAggregate - Group Key: users_table.user_id - -> Nested Loop - Join Filter: (users_table.user_id = users_table_1.user_id) - -> Sort - Sort Key: (max(users_table_1."time")) DESC - -> HashAggregate - Group Key: users_table_1.user_id - Filter: (max(users_table_1."time") > '2017-11-23 16:20:33.264457'::timestamp without time zone) - -> Seq Scan on users_table_1400256 users_table_1 - -> Seq Scan on users_table_1400256 users_table - Filter: ((value_1 >= 1) AND (value_1 < 3)) -(21 rows) + -> Nested Loop + Join Filter: (users_table.user_id = users_table_1.user_id) + -> Sort + Sort Key: (max(users_table_1."time")) DESC + -> HashAggregate + Group Key: users_table_1.user_id + Filter: (max(users_table_1."time") > '2017-11-23 16:20:33.264457'::timestamp without time zone) + -> Seq Scan on users_table_1400256 users_table_1 + -> Seq Scan on users_table_1400256 users_table + Filter: ((value_1 >= 1) AND (value_1 < 3)) +(19 rows) EXPLAIN (COSTS FALSE) SELECT * FROM ( diff --git a/src/test/regress/expected/set_operations.out b/src/test/regress/expected/set_operations.out index 7ef4deeb0..13b99f81d 100644 --- a/src/test/regress/expected/set_operations.out +++ b/src/test/regress/expected/set_operations.out @@ -667,7 +667,7 @@ DEBUG: Plan is router executable 2 | 2 (2 rows) --- set operations works fine with pushdownable window functions +-- set operations work fine with pushdownable window functions SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY x ORDER BY y DESC)) as foo UNION SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY x ORDER BY y DESC)) as bar @@ -686,14 +686,25 @@ DEBUG: Plan is router executable 1 | 1 | 1 (2 rows) --- set operations errors out with non-pushdownable window functions +-- set operations work fine with non-pushdownable window functions SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC)) as foo UNION -SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC)) as bar; +SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC)) as bar +ORDER BY 1 DESC, 2 DESC, 3 DESC; DEBUG: Router planner cannot handle multi-shard select queries DEBUG: Router planner cannot handle multi-shard select queries -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column +DEBUG: generating subplan XXX_1 for subquery SELECT x, y, rank() OVER my_win AS rnk FROM recursive_union.test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC) +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: generating subplan XXX_2 for subquery SELECT x, y, rank() OVER my_win AS rnk FROM recursive_union.test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.x, foo.y, foo.rnk FROM (SELECT intermediate_result.x, intermediate_result.y, intermediate_result.rnk FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer, rnk bigint)) foo UNION SELECT bar.x, bar.y, bar.rnk FROM (SELECT intermediate_result.x, intermediate_result.y, intermediate_result.rnk FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer, rnk bigint)) bar ORDER BY 1 DESC, 2 DESC, 3 DESC +DEBUG: Creating router plan +DEBUG: Plan is router executable + x | y | rnk +--------------------------------------------------------------------- + 2 | 2 | 1 + 1 | 1 | 1 +(2 rows) + -- other set operations in joins also cannot be pushed down SELECT * FROM ((SELECT * FROM test) EXCEPT (SELECT * FROM test ORDER BY x LIMIT 1)) u JOIN test USING (x) ORDER BY 1,2; DEBUG: Router planner cannot handle multi-shard select queries diff --git a/src/test/regress/expected/subqueries_deep.out b/src/test/regress/expected/subqueries_deep.out index 08524c086..e8b2d9439 100644 --- a/src/test/regress/expected/subqueries_deep.out +++ b/src/test/regress/expected/subqueries_deep.out @@ -2,11 +2,11 @@ -- test recursive planning functionality with subqueries and CTEs -- =================================================================== CREATE SCHEMA subquery_deep; -SET search_path TO subquery_and_ctes, public; +SET search_path TO subquery_deep, public; SET client_min_messages TO DEBUG1; -- subquery in FROM -> FROM -> FROM should be replaced due to OFFSET -- one level up subquery should be replaced due to GROUP BY on non partition key --- one level up subquery should be replaced due to LIMUT +-- one level up subquery should be replaced due to LIMIT SELECT DISTINCT user_id FROM diff --git a/src/test/regress/expected/subqueries_not_supported.out b/src/test/regress/expected/subqueries_not_supported.out index 2903ce2aa..589e0d922 100644 --- a/src/test/regress/expected/subqueries_not_supported.out +++ b/src/test/regress/expected/subqueries_not_supported.out @@ -81,27 +81,6 @@ DEBUG: generating subplan XXX_1 for subquery SELECT DISTINCT users_table.user_i DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo ORDER BY user_id DESC ERROR: cannot handle complex subqueries when the router executor is disabled SET citus.enable_router_execution TO true; --- window functions are not allowed if they're not partitioned on the distribution column -SELECT - * -FROM -( -SELECT - user_id, time, rnk -FROM -( - SELECT - *, rank() OVER my_win as rnk - FROM - events_table - WINDOW my_win AS (PARTITION BY event_type ORDER BY time DESC) -) as foo -ORDER BY - 3 DESC, 1 DESC, 2 DESC -LIMIT - 10) as foo; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column -- OUTER JOINs where the outer part is recursively planned and not the other way -- around is not supported SELECT diff --git a/src/test/regress/expected/window_functions.out b/src/test/regress/expected/window_functions.out index 3cdfdff85..01094c544 100644 --- a/src/test/regress/expected/window_functions.out +++ b/src/test/regress/expected/window_functions.out @@ -248,8 +248,7 @@ LIMIT 10; 3 | 18 (10 rows) --- similar query with no distribution column is on the partition by clause --- is not supported +-- similar query with no distribution column on the partition by clause SELECT DISTINCT ON (events_table.user_id, rnk) events_table.user_id, rank() OVER my_win AS rnk FROM @@ -261,8 +260,20 @@ WINDOW ORDER BY rnk DESC, 1 DESC LIMIT 10; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + user_id | rnk +--------------------------------------------------------------------- + 3 | 7 + 2 | 7 + 3 | 6 + 2 | 6 + 4 | 5 + 3 | 5 + 2 | 5 + 1 | 5 + 6 | 4 + 5 | 4 +(10 rows) + -- ORDER BY in the window function is an aggregate SELECT user_id, rank() OVER my_win as rnk, avg(value_2) as avg_val_2 @@ -656,6 +667,237 @@ ORDER BY 5 | 5 | {5,5,5} | {5,5} (66 rows) +-- repeat above 3 tests without grouping by distribution column +SELECT + value_2, + rank() OVER (PARTITION BY value_2 ROWS BETWEEN + UNBOUNDED PRECEDING AND CURRENT ROW), + dense_rank() OVER (PARTITION BY value_2 RANGE BETWEEN + UNBOUNDED PRECEDING AND CURRENT ROW), + CUME_DIST() OVER (PARTITION BY value_2 RANGE BETWEEN + UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), + PERCENT_RANK() OVER (PARTITION BY value_2 ORDER BY avg(value_1) RANGE BETWEEN + UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +FROM + users_table +GROUP BY + 1 +ORDER BY + 4 DESC,3 DESC,2 DESC ,1 DESC; + value_2 | rank | dense_rank | cume_dist | percent_rank +--------------------------------------------------------------------- + 5 | 1 | 1 | 1 | 0 + 4 | 1 | 1 | 1 | 0 + 3 | 1 | 1 | 1 | 0 + 2 | 1 | 1 | 1 | 0 + 1 | 1 | 1 | 1 | 0 + 0 | 1 | 1 | 1 | 0 +(6 rows) + +-- test exclude supported +SELECT + value_2, + value_1, + array_agg(value_1) OVER (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), + array_agg(value_1) OVER (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW EXCLUDE CURRENT ROW) +FROM + users_table +WHERE + value_2 > 2 AND value_2 < 6 +ORDER BY + value_2, value_1, 3, 4; + value_2 | value_1 | array_agg | array_agg +--------------------------------------------------------------------- + 3 | 0 | {0,0,0} | {0,0} + 3 | 0 | {0,0,0} | {0,0} + 3 | 0 | {0,0,0} | {0,0} + 3 | 1 | {0,0,0,1,1,1,1} | {0,0,0,1,1,1} + 3 | 1 | {0,0,0,1,1,1,1} | {0,0,0,1,1,1} + 3 | 1 | {0,0,0,1,1,1,1} | {0,0,0,1,1,1} + 3 | 1 | {0,0,0,1,1,1,1} | {0,0,0,1,1,1} + 3 | 2 | {0,0,0,1,1,1,1,2,2} | {0,0,0,1,1,1,1,2} + 3 | 2 | {0,0,0,1,1,1,1,2,2} | {0,0,0,1,1,1,1,2} + 3 | 3 | {0,0,0,1,1,1,1,2,2,3,3} | {0,0,0,1,1,1,1,2,2,3} + 3 | 3 | {0,0,0,1,1,1,1,2,2,3,3} | {0,0,0,1,1,1,1,2,2,3} + 3 | 4 | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4,4} | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4} + 3 | 4 | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4,4} | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4} + 3 | 4 | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4,4} | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4} + 3 | 4 | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4,4} | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4} + 3 | 4 | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4,4} | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4} + 3 | 5 | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4,4,5} | {0,0,0,1,1,1,1,2,2,3,3,4,4,4,4,4} + 4 | 0 | {0,0} | {0} + 4 | 0 | {0,0} | {0} + 4 | 1 | {0,0,1,1} | {0,0,1} + 4 | 1 | {0,0,1,1} | {0,0,1} + 4 | 2 | {0,0,1,1,2,2,2} | {0,0,1,1,2,2} + 4 | 2 | {0,0,1,1,2,2,2} | {0,0,1,1,2,2} + 4 | 2 | {0,0,1,1,2,2,2} | {0,0,1,1,2,2} + 4 | 3 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3} | {0,0,1,1,2,2,2,3,3,3,3,3,3} + 4 | 3 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3} | {0,0,1,1,2,2,2,3,3,3,3,3,3} + 4 | 3 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3} | {0,0,1,1,2,2,2,3,3,3,3,3,3} + 4 | 3 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3} | {0,0,1,1,2,2,2,3,3,3,3,3,3} + 4 | 3 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3} | {0,0,1,1,2,2,2,3,3,3,3,3,3} + 4 | 3 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3} | {0,0,1,1,2,2,2,3,3,3,3,3,3} + 4 | 3 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3} | {0,0,1,1,2,2,2,3,3,3,3,3,3} + 4 | 4 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4} + 4 | 4 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4} + 4 | 4 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4} + 4 | 4 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4} + 4 | 5 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4,5,5} | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4,5} + 4 | 5 | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4,5,5} | {0,0,1,1,2,2,2,3,3,3,3,3,3,3,4,4,4,4,5} + 5 | 0 | {0,0} | {0} + 5 | 0 | {0,0} | {0} + 5 | 1 | {0,0,1} | {0,0} + 5 | 2 | {0,0,1,2,2} | {0,0,1,2} + 5 | 2 | {0,0,1,2,2} | {0,0,1,2} + 5 | 3 | {0,0,1,2,2,3,3,3,3} | {0,0,1,2,2,3,3,3} + 5 | 3 | {0,0,1,2,2,3,3,3,3} | {0,0,1,2,2,3,3,3} + 5 | 3 | {0,0,1,2,2,3,3,3,3} | {0,0,1,2,2,3,3,3} + 5 | 3 | {0,0,1,2,2,3,3,3,3} | {0,0,1,2,2,3,3,3} + 5 | 4 | {0,0,1,2,2,3,3,3,3,4,4} | {0,0,1,2,2,3,3,3,3,4} + 5 | 4 | {0,0,1,2,2,3,3,3,3,4,4} | {0,0,1,2,2,3,3,3,3,4} + 5 | 5 | {0,0,1,2,2,3,3,3,3,4,4,5,5} | {0,0,1,2,2,3,3,3,3,4,4,5} + 5 | 5 | {0,0,1,2,2,3,3,3,3,4,4,5,5} | {0,0,1,2,2,3,3,3,3,4,4,5} +(50 rows) + +-- test preceding and following on RANGE window +SELECT + value_2, + value_1, + array_agg(value_1) OVER range_window, + array_agg(value_1) OVER range_window_exclude +FROM + users_table +WHERE + value_2 > 2 AND value_2 < 6 +WINDOW + range_window as (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING), + range_window_exclude as (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING EXCLUDE CURRENT ROW) +ORDER BY + value_2, value_1, 3, 4; + value_2 | value_1 | array_agg | array_agg +--------------------------------------------------------------------- + 3 | 0 | {0,0,0,1,1,1,1} | {0,0,1,1,1,1} + 3 | 0 | {0,0,0,1,1,1,1} | {0,0,1,1,1,1} + 3 | 0 | {0,0,0,1,1,1,1} | {0,0,1,1,1,1} + 3 | 1 | {0,0,0,1,1,1,1,2,2} | {0,0,0,1,1,1,2,2} + 3 | 1 | {0,0,0,1,1,1,1,2,2} | {0,0,0,1,1,1,2,2} + 3 | 1 | {0,0,0,1,1,1,1,2,2} | {0,0,0,1,1,1,2,2} + 3 | 1 | {0,0,0,1,1,1,1,2,2} | {0,0,0,1,1,1,2,2} + 3 | 2 | {1,1,1,1,2,2,3,3} | {1,1,1,1,2,3,3} + 3 | 2 | {1,1,1,1,2,2,3,3} | {1,1,1,1,2,3,3} + 3 | 3 | {2,2,3,3,4,4,4,4,4} | {2,2,3,4,4,4,4,4} + 3 | 3 | {2,2,3,3,4,4,4,4,4} | {2,2,3,4,4,4,4,4} + 3 | 4 | {3,3,4,4,4,4,4,5} | {3,3,4,4,4,4,5} + 3 | 4 | {3,3,4,4,4,4,4,5} | {3,3,4,4,4,4,5} + 3 | 4 | {3,3,4,4,4,4,4,5} | {3,3,4,4,4,4,5} + 3 | 4 | {3,3,4,4,4,4,4,5} | {3,3,4,4,4,4,5} + 3 | 4 | {3,3,4,4,4,4,4,5} | {3,3,4,4,4,4,5} + 3 | 5 | {4,4,4,4,4,5} | {4,4,4,4,4} + 4 | 0 | {0,0,1,1} | {0,1,1} + 4 | 0 | {0,0,1,1} | {0,1,1} + 4 | 1 | {0,0,1,1,2,2,2} | {0,0,1,2,2,2} + 4 | 1 | {0,0,1,1,2,2,2} | {0,0,1,2,2,2} + 4 | 2 | {1,1,2,2,2,3,3,3,3,3,3,3} | {1,1,2,2,3,3,3,3,3,3,3} + 4 | 2 | {1,1,2,2,2,3,3,3,3,3,3,3} | {1,1,2,2,3,3,3,3,3,3,3} + 4 | 2 | {1,1,2,2,2,3,3,3,3,3,3,3} | {1,1,2,2,3,3,3,3,3,3,3} + 4 | 3 | {2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {2,2,2,3,3,3,3,3,3,4,4,4,4} + 4 | 3 | {2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {2,2,2,3,3,3,3,3,3,4,4,4,4} + 4 | 3 | {2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {2,2,2,3,3,3,3,3,3,4,4,4,4} + 4 | 3 | {2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {2,2,2,3,3,3,3,3,3,4,4,4,4} + 4 | 3 | {2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {2,2,2,3,3,3,3,3,3,4,4,4,4} + 4 | 3 | {2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {2,2,2,3,3,3,3,3,3,4,4,4,4} + 4 | 3 | {2,2,2,3,3,3,3,3,3,3,4,4,4,4} | {2,2,2,3,3,3,3,3,3,4,4,4,4} + 4 | 4 | {3,3,3,3,3,3,3,4,4,4,4,5,5} | {3,3,3,3,3,3,3,4,4,4,5,5} + 4 | 4 | {3,3,3,3,3,3,3,4,4,4,4,5,5} | {3,3,3,3,3,3,3,4,4,4,5,5} + 4 | 4 | {3,3,3,3,3,3,3,4,4,4,4,5,5} | {3,3,3,3,3,3,3,4,4,4,5,5} + 4 | 4 | {3,3,3,3,3,3,3,4,4,4,4,5,5} | {3,3,3,3,3,3,3,4,4,4,5,5} + 4 | 5 | {4,4,4,4,5,5} | {4,4,4,4,5} + 4 | 5 | {4,4,4,4,5,5} | {4,4,4,4,5} + 5 | 0 | {0,0,1} | {0,1} + 5 | 0 | {0,0,1} | {0,1} + 5 | 1 | {0,0,1,2,2} | {0,0,2,2} + 5 | 2 | {1,2,2,3,3,3,3} | {1,2,3,3,3,3} + 5 | 2 | {1,2,2,3,3,3,3} | {1,2,3,3,3,3} + 5 | 3 | {2,2,3,3,3,3,4,4} | {2,2,3,3,3,4,4} + 5 | 3 | {2,2,3,3,3,3,4,4} | {2,2,3,3,3,4,4} + 5 | 3 | {2,2,3,3,3,3,4,4} | {2,2,3,3,3,4,4} + 5 | 3 | {2,2,3,3,3,3,4,4} | {2,2,3,3,3,4,4} + 5 | 4 | {3,3,3,3,4,4,5,5} | {3,3,3,3,4,5,5} + 5 | 4 | {3,3,3,3,4,4,5,5} | {3,3,3,3,4,5,5} + 5 | 5 | {4,4,5,5} | {4,4,5} + 5 | 5 | {4,4,5,5} | {4,4,5} +(50 rows) + +-- test preceding and following on ROW window +SELECT + value_2, + value_1, + array_agg(value_1) OVER row_window, + array_agg(value_1) OVER row_window_exclude +FROM + users_table +WHERE + value_2 > 2 and value_2 < 6 +WINDOW + row_window as (PARTITION BY value_2 ORDER BY value_1 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING), + row_window_exclude as (PARTITION BY value_2 ORDER BY value_1 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING EXCLUDE CURRENT ROW) +ORDER BY + value_2, value_1, 3, 4; + value_2 | value_1 | array_agg | array_agg +--------------------------------------------------------------------- + 3 | 0 | {0,0} | {0} + 3 | 0 | {0,0,0} | {0,0} + 3 | 0 | {0,0,1} | {0,1} + 3 | 1 | {0,1,1} | {0,1} + 3 | 1 | {1,1,1} | {1,1} + 3 | 1 | {1,1,1} | {1,1} + 3 | 1 | {1,1,2} | {1,2} + 3 | 2 | {1,2,2} | {1,2} + 3 | 2 | {2,2,3} | {2,3} + 3 | 3 | {2,3,3} | {2,3} + 3 | 3 | {3,3,4} | {3,4} + 3 | 4 | {3,4,4} | {3,4} + 3 | 4 | {4,4,4} | {4,4} + 3 | 4 | {4,4,4} | {4,4} + 3 | 4 | {4,4,4} | {4,4} + 3 | 4 | {4,4,5} | {4,5} + 3 | 5 | {4,5} | {4} + 4 | 0 | {0,0} | {0} + 4 | 0 | {0,0,1} | {0,1} + 4 | 1 | {0,1,1} | {0,1} + 4 | 1 | {1,1,2} | {1,2} + 4 | 2 | {1,2,2} | {1,2} + 4 | 2 | {2,2,2} | {2,2} + 4 | 2 | {2,2,3} | {2,3} + 4 | 3 | {2,3,3} | {2,3} + 4 | 3 | {3,3,3} | {3,3} + 4 | 3 | {3,3,3} | {3,3} + 4 | 3 | {3,3,3} | {3,3} + 4 | 3 | {3,3,3} | {3,3} + 4 | 3 | {3,3,3} | {3,3} + 4 | 3 | {3,3,4} | {3,4} + 4 | 4 | {3,4,4} | {3,4} + 4 | 4 | {4,4,4} | {4,4} + 4 | 4 | {4,4,4} | {4,4} + 4 | 4 | {4,4,5} | {4,5} + 4 | 5 | {4,5,5} | {4,5} + 4 | 5 | {5,5} | {5} + 5 | 0 | {0,0} | {0} + 5 | 0 | {0,0,1} | {0,1} + 5 | 1 | {0,1,2} | {0,2} + 5 | 2 | {1,2,2} | {1,2} + 5 | 2 | {2,2,3} | {2,3} + 5 | 3 | {2,3,3} | {2,3} + 5 | 3 | {3,3,3} | {3,3} + 5 | 3 | {3,3,3} | {3,3} + 5 | 3 | {3,3,4} | {3,4} + 5 | 4 | {3,4,4} | {3,4} + 5 | 4 | {4,4,5} | {4,5} + 5 | 5 | {4,5,5} | {4,5} + 5 | 5 | {5,5} | {5} +(50 rows) + -- some tests with GROUP BY, HAVING and LIMIT SELECT user_id, sum(event_type) OVER my_win , event_type @@ -678,6 +920,27 @@ LIMIT 5 | 3 | 3 (5 rows) +-- test PARTITION BY avg(...) ORDER BY avg(...) +SELECT + value_1, + avg(value_3), + dense_rank() OVER (PARTITION BY avg(value_3) ORDER BY avg(value_2)) +FROM + users_table +GROUP BY + 1 +ORDER BY + 1; + value_1 | avg | dense_rank +--------------------------------------------------------------------- + 0 | 3.08333333333333 | 1 + 1 | 2.93333333333333 | 1 + 2 | 2.22222222222222 | 1 + 3 | 2.73076923076923 | 1 + 4 | 2.9047619047619 | 1 + 5 | 2.22222222222222 | 2 +(6 rows) + -- Group by has more columns than partition by SELECT DISTINCT user_id, SUM(value_2) OVER (PARTITION BY user_id) @@ -768,10 +1031,89 @@ $Q$); --------------------------------------------------------------------- Sort Sort Key: remote_scan.avg_1 DESC, remote_scan.avg DESC, remote_scan.user_id DESC - -> HashAggregate - Group Key: remote_scan.user_id - -> Custom Scan (Citus Adaptive) - Task Count: 4 + -> Custom Scan (Citus Adaptive) + Task Count: 4 +(4 rows) + +SELECT + value_2, + AVG(avg(value_1)) OVER (PARTITION BY value_2, max(value_2), MIN(value_2)), + AVG(avg(value_2)) OVER (PARTITION BY value_2, min(value_2), AVG(value_1)) +FROM + users_table +GROUP BY + 1 +ORDER BY + 3 DESC, 2 DESC, 1 DESC; + value_2 | avg | avg +--------------------------------------------------------------------- + 5 | 2.6923076923076923 | 5.0000000000000000 + 4 | 2.7500000000000000 | 4.0000000000000000 + 3 | 2.2941176470588235 | 3.0000000000000000 + 2 | 2.7619047619047619 | 2.0000000000000000 + 1 | 2.4285714285714286 | 1.00000000000000000000 + 0 | 2.2222222222222222 | 0.00000000000000000000 +(6 rows) + +SELECT + value_2, user_id, + AVG(avg(value_1)) OVER (PARTITION BY value_2, max(value_2), MIN(value_2)), + AVG(avg(value_2)) OVER (PARTITION BY user_id, min(value_2), AVG(value_1)) +FROM + users_table +GROUP BY + 1, 2 +ORDER BY + 3 DESC, 2 DESC, 1 DESC; + value_2 | user_id | avg | avg +--------------------------------------------------------------------- + 5 | 5 | 2.6666666666666667 | 5.0000000000000000 + 5 | 4 | 2.6666666666666667 | 5.0000000000000000 + 5 | 3 | 2.6666666666666667 | 5.0000000000000000 + 5 | 2 | 2.6666666666666667 | 5.0000000000000000 + 2 | 6 | 2.54583333333333333333 | 2.0000000000000000 + 2 | 5 | 2.54583333333333333333 | 2.0000000000000000 + 2 | 4 | 2.54583333333333333333 | 2.0000000000000000 + 2 | 3 | 2.54583333333333333333 | 2.0000000000000000 + 2 | 2 | 2.54583333333333333333 | 2.0000000000000000 + 2 | 1 | 2.54583333333333333333 | 2.0000000000000000 + 0 | 6 | 2.50000000000000000000 | 0.00000000000000000000 + 0 | 5 | 2.50000000000000000000 | 0.00000000000000000000 + 0 | 4 | 2.50000000000000000000 | 0.00000000000000000000 + 0 | 2 | 2.50000000000000000000 | 0.00000000000000000000 + 0 | 1 | 2.50000000000000000000 | 0.00000000000000000000 + 4 | 6 | 2.45555555555555555000 | 4.0000000000000000 + 4 | 5 | 2.45555555555555555000 | 4.0000000000000000 + 4 | 4 | 2.45555555555555555000 | 4.0000000000000000 + 4 | 3 | 2.45555555555555555000 | 4.0000000000000000 + 4 | 2 | 2.45555555555555555000 | 4.0000000000000000 + 4 | 1 | 2.45555555555555555000 | 4.0000000000000000 + 3 | 6 | 2.3500000000000000 | 3.0000000000000000 + 3 | 5 | 2.3500000000000000 | 3.0000000000000000 + 3 | 4 | 2.3500000000000000 | 3.0000000000000000 + 3 | 3 | 2.3500000000000000 | 3.0000000000000000 + 3 | 2 | 2.3500000000000000 | 3.0000000000000000 + 3 | 1 | 2.3500000000000000 | 3.0000000000000000 + 1 | 6 | 1.90666666666666666000 | 1.00000000000000000000 + 1 | 5 | 1.90666666666666666000 | 1.00000000000000000000 + 1 | 4 | 1.90666666666666666000 | 1.00000000000000000000 + 1 | 3 | 1.90666666666666666000 | 1.00000000000000000000 + 1 | 2 | 1.90666666666666666000 | 1.00000000000000000000 +(32 rows) + +SELECT user_id, sum(avg(user_id)) OVER () +FROM users_table +GROUP BY user_id +ORDER BY 1 +LIMIT 10; + user_id | sum +--------------------------------------------------------------------- + 1 | 21.00000000000000000000 + 2 | 21.00000000000000000000 + 3 | 21.00000000000000000000 + 4 | 21.00000000000000000000 + 5 | 21.00000000000000000000 + 6 | 21.00000000000000000000 (6 rows) SELECT @@ -939,24 +1281,22 @@ FROM users_table GROUP BY user_id, value_2 ORDER BY user_id, avg(value_1) DESC; - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- Sort - Sort Key: remote_scan.user_id, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))) DESC - -> HashAggregate - Group Key: remote_scan.user_id, remote_scan.worker_column_5 - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> WindowAgg - -> Sort - Sort Key: users_table.user_id, (('1'::numeric / ('1'::numeric + avg(users_table.value_1)))) - -> HashAggregate - Group Key: users_table.user_id, users_table.value_2 - -> Seq Scan on users_table_1400256 users_table -(15 rows) + Sort Key: remote_scan.user_id, remote_scan.avg DESC + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> WindowAgg + -> Sort + Sort Key: users_table.user_id, (('1'::numeric / ('1'::numeric + avg(users_table.value_1)))) + -> HashAggregate + Group Key: users_table.user_id, users_table.value_2 + -> Seq Scan on users_table_1400256 users_table +(13 rows) -- order by in the window function is same as avg(value_1) DESC SELECT @@ -1014,25 +1354,23 @@ FROM GROUP BY user_id, value_2 ORDER BY user_id, avg(value_1) DESC LIMIT 5; - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- Limit -> Sort - Sort Key: remote_scan.user_id, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))) DESC - -> HashAggregate - Group Key: remote_scan.user_id, remote_scan.worker_column_5 - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> WindowAgg - -> Sort - Sort Key: users_table.user_id, (('1'::numeric / ('1'::numeric + avg(users_table.value_1)))) - -> HashAggregate - Group Key: users_table.user_id, users_table.value_2 - -> Seq Scan on users_table_1400256 users_table -(16 rows) + Sort Key: remote_scan.user_id, remote_scan.avg DESC + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> WindowAgg + -> Sort + Sort Key: users_table.user_id, (('1'::numeric / ('1'::numeric + avg(users_table.value_1)))) + -> HashAggregate + Group Key: users_table.user_id, users_table.value_2 + -> Seq Scan on users_table_1400256 users_table +(14 rows) EXPLAIN (COSTS FALSE) SELECT @@ -1044,25 +1382,23 @@ FROM GROUP BY user_id, value_2 ORDER BY user_id, avg(value_1) DESC LIMIT 5; - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- Limit -> Sort - Sort Key: remote_scan.user_id, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))) DESC - -> HashAggregate - Group Key: remote_scan.user_id, remote_scan.worker_column_5 - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> WindowAgg - -> Sort - Sort Key: users_table.user_id, (('1'::numeric / ('1'::numeric + avg(users_table.value_1)))) - -> HashAggregate - Group Key: users_table.user_id, users_table.value_2 - -> Seq Scan on users_table_1400256 users_table -(16 rows) + Sort Key: remote_scan.user_id, remote_scan.avg DESC + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> WindowAgg + -> Sort + Sort Key: users_table.user_id, (('1'::numeric / ('1'::numeric + avg(users_table.value_1)))) + -> HashAggregate + Group Key: users_table.user_id, users_table.value_2 + -> Seq Scan on users_table_1400256 users_table +(14 rows) EXPLAIN (COSTS FALSE) SELECT @@ -1074,25 +1410,23 @@ FROM GROUP BY user_id, value_2 ORDER BY user_id, avg(value_1) DESC LIMIT 5; - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- Limit -> Sort - Sort Key: remote_scan.user_id, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))) DESC - -> HashAggregate - Group Key: remote_scan.user_id, remote_scan.worker_column_5 - -> Custom Scan (Citus Adaptive) - Task Count: 4 - Tasks Shown: One of 4 - -> Task - Node: host=localhost port=xxxxx dbname=regression - -> WindowAgg - -> Sort - Sort Key: users_table.user_id, ((1 / (1 + sum(users_table.value_2)))) - -> HashAggregate - Group Key: users_table.user_id, users_table.value_2 - -> Seq Scan on users_table_1400256 users_table -(16 rows) + Sort Key: remote_scan.user_id, remote_scan.avg DESC + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> WindowAgg + -> Sort + Sort Key: users_table.user_id, ((1 / (1 + sum(users_table.value_2)))) + -> HashAggregate + Group Key: users_table.user_id, users_table.value_2 + -> Seq Scan on users_table_1400256 users_table +(14 rows) EXPLAIN (COSTS FALSE) SELECT @@ -1104,23 +1438,117 @@ FROM GROUP BY user_id, value_2 ORDER BY user_id, avg(value_1) DESC LIMIT 5; - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- Limit -> Sort - Sort Key: remote_scan.user_id, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))) DESC - -> HashAggregate - Group Key: remote_scan.user_id, remote_scan.worker_column_5 + Sort Key: remote_scan.user_id, remote_scan.avg DESC + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> WindowAgg + -> Sort + Sort Key: users_table.user_id, (sum(users_table.value_2)) + -> HashAggregate + Group Key: users_table.user_id, users_table.value_2 + -> Seq Scan on users_table_1400256 users_table +(14 rows) + +-- Grouping can be pushed down with aggregates even when window function can't +EXPLAIN (COSTS FALSE) +SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random()) +FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1; + QUERY PLAN +--------------------------------------------------------------------- + Limit + -> WindowAgg + -> Sort + Sort Key: remote_scan.worker_column_5 -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task Node: host=localhost port=xxxxx dbname=regression - -> WindowAgg - -> Sort - Sort Key: users_table.user_id, (sum(users_table.value_2)) - -> HashAggregate - Group Key: users_table.user_id, users_table.value_2 - -> Seq Scan on users_table_1400256 users_table -(16 rows) + -> HashAggregate + Group Key: user_id + Filter: (avg(value_1) > '2'::numeric) + -> Seq Scan on users_table_1400256 users_table +(13 rows) + +-- Window function with inlined CTE +WITH cte as ( + SELECT uref.id user_id, events_table.value_2, count(*) c + FROM events_table + JOIN users_ref_test_table uref ON uref.id = events_table.user_id + GROUP BY 1, 2 +) +SELECT DISTINCT cte.value_2, cte.c, sum(cte.value_2) OVER (PARTITION BY cte.c) +FROM cte JOIN events_table et ON et.value_2 = cte.value_2 and et.value_2 = cte.c +ORDER BY 1; + value_2 | c | sum +--------------------------------------------------------------------- + 3 | 3 | 108 + 4 | 4 | 56 +(2 rows) + +-- There was a strange bug where this wouldn't have window functions being pushed down +-- Bug dependent on column ordering +CREATE TABLE daily_uniques (value_2 float, user_id bigint); +SELECT create_distributed_table('daily_uniques', 'user_id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +EXPLAIN (COSTS FALSE) SELECT + user_id, + sum(value_2) AS commits, + RANK () OVER ( + PARTITION BY user_id + ORDER BY + sum(value_2) DESC + ) +FROM daily_uniques +GROUP BY user_id +HAVING + sum(value_2) > 0 +ORDER BY commits DESC +LIMIT 10; + QUERY PLAN +--------------------------------------------------------------------- + Limit + -> Sort + Sort Key: remote_scan.commits DESC + -> Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> WindowAgg + -> Sort + Sort Key: daily_uniques.user_id, (sum(daily_uniques.value_2)) DESC + -> HashAggregate + Group Key: daily_uniques.user_id + Filter: (sum(daily_uniques.value_2) > '0'::double precision) + -> Seq Scan on daily_uniques_xxxxxxx daily_uniques +(15 rows) + +DROP TABLE daily_uniques; +-- Partition by reference table column joined to distribution column +SELECT DISTINCT value_2, array_agg(rnk ORDER BY rnk) FROM ( +SELECT events_table.value_2, sum(uref.k_no) OVER (PARTITION BY uref.id) AS rnk +FROM events_table +JOIN users_ref_test_table uref ON uref.id = events_table.user_id) sq +GROUP BY 1 ORDER BY 1; + value_2 | array_agg +--------------------------------------------------------------------- + 0 | {686,686,816,816,987,987,1104} + 1 | {500,500,675,675,675,686,686,816,816,816,987,987,987,987,987,1104,1104,1104,1104,1104,1104,1104} + 2 | {500,500,500,500,675,675,675,675,675,686,686,686,686,816,816,816,816,816,987,987,987,987,987,987,987,1104,1104,1104,1104,1104,1104} + 3 | {500,500,500,500,675,686,686,686,816,816,987,987,987,1104,1104,1104,1104,1104} + 4 | {675,675,675,675,686,686,686,816,816,816,987,987,1104,1104} + 5 | {675,675,816,816,987,987,1104,1104,1104} +(6 rows) diff --git a/src/test/regress/expected/with_basics.out b/src/test/regress/expected/with_basics.out index 8c7fce459..4cd27d624 100644 --- a/src/test/regress/expected/with_basics.out +++ b/src/test/regress/expected/with_basics.out @@ -461,8 +461,15 @@ ORDER BY user_id LIMIT 5; -ERROR: could not run distributed query because the window function that is used cannot be pushed down -HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions with a PARTITION BY clause containing the distribution column + user_id +--------------------------------------------------------------------- + 6 + 6 + 6 + 6 + 6 +(5 rows) + -- Window functions that partition by the distribution column in subqueries in CTEs are ok WITH top_users AS (SELECT * diff --git a/src/test/regress/sql/custom_aggregate_support.sql b/src/test/regress/sql/custom_aggregate_support.sql index f601fb8b8..55bc76316 100644 --- a/src/test/regress/sql/custom_aggregate_support.sql +++ b/src/test/regress/sql/custom_aggregate_support.sql @@ -60,14 +60,15 @@ WHERE day >= '2018-06-23' AND day <= '2018-07-01' GROUP BY 1 ORDER BY 1; --- These are going to be supported after window function support SELECT day, hll_cardinality(hll_union_agg(unique_users) OVER seven_days) FROM daily_uniques -WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING); +WINDOW seven_days AS (ORDER BY day ASC ROWS 6 PRECEDING) +ORDER BY 1; SELECT day, (hll_cardinality(hll_union_agg(unique_users) OVER two_days)) - hll_cardinality(unique_users) AS lost_uniques FROM daily_uniques -WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING); +WINDOW two_days AS (ORDER BY day ASC ROWS 1 PRECEDING) +ORDER BY 1; -- Test disabling hash_agg on coordinator query SET citus.explain_all_tasks to true; @@ -219,7 +220,6 @@ SELECT (topn(topn_add_agg(user_id::text), 10)).* FROM customer_reviews ORDER BY 2 DESC, 1; --- The following is going to be supported after window function support SELECT day, (topn(agg, 10)).* FROM ( SELECT day, topn_union_agg(reviewers) OVER seven_days AS agg diff --git a/src/test/regress/sql/multi_complex_expressions.sql b/src/test/regress/sql/multi_complex_expressions.sql index 4b3d3842f..0aa6cdb39 100644 --- a/src/test/regress/sql/multi_complex_expressions.sql +++ b/src/test/regress/sql/multi_complex_expressions.sql @@ -181,6 +181,30 @@ ORDER BY customer_keys.o_custkey DESC LIMIT 10 OFFSET 20; +SELECT + customer_keys.o_custkey, + SUM(order_count) AS total_order_count +FROM + (SELECT o_custkey, o_orderstatus, COUNT(*) over (partition by o_orderstatus) AS order_count + FROM orders GROUP BY o_custkey, o_orderstatus ) customer_keys +GROUP BY + customer_keys.o_custkey +ORDER BY + customer_keys.o_custkey DESC +LIMIT 10 OFFSET 20; + +SELECT + customer_keys.o_custkey, + SUM(order_count1 + order_count) AS total_order_count +FROM + (SELECT o_custkey, o_orderstatus, count(*) order_count1, COUNT(*) over (partition by o_orderstatus) AS order_count + FROM orders GROUP BY o_custkey, o_orderstatus ) customer_keys +GROUP BY + customer_keys.o_custkey +ORDER BY + customer_keys.o_custkey DESC +LIMIT 10 OFFSET 20; + RESET citus.task_executor_type; SET client_min_messages TO DEBUG1; diff --git a/src/test/regress/sql/multi_insert_select_window.sql b/src/test/regress/sql/multi_insert_select_window.sql index bcf1e96a9..10c527ad4 100644 --- a/src/test/regress/sql/multi_insert_select_window.sql +++ b/src/test/regress/sql/multi_insert_select_window.sql @@ -561,7 +561,7 @@ FROM ( SELECT count(*), count(DISTINCT user_id), avg(user_id) FROM agg_results_window; TRUNCATE agg_results_window; --- lets have some queries that Citus shouldn't push down +-- test queries where the window function isn't pushed down INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, rnk @@ -578,8 +578,6 @@ ORDER BY LIMIT 10; --- user needs to supply partition by which should --- include the distribution key INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, rnk @@ -596,8 +594,6 @@ ORDER BY LIMIT 10; --- user needs to supply partition by which should --- include the distribution key INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, rnk @@ -614,7 +610,6 @@ ORDER BY LIMIT 10; --- w2 should not be pushed down INSERT INTO agg_results_window (user_id, value_1_agg, value_2_agg) SELECT * FROM ( @@ -631,7 +626,6 @@ SELECT * FROM LIMIT 10; --- GROUP BY includes the partition key, but not the WINDOW function INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, my_rank @@ -648,7 +642,6 @@ FROM WHERE my_rank > 125; --- GROUP BY includes the partition key, but not the WINDOW function INSERT INTO agg_results_window (user_id, agg_time, value_2_agg) SELECT user_id, time, my_rank @@ -665,7 +658,6 @@ FROM WHERE my_rank > 125; --- w2 should not be allowed INSERT INTO agg_results_window (user_id, value_2_agg, value_3_agg) SELECT * FROM ( @@ -680,7 +672,6 @@ SELECT * FROM w2 AS (ORDER BY events_table.time) ) as foo; --- unsupported window function with an override INSERT INTO agg_results_window(user_id, agg_time, value_2_agg) SELECT * FROM ( SELECT @@ -697,7 +688,6 @@ SELECT * FROM ( w2 as (PARTITION BY user_id, time) ) a; - -- Subquery in where with unsupported window function INSERT INTO agg_results_window(user_id) SELECT user_id @@ -716,7 +706,6 @@ WHERE GROUP BY user_id; --- Aggregate function on distribution column should error out INSERT INTO agg_results_window(user_id, value_2_agg) SELECT * FROM ( SELECT @@ -727,8 +716,6 @@ SELECT * FROM ( user_id ) a; --- UNION with only one subquery which has a partition on non-distribution column should --- error out INSERT INTO agg_results_window(user_id, value_1_agg) SELECT * FROM ( diff --git a/src/test/regress/sql/multi_mx_router_planner.sql b/src/test/regress/sql/multi_mx_router_planner.sql index 628118f70..fe7cb54ee 100644 --- a/src/test/regress/sql/multi_mx_router_planner.sql +++ b/src/test/regress/sql/multi_mx_router_planner.sql @@ -501,14 +501,15 @@ SELECT word_count, rank() OVER (PARTITION BY author_id ORDER BY word_count) FROM articles_hash_mx WHERE author_id = 1; --- window functions are not supported for not router plannable queries SELECT id, MIN(id) over (order by word_count) FROM articles_hash_mx - WHERE author_id = 1 or author_id = 2; + WHERE author_id = 1 or author_id = 2 + ORDER BY 1; SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count FROM articles_hash_mx - WHERE author_id = 5 or author_id = 2; + WHERE author_id = 5 or author_id = 2 + ORDER BY 2; -- complex query hitting a single shard SELECT diff --git a/src/test/regress/sql/multi_router_planner.sql b/src/test/regress/sql/multi_router_planner.sql index 9315f1ac7..f396d384a 100644 --- a/src/test/regress/sql/multi_router_planner.sql +++ b/src/test/regress/sql/multi_router_planner.sql @@ -572,7 +572,6 @@ SELECT * FROM articles_hash WHERE (title like '%s' or title like 'a%') and (author_id = 1) and (word_count < 3000 or word_count > 8000); --- window functions are supported if query is router plannable SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count FROM articles_hash WHERE author_id = 5; @@ -594,14 +593,15 @@ SELECT word_count, rank() OVER (PARTITION BY author_id ORDER BY word_count) FROM articles_hash WHERE author_id = 1; --- window functions are not supported for not router plannable queries SELECT id, MIN(id) over (order by word_count) FROM articles_hash - WHERE author_id = 1 or author_id = 2; + WHERE author_id = 1 or author_id = 2 + ORDER BY 1; SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count FROM articles_hash - WHERE author_id = 5 or author_id = 2; + WHERE author_id = 5 or author_id = 2 + ORDER BY 2; -- where false queries are router plannable SELECT * diff --git a/src/test/regress/sql/multi_select_distinct.sql b/src/test/regress/sql/multi_select_distinct.sql index fdfeec76c..8254da5ba 100644 --- a/src/test/regress/sql/multi_select_distinct.sql +++ b/src/test/regress/sql/multi_select_distinct.sql @@ -396,13 +396,13 @@ EXPLAIN (COSTS FALSE) SELECT DISTINCT ON (l_orderkey) l_orderkey, l_partkey, l_suppkey FROM lineitem_hash_part WHERE l_orderkey < 35 - ORDER BY 1; + ORDER BY 1, 2, 3; EXPLAIN (COSTS FALSE) SELECT DISTINCT ON (l_orderkey) l_orderkey, l_partkey, l_suppkey FROM lineitem_hash_part WHERE l_orderkey < 35 - ORDER BY 1; + ORDER BY 1, 2, 3; -- distinct on non-partition column -- note order by is required here diff --git a/src/test/regress/sql/multi_subquery_window_functions.sql b/src/test/regress/sql/multi_subquery_window_functions.sql index 0e9b78444..b5c0332d7 100644 --- a/src/test/regress/sql/multi_subquery_window_functions.sql +++ b/src/test/regress/sql/multi_subquery_window_functions.sql @@ -476,7 +476,7 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE) ORDER BY 2 DESC, 1 DESC LIMIT 5; --- lets have some queries that Citus shouldn't push down +-- test with window functions which aren't pushed down SELECT user_id, time, rnk FROM @@ -492,8 +492,6 @@ ORDER BY LIMIT 10; --- user needs to supply partition by which should --- include the distribution key SELECT user_id, time, rnk FROM @@ -509,8 +507,6 @@ ORDER BY LIMIT 10; --- user needs to supply partition by which should --- include the distribution key SELECT user_id, time, rnk FROM @@ -526,7 +522,6 @@ ORDER BY LIMIT 10; --- w2 should not be pushed down SELECT * FROM ( SELECT @@ -542,7 +537,6 @@ SELECT * FROM ORDER BY 3 DESC, 1 DESC, 2 DESC NULLS LAST LIMIT 10; --- w2 should not be pushed down SELECT * FROM ( SELECT @@ -560,7 +554,6 @@ ORDER BY LIMIT 10; --- GROUP BY includes the partition key, but not the WINDOW function SELECT user_id, time, my_rank FROM @@ -580,7 +573,6 @@ ORDER BY LIMIT 10; --- GROUP BY includes the partition key, but not the WINDOW function SELECT user_id, time, my_rank FROM @@ -600,7 +592,6 @@ ORDER BY LIMIT 10; --- Overriding window function but not supported SELECT * FROM ( SELECT user_id, date_trunc('day', time) as time, sum(rank) OVER w2 @@ -619,7 +610,6 @@ ORDER BY 1,2,3; --- Aggregate function on distribution column should error out SELECT * FROM ( SELECT user_id, COUNT(*) OVER (PARTITION BY sum(user_id), MIN(value_2)) @@ -645,8 +635,6 @@ ORDER BY LIMIT 20; --- UNION ALL with only one of them is not partitioned over distribution column which --- should not be allowed. SELECT max(avg) FROM @@ -667,8 +655,6 @@ GROUP BY user_id ORDER BY 1 DESC LIMIT 5; --- UNION with only one subquery which has a partition on non-distribution column should --- error out SELECT * FROM ( ( SELECT user_id, diff --git a/src/test/regress/sql/set_operations.sql b/src/test/regress/sql/set_operations.sql index 03481fe5b..be9554835 100644 --- a/src/test/regress/sql/set_operations.sql +++ b/src/test/regress/sql/set_operations.sql @@ -104,16 +104,17 @@ SELECT * FROM ((SELECT x,y FROM test) UNION (SELECT y,x FROM test)) foo WHERE x -- set operations and the sublink can be recursively planned SELECT * FROM ((SELECT x,y FROM test) UNION (SELECT y,x FROM test)) foo WHERE x IN (SELECT y FROM test) ORDER BY 1; --- set operations works fine with pushdownable window functions +-- set operations work fine with pushdownable window functions SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY x ORDER BY y DESC)) as foo UNION SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY x ORDER BY y DESC)) as bar ORDER BY 1 DESC, 2 DESC, 3 DESC; --- set operations errors out with non-pushdownable window functions +-- set operations work fine with non-pushdownable window functions SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC)) as foo UNION -SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC)) as bar; +SELECT x, y, rnk FROM (SELECT *, rank() OVER my_win as rnk FROM test WINDOW my_win AS (PARTITION BY y ORDER BY x DESC)) as bar +ORDER BY 1 DESC, 2 DESC, 3 DESC; -- other set operations in joins also cannot be pushed down SELECT * FROM ((SELECT * FROM test) EXCEPT (SELECT * FROM test ORDER BY x LIMIT 1)) u JOIN test USING (x) ORDER BY 1,2; diff --git a/src/test/regress/sql/subqueries_deep.sql b/src/test/regress/sql/subqueries_deep.sql index 0d8301d25..f3415f464 100644 --- a/src/test/regress/sql/subqueries_deep.sql +++ b/src/test/regress/sql/subqueries_deep.sql @@ -2,12 +2,12 @@ -- test recursive planning functionality with subqueries and CTEs -- =================================================================== CREATE SCHEMA subquery_deep; -SET search_path TO subquery_and_ctes, public; +SET search_path TO subquery_deep, public; SET client_min_messages TO DEBUG1; -- subquery in FROM -> FROM -> FROM should be replaced due to OFFSET -- one level up subquery should be replaced due to GROUP BY on non partition key --- one level up subquery should be replaced due to LIMUT +-- one level up subquery should be replaced due to LIMIT SELECT DISTINCT user_id FROM diff --git a/src/test/regress/sql/subqueries_not_supported.sql b/src/test/regress/sql/subqueries_not_supported.sql index 4af92d435..a91c9a097 100644 --- a/src/test/regress/sql/subqueries_not_supported.sql +++ b/src/test/regress/sql/subqueries_not_supported.sql @@ -79,27 +79,6 @@ FROM ORDER BY 1 DESC; SET citus.enable_router_execution TO true; - --- window functions are not allowed if they're not partitioned on the distribution column -SELECT - * -FROM -( -SELECT - user_id, time, rnk -FROM -( - SELECT - *, rank() OVER my_win as rnk - FROM - events_table - WINDOW my_win AS (PARTITION BY event_type ORDER BY time DESC) -) as foo -ORDER BY - 3 DESC, 1 DESC, 2 DESC -LIMIT - 10) as foo; - -- OUTER JOINs where the outer part is recursively planned and not the other way -- around is not supported SELECT diff --git a/src/test/regress/sql/window_functions.sql b/src/test/regress/sql/window_functions.sql index 6fb378f9e..ed67fe22d 100644 --- a/src/test/regress/sql/window_functions.sql +++ b/src/test/regress/sql/window_functions.sql @@ -136,8 +136,7 @@ ORDER BY rnk DESC, 1 DESC LIMIT 10; --- similar query with no distribution column is on the partition by clause --- is not supported +-- similar query with no distribution column on the partition by clause SELECT DISTINCT ON (events_table.user_id, rnk) events_table.user_id, rank() OVER my_win AS rnk FROM @@ -270,6 +269,69 @@ WINDOW ORDER BY user_id, value_1, 3, 4; +-- repeat above 3 tests without grouping by distribution column +SELECT + value_2, + rank() OVER (PARTITION BY value_2 ROWS BETWEEN + UNBOUNDED PRECEDING AND CURRENT ROW), + dense_rank() OVER (PARTITION BY value_2 RANGE BETWEEN + UNBOUNDED PRECEDING AND CURRENT ROW), + CUME_DIST() OVER (PARTITION BY value_2 RANGE BETWEEN + UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), + PERCENT_RANK() OVER (PARTITION BY value_2 ORDER BY avg(value_1) RANGE BETWEEN + UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +FROM + users_table +GROUP BY + 1 +ORDER BY + 4 DESC,3 DESC,2 DESC ,1 DESC; + +-- test exclude supported +SELECT + value_2, + value_1, + array_agg(value_1) OVER (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), + array_agg(value_1) OVER (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW EXCLUDE CURRENT ROW) +FROM + users_table +WHERE + value_2 > 2 AND value_2 < 6 +ORDER BY + value_2, value_1, 3, 4; + +-- test preceding and following on RANGE window +SELECT + value_2, + value_1, + array_agg(value_1) OVER range_window, + array_agg(value_1) OVER range_window_exclude +FROM + users_table +WHERE + value_2 > 2 AND value_2 < 6 +WINDOW + range_window as (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING), + range_window_exclude as (PARTITION BY value_2 ORDER BY value_1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING EXCLUDE CURRENT ROW) +ORDER BY + value_2, value_1, 3, 4; + +-- test preceding and following on ROW window +SELECT + value_2, + value_1, + array_agg(value_1) OVER row_window, + array_agg(value_1) OVER row_window_exclude +FROM + users_table +WHERE + value_2 > 2 and value_2 < 6 +WINDOW + row_window as (PARTITION BY value_2 ORDER BY value_1 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING), + row_window_exclude as (PARTITION BY value_2 ORDER BY value_1 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING EXCLUDE CURRENT ROW) +ORDER BY + value_2, value_1, 3, 4; + -- some tests with GROUP BY, HAVING and LIMIT SELECT user_id, sum(event_type) OVER my_win , event_type @@ -284,6 +346,18 @@ ORDER BY LIMIT 5; +-- test PARTITION BY avg(...) ORDER BY avg(...) +SELECT + value_1, + avg(value_3), + dense_rank() OVER (PARTITION BY avg(value_3) ORDER BY avg(value_2)) +FROM + users_table +GROUP BY + 1 +ORDER BY + 1; + -- Group by has more columns than partition by SELECT DISTINCT user_id, SUM(value_2) OVER (PARTITION BY user_id) @@ -347,6 +421,34 @@ ORDER BY 3 DESC, 2 DESC, 1 DESC; $Q$); +SELECT + value_2, + AVG(avg(value_1)) OVER (PARTITION BY value_2, max(value_2), MIN(value_2)), + AVG(avg(value_2)) OVER (PARTITION BY value_2, min(value_2), AVG(value_1)) +FROM + users_table +GROUP BY + 1 +ORDER BY + 3 DESC, 2 DESC, 1 DESC; + +SELECT + value_2, user_id, + AVG(avg(value_1)) OVER (PARTITION BY value_2, max(value_2), MIN(value_2)), + AVG(avg(value_2)) OVER (PARTITION BY user_id, min(value_2), AVG(value_1)) +FROM + users_table +GROUP BY + 1, 2 +ORDER BY + 3 DESC, 2 DESC, 1 DESC; + +SELECT user_id, sum(avg(user_id)) OVER () +FROM users_table +GROUP BY user_id +ORDER BY 1 +LIMIT 10; + SELECT user_id, 1 + sum(value_1), @@ -454,3 +556,49 @@ FROM GROUP BY user_id, value_2 ORDER BY user_id, avg(value_1) DESC LIMIT 5; + +-- Grouping can be pushed down with aggregates even when window function can't +EXPLAIN (COSTS FALSE) +SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random()) +FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1; + +-- Window function with inlined CTE +WITH cte as ( + SELECT uref.id user_id, events_table.value_2, count(*) c + FROM events_table + JOIN users_ref_test_table uref ON uref.id = events_table.user_id + GROUP BY 1, 2 +) +SELECT DISTINCT cte.value_2, cte.c, sum(cte.value_2) OVER (PARTITION BY cte.c) +FROM cte JOIN events_table et ON et.value_2 = cte.value_2 and et.value_2 = cte.c +ORDER BY 1; + +-- There was a strange bug where this wouldn't have window functions being pushed down +-- Bug dependent on column ordering +CREATE TABLE daily_uniques (value_2 float, user_id bigint); +SELECT create_distributed_table('daily_uniques', 'user_id'); + +EXPLAIN (COSTS FALSE) SELECT + user_id, + sum(value_2) AS commits, + RANK () OVER ( + PARTITION BY user_id + ORDER BY + sum(value_2) DESC + ) +FROM daily_uniques +GROUP BY user_id +HAVING + sum(value_2) > 0 +ORDER BY commits DESC +LIMIT 10; + +DROP TABLE daily_uniques; + +-- Partition by reference table column joined to distribution column +SELECT DISTINCT value_2, array_agg(rnk ORDER BY rnk) FROM ( +SELECT events_table.value_2, sum(uref.k_no) OVER (PARTITION BY uref.id) AS rnk +FROM events_table +JOIN users_ref_test_table uref ON uref.id = events_table.user_id) sq +GROUP BY 1 ORDER BY 1; +