diff --git a/src/backend/distributed/planner/multi_logical_optimizer.c b/src/backend/distributed/planner/multi_logical_optimizer.c index 00183f201..c32bbe7a0 100644 --- a/src/backend/distributed/planner/multi_logical_optimizer.c +++ b/src/backend/distributed/planner/multi_logical_optimizer.c @@ -43,11 +43,11 @@ #include "parser/parse_agg.h" #include "parser/parse_coerce.h" #include "parser/parse_oper.h" +#include "parser/parsetree.h" #include "utils/builtins.h" #include "utils/fmgroids.h" #include "utils/lsyscache.h" #include "utils/rel.h" -#include "utils/relcache.h" #include "utils/syscache.h" #include "utils/tqual.h" @@ -147,27 +147,6 @@ static bool TablePartitioningSupportsDistinct(List *tableNodeList, Var *distinctColumn); static bool GroupedByColumn(List *groupClauseList, List *targetList, Var *column); -/* Local functions forward declarations for subquery pushdown checks */ -static void ErrorIfContainsUnsupportedSubquery(MultiNode *logicalPlanNode); -static void ErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerQueryHasLimit); -static void ErrorIfUnsupportedTableCombination(Query *queryTree); -static void ErrorIfUnsupportedUnionQuery(Query *unionQuery); -static bool TargetListOnPartitionColumn(Query *query, List *targetEntryList); -static FieldSelect * CompositeFieldRecursive(Expr *expression, Query *query); -static bool FullCompositeFieldList(List *compositeFieldList); -static Query * LateralQuery(Query *query); -static bool SupportedLateralQuery(Query *parentQuery, Query *lateralQuery); -static bool JoinOnPartitionColumn(Query *query); -static void ErrorIfUnsupportedShardDistribution(Query *query); -static List * RelationIdList(Query *query); -static bool CoPartitionedTables(Oid firstRelationId, Oid secondRelationId); -static bool ShardIntervalsEqual(FmgrInfo *comparisonFunction, - ShardInterval *firstInterval, - ShardInterval *secondInterval); -static void ErrorIfUnsupportedFilters(Query *subquery); -static bool EqualOpExpressionLists(List *firstOpExpressionList, - List *secondOpExpressionList); - /* Local functions forward declarations for limit clauses */ static Node * WorkerLimitCount(MultiExtendedOp *originalOpNode); static List * WorkerSortClauseList(MultiExtendedOp *originalOpNode); @@ -211,9 +190,6 @@ MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan) /* check that we can optimize aggregates in the plan */ ErrorIfContainsUnsupportedAggregate(logicalPlanNode); - /* check that we can pushdown subquery in the plan */ - ErrorIfContainsUnsupportedSubquery(logicalPlanNode); - /* * If a select node exists, we use the idempower property to split the node * into two nodes that contain And and Or clauses. If both And and Or nodes @@ -2799,48 +2775,6 @@ GroupedByColumn(List *groupClauseList, List *targetList, Var *column) } -/* - * ErrorIfContainsUnsupportedSubquery extracts subquery multi table from the - * logical plan and uses helper functions to check if we can push down subquery - * to worker nodes. These helper functions error out if we cannot push down the - * the subquery. - */ -static void -ErrorIfContainsUnsupportedSubquery(MultiNode *logicalPlanNode) -{ - Query *subquery = NULL; - List *extendedOpNodeList = NIL; - MultiTable *multiTable = NULL; - MultiExtendedOp *extendedOpNode = NULL; - bool outerQueryHasLimit = false; - - /* check if logical plan includes a subquery */ - List *subqueryMultiTableList = SubqueryMultiTableList(logicalPlanNode); - if (subqueryMultiTableList == NIL) - { - return; - } - - /* currently in the planner we only allow one subquery in from-clause*/ - Assert(list_length(subqueryMultiTableList) == 1); - - multiTable = (MultiTable *) linitial(subqueryMultiTableList); - subquery = multiTable->subquery; - - extendedOpNodeList = FindNodesOfType(logicalPlanNode, T_MultiExtendedOp); - extendedOpNode = (MultiExtendedOp *) linitial(extendedOpNodeList); - - if (extendedOpNode->limitCount) - { - outerQueryHasLimit = true; - } - - ErrorIfCannotPushdownSubquery(subquery, outerQueryHasLimit); - ErrorIfUnsupportedShardDistribution(subquery); - ErrorIfUnsupportedFilters(subquery); -} - - /* * SubqueryMultiTableList extracts multi tables in the given logical plan tree * and returns subquery multi tables in a new list. @@ -2867,382 +2801,6 @@ SubqueryMultiTableList(MultiNode *multiNode) } -/* - * ErrorIfCannotPushdownSubquery recursively checks if we can push down the given - * subquery to worker nodes. If we cannot push down the subquery, this function - * errors out. - * - * We can push down a subquery if it follows rules below. We support nested queries - * as long as they follow the same rules, and we recurse to validate each subquery - * for this given query. - * a. If there is an aggregate, it must be grouped on partition column. - * b. If there is a join, it must be between two regular tables or two subqueries. - * We don't support join between a regular table and a subquery. And columns on - * the join condition must be partition columns. - * c. If there is a distinct clause, it must be on the partition column. - * - * This function is very similar to ErrorIfQueryNotSupported() in logical - * planner, but we don't reuse it, because differently for subqueries we support - * a subset of distinct, union and left joins. - * - * Note that this list of checks is not exhaustive, there can be some cases - * which we let subquery to run but returned results would be wrong. Such as if - * a subquery has a group by on another subquery which includes order by with - * limit, we let this query to run, but results could be wrong depending on the - * features of underlying tables. - */ -static void -ErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerQueryHasLimit) -{ - bool preconditionsSatisfied = true; - char *errorDetail = NULL; - Query *lateralQuery = NULL; - List *subqueryEntryList = NIL; - ListCell *rangeTableEntryCell = NULL; - - ErrorIfUnsupportedTableCombination(subqueryTree); - - if (subqueryTree->hasSubLinks) - { - preconditionsSatisfied = false; - errorDetail = "Subqueries other than from-clause subqueries are unsupported"; - } - - if (subqueryTree->hasWindowFuncs) - { - preconditionsSatisfied = false; - errorDetail = "Window functions are currently unsupported"; - } - - if (subqueryTree->limitOffset) - { - preconditionsSatisfied = false; - errorDetail = "Offset clause is currently unsupported"; - } - - if (subqueryTree->limitCount && !outerQueryHasLimit) - { - preconditionsSatisfied = false; - errorDetail = "Limit in subquery without limit in the outer query is unsupported"; - } - - if (subqueryTree->setOperations) - { - SetOperationStmt *setOperationStatement = - (SetOperationStmt *) subqueryTree->setOperations; - - if (setOperationStatement->op == SETOP_UNION) - { - ErrorIfUnsupportedUnionQuery(subqueryTree); - } - else - { - preconditionsSatisfied = false; - errorDetail = "Intersect and Except are currently unsupported"; - } - } - - if (subqueryTree->hasRecursive) - { - preconditionsSatisfied = false; - errorDetail = "Recursive queries are currently unsupported"; - } - - if (subqueryTree->cteList) - { - preconditionsSatisfied = false; - errorDetail = "Common Table Expressions are currently unsupported"; - } - - if (subqueryTree->hasForUpdate) - { - preconditionsSatisfied = false; - errorDetail = "For Update/Share commands are currently unsupported"; - } - - /* group clause list must include partition column */ - if (subqueryTree->groupClause) - { - List *groupClauseList = subqueryTree->groupClause; - List *targetEntryList = subqueryTree->targetList; - List *groupTargetEntryList = GroupTargetEntryList(groupClauseList, - targetEntryList); - bool groupOnPartitionColumn = TargetListOnPartitionColumn(subqueryTree, - groupTargetEntryList); - if (!groupOnPartitionColumn) - { - preconditionsSatisfied = false; - errorDetail = "Group by list without partition column is currently " - "unsupported"; - } - } - - /* we don't support aggregates without group by */ - if (subqueryTree->hasAggs && (subqueryTree->groupClause == NULL)) - { - preconditionsSatisfied = false; - errorDetail = "Aggregates without group by are currently unsupported"; - } - - /* having clause without group by on partition column is not supported */ - if (subqueryTree->havingQual && (subqueryTree->groupClause == NULL)) - { - preconditionsSatisfied = false; - errorDetail = "Having qual without group by on partition column is " - "currently unsupported"; - } - - /* - * Check if join is supported. We check lateral joins differently, because - * lateral join representation in query tree is a bit different than normal - * join queries. - */ - lateralQuery = LateralQuery(subqueryTree); - if (lateralQuery != NULL) - { - bool supportedLateralQuery = SupportedLateralQuery(subqueryTree, lateralQuery); - if (!supportedLateralQuery) - { - preconditionsSatisfied = false; - errorDetail = "This type of lateral query in subquery is currently " - "unsupported"; - } - } - else - { - List *joinTreeTableIndexList = NIL; - uint32 joiningTableCount = 0; - - ExtractRangeTableIndexWalker((Node *) subqueryTree->jointree, - &joinTreeTableIndexList); - joiningTableCount = list_length(joinTreeTableIndexList); - - /* if this is a join query, check if join clause is on partition columns */ - if ((joiningTableCount > 1)) - { - bool joinOnPartitionColumn = JoinOnPartitionColumn(subqueryTree); - if (!joinOnPartitionColumn) - { - preconditionsSatisfied = false; - errorDetail = "Relations need to be joining on partition columns"; - } - } - } - - /* distinct clause list must include partition column */ - if (subqueryTree->distinctClause) - { - List *distinctClauseList = subqueryTree->distinctClause; - List *targetEntryList = subqueryTree->targetList; - List *distinctTargetEntryList = GroupTargetEntryList(distinctClauseList, - targetEntryList); - bool distinctOnPartitionColumn = - TargetListOnPartitionColumn(subqueryTree, distinctTargetEntryList); - if (!distinctOnPartitionColumn) - { - preconditionsSatisfied = false; - errorDetail = "Distinct on columns without partition column is " - "currently unsupported"; - } - } - - /* finally check and error out if not satisfied */ - if (!preconditionsSatisfied) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot push down this subquery"), - errdetail("%s", errorDetail))); - } - - /* recursively do same check for subqueries of this query */ - subqueryEntryList = SubqueryEntryList(subqueryTree); - foreach(rangeTableEntryCell, subqueryEntryList) - { - RangeTblEntry *rangeTableEntry = - (RangeTblEntry *) lfirst(rangeTableEntryCell); - - Query *innerSubquery = rangeTableEntry->subquery; - ErrorIfCannotPushdownSubquery(innerSubquery, outerQueryHasLimit); - } -} - - -/* - * ErrorIfUnsupportedTableCombination checks if the given query tree contains any - * unsupported range table combinations. For this, the function walks over all - * range tables in the join tree, and checks if they correspond to simple relations - * or subqueries. It also checks if there is a join between a regular table and - * a subquery and if join is on more than two range table entries. - */ -static void -ErrorIfUnsupportedTableCombination(Query *queryTree) -{ - List *rangeTableList = queryTree->rtable; - List *joinTreeTableIndexList = NIL; - ListCell *joinTreeTableIndexCell = NULL; - bool unsupporteTableCombination = false; - char *errorDetail = NULL; - uint32 relationRangeTableCount = 0; - uint32 subqueryRangeTableCount = 0; - - /* - * Extract all range table indexes from the join tree. Note that sub-queries - * that get pulled up by PostgreSQL don't appear in this join tree. - */ - ExtractRangeTableIndexWalker((Node *) queryTree->jointree, &joinTreeTableIndexList); - foreach(joinTreeTableIndexCell, joinTreeTableIndexList) - { - /* - * Join tree's range table index starts from 1 in the query tree. But, - * list indexes start from 0. - */ - int joinTreeTableIndex = lfirst_int(joinTreeTableIndexCell); - int rangeTableListIndex = joinTreeTableIndex - 1; - - RangeTblEntry *rangeTableEntry = - (RangeTblEntry *) list_nth(rangeTableList, rangeTableListIndex); - - /* - * Check if the range table in the join tree is a simple relation or a - * subquery. - */ - if (rangeTableEntry->rtekind == RTE_RELATION) - { - relationRangeTableCount++; - } - else if (rangeTableEntry->rtekind == RTE_SUBQUERY) - { - subqueryRangeTableCount++; - } - else - { - unsupporteTableCombination = true; - errorDetail = "Table expressions other than simple relations and " - "subqueries are currently unsupported"; - break; - } - } - - if ((subqueryRangeTableCount > 0) && (relationRangeTableCount > 0)) - { - unsupporteTableCombination = true; - errorDetail = "Joins between regular tables and subqueries are unsupported"; - } - - if ((relationRangeTableCount > 2) || (subqueryRangeTableCount > 2)) - { - unsupporteTableCombination = true; - errorDetail = "Joins between more than two relations and subqueries are " - "unsupported"; - } - - /* finally check and error out if not satisfied */ - if (unsupporteTableCombination) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot push down this subquery"), - errdetail("%s", errorDetail))); - } -} - - -/* - * ErrorIfUnsupportedUnionQuery checks if the given union query is a supported - * one., otherwise it errors out. For these purpose it checks tree conditions; - * a. Are count of partition column filters same for union subqueries. - * b. Are target lists of union subquries include partition column. - * c. Is it a union clause without All option. - * - * Note that we check equality of filters in ErrorIfUnsupportedFilters(). We - * allow leaf queries not having a filter clause on the partition column. We - * check if a leaf query has a filter on the partition column, it must be same - * with other queries or if leaf query must not have any filter on the partition - * column, both are ok. Because joins and nested queries are transitive, it is - * enough one leaf query to have a filter on the partition column. But unions - * are not transitive, so here we check if they have same count of filters on - * the partition column. If count is more than 0, we already checked that they - * are same, of if count is 0 then both don't have any filter on the partition - * column. - */ -static void -ErrorIfUnsupportedUnionQuery(Query *unionQuery) -{ - bool supportedUnionQuery = true; - bool leftQueryOnPartitionColumn = false; - bool rightQueryOnPartitionColumn = false; - List *rangeTableList = unionQuery->rtable; - SetOperationStmt *unionStatement = (SetOperationStmt *) unionQuery->setOperations; - Query *leftQuery = NULL; - Query *rightQuery = NULL; - List *leftOpExpressionList = NIL; - List *rightOpExpressionList = NIL; - uint32 leftOpExpressionCount = 0; - uint32 rightOpExpressionCount = 0; - char *errorDetail = NULL; - - RangeTblRef *leftRangeTableReference = (RangeTblRef *) unionStatement->larg; - RangeTblRef *rightRangeTableReference = (RangeTblRef *) unionStatement->rarg; - - int leftTableIndex = leftRangeTableReference->rtindex - 1; - int rightTableIndex = rightRangeTableReference->rtindex - 1; - - RangeTblEntry *leftRangeTableEntry = (RangeTblEntry *) list_nth(rangeTableList, - leftTableIndex); - RangeTblEntry *rightRangeTableEntry = (RangeTblEntry *) list_nth(rangeTableList, - rightTableIndex); - - Assert(leftRangeTableEntry->rtekind == RTE_SUBQUERY); - Assert(rightRangeTableEntry->rtekind == RTE_SUBQUERY); - - leftQuery = leftRangeTableEntry->subquery; - rightQuery = rightRangeTableEntry->subquery; - - /* - * Check if subqueries of union have same count of filters on partition - * column. - */ - leftOpExpressionList = PartitionColumnOpExpressionList(leftQuery); - rightOpExpressionList = PartitionColumnOpExpressionList(rightQuery); - - leftOpExpressionCount = list_length(leftOpExpressionList); - rightOpExpressionCount = list_length(rightOpExpressionList); - - if (leftOpExpressionCount != rightOpExpressionCount) - { - supportedUnionQuery = false; - errorDetail = "Union clauses need to have same count of filters on " - "partition column"; - } - - /* check if union subqueries have partition column in their target lists */ - leftQueryOnPartitionColumn = TargetListOnPartitionColumn(leftQuery, - leftQuery->targetList); - rightQueryOnPartitionColumn = TargetListOnPartitionColumn(rightQuery, - rightQuery->targetList); - - if (!(leftQueryOnPartitionColumn && rightQueryOnPartitionColumn)) - { - supportedUnionQuery = false; - errorDetail = "Union clauses need to select partition columns"; - } - - /* check if it is a union all operation */ - if (unionStatement->all) - { - supportedUnionQuery = false; - errorDetail = "Union All clauses are currently unsupported"; - } - - /* finally check and error out if not satisfied */ - if (!supportedUnionQuery) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot push down this subquery"), - errdetail("%s", errorDetail))); - } -} - - /* * GroupTargetEntryList walks over group clauses in the given list, finds * matching target entries and return them in a new list. @@ -3265,53 +2823,6 @@ GroupTargetEntryList(List *groupClauseList, List *targetEntryList) } -/* - * TargetListOnPartitionColumn checks if at least one target list entry is on - * partition column. - */ -static bool -TargetListOnPartitionColumn(Query *query, List *targetEntryList) -{ - bool targetListOnPartitionColumn = false; - List *compositeFieldList = NIL; - - ListCell *targetEntryCell = NULL; - foreach(targetEntryCell, targetEntryList) - { - TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); - Expr *targetExpression = targetEntry->expr; - - bool isPartitionColumn = IsPartitionColumn(targetExpression, query); - if (isPartitionColumn) - { - FieldSelect *compositeField = CompositeFieldRecursive(targetExpression, - query); - if (compositeField) - { - compositeFieldList = lappend(compositeFieldList, compositeField); - } - else - { - targetListOnPartitionColumn = true; - break; - } - } - } - - /* check composite fields */ - if (!targetListOnPartitionColumn) - { - bool fullCompositeFieldList = FullCompositeFieldList(compositeFieldList); - if (fullCompositeFieldList) - { - targetListOnPartitionColumn = true; - } - } - - return targetListOnPartitionColumn; -} - - /* * IsPartitionColumn returns true if the given column is a partition column. * The function uses FindReferencedTableColumn to find the original relation @@ -3465,667 +2976,6 @@ FindReferencedTableColumn(Expr *columnExpression, List *parentQueryList, Query * } -/* - * CompositeFieldRecursive recursively finds composite field in the query tree - * referred by given expression. If expression does not refer to a composite - * field, then it returns NULL. - * - * If expression is a field select we directly return composite field. If it is - * a column is referenced from a subquery, then we recursively check that subquery - * until we reach the source of that column, and find composite field. If this - * column is referenced from join range table entry, then we resolve which join - * column it refers and recursively use this column with the same query. - */ -static FieldSelect * -CompositeFieldRecursive(Expr *expression, Query *query) -{ - FieldSelect *compositeField = NULL; - List *rangetableList = query->rtable; - Index rangeTableEntryIndex = 0; - RangeTblEntry *rangeTableEntry = NULL; - Var *candidateColumn = NULL; - - if (IsA(expression, FieldSelect)) - { - compositeField = (FieldSelect *) expression; - return compositeField; - } - - if (IsA(expression, Var)) - { - candidateColumn = (Var *) expression; - } - else - { - return NULL; - } - - rangeTableEntryIndex = candidateColumn->varno - 1; - rangeTableEntry = list_nth(rangetableList, rangeTableEntryIndex); - - if (rangeTableEntry->rtekind == RTE_SUBQUERY) - { - Query *subquery = rangeTableEntry->subquery; - List *targetEntryList = subquery->targetList; - AttrNumber targetEntryIndex = candidateColumn->varattno - 1; - TargetEntry *subqueryTargetEntry = list_nth(targetEntryList, targetEntryIndex); - - Expr *subqueryExpression = subqueryTargetEntry->expr; - compositeField = CompositeFieldRecursive(subqueryExpression, subquery); - } - else if (rangeTableEntry->rtekind == RTE_JOIN) - { - List *joinColumnList = rangeTableEntry->joinaliasvars; - AttrNumber joinColumnIndex = candidateColumn->varattno - 1; - Expr *joinColumn = list_nth(joinColumnList, joinColumnIndex); - - compositeField = CompositeFieldRecursive(joinColumn, query); - } - - return compositeField; -} - - -/* - * FullCompositeFieldList gets a composite field list, and checks if all fields - * of composite type are used in the list. - */ -static bool -FullCompositeFieldList(List *compositeFieldList) -{ - bool fullCompositeFieldList = true; - bool *compositeFieldArray = NULL; - uint32 compositeFieldCount = 0; - uint32 fieldIndex = 0; - - ListCell *fieldSelectCell = NULL; - foreach(fieldSelectCell, compositeFieldList) - { - FieldSelect *fieldSelect = (FieldSelect *) lfirst(fieldSelectCell); - uint32 compositeFieldIndex = 0; - - Expr *fieldExpression = fieldSelect->arg; - if (!IsA(fieldExpression, Var)) - { - continue; - } - - if (compositeFieldArray == NULL) - { - uint32 index = 0; - Var *compositeColumn = (Var *) fieldExpression; - Oid compositeTypeId = compositeColumn->vartype; - Oid compositeRelationId = get_typ_typrelid(compositeTypeId); - - /* get composite type attribute count */ - Relation relation = relation_open(compositeRelationId, AccessShareLock); - compositeFieldCount = relation->rd_att->natts; - compositeFieldArray = palloc0(compositeFieldCount * sizeof(bool)); - relation_close(relation, AccessShareLock); - - for (index = 0; index < compositeFieldCount; index++) - { - compositeFieldArray[index] = false; - } - } - - compositeFieldIndex = fieldSelect->fieldnum - 1; - compositeFieldArray[compositeFieldIndex] = true; - } - - for (fieldIndex = 0; fieldIndex < compositeFieldCount; fieldIndex++) - { - if (!compositeFieldArray[fieldIndex]) - { - fullCompositeFieldList = false; - } - } - - if (compositeFieldCount == 0) - { - fullCompositeFieldList = false; - } - - return fullCompositeFieldList; -} - - -/* - * LateralQuery walks over the given range table list and if there is a subquery - * columns with other sibling subquery. - */ -static Query * -LateralQuery(Query *query) -{ - Query *lateralQuery = NULL; - List *rangeTableList = query->rtable; - - ListCell *rangeTableCell = NULL; - foreach(rangeTableCell, rangeTableList) - { - RangeTblEntry *rangeTableEntry = (RangeTblEntry *) lfirst(rangeTableCell); - if (rangeTableEntry->rtekind == RTE_SUBQUERY && rangeTableEntry->lateral) - { - lateralQuery = rangeTableEntry->subquery; - break; - } - } - - return lateralQuery; -} - - -/* - * SupportedLateralQuery checks if the given lateral query is joined on partition - * columns with another siblings subquery. - */ -static bool -SupportedLateralQuery(Query *parentQuery, Query *lateralQuery) -{ - bool supportedLateralQuery = false; - List *outerCompositeFieldList = NIL; - List *localCompositeFieldList = NIL; - ListCell *qualifierCell = NULL; - - List *qualifierList = QualifierList(lateralQuery->jointree); - foreach(qualifierCell, qualifierList) - { - OpExpr *operatorExpression = NULL; - List *argumentList = NIL; - bool equalsOperator = false; - Expr *leftArgument = NULL; - Expr *rightArgument = NULL; - Expr *outerQueryExpression = NULL; - Expr *localQueryExpression = NULL; - Var *leftColumn = NULL; - Var *rightColumn = NULL; - bool outerColumnIsPartitionColumn = false; - bool localColumnIsPartitionColumn = false; - - Node *qualifier = (Node *) lfirst(qualifierCell); - if (!IsA(qualifier, OpExpr)) - { - continue; - } - - operatorExpression = (OpExpr *) qualifier; - argumentList = operatorExpression->args; - - /* - * Join clauses must have two arguments. Note that logic here use to find - * join clauses is very similar to IsJoinClause(). But we are not able to - * reuse it, because it calls pull_var_clause_default() which in return - * deep down calls pull_var_clause_walker(), and this function errors out - * for variable level other than 0 which is the case for lateral joins. - */ - if (list_length(argumentList) != 2) - { - continue; - } - - equalsOperator = OperatorImplementsEquality(operatorExpression->opno); - if (!equalsOperator) - { - continue; - } - - /* get left and right side of the expression */ - leftArgument = (Expr *) linitial(argumentList); - rightArgument = (Expr *) lsecond(argumentList); - - if (IsA(leftArgument, Var)) - { - leftColumn = (Var *) leftArgument; - } - else if (IsA(leftArgument, FieldSelect)) - { - FieldSelect *fieldSelect = (FieldSelect *) leftArgument; - Expr *fieldExpression = fieldSelect->arg; - - if (!IsA(fieldExpression, Var)) - { - continue; - } - - leftColumn = (Var *) fieldExpression; - } - else - { - continue; - } - - if (IsA(rightArgument, Var)) - { - rightColumn = (Var *) rightArgument; - } - else if (IsA(rightArgument, FieldSelect)) - { - FieldSelect *fieldSelect = (FieldSelect *) rightArgument; - Expr *fieldExpression = fieldSelect->arg; - - if (!IsA(fieldExpression, Var)) - { - continue; - } - - rightColumn = (Var *) fieldExpression; - } - else - { - continue; - } - - if (leftColumn->varlevelsup == 1 && rightColumn->varlevelsup == 0) - { - outerQueryExpression = leftArgument; - localQueryExpression = rightArgument; - } - else if (leftColumn->varlevelsup == 0 && rightColumn->varlevelsup == 1) - { - outerQueryExpression = rightArgument; - localQueryExpression = leftArgument; - } - else - { - continue; - } - - outerColumnIsPartitionColumn = IsPartitionColumn(outerQueryExpression, - parentQuery); - localColumnIsPartitionColumn = IsPartitionColumn(localQueryExpression, - lateralQuery); - - if (outerColumnIsPartitionColumn && localColumnIsPartitionColumn) - { - FieldSelect *outerCompositeField = - CompositeFieldRecursive(outerQueryExpression, parentQuery); - FieldSelect *localCompositeField = - CompositeFieldRecursive(localQueryExpression, lateralQuery); - - /* - * If partition colums are composite fields, add them to list to - * check later if all composite fields are used. - */ - if (outerCompositeField && localCompositeField) - { - outerCompositeFieldList = lappend(outerCompositeFieldList, - outerCompositeField); - localCompositeFieldList = lappend(localCompositeFieldList, - localCompositeField); - } - - /* if both sides are not composite fields, they are normal columns */ - if (!(outerCompositeField || localCompositeField)) - { - supportedLateralQuery = true; - break; - } - } - } - - /* check composite fields */ - if (!supportedLateralQuery) - { - bool outerFullCompositeFieldList = - FullCompositeFieldList(outerCompositeFieldList); - bool localFullCompositeFieldList = - FullCompositeFieldList(localCompositeFieldList); - - if (outerFullCompositeFieldList && localFullCompositeFieldList) - { - supportedLateralQuery = true; - } - } - - return supportedLateralQuery; -} - - -/* - * JoinOnPartitionColumn checks if both sides of at least one join clause are on - * partition columns. - */ -static bool -JoinOnPartitionColumn(Query *query) -{ - bool joinOnPartitionColumn = false; - List *leftCompositeFieldList = NIL; - List *rightCompositeFieldList = NIL; - List *qualifierList = QualifierList(query->jointree); - List *joinClauseList = JoinClauseList(qualifierList); - - ListCell *joinClauseCell = NULL; - foreach(joinClauseCell, joinClauseList) - { - OpExpr *joinClause = (OpExpr *) lfirst(joinClauseCell); - List *joinArgumentList = joinClause->args; - Expr *leftArgument = NULL; - Expr *rightArgument = NULL; - bool isLeftColumnPartitionColumn = false; - bool isRightColumnPartitionColumn = false; - - /* get left and right side of the expression */ - leftArgument = (Expr *) linitial(joinArgumentList); - rightArgument = (Expr *) lsecond(joinArgumentList); - - isLeftColumnPartitionColumn = IsPartitionColumn(leftArgument, query); - isRightColumnPartitionColumn = IsPartitionColumn(rightArgument, query); - - if (isLeftColumnPartitionColumn && isRightColumnPartitionColumn) - { - FieldSelect *leftCompositeField = - CompositeFieldRecursive(leftArgument, query); - FieldSelect *rightCompositeField = - CompositeFieldRecursive(rightArgument, query); - - /* - * If partition colums are composite fields, add them to list to - * check later if all composite fields are used. - */ - if (leftCompositeField && rightCompositeField) - { - leftCompositeFieldList = lappend(leftCompositeFieldList, - leftCompositeField); - rightCompositeFieldList = lappend(rightCompositeFieldList, - rightCompositeField); - } - - /* if both sides are not composite fields, they are normal columns */ - if (!(leftCompositeField && rightCompositeField)) - { - joinOnPartitionColumn = true; - break; - } - } - } - - /* check composite fields */ - if (!joinOnPartitionColumn) - { - bool leftFullCompositeFieldList = - FullCompositeFieldList(leftCompositeFieldList); - bool rightFullCompositeFieldList = - FullCompositeFieldList(rightCompositeFieldList); - - if (leftFullCompositeFieldList && rightFullCompositeFieldList) - { - joinOnPartitionColumn = true; - } - } - - return joinOnPartitionColumn; -} - - -/* - * ErrorIfUnsupportedShardDistribution gets list of relations in the given query - * and checks if two conditions below hold for them, otherwise it errors out. - * a. Every relation is distributed by range or hash. This means shards are - * disjoint based on the partition column. - * b. All relations have 1-to-1 shard partitioning between them. This means - * shard count for every relation is same and for every shard in a relation - * there is exactly one shard in other relations with same min/max values. - */ -static void -ErrorIfUnsupportedShardDistribution(Query *query) -{ - Oid firstTableRelationId = InvalidOid; - List *relationIdList = RelationIdList(query); - ListCell *relationIdCell = NULL; - uint32 relationIndex = 0; - uint32 rangeDistributedRelationCount = 0; - uint32 hashDistributedRelationCount = 0; - - foreach(relationIdCell, relationIdList) - { - Oid relationId = lfirst_oid(relationIdCell); - char partitionMethod = PartitionMethod(relationId); - if (partitionMethod == DISTRIBUTE_BY_RANGE) - { - rangeDistributedRelationCount++; - } - else if (partitionMethod == DISTRIBUTE_BY_HASH) - { - hashDistributedRelationCount++; - } - else - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot push down this subquery"), - errdetail("Currently range and hash partitioned " - "relations are supported"))); - } - } - - if ((rangeDistributedRelationCount > 0) && (hashDistributedRelationCount > 0)) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot push down this subquery"), - errdetail("A query including both range and hash " - "partitioned relations are unsupported"))); - } - - foreach(relationIdCell, relationIdList) - { - Oid relationId = lfirst_oid(relationIdCell); - bool coPartitionedTables = false; - Oid currentRelationId = relationId; - - /* get shard list of first relation and continue for the next relation */ - if (relationIndex == 0) - { - firstTableRelationId = relationId; - relationIndex++; - - continue; - } - - /* check if this table has 1-1 shard partitioning with first table */ - coPartitionedTables = CoPartitionedTables(firstTableRelationId, - currentRelationId); - if (!coPartitionedTables) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot push down this subquery"), - errdetail("Shards of relations in subquery need to " - "have 1-to-1 shard partitioning"))); - } - } -} - - -/* - * RelationIdList returns list of unique relation ids in query tree. - */ -static List * -RelationIdList(Query *query) -{ - List *rangeTableList = NIL; - List *tableEntryList = NIL; - List *relationIdList = NIL; - ListCell *tableEntryCell = NULL; - - ExtractRangeTableRelationWalker((Node *) query, &rangeTableList); - tableEntryList = TableEntryList(rangeTableList); - - foreach(tableEntryCell, tableEntryList) - { - TableEntry *tableEntry = (TableEntry *) lfirst(tableEntryCell); - Oid relationId = tableEntry->relationId; - - relationIdList = list_append_unique_oid(relationIdList, relationId); - } - - return relationIdList; -} - - -/* - * CoPartitionedTables checks if given two distributed tables have 1-to-1 - * shard partitioning. - */ -static bool -CoPartitionedTables(Oid firstRelationId, Oid secondRelationId) -{ - bool coPartitionedTables = true; - uint32 intervalIndex = 0; - DistTableCacheEntry *firstTableCache = DistributedTableCacheEntry(firstRelationId); - DistTableCacheEntry *secondTableCache = DistributedTableCacheEntry(secondRelationId); - ShardInterval **sortedFirstIntervalArray = firstTableCache->sortedShardIntervalArray; - ShardInterval **sortedSecondIntervalArray = - secondTableCache->sortedShardIntervalArray; - uint32 firstListShardCount = firstTableCache->shardIntervalArrayLength; - uint32 secondListShardCount = secondTableCache->shardIntervalArrayLength; - FmgrInfo *comparisonFunction = firstTableCache->shardIntervalCompareFunction; - - if (firstListShardCount != secondListShardCount) - { - return false; - } - - /* if there are not any shards just return true */ - if (firstListShardCount == 0) - { - return true; - } - - Assert(comparisonFunction != NULL); - - /* - * Check if the tables have the same colocation ID - if so, we know - * they're colocated. - */ - if (firstTableCache->colocationId != INVALID_COLOCATION_ID && - firstTableCache->colocationId == secondTableCache->colocationId) - { - return true; - } - - /* - * If not known to be colocated check if the remaining shards are - * anyway. Do so by comparing the shard interval arrays that are sorted on - * interval minimum values. Then it compares every shard interval in order - * and if any pair of shard intervals are not equal it returns false. - */ - for (intervalIndex = 0; intervalIndex < firstListShardCount; intervalIndex++) - { - ShardInterval *firstInterval = sortedFirstIntervalArray[intervalIndex]; - ShardInterval *secondInterval = sortedSecondIntervalArray[intervalIndex]; - - bool shardIntervalsEqual = ShardIntervalsEqual(comparisonFunction, - firstInterval, - secondInterval); - if (!shardIntervalsEqual) - { - coPartitionedTables = false; - break; - } - } - - return coPartitionedTables; -} - - -/* - * ShardIntervalsEqual checks if given shard intervals have equal min/max values. - */ -static bool -ShardIntervalsEqual(FmgrInfo *comparisonFunction, ShardInterval *firstInterval, - ShardInterval *secondInterval) -{ - bool shardIntervalsEqual = false; - Datum firstMin = 0; - Datum firstMax = 0; - Datum secondMin = 0; - Datum secondMax = 0; - - firstMin = firstInterval->minValue; - firstMax = firstInterval->maxValue; - secondMin = secondInterval->minValue; - secondMax = secondInterval->maxValue; - - if (firstInterval->minValueExists && firstInterval->maxValueExists && - secondInterval->minValueExists && secondInterval->maxValueExists) - { - Datum minDatum = CompareCall2(comparisonFunction, firstMin, secondMin); - Datum maxDatum = CompareCall2(comparisonFunction, firstMax, secondMax); - int firstComparison = DatumGetInt32(minDatum); - int secondComparison = DatumGetInt32(maxDatum); - - if (firstComparison == 0 && secondComparison == 0) - { - shardIntervalsEqual = true; - } - } - - return shardIntervalsEqual; -} - - -/* - * ErrorIfUnsupportedFilters checks if all leaf queries in the given query have - * same filter on the partition column. Note that if there are queries without - * any filter on the partition column, they don't break this prerequisite. - */ -static void -ErrorIfUnsupportedFilters(Query *subquery) -{ - List *queryList = NIL; - ListCell *queryCell = NULL; - List *subqueryOpExpressionList = NIL; - List *relationIdList = RelationIdList(subquery); - - /* - * Get relation id of any relation in the subquery and create partiton column - * for this relation. We will use this column to replace columns on operator - * expressions on different tables. Then we compare these operator expressions - * to see if they consist of same operator and constant value. - */ - Oid relationId = linitial_oid(relationIdList); - Var *partitionColumn = PartitionColumn(relationId, 0); - - ExtractQueryWalker((Node *) subquery, &queryList); - foreach(queryCell, queryList) - { - Query *query = (Query *) lfirst(queryCell); - List *opExpressionList = NIL; - List *newOpExpressionList = NIL; - - bool leafQuery = LeafQuery(query); - if (!leafQuery) - { - continue; - } - - opExpressionList = PartitionColumnOpExpressionList(query); - if (opExpressionList == NIL) - { - continue; - } - - newOpExpressionList = ReplaceColumnsInOpExpressionList(opExpressionList, - partitionColumn); - - if (subqueryOpExpressionList == NIL) - { - subqueryOpExpressionList = newOpExpressionList; - } - else - { - bool equalOpExpressionLists = EqualOpExpressionLists(subqueryOpExpressionList, - newOpExpressionList); - if (!equalOpExpressionLists) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot push down this subquery"), - errdetail("Currently all leaf queries need to " - "have same filters on partition column"))); - } - } - } -} - - /* * ExtractQueryWalker walks over a query, and finds all queries in the query * tree and returns these queries. @@ -4324,51 +3174,6 @@ ReplaceColumnsInOpExpressionList(List *opExpressionList, Var *newColumn) } -/* - * EqualOpExpressionLists checks if given two operator expression lists are - * equal. - */ -static bool -EqualOpExpressionLists(List *firstOpExpressionList, List *secondOpExpressionList) -{ - bool equalOpExpressionLists = false; - ListCell *firstOpExpressionCell = NULL; - uint32 equalOpExpressionCount = 0; - uint32 firstOpExpressionCount = list_length(firstOpExpressionList); - uint32 secondOpExpressionCount = list_length(secondOpExpressionList); - - if (firstOpExpressionCount != secondOpExpressionCount) - { - return false; - } - - foreach(firstOpExpressionCell, firstOpExpressionList) - { - OpExpr *firstOpExpression = (OpExpr *) lfirst(firstOpExpressionCell); - ListCell *secondOpExpressionCell = NULL; - - foreach(secondOpExpressionCell, secondOpExpressionList) - { - OpExpr *secondOpExpression = (OpExpr *) lfirst(secondOpExpressionCell); - bool equalExpressions = equal(firstOpExpression, secondOpExpression); - - if (equalExpressions) - { - equalOpExpressionCount++; - continue; - } - } - } - - if (equalOpExpressionCount == firstOpExpressionCount) - { - equalOpExpressionLists = true; - } - - return equalOpExpressionLists; -} - - /* * WorkerLimitCount checks if the given extended node contains a limit node, and * if that node can be pushed down. For this, the function checks if this limit @@ -4409,6 +3214,15 @@ WorkerLimitCount(MultiExtendedOp *originalOpNode) return NULL; } + /* + * During subquery pushdown planning original query is used. In that case, + * certain expressions such as parameters are not evaluated and converted + * into Consts on the op node. + */ + Assert(IsA(originalOpNode->limitCount, Const)); + Assert(originalOpNode->limitOffset == NULL || + IsA(originalOpNode->limitOffset, Const)); + /* * If we don't have group by clauses, or if we have order by clauses without * aggregates, we can push down the original limit. Else if we have order by diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index 1d2e071cd..f2b08b096 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -14,15 +14,18 @@ #include "postgres.h" +#include "access/heapam.h" #include "access/nbtree.h" #include "catalog/pg_am.h" #include "catalog/pg_class.h" #include "commands/defrem.h" +#include "distributed/citus_clauses.h" #include "distributed/colocation_utils.h" #include "distributed/metadata_cache.h" #include "distributed/multi_logical_optimizer.h" #include "distributed/multi_logical_planner.h" #include "distributed/multi_physical_planner.h" +#include "distributed/relation_restriction_equivalence.h" #include "distributed/worker_protocol.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" @@ -34,6 +37,8 @@ #include "utils/datum.h" #include "utils/lsyscache.h" #include "utils/syscache.h" +#include "utils/rel.h" +#include "utils/relcache.h" /* Config variable managed via guc.c */ @@ -56,11 +61,32 @@ typedef MultiNode *(*RuleApplyFunction) (MultiNode *leftNode, MultiNode *rightNo static RuleApplyFunction RuleApplyFunctionArray[JOIN_RULE_LAST] = { 0 }; /* join rules */ /* Local functions forward declarations */ +static bool SingleRelationRepartitionSubquery(Query *queryTree); +static DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryPushdown(Query * + originalQuery, + PlannerRestrictionContext + * + plannerRestrictionContext); +static DeferredErrorMessage * DeferErrorIfUnsupportedFilters(Query *subquery); +static bool EqualOpExpressionLists(List *firstOpExpressionList, + List *secondOpExpressionList); +static DeferredErrorMessage * DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, + bool + outerMostQueryHasLimit); +static DeferredErrorMessage * DeferErrorIfUnsupportedUnionQuery(Query *queryTree, + bool + outerMostQueryHasLimit); +static bool ExtractSetOperationStatmentWalker(Node *node, List **setOperationList); +static DeferredErrorMessage * DeferErrorIfUnsupportedTableCombination(Query *queryTree); +static bool TargetListOnPartitionColumn(Query *query, List *targetEntryList); +static FieldSelect * CompositeFieldRecursive(Expr *expression, Query *query); +static bool FullCompositeFieldList(List *compositeFieldList); static MultiNode * MultiPlanTree(Query *queryTree); static void ErrorIfQueryNotSupported(Query *queryTree); static bool HasUnsupportedJoinWalker(Node *node, void *context); static bool ErrorHintRequired(const char *errorHint, Query *queryTree); -static void ErrorIfSubqueryNotSupported(Query *subqueryTree); +static DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryRepartition(Query * + subqueryTree); static bool HasTablesample(Query *queryTree); static bool HasOuterJoin(Query *queryTree); static bool HasOuterJoinWalker(Node *node, void *maxJoinLevel); @@ -104,34 +130,47 @@ static MultiNode * ApplyCartesianProduct(MultiNode *leftNode, MultiNode *rightNo * Local functions forward declarations for subquery pushdown. Note that these * functions will be removed with upcoming subqery changes. */ -static MultiNode * SubqueryPushdownMultiPlanTree(Query *queryTree, - List *subqueryEntryList); -static void ErrorIfSubqueryJoin(Query *queryTree); -static MultiTable * MultiSubqueryPushdownTable(RangeTblEntry *subqueryRangeTableEntry); +static MultiNode * MultiSubqueryPlanTree(Query *originalQuery, + Query *queryTree, + PlannerRestrictionContext * + plannerRestrictionContext); +static MultiNode * SubqueryPushdownMultiPlanTree(Query *queryTree); + +static List * CreateSubqueryTargetEntryList(List *columnList); +static void UpdateVarMappingsForExtendedOpNode(List *columnList, + List *subqueryTargetEntryList); +static MultiTable * MultiSubqueryPushdownTable(Query *subquery); /* - * MultiLogicalPlanCreate takes in a parsed query tree, uses helper functions to - * create logical plan and adds a root node to top of it. + * MultiLogicalPlanCreate takes in both the original query and its corresponding modified + * query tree yield by the standard planner. It uses helper functions to create logical + * plan and adds a root node to top of it. The original query is only used for subquery + * pushdown planning. + * + * We also pass queryTree and plannerRestrictionContext to the planner. They + * are primarily used to decide whether the subquery is safe to pushdown. + * If not, it helps to produce meaningful error messages for subquery + * pushdown planning. */ MultiTreeRoot * -MultiLogicalPlanCreate(Query *queryTree) +MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree, + PlannerRestrictionContext *plannerRestrictionContext) { MultiNode *multiQueryNode = NULL; MultiTreeRoot *rootNode = NULL; + List *subqueryEntryList = NULL; - List *subqueryEntryList = SubqueryEntryList(queryTree); + /* + * We check the existence of subqueries in the modified query given that + * if postgres already flattened the subqueries, MultiPlanTree() can plan + * corresponding distributed plan. + */ + subqueryEntryList = SubqueryEntryList(queryTree); if (subqueryEntryList != NIL) { - if (SubqueryPushdown) - { - multiQueryNode = SubqueryPushdownMultiPlanTree(queryTree, subqueryEntryList); - } - else - { - ErrorIfSubqueryJoin(queryTree); - multiQueryNode = MultiPlanTree(queryTree); - } + multiQueryNode = MultiSubqueryPlanTree(originalQuery, queryTree, + plannerRestrictionContext); } else { @@ -146,6 +185,838 @@ MultiLogicalPlanCreate(Query *queryTree) } +/* + * MultiSubqueryPlanTree gets the query objects and returns logical plan + * for subqueries. + * + * We currently have two different code paths for creating logic plan for subqueries: + * (i) subquery pushdown + * (ii) single relation repartition subquery + * + * In order to create the logical plan, we follow the algorithm below: + * - If subquery pushdown planner can plan the query + * - We're done, we create the multi plan tree and return + * - Else + * - If the query is not eligible for single table repartition subquery planning + * - Throw the error that the subquery pushdown planner generated + * - If it is eligible for single table repartition subquery planning + * - Check for the errors for single table repartition subquery planning + * - If no errors found, we're done. Create the multi plan and return + * - If found errors, throw it + */ +static MultiNode * +MultiSubqueryPlanTree(Query *originalQuery, Query *queryTree, + PlannerRestrictionContext *plannerRestrictionContext) +{ + MultiNode *multiQueryNode = NULL; + DeferredErrorMessage *subqueryPushdownError = NULL; + + /* + * This is a generic error check that applies to both subquery pushdown + * and single table repartition subquery. + */ + ErrorIfQueryNotSupported(originalQuery); + + /* + * In principle, we're first trying subquery pushdown planner. If it fails + * to create a logical plan, continue with trying the single table + * repartition subquery planning. + */ + subqueryPushdownError = DeferErrorIfUnsupportedSubqueryPushdown(originalQuery, + plannerRestrictionContext); + if (!subqueryPushdownError) + { + multiQueryNode = SubqueryPushdownMultiPlanTree(originalQuery); + } + else if (subqueryPushdownError) + { + bool singleRelationRepartitionSubquery = false; + RangeTblEntry *subqueryRangeTableEntry = NULL; + Query *subqueryTree = NULL; + DeferredErrorMessage *repartitionQueryError = NULL; + List *subqueryEntryList = NULL; + + /* + * If not eligible for single relation repartition query, we should raise + * subquery pushdown error. + */ + singleRelationRepartitionSubquery = + SingleRelationRepartitionSubquery(originalQuery); + if (!singleRelationRepartitionSubquery) + { + RaiseDeferredErrorInternal(subqueryPushdownError, ERROR); + } + + subqueryEntryList = SubqueryEntryList(queryTree); + subqueryRangeTableEntry = (RangeTblEntry *) linitial(subqueryEntryList); + Assert(subqueryRangeTableEntry->rtekind == RTE_SUBQUERY); + + subqueryTree = subqueryRangeTableEntry->subquery; + + repartitionQueryError = DeferErrorIfUnsupportedSubqueryRepartition(subqueryTree); + if (repartitionQueryError) + { + RaiseDeferredErrorInternal(repartitionQueryError, ERROR); + } + + /* all checks has passed, safe to create the multi plan */ + multiQueryNode = MultiPlanTree(queryTree); + } + + Assert(multiQueryNode != NULL); + + return multiQueryNode; +} + + +/* + * SingleRelationRepartitionSubquery returns true if it is eligible single + * repartition query planning in the sense that: + * - None of the levels of the subquery contains a join + * - 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 + * + * Note that the caller should still call DeferErrorIfUnsupportedSubqueryRepartition() + * to ensure that Citus supports the subquery. Also, this function is designed to run + * on the original query. + */ +static bool +SingleRelationRepartitionSubquery(Query *queryTree) +{ + List *rangeTableIndexList = NULL; + RangeTblEntry *rangeTableEntry = NULL; + List *rangeTableList = queryTree->rtable; + int rangeTableIndex = 0; + + /* we don't support subqueries in WHERE */ + if (queryTree->hasSubLinks) + { + return false; + } + + /* + * Don't allow joins and set operations. If join appears in the queryTree, the + * length would be greater than 1. If only set operations exists, the length + * would be 0. + */ + ExtractRangeTableIndexWalker((Node *) queryTree->jointree, + &rangeTableIndexList); + if (list_length(rangeTableIndexList) != 1) + { + return false; + } + + rangeTableIndex = linitial_int(rangeTableIndexList); + rangeTableEntry = rt_fetch(rangeTableIndex, rangeTableList); + if (rangeTableEntry->rtekind == RTE_RELATION) + { + return true; + } + else if (rangeTableEntry->rtekind == RTE_SUBQUERY) + { + Query *subqueryTree = rangeTableEntry->subquery; + + return SingleRelationRepartitionSubquery(subqueryTree); + } + + return false; +} + + +/* + * DeferErrorIfContainsUnsupportedSubqueryPushdown iterates on the query's subquery + * entry list and uses helper functions to check if we can push down subquery + * to worker nodes. These helper functions returns a deferred error if we + * cannot push down the subquery. + */ +static DeferredErrorMessage * +DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery, + PlannerRestrictionContext * + plannerRestrictionContext) +{ + ListCell *rangeTableEntryCell = NULL; + List *subqueryEntryList = NIL; + bool outerMostQueryHasLimit = false; + DeferredErrorMessage *error = NULL; + RelationRestrictionContext *relationRestrictionContext = + plannerRestrictionContext->relationRestrictionContext; + + if (originalQuery->limitCount != NULL) + { + outerMostQueryHasLimit = true; + } + + /* + * We're checking two things here: + * (i) If the query contains a top level union, ensure that all leaves + * return the partition key at the same position + * (ii) Else, check whether all relations joined on the partition key or not + */ + if (ContainsUnionSubquery(originalQuery)) + { + if (!SafeToPushdownUnionSubquery(relationRestrictionContext)) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot pushdown the subquery since all leaves of " + "the UNION does not include partition key at the " + "same position", + "Each leaf query of the UNION should return " + "partition key at the same position on its " + "target list.", NULL); + } + } + else if (!RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext)) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot pushdown the subquery since all relations are not " + "joined using distribution keys", + "Each relation should be joined with at least " + "one another relation using distribution keys and " + "equality operator.", NULL); + } + + subqueryEntryList = SubqueryEntryList(originalQuery); + foreach(rangeTableEntryCell, subqueryEntryList) + { + RangeTblEntry *rangeTableEntry = lfirst(rangeTableEntryCell); + Query *subquery = rangeTableEntry->subquery; + + error = DeferErrorIfCannotPushdownSubquery(subquery, outerMostQueryHasLimit); + if (error) + { + return error; + } + + error = DeferErrorIfUnsupportedFilters(subquery); + if (error) + { + return error; + } + } + + return NULL; +} + + +/* + * DeferErrorIfUnsupportedFilters checks if all leaf queries in the given query have + * same filter on the partition column. Note that if there are queries without + * any filter on the partition column, they don't break this prerequisite. + */ +static DeferredErrorMessage * +DeferErrorIfUnsupportedFilters(Query *subquery) +{ + List *queryList = NIL; + ListCell *queryCell = NULL; + List *subqueryOpExpressionList = NIL; + List *relationIdList = RelationIdList(subquery); + + /* + * Get relation id of any relation in the subquery and create partiton column + * for this relation. We will use this column to replace columns on operator + * expressions on different tables. Then we compare these operator expressions + * to see if they consist of same operator and constant value. + */ + Oid relationId = linitial_oid(relationIdList); + Var *partitionColumn = PartitionColumn(relationId, 0); + + ExtractQueryWalker((Node *) subquery, &queryList); + foreach(queryCell, queryList) + { + Query *query = (Query *) lfirst(queryCell); + List *opExpressionList = NIL; + List *newOpExpressionList = NIL; + + bool leafQuery = LeafQuery(query); + if (!leafQuery) + { + continue; + } + + opExpressionList = PartitionColumnOpExpressionList(query); + if (opExpressionList == NIL) + { + continue; + } + + newOpExpressionList = ReplaceColumnsInOpExpressionList(opExpressionList, + partitionColumn); + + if (subqueryOpExpressionList == NIL) + { + subqueryOpExpressionList = newOpExpressionList; + } + else + { + bool equalOpExpressionLists = EqualOpExpressionLists(subqueryOpExpressionList, + newOpExpressionList); + if (!equalOpExpressionLists) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot push down this subquery", + "Currently all leaf queries need to " + "have same filters on partition column", NULL); + } + } + } + + return NULL; +} + + +/* + * EqualOpExpressionLists checks if given two operator expression lists are + * equal. + */ +static bool +EqualOpExpressionLists(List *firstOpExpressionList, List *secondOpExpressionList) +{ + bool equalOpExpressionLists = false; + ListCell *firstOpExpressionCell = NULL; + uint32 equalOpExpressionCount = 0; + uint32 firstOpExpressionCount = list_length(firstOpExpressionList); + uint32 secondOpExpressionCount = list_length(secondOpExpressionList); + + if (firstOpExpressionCount != secondOpExpressionCount) + { + return false; + } + + foreach(firstOpExpressionCell, firstOpExpressionList) + { + OpExpr *firstOpExpression = (OpExpr *) lfirst(firstOpExpressionCell); + ListCell *secondOpExpressionCell = NULL; + + foreach(secondOpExpressionCell, secondOpExpressionList) + { + OpExpr *secondOpExpression = (OpExpr *) lfirst(secondOpExpressionCell); + bool equalExpressions = equal(firstOpExpression, secondOpExpression); + + if (equalExpressions) + { + equalOpExpressionCount++; + continue; + } + } + } + + if (equalOpExpressionCount == firstOpExpressionCount) + { + equalOpExpressionLists = true; + } + + return equalOpExpressionLists; +} + + +/* + * DeferErrorIfCannotPushdownSubquery recursively checks if we can push down the given + * subquery to worker nodes. If we cannot push down the subquery, this function + * returns a deferred error. + * + * We can push down a subquery if it follows rules below. We support nested queries + * as long as they follow the same rules, and we recurse to validate each subquery + * for this given query. + * a. If there is an aggregate, it must be grouped on partition column. + * b. If there is a join, it must be between two regular tables or two subqueries. + * We don't support join between a regular table and a subquery. And columns on + * the join condition must be partition columns. + * c. If there is a distinct clause, it must be on the partition column. + * + * This function is very similar to ErrorIfQueryNotSupported() in logical + * planner, but we don't reuse it, because differently for subqueries we support + * a subset of distinct, union and left joins. + * + * Note that this list of checks is not exhaustive, there can be some cases + * which we let subquery to run but returned results would be wrong. Such as if + * a subquery has a group by on another subquery which includes order by with + * limit, we let this query to run, but results could be wrong depending on the + * features of underlying tables. + */ +static DeferredErrorMessage * +DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerMostQueryHasLimit) +{ + bool preconditionsSatisfied = true; + char *errorDetail = NULL; + List *subqueryEntryList = NIL; + ListCell *rangeTableEntryCell = NULL; + DeferredErrorMessage *deferredError = NULL; + + deferredError = DeferErrorIfUnsupportedTableCombination(subqueryTree); + if (deferredError) + { + return deferredError; + } + + if (subqueryTree->hasSubLinks) + { + preconditionsSatisfied = false; + errorDetail = "Subqueries other than from-clause subqueries are unsupported"; + } + + if (subqueryTree->rtable == NIL) + { + preconditionsSatisfied = false; + errorDetail = "Subqueries without relations are unsupported"; + } + + if (subqueryTree->hasWindowFuncs) + { + preconditionsSatisfied = false; + errorDetail = "Window functions are currently unsupported"; + } + + if (subqueryTree->limitOffset) + { + preconditionsSatisfied = false; + errorDetail = "Offset clause is currently unsupported"; + } + + /* limit is not supported when SubqueryPushdown is not set */ + if (subqueryTree->limitCount && !SubqueryPushdown) + { + preconditionsSatisfied = false; + errorDetail = "Limit in subquery is currently unsupported"; + } + + /* + * Limit is partially supported when SubqueryPushdown is set. + * The outermost query must have a limit clause. + */ + if (subqueryTree->limitCount && SubqueryPushdown && !outerMostQueryHasLimit) + { + preconditionsSatisfied = false; + errorDetail = "Limit in subquery without limit in the outermost query is " + "unsupported"; + } + + if (subqueryTree->setOperations) + { + deferredError = DeferErrorIfUnsupportedUnionQuery(subqueryTree, + outerMostQueryHasLimit); + if (deferredError) + { + return deferredError; + } + } + + if (subqueryTree->hasRecursive) + { + preconditionsSatisfied = false; + errorDetail = "Recursive queries are currently unsupported"; + } + + if (subqueryTree->cteList) + { + preconditionsSatisfied = false; + errorDetail = "Common Table Expressions are currently unsupported"; + } + + if (subqueryTree->hasForUpdate) + { + preconditionsSatisfied = false; + errorDetail = "For Update/Share commands are currently unsupported"; + } + + /* group clause list must include partition column */ + if (subqueryTree->groupClause) + { + List *groupClauseList = subqueryTree->groupClause; + List *targetEntryList = subqueryTree->targetList; + List *groupTargetEntryList = GroupTargetEntryList(groupClauseList, + targetEntryList); + bool groupOnPartitionColumn = TargetListOnPartitionColumn(subqueryTree, + groupTargetEntryList); + if (!groupOnPartitionColumn) + { + preconditionsSatisfied = false; + errorDetail = "Group by list without partition column is currently " + "unsupported"; + } + } + + /* we don't support aggregates without group by */ + if (subqueryTree->hasAggs && (subqueryTree->groupClause == NULL)) + { + preconditionsSatisfied = false; + errorDetail = "Aggregates without group by are currently unsupported"; + } + + /* having clause without group by on partition column is not supported */ + if (subqueryTree->havingQual && (subqueryTree->groupClause == NULL)) + { + preconditionsSatisfied = false; + errorDetail = "Having qual without group by on partition column is " + "currently unsupported"; + } + + /* distinct clause list must include partition column */ + if (subqueryTree->distinctClause) + { + List *distinctClauseList = subqueryTree->distinctClause; + List *targetEntryList = subqueryTree->targetList; + List *distinctTargetEntryList = GroupTargetEntryList(distinctClauseList, + targetEntryList); + bool distinctOnPartitionColumn = + TargetListOnPartitionColumn(subqueryTree, distinctTargetEntryList); + if (!distinctOnPartitionColumn) + { + preconditionsSatisfied = false; + errorDetail = "Distinct on columns without partition column is " + "currently unsupported"; + } + } + + /* finally check and return deferred if not satisfied */ + if (!preconditionsSatisfied) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot push down this subquery", + errorDetail, NULL); + } + + /* recursively do same check for subqueries of this query */ + subqueryEntryList = SubqueryEntryList(subqueryTree); + foreach(rangeTableEntryCell, subqueryEntryList) + { + RangeTblEntry *rangeTableEntry = + (RangeTblEntry *) lfirst(rangeTableEntryCell); + + Query *innerSubquery = rangeTableEntry->subquery; + deferredError = DeferErrorIfCannotPushdownSubquery(innerSubquery, + outerMostQueryHasLimit); + if (deferredError) + { + return deferredError; + } + } + + return NULL; +} + + +/* + * DeferErrorIfUnsupportedUnionQuery is a helper function for ErrorIfCannotPushdownSubquery(). + * It basically iterates over the subqueries that reside under the given set operations. + * + * The function also errors out for set operations INTERSECT and EXCEPT. + */ +static DeferredErrorMessage * +DeferErrorIfUnsupportedUnionQuery(Query *subqueryTree, + bool outerMostQueryHasLimit) +{ + List *rangeTableIndexList = NIL; + ListCell *rangeTableIndexCell = NULL; + List *setOperationStatementList = NIL; + ListCell *setOperationStatmentCell = NULL; + List *rangeTableList = subqueryTree->rtable; + + ExtractSetOperationStatmentWalker((Node *) subqueryTree->setOperations, + &setOperationStatementList); + foreach(setOperationStatmentCell, setOperationStatementList) + { + SetOperationStmt *setOperation = + (SetOperationStmt *) lfirst(setOperationStatmentCell); + + if (setOperation->op != SETOP_UNION) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot push down this subquery", + "Intersect and Except are currently unsupported", NULL); + } + } + + ExtractRangeTableIndexWalker((Node *) subqueryTree->setOperations, + &rangeTableIndexList); + foreach(rangeTableIndexCell, rangeTableIndexList) + { + int rangeTableIndex = lfirst_int(rangeTableIndexCell); + RangeTblEntry *rangeTableEntry = rt_fetch(rangeTableIndex, rangeTableList); + DeferredErrorMessage *deferredError = NULL; + + Assert(rangeTableEntry->rtekind == RTE_SUBQUERY); + + deferredError = DeferErrorIfCannotPushdownSubquery(rangeTableEntry->subquery, + outerMostQueryHasLimit); + if (deferredError) + { + return deferredError; + } + } + + return NULL; +} + + +/* + * ExtractSetOperationStatementWalker walks over a set operations statment, + * and finds all set operations in the tree. + */ +static bool +ExtractSetOperationStatmentWalker(Node *node, List **setOperationList) +{ + bool walkerResult = false; + if (node == NULL) + { + return false; + } + + if (IsA(node, SetOperationStmt)) + { + SetOperationStmt *setOperation = (SetOperationStmt *) node; + + (*setOperationList) = lappend(*setOperationList, setOperation); + } + + walkerResult = expression_tree_walker(node, ExtractSetOperationStatmentWalker, + setOperationList); + + return walkerResult; +} + + +/* + * DeferErrorIfUnsupportedTableCombination checks if the given query tree contains any + * unsupported range table combinations. For this, the function walks over all + * range tables in the join tree, and checks if they correspond to simple relations + * or subqueries. It also checks if there is a join between a regular table and + * a subquery and if join is on more than two range table entries. If any error is found, + * a deferred error is returned. Else, NULL is returned. + */ +static DeferredErrorMessage * +DeferErrorIfUnsupportedTableCombination(Query *queryTree) +{ + List *rangeTableList = queryTree->rtable; + List *joinTreeTableIndexList = NIL; + ListCell *joinTreeTableIndexCell = NULL; + bool unsupporteTableCombination = false; + char *errorDetail = NULL; + uint32 relationRangeTableCount = 0; + uint32 subqueryRangeTableCount = 0; + + /* + * Extract all range table indexes from the join tree. Note that sub-queries + * that get pulled up by PostgreSQL don't appear in this join tree. + */ + ExtractRangeTableIndexWalker((Node *) queryTree->jointree, &joinTreeTableIndexList); + foreach(joinTreeTableIndexCell, joinTreeTableIndexList) + { + /* + * Join tree's range table index starts from 1 in the query tree. But, + * list indexes start from 0. + */ + int joinTreeTableIndex = lfirst_int(joinTreeTableIndexCell); + int rangeTableListIndex = joinTreeTableIndex - 1; + + RangeTblEntry *rangeTableEntry = + (RangeTblEntry *) list_nth(rangeTableList, rangeTableListIndex); + + /* + * Check if the range table in the join tree is a simple relation or a + * subquery. + */ + if (rangeTableEntry->rtekind == RTE_RELATION) + { + relationRangeTableCount++; + } + else if (rangeTableEntry->rtekind == RTE_SUBQUERY) + { + subqueryRangeTableCount++; + } + else + { + unsupporteTableCombination = true; + errorDetail = "Table expressions other than simple relations and " + "subqueries are currently unsupported"; + break; + } + } + + /* finally check and error out if not satisfied */ + if (unsupporteTableCombination) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot push down this subquery", + errorDetail, NULL); + } + + return NULL; +} + + +/* + * TargetListOnPartitionColumn checks if at least one target list entry is on + * partition column. + */ +static bool +TargetListOnPartitionColumn(Query *query, List *targetEntryList) +{ + bool targetListOnPartitionColumn = false; + List *compositeFieldList = NIL; + + ListCell *targetEntryCell = NULL; + foreach(targetEntryCell, targetEntryList) + { + TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); + Expr *targetExpression = targetEntry->expr; + + bool isPartitionColumn = IsPartitionColumn(targetExpression, query); + if (isPartitionColumn) + { + FieldSelect *compositeField = CompositeFieldRecursive(targetExpression, + query); + if (compositeField) + { + compositeFieldList = lappend(compositeFieldList, compositeField); + } + else + { + targetListOnPartitionColumn = true; + break; + } + } + } + + /* check composite fields */ + if (!targetListOnPartitionColumn) + { + bool fullCompositeFieldList = FullCompositeFieldList(compositeFieldList); + if (fullCompositeFieldList) + { + targetListOnPartitionColumn = true; + } + } + + return targetListOnPartitionColumn; +} + + +/* + * FullCompositeFieldList gets a composite field list, and checks if all fields + * of composite type are used in the list. + */ +static bool +FullCompositeFieldList(List *compositeFieldList) +{ + bool fullCompositeFieldList = true; + bool *compositeFieldArray = NULL; + uint32 compositeFieldCount = 0; + uint32 fieldIndex = 0; + + ListCell *fieldSelectCell = NULL; + foreach(fieldSelectCell, compositeFieldList) + { + FieldSelect *fieldSelect = (FieldSelect *) lfirst(fieldSelectCell); + uint32 compositeFieldIndex = 0; + + Expr *fieldExpression = fieldSelect->arg; + if (!IsA(fieldExpression, Var)) + { + continue; + } + + if (compositeFieldArray == NULL) + { + uint32 index = 0; + Var *compositeColumn = (Var *) fieldExpression; + Oid compositeTypeId = compositeColumn->vartype; + Oid compositeRelationId = get_typ_typrelid(compositeTypeId); + + /* get composite type attribute count */ + Relation relation = relation_open(compositeRelationId, AccessShareLock); + compositeFieldCount = relation->rd_att->natts; + compositeFieldArray = palloc0(compositeFieldCount * sizeof(bool)); + relation_close(relation, AccessShareLock); + + for (index = 0; index < compositeFieldCount; index++) + { + compositeFieldArray[index] = false; + } + } + + compositeFieldIndex = fieldSelect->fieldnum - 1; + compositeFieldArray[compositeFieldIndex] = true; + } + + for (fieldIndex = 0; fieldIndex < compositeFieldCount; fieldIndex++) + { + if (!compositeFieldArray[fieldIndex]) + { + fullCompositeFieldList = false; + } + } + + if (compositeFieldCount == 0) + { + fullCompositeFieldList = false; + } + + return fullCompositeFieldList; +} + + +/* + * CompositeFieldRecursive recursively finds composite field in the query tree + * referred by given expression. If expression does not refer to a composite + * field, then it returns NULL. + * + * If expression is a field select we directly return composite field. If it is + * a column is referenced from a subquery, then we recursively check that subquery + * until we reach the source of that column, and find composite field. If this + * column is referenced from join range table entry, then we resolve which join + * column it refers and recursively use this column with the same query. + */ +static FieldSelect * +CompositeFieldRecursive(Expr *expression, Query *query) +{ + FieldSelect *compositeField = NULL; + List *rangetableList = query->rtable; + Index rangeTableEntryIndex = 0; + RangeTblEntry *rangeTableEntry = NULL; + Var *candidateColumn = NULL; + + if (IsA(expression, FieldSelect)) + { + compositeField = (FieldSelect *) expression; + return compositeField; + } + + if (IsA(expression, Var)) + { + candidateColumn = (Var *) expression; + } + else + { + return NULL; + } + + rangeTableEntryIndex = candidateColumn->varno - 1; + rangeTableEntry = list_nth(rangetableList, rangeTableEntryIndex); + + if (rangeTableEntry->rtekind == RTE_SUBQUERY) + { + Query *subquery = rangeTableEntry->subquery; + List *targetEntryList = subquery->targetList; + AttrNumber targetEntryIndex = candidateColumn->varattno - 1; + TargetEntry *subqueryTargetEntry = list_nth(targetEntryList, targetEntryIndex); + + Expr *subqueryExpression = subqueryTargetEntry->expr; + compositeField = CompositeFieldRecursive(subqueryExpression, subquery); + } + else if (rangeTableEntry->rtekind == RTE_JOIN) + { + List *joinColumnList = rangeTableEntry->joinaliasvars; + AttrNumber joinColumnIndex = candidateColumn->varattno - 1; + Expr *joinColumn = list_nth(joinColumnList, joinColumnIndex); + + compositeField = CompositeFieldRecursive(joinColumn, query); + } + + return compositeField; +} + + /* * SubqueryEntryList finds the subquery nodes in the range table entry list, and * builds a list of subquery range table entries from these subquery nodes. Range @@ -248,14 +1119,14 @@ MultiPlanTree(Query *queryTree) List *columnList = NIL; ListCell *columnCell = NULL; + /* we only support single subquery in the entry list */ + Assert(list_length(subqueryEntryList) == 1); + subqueryRangeTableEntry = (RangeTblEntry *) linitial(subqueryEntryList); subqueryTree = subqueryRangeTableEntry->subquery; - /* check if subquery satisfies preconditons */ - ErrorIfSubqueryNotSupported(subqueryTree); - - /* check if subquery has joining tables */ - ErrorIfSubqueryJoin(subqueryTree); + /* ensure if subquery satisfies preconditions */ + Assert(DeferErrorIfUnsupportedSubqueryRepartition(subqueryTree) == NULL); subqueryNode = CitusMakeNode(MultiTable); subqueryNode->relationId = SUBQUERY_RELATION_ID; @@ -591,14 +1462,19 @@ ErrorHintRequired(const char *errorHint, Query *queryTree) /* - * ErrorIfSubqueryNotSupported checks that we can perform distributed planning for - * the given subquery. + * DeferErrorIfSubqueryNotSupported checks that we can perform distributed planning for + * the given subquery. If not, a deferred error is returned. The function recursively + * does this check to all lower levels of the subquery. */ -static void -ErrorIfSubqueryNotSupported(Query *subqueryTree) +static DeferredErrorMessage * +DeferErrorIfUnsupportedSubqueryRepartition(Query *subqueryTree) { char *errorDetail = NULL; bool preconditionsSatisfied = true; + List *joinTreeTableIndexList = NIL; + int rangeTableIndex = 0; + RangeTblEntry *rangeTableEntry = NULL; + Query *innerSubquery = NULL; if (!subqueryTree->hasAggs) { @@ -630,13 +1506,35 @@ ErrorIfSubqueryNotSupported(Query *subqueryTree) errorDetail = "Subqueries with offset are not supported yet"; } - /* finally check and error out if not satisfied */ + /* finally check and return error if conditions are not satisfied */ if (!preconditionsSatisfied) { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot perform distributed planning on this query"), - errdetail("%s", errorDetail))); + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot perform distributed planning on this query", + errorDetail, NULL); } + + /* + * Extract all range table indexes from the join tree. Note that sub-queries + * that get pulled up by PostgreSQL don't appear in this join tree. + */ + ExtractRangeTableIndexWalker((Node *) subqueryTree->jointree, + &joinTreeTableIndexList); + Assert(list_length(joinTreeTableIndexList) == 1); + + /* continue with the inner subquery */ + rangeTableIndex = linitial_int(joinTreeTableIndexList); + rangeTableEntry = rt_fetch(rangeTableIndex, subqueryTree->rtable); + if (rangeTableEntry->rtekind == RTE_RELATION) + { + return NULL; + } + + Assert(rangeTableEntry->rtekind == RTE_SUBQUERY); + innerSubquery = rangeTableEntry->subquery; + + /* recursively continue to the inner subqueries */ + return DeferErrorIfUnsupportedSubqueryRepartition(innerSubquery); } @@ -1980,66 +2878,109 @@ ApplyCartesianProduct(MultiNode *leftNode, MultiNode *rightNode, * SubqueryPushdownMultiTree creates logical plan for subquery pushdown logic. * Note that this logic will be changed in next iterations, so we decoupled it * from other parts of code although it causes some code duplication. + * + * Current subquery pushdown support in MultiTree logic requires a single range + * table entry in the top most from clause. Therefore we inject an synthetic + * query derived from the top level query and make it the only range table + * entry for the top level query. This way we can push down any subquery joins + * down to workers without invoking join order planner. */ static MultiNode * -SubqueryPushdownMultiPlanTree(Query *queryTree, List *subqueryEntryList) +SubqueryPushdownMultiPlanTree(Query *queryTree) { List *targetEntryList = queryTree->targetList; List *qualifierList = NIL; - List *qualifierColumnList = NIL; - List *targetListColumnList = NIL; List *columnList = NIL; - ListCell *columnCell = NULL; + List *targetColumnList = NIL; MultiCollect *subqueryCollectNode = CitusMakeNode(MultiCollect); MultiTable *subqueryNode = NULL; - MultiSelect *selectNode = NULL; MultiProject *projectNode = NULL; MultiExtendedOp *extendedOpNode = NULL; MultiNode *currentTopNode = NULL; - RangeTblEntry *subqueryRangeTableEntry = NULL; + Query *pushedDownQuery = NULL; + List *subqueryTargetEntryList = NIL; + List *havingClauseColumnList = NIL; /* verify we can perform distributed planning on this query */ ErrorIfQueryNotSupported(queryTree); - ErrorIfSubqueryJoin(queryTree); /* extract qualifiers and verify we can plan for them */ qualifierList = QualifierList(queryTree->jointree); ValidateClauseList(qualifierList); /* - * We disregard pulled subqueries. This changes order of range table list. - * We do not allow subquery joins, so we will have only one range table - * entry in range table list after dropping pulled subquery. For this reason, - * here we are updating columns in the most outer query for where clause - * list and target list accordingly. + * We would be creating a new Query and pushing down top level query's + * contents down to it. Join and filter clauses in higher level query would + * be transferred to lower query. Therefore after this function we would + * only have a single range table entry in the top level query. We need to + * create a target list entry in lower query for each column reference in + * upper level query's target list and having clauses. Any column reference + * in the upper query will be updated to have varno=1, and varattno= + * of matching target entry in pushed down query. + * Consider query + * SELECT s1.a, sum(s2.c) + * FROM (some subquery) s1, (some subquery) s2 + * WHERE s1.a = s2.a + * GROUP BY s1.a + * HAVING avg(s2.b); + * + * We want to prepare a multi tree to avoid subquery joins at top level, + * therefore above query is converted to an equivalent + * SELECT worker_column_0, sum(worker_column_1) + * FROM ( + * SELECT + * s1.a AS worker_column_0, + * s2.c AS worker_column_1, + * s2.b AS as worker_column_2 + * FROM (some subquery) s1, (some subquery) s2 + * WHERE s1.a = s2.a) worker_subquery + * GROUP BY worker_column_0 + * HAVING avg(worker_column_2); + * After this conversion MultiTree is created as follows + * + * MultiExtendedOpNode( + * targetList : worker_column_0, sum(worker_column_1) + * groupBy : worker_column_0 + * having : avg(worker_column_2)) + * --->MultiProject (worker_column_0, worker_column_1, worker_column_2) + * --->---> MultiTable (subquery : worker_subquery) + * + * Master and worker queries will be created out of this MultiTree at later stages. */ - Assert(list_length(subqueryEntryList) == 1); - qualifierColumnList = pull_var_clause_default((Node *) qualifierList); - targetListColumnList = pull_var_clause_default((Node *) targetEntryList); + /* + * uniqueColumnList contains all columns returned by subquery. Subquery target + * entry list, subquery range table entry's column name list are derived from + * uniqueColumnList. Columns mentioned in multiProject node and multiExtendedOp + * node are indexed with their respective position in uniqueColumnList. + */ + targetColumnList = pull_var_clause_default((Node *) targetEntryList); + havingClauseColumnList = pull_var_clause_default(queryTree->havingQual); + columnList = list_concat(targetColumnList, havingClauseColumnList); - columnList = list_concat(qualifierColumnList, targetListColumnList); - foreach(columnCell, columnList) - { - Var *column = (Var *) lfirst(columnCell); - column->varno = 1; - } + /* create a target entry for each unique column */ + subqueryTargetEntryList = CreateSubqueryTargetEntryList(columnList); - /* create multi node for the subquery */ - subqueryRangeTableEntry = (RangeTblEntry *) linitial(subqueryEntryList); - subqueryNode = MultiSubqueryPushdownTable(subqueryRangeTableEntry); + /* + * Update varno/varattno fields of columns in columnList to + * point to corresponding target entry in subquery target entry list. + */ + UpdateVarMappingsForExtendedOpNode(columnList, subqueryTargetEntryList); + + /* new query only has target entries, join tree, and rtable*/ + pushedDownQuery = makeNode(Query); + pushedDownQuery->commandType = queryTree->commandType; + pushedDownQuery->targetList = subqueryTargetEntryList; + pushedDownQuery->jointree = copyObject(queryTree->jointree); + pushedDownQuery->rtable = copyObject(queryTree->rtable); + pushedDownQuery->setOperations = copyObject(queryTree->setOperations); + pushedDownQuery->querySource = queryTree->querySource; + + subqueryNode = MultiSubqueryPushdownTable(pushedDownQuery); SetChild((MultiUnaryNode *) subqueryCollectNode, (MultiNode *) subqueryNode); currentTopNode = (MultiNode *) subqueryCollectNode; - /* build select node if the query has selection criteria */ - selectNode = MultiSelectNode(qualifierList); - if (selectNode != NULL) - { - SetChild((MultiUnaryNode *) selectNode, currentTopNode); - currentTopNode = (MultiNode *) selectNode; - } - /* build project node for the columns to project */ projectNode = MultiProjectNode(targetEntryList); SetChild((MultiUnaryNode *) projectNode, currentTopNode); @@ -2052,6 +2993,31 @@ SubqueryPushdownMultiPlanTree(Query *queryTree, List *subqueryEntryList) * in the logical optimizer. */ extendedOpNode = MultiExtendedOpNode(queryTree); + + /* + * Postgres standard planner converts having qual node to a list of and + * clauses and expects havingQual to be of type List when executing the + * query later. This function is called on an original query, therefore + * havingQual has not been converted yet. Perform conversion here. + */ + if (extendedOpNode->havingQual != NULL && + !IsA(extendedOpNode->havingQual, List)) + { + extendedOpNode->havingQual = + (Node *) make_ands_implicit((Expr *) extendedOpNode->havingQual); + } + + /* + * Postgres standard planner evaluates expressions in the LIMIT/OFFSET clauses. + * Since we're using original query here, we should manually evaluate the + * expression on the LIMIT and OFFSET clauses. Note that logical optimizer + * expects those clauses to be already evaluated. + */ + extendedOpNode->limitCount = + PartiallyEvaluateExpression(extendedOpNode->limitCount, NULL); + extendedOpNode->limitOffset = + PartiallyEvaluateExpression(extendedOpNode->limitOffset, NULL); + SetChild((MultiUnaryNode *) extendedOpNode, currentTopNode); currentTopNode = (MultiNode *) extendedOpNode; @@ -2060,48 +3026,105 @@ SubqueryPushdownMultiPlanTree(Query *queryTree, List *subqueryEntryList) /* - * ErrorIfSubqueryJoin errors out if the given query is a join query. Note that - * this function will not be required once we implement subquery joins. + * CreateSubqueryTargetEntryList creates a target entry for each unique column + * in the column list and returns the target entry list. + */ +static List * +CreateSubqueryTargetEntryList(List *columnList) +{ + AttrNumber resNo = 1; + ListCell *columnCell = NULL; + List *uniqueColumnList = NIL; + List *subqueryTargetEntryList = NIL; + + foreach(columnCell, columnList) + { + Var *column = (Var *) lfirst(columnCell); + uniqueColumnList = list_append_unique(uniqueColumnList, copyObject(column)); + } + + foreach(columnCell, uniqueColumnList) + { + Var *column = (Var *) lfirst(columnCell); + TargetEntry *newTargetEntry = makeNode(TargetEntry); + StringInfo columnNameString = makeStringInfo(); + + newTargetEntry->expr = (Expr *) copyObject(column); + appendStringInfo(columnNameString, WORKER_COLUMN_FORMAT, resNo); + newTargetEntry->resname = columnNameString->data; + newTargetEntry->resjunk = false; + newTargetEntry->resno = resNo; + + subqueryTargetEntryList = lappend(subqueryTargetEntryList, newTargetEntry); + resNo++; + } + + return subqueryTargetEntryList; +} + + +/* + * UpdateVarMappingsForExtendedOpNode updates varno/varattno fields of columns + * in columnList to point to corresponding target in subquery target entry + * list. */ static void -ErrorIfSubqueryJoin(Query *queryTree) +UpdateVarMappingsForExtendedOpNode(List *columnList, List *subqueryTargetEntryList) { - List *joinTreeTableIndexList = NIL; - uint32 joiningRangeTableCount = 0; - - /* - * Extract all range table indexes from the join tree. Note that sub-queries - * that get pulled up by PostgreSQL don't appear in this join tree. - */ - ExtractRangeTableIndexWalker((Node *) queryTree->jointree, &joinTreeTableIndexList); - joiningRangeTableCount = list_length(joinTreeTableIndexList); - - if (joiningRangeTableCount > 1) + ListCell *columnCell = NULL; + foreach(columnCell, columnList) { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot perform distributed planning on this query"), - errdetail("Join in subqueries is not supported yet"))); + Var *columnOnTheExtendedNode = (Var *) lfirst(columnCell); + ListCell *targetEntryCell = NULL; + foreach(targetEntryCell, subqueryTargetEntryList) + { + TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); + Var *targetColumn = NULL; + + Assert(IsA(targetEntry->expr, Var)); + targetColumn = (Var *) targetEntry->expr; + if (columnOnTheExtendedNode->varno == targetColumn->varno && + columnOnTheExtendedNode->varattno == targetColumn->varattno) + { + columnOnTheExtendedNode->varno = 1; + columnOnTheExtendedNode->varattno = targetEntry->resno; + break; + } + } } } /* - * MultiSubqueryPushdownTable creates a MultiTable from the given subquery range - * table entry and returns it. Note that this sets subquery field of MultiTable - * to subquery of the given range table entry. + * MultiSubqueryPushdownTable creates a MultiTable from the given subquery, + * populates column list and returns the multitable. */ static MultiTable * -MultiSubqueryPushdownTable(RangeTblEntry *subqueryRangeTableEntry) +MultiSubqueryPushdownTable(Query *subquery) { - Query *subquery = subqueryRangeTableEntry->subquery; + MultiTable *subqueryTableNode = NULL; + StringInfo rteName = makeStringInfo(); + List *columnNamesList = NIL; + ListCell *targetEntryCell = NULL; - MultiTable *subqueryTableNode = CitusMakeNode(MultiTable); + appendStringInfo(rteName, "worker_subquery"); + + foreach(targetEntryCell, subquery->targetList) + { + TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); + columnNamesList = lappend(columnNamesList, makeString(targetEntry->resname)); + } + + subqueryTableNode = CitusMakeNode(MultiTable); subqueryTableNode->subquery = subquery; - subqueryTableNode->relationId = HEAP_ANALYTICS_SUBQUERY_RELATION_ID; + subqueryTableNode->relationId = SUBQUERY_PUSHDOWN_RELATION_ID; subqueryTableNode->rangeTableId = SUBQUERY_RANGE_TABLE_ID; subqueryTableNode->partitionColumn = NULL; - subqueryTableNode->alias = subqueryRangeTableEntry->alias; - subqueryTableNode->referenceNames = subqueryRangeTableEntry->eref; + subqueryTableNode->alias = makeNode(Alias); + subqueryTableNode->alias->aliasname = rteName->data; + subqueryTableNode->referenceNames = makeNode(Alias); + subqueryTableNode->referenceNames->aliasname = rteName->data; + subqueryTableNode->referenceNames->colnames = columnNamesList; return subqueryTableNode; } diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 9d2bb9f5b..f4c87aaec 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -121,8 +121,18 @@ static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray, /* Local functions forward declarations for task list creation and helper functions */ static bool MultiPlanRouterExecutable(MultiPlan *multiPlan); -static Job * BuildJobTreeTaskList(Job *jobTree); -static List * SubquerySqlTaskList(Job *job); +static Job * BuildJobTreeTaskList(Job *jobTree, + PlannerRestrictionContext *plannerRestrictionContext); +static List * SubquerySqlTaskList(Job *job, + PlannerRestrictionContext *plannerRestrictionContext); +static void ErrorIfUnsupportedShardDistribution(Query *query); +static bool CoPartitionedTables(Oid firstRelationId, Oid secondRelationId); +static bool ShardIntervalsEqual(FmgrInfo *comparisonFunction, + ShardInterval *firstInterval, + ShardInterval *secondInterval); +static Task * SubqueryTaskCreate(Query *originalQuery, ShardInterval *shardInterval, + RelationRestrictionContext *restrictionContext, + uint32 taskId); static List * SqlTaskList(Job *job); static bool DependsOnHashPartitionJob(Job *job); static uint32 AnchorRangeTableId(List *rangeTableList); @@ -145,7 +155,6 @@ static bool JoinPrunable(RangeTableFragment *leftFragment, RangeTableFragment *rightFragment); static ShardInterval * FragmentInterval(RangeTableFragment *fragment); static StringInfo FragmentIntervalString(ShardInterval *fragmentInterval); -static List * UniqueFragmentList(List *fragmentList); static List * DataFetchTaskList(uint64 jobId, uint32 taskIdIndex, List *fragmentList); static StringInfo NodeNameArrayString(List *workerNodeList); static StringInfo NodePortArrayString(List *workerNodeList); @@ -193,7 +202,8 @@ static uint32 FinalTargetEntryCount(List *targetEntryList); * executed on worker nodes, and the final query to run on the master node. */ MultiPlan * -MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) +MultiPhysicalPlanCreate(MultiTreeRoot *multiTree, + PlannerRestrictionContext *plannerRestrictionContext) { MultiPlan *multiPlan = NULL; Job *workerJob = NULL; @@ -204,7 +214,7 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) workerJob = BuildJobTree(multiTree); /* create the tree of executable tasks for the worker job */ - workerJob = BuildJobTreeTaskList(workerJob); + workerJob = BuildJobTreeTaskList(workerJob, plannerRestrictionContext); /* build the final merge query to execute on the master */ masterDependedJobList = list_make1(workerJob); @@ -810,7 +820,7 @@ BaseRangeTableList(MultiNode *multiNode) */ MultiTable *multiTable = (MultiTable *) multiNode; if (multiTable->relationId != SUBQUERY_RELATION_ID && - multiTable->relationId != HEAP_ANALYTICS_SUBQUERY_RELATION_ID) + multiTable->relationId != SUBQUERY_PUSHDOWN_RELATION_ID) { RangeTblEntry *rangeTableEntry = makeNode(RangeTblEntry); rangeTableEntry->inFromCl = true; @@ -1390,6 +1400,7 @@ BuildSubqueryJobQuery(MultiNode *multiNode) List *sortClauseList = NIL; List *groupClauseList = NIL; List *whereClauseList = NIL; + Node *havingQual = NULL; Node *limitCount = NULL; Node *limitOffset = NULL; FromExpr *joinTree = NULL; @@ -1429,7 +1440,7 @@ BuildSubqueryJobQuery(MultiNode *multiNode) targetList = QueryTargetList(multiNode); } - /* extract limit count/offset and sort clauses */ + /* extract limit count/offset, sort and having clauses */ if (extendedOpNodeList != NIL) { MultiExtendedOp *extendedOp = (MultiExtendedOp *) linitial(extendedOpNodeList); @@ -1437,6 +1448,7 @@ BuildSubqueryJobQuery(MultiNode *multiNode) limitCount = extendedOp->limitCount; limitOffset = extendedOp->limitOffset; sortClauseList = extendedOp->sortClauseList; + havingQual = extendedOp->havingQual; } /* build group clauses */ @@ -1466,7 +1478,9 @@ BuildSubqueryJobQuery(MultiNode *multiNode) jobQuery->groupClause = groupClauseList; jobQuery->limitOffset = limitOffset; jobQuery->limitCount = limitCount; - jobQuery->hasAggs = contain_agg_clause((Node *) targetList); + jobQuery->havingQual = havingQual; + jobQuery->hasAggs = contain_agg_clause((Node *) targetList) || + contain_agg_clause((Node *) havingQual); return jobQuery; } @@ -1909,7 +1923,7 @@ SplitPointObject(ShardInterval **shardIntervalArray, uint32 shardIntervalCount) * tasks to worker nodes. */ static Job * -BuildJobTreeTaskList(Job *jobTree) +BuildJobTreeTaskList(Job *jobTree, PlannerRestrictionContext *plannerRestrictionContext) { List *flattenedJobList = NIL; uint32 flattenedJobCount = 0; @@ -1947,7 +1961,7 @@ BuildJobTreeTaskList(Job *jobTree) /* create sql tasks for the job, and prune redundant data fetch tasks */ if (job->subqueryPushdown) { - sqlTaskList = SubquerySqlTaskList(job); + sqlTaskList = SubquerySqlTaskList(job, plannerRestrictionContext); } else { @@ -1999,131 +2013,329 @@ BuildJobTreeTaskList(Job *jobTree) /* * SubquerySqlTaskList creates a list of SQL tasks to execute the given subquery - * pushdown job. For this, it gets all range tables in the subquery tree, then - * walks over each range table in the list, gets shards for each range table, - * and prunes unneeded shards. Then for remaining shards, fragments are created - * and merged to create fragment combinations. For each created combination, the - * function builds a SQL task, and appends this task to a task list. + * pushdown job. For this, the it is being checked whether the query is router + * plannable per target shard interval. For those router plannable worker + * queries, we create a SQL task and append the task to the task list that is going + * to be executed. */ static List * -SubquerySqlTaskList(Job *job) +SubquerySqlTaskList(Job *job, PlannerRestrictionContext *plannerRestrictionContext) { Query *subquery = job->jobQuery; uint64 jobId = job->jobId; List *sqlTaskList = NIL; - List *fragmentCombinationList = NIL; - List *opExpressionList = NIL; - List *queryList = NIL; List *rangeTableList = NIL; - ListCell *fragmentCombinationCell = NULL; ListCell *rangeTableCell = NULL; - ListCell *queryCell = NULL; - Node *whereClauseTree = NULL; uint32 taskIdIndex = 1; /* 0 is reserved for invalid taskId */ - uint32 anchorRangeTableId = 0; - uint32 rangeTableIndex = 0; - const uint32 fragmentSize = sizeof(RangeTableFragment); - uint64 largestTableSize = 0; + Oid relationId = 0; + int shardCount = 0; + int shardOffset = 0; + DistTableCacheEntry *targetCacheEntry = NULL; + RelationRestrictionContext *relationRestrictionContext = + plannerRestrictionContext->relationRestrictionContext; - /* find filters on partition columns */ - ExtractQueryWalker((Node *) subquery, &queryList); - foreach(queryCell, queryList) - { - Query *query = (Query *) lfirst(queryCell); - bool leafQuery = LeafQuery(query); - - if (!leafQuery) - { - continue; - } - - /* we have some filters on partition column */ - opExpressionList = PartitionColumnOpExpressionList(query); - if (opExpressionList != NIL) - { - break; - } - } + /* error if shards are not co-partitioned */ + ErrorIfUnsupportedShardDistribution(subquery); /* get list of all range tables in subquery tree */ ExtractRangeTableRelationWalker((Node *) subquery, &rangeTableList); /* - * For each range table entry, first we prune shards for the relation - * referenced in the range table. Then we sort remaining shards and create - * fragments in this order and add these fragments to fragment combination - * list. + * Find the first relation that is not a reference table. We'll use the shards + * of that relation as the target shards. */ foreach(rangeTableCell, rangeTableList) { RangeTblEntry *rangeTableEntry = (RangeTblEntry *) lfirst(rangeTableCell); - Oid relationId = rangeTableEntry->relid; - List *finalShardIntervalList = NIL; - ListCell *fragmentCombinationCell = NULL; - ListCell *shardIntervalCell = NULL; - uint32 tableId = rangeTableIndex + 1; /* tableId starts from 1 */ - uint32 finalShardCount = 0; - uint64 tableSize = 0; + DistTableCacheEntry *cacheEntry = NULL; - if (opExpressionList != NIL) + relationId = rangeTableEntry->relid; + cacheEntry = DistributedTableCacheEntry(relationId); + if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE) { - Var *partitionColumn = PartitionColumn(relationId, tableId); - List *whereClauseList = ReplaceColumnsInOpExpressionList(opExpressionList, - partitionColumn); - finalShardIntervalList = PruneShards(relationId, tableId, whereClauseList); + continue; + } + + targetCacheEntry = DistributedTableCacheEntry(relationId); + break; + } + + Assert(targetCacheEntry != NULL); + + shardCount = targetCacheEntry->shardIntervalArrayLength; + for (shardOffset = 0; shardOffset < shardCount; shardOffset++) + { + ShardInterval *targetShardInterval = + targetCacheEntry->sortedShardIntervalArray[shardOffset]; + Task *subqueryTask = NULL; + + subqueryTask = SubqueryTaskCreate(subquery, targetShardInterval, + relationRestrictionContext, taskIdIndex); + + + /* add the task if it could be created */ + if (subqueryTask != NULL) + { + subqueryTask->jobId = jobId; + sqlTaskList = lappend(sqlTaskList, subqueryTask); + + ++taskIdIndex; + } + } + + return sqlTaskList; +} + + +/* + * ErrorIfUnsupportedShardDistribution gets list of relations in the given query + * and checks if two conditions below hold for them, otherwise it errors out. + * a. Every relation is distributed by range or hash. This means shards are + * disjoint based on the partition column. + * b. All relations have 1-to-1 shard partitioning between them. This means + * shard count for every relation is same and for every shard in a relation + * there is exactly one shard in other relations with same min/max values. + */ +static void +ErrorIfUnsupportedShardDistribution(Query *query) +{ + Oid firstTableRelationId = InvalidOid; + List *relationIdList = RelationIdList(query); + ListCell *relationIdCell = NULL; + uint32 relationIndex = 0; + uint32 rangeDistributedRelationCount = 0; + uint32 hashDistributedRelationCount = 0; + + foreach(relationIdCell, relationIdList) + { + Oid relationId = lfirst_oid(relationIdCell); + char partitionMethod = PartitionMethod(relationId); + if (partitionMethod == DISTRIBUTE_BY_RANGE) + { + rangeDistributedRelationCount++; + } + else if (partitionMethod == DISTRIBUTE_BY_HASH) + { + hashDistributedRelationCount++; } else { - finalShardIntervalList = LoadShardIntervalList(relationId); + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot push down this subquery"), + errdetail("Currently range and hash partitioned " + "relations are supported"))); } + } - /* if all shards are pruned away, we return an empty task list */ - finalShardCount = list_length(finalShardIntervalList); - if (finalShardCount == 0) + if ((rangeDistributedRelationCount > 0) && (hashDistributedRelationCount > 0)) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot push down this subquery"), + errdetail("A query including both range and hash " + "partitioned relations are unsupported"))); + } + + foreach(relationIdCell, relationIdList) + { + Oid relationId = lfirst_oid(relationIdCell); + bool coPartitionedTables = false; + Oid currentRelationId = relationId; + + /* get shard list of first relation and continue for the next relation */ + if (relationIndex == 0) { - return NIL; + firstTableRelationId = relationId; + relationIndex++; + + continue; } - fragmentCombinationCell = list_head(fragmentCombinationList); - - foreach(shardIntervalCell, finalShardIntervalList) + /* check if this table has 1-1 shard partitioning with first table */ + coPartitionedTables = CoPartitionedTables(firstTableRelationId, + currentRelationId); + if (!coPartitionedTables) { - ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell); - - RangeTableFragment *shardFragment = palloc0(fragmentSize); - shardFragment->fragmentReference = shardInterval; - shardFragment->fragmentType = CITUS_RTE_RELATION; - shardFragment->rangeTableId = tableId; - - tableSize += ShardLength(shardInterval->shardId); - - if (tableId == 1) - { - List *fragmentCombination = list_make1(shardFragment); - fragmentCombinationList = lappend(fragmentCombinationList, - fragmentCombination); - } - else - { - List *fragmentCombination = (List *) lfirst(fragmentCombinationCell); - fragmentCombination = lappend(fragmentCombination, shardFragment); - - /* get next fragment for the first relation list */ - fragmentCombinationCell = lnext(fragmentCombinationCell); - } + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot push down this subquery"), + errdetail("Shards of relations in subquery need to " + "have 1-to-1 shard partitioning"))); } + } +} - /* - * Determine anchor table using shards which survive pruning instead of calling - * AnchorRangeTableId - */ - if (anchorRangeTableId == 0 || tableSize > largestTableSize) + +/* + * CoPartitionedTables checks if given two distributed tables have 1-to-1 shard + * partitioning. It uses shard interval array that are sorted on interval minimum + * values. Then it compares every shard interval in order and if any pair of + * shard intervals are not equal it returns false. + */ +static bool +CoPartitionedTables(Oid firstRelationId, Oid secondRelationId) +{ + bool coPartitionedTables = true; + uint32 intervalIndex = 0; + DistTableCacheEntry *firstTableCache = DistributedTableCacheEntry(firstRelationId); + DistTableCacheEntry *secondTableCache = DistributedTableCacheEntry(secondRelationId); + ShardInterval **sortedFirstIntervalArray = firstTableCache->sortedShardIntervalArray; + ShardInterval **sortedSecondIntervalArray = + secondTableCache->sortedShardIntervalArray; + uint32 firstListShardCount = firstTableCache->shardIntervalArrayLength; + uint32 secondListShardCount = secondTableCache->shardIntervalArrayLength; + FmgrInfo *comparisonFunction = firstTableCache->shardIntervalCompareFunction; + + if (firstListShardCount != secondListShardCount) + { + return false; + } + + /* if there are not any shards just return true */ + if (firstListShardCount == 0) + { + return true; + } + + Assert(comparisonFunction != NULL); + + for (intervalIndex = 0; intervalIndex < firstListShardCount; intervalIndex++) + { + ShardInterval *firstInterval = sortedFirstIntervalArray[intervalIndex]; + ShardInterval *secondInterval = sortedSecondIntervalArray[intervalIndex]; + + bool shardIntervalsEqual = ShardIntervalsEqual(comparisonFunction, + firstInterval, + secondInterval); + if (!shardIntervalsEqual) { - largestTableSize = tableSize; - anchorRangeTableId = tableId; + coPartitionedTables = false; + break; } + } - rangeTableIndex++; + return coPartitionedTables; +} + + +/* + * ShardIntervalsEqual checks if given shard intervals have equal min/max values. + */ +static bool +ShardIntervalsEqual(FmgrInfo *comparisonFunction, ShardInterval *firstInterval, + ShardInterval *secondInterval) +{ + bool shardIntervalsEqual = false; + Datum firstMin = 0; + Datum firstMax = 0; + Datum secondMin = 0; + Datum secondMax = 0; + + firstMin = firstInterval->minValue; + firstMax = firstInterval->maxValue; + secondMin = secondInterval->minValue; + secondMax = secondInterval->maxValue; + + if (firstInterval->minValueExists && firstInterval->maxValueExists && + secondInterval->minValueExists && secondInterval->maxValueExists) + { + Datum minDatum = CompareCall2(comparisonFunction, firstMin, secondMin); + Datum maxDatum = CompareCall2(comparisonFunction, firstMax, secondMax); + int firstComparison = DatumGetInt32(minDatum); + int secondComparison = DatumGetInt32(maxDatum); + + if (firstComparison == 0 && secondComparison == 0) + { + shardIntervalsEqual = true; + } + } + + return shardIntervalsEqual; +} + + +/* + * SubqueryTaskCreate creates a sql task by replacing the target + * shardInterval's boundary value.. Then performs the normal + * shard pruning on the subquery via RouterSelectQuery(). + * + * The function errors out if the subquery is not router select query (i.e., + * subqueries with non equi-joins.). + */ +static Task * +SubqueryTaskCreate(Query *originalQuery, ShardInterval *shardInterval, + RelationRestrictionContext *restrictionContext, + uint32 taskId) +{ + Query *taskQuery = copyObject(originalQuery); + + uint64 shardId = shardInterval->shardId; + Oid distributedTableId = shardInterval->relationId; + StringInfo queryString = makeStringInfo(); + ListCell *restrictionCell = NULL; + Task *subqueryTask = NULL; + List *selectPlacementList = NIL; + uint64 selectAnchorShardId = INVALID_SHARD_ID; + List *relationShardList = NIL; + uint64 jobId = INVALID_JOB_ID; + bool routerPlannable = false; + bool replacePrunedQueryWithDummy = false; + RelationRestrictionContext *copiedRestrictionContext = + CopyRelationRestrictionContext(restrictionContext); + List *shardOpExpressions = NIL; + RestrictInfo *shardRestrictionList = NULL; + + /* such queries should go through router planner */ + Assert(!restrictionContext->allReferenceTables); + + /* + * Add the restriction qual parameter value in all baserestrictinfos. + * Note that this has to be done on a copy, as the originals are needed + * per target shard interval. + */ + foreach(restrictionCell, copiedRestrictionContext->relationRestrictionList) + { + RelationRestriction *restriction = lfirst(restrictionCell); + Index rteIndex = restriction->index; + List *originalBaseRestrictInfo = restriction->relOptInfo->baserestrictinfo; + List *extendedBaseRestrictInfo = originalBaseRestrictInfo; + + shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex); + + shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions); + extendedBaseRestrictInfo = lappend(extendedBaseRestrictInfo, + shardRestrictionList); + + restriction->relOptInfo->baserestrictinfo = extendedBaseRestrictInfo; + } + + /* mark that we don't want the router planner to generate dummy hosts/queries */ + replacePrunedQueryWithDummy = false; + + /* + * Use router select planner to decide on whether we can push down the query + * or not. If we can, we also rely on the side-effects that all RTEs have been + * updated to point to the relevant nodes and selectPlacementList is determined. + */ + routerPlannable = RouterSelectQuery(taskQuery, copiedRestrictionContext, + &selectPlacementList, &selectAnchorShardId, + &relationShardList, replacePrunedQueryWithDummy); + + /* we don't expect to this this error but keeping it as a precaution for future changes */ + if (!routerPlannable) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot perform distributed planning for the given " + "query"), + errdetail("Select query cannot be pushed down to the worker."))); + } + + /* ensure that we do not send queries where select is pruned away completely */ + if (list_length(selectPlacementList) == 0) + { + ereport(DEBUG2, (errmsg("Skipping the target shard interval %ld because " + "SELECT query is pruned away for the interval", + shardId))); + + return NULL; } /* @@ -2132,46 +2344,22 @@ SubquerySqlTaskList(Job *job) * that the query string is generated as (...) AND (...) as opposed to * (...), (...). */ - whereClauseTree = (Node *) make_ands_explicit((List *) subquery->jointree->quals); - subquery->jointree->quals = whereClauseTree; + taskQuery->jointree->quals = + (Node *) make_ands_explicit((List *) taskQuery->jointree->quals); - /* create tasks from every fragment combination */ - foreach(fragmentCombinationCell, fragmentCombinationList) - { - List *fragmentCombination = (List *) lfirst(fragmentCombinationCell); - List *taskRangeTableList = NIL; - Query *taskQuery = copyObject(subquery); - Task *sqlTask = NULL; - StringInfo sqlQueryString = NULL; + /* and generate the full query string */ + deparse_shard_query(taskQuery, distributedTableId, shardInterval->shardId, + queryString); + ereport(DEBUG4, (errmsg("distributed statement: %s", queryString->data))); - /* create tasks to fetch fragments required for the sql task */ - List *uniqueFragmentList = UniqueFragmentList(fragmentCombination); - List *dataFetchTaskList = DataFetchTaskList(jobId, taskIdIndex, - uniqueFragmentList); - int32 dataFetchTaskCount = list_length(dataFetchTaskList); - taskIdIndex += dataFetchTaskCount; + subqueryTask = CreateBasicTask(jobId, taskId, SQL_TASK, queryString->data); + subqueryTask->dependedTaskList = NULL; + subqueryTask->anchorShardId = selectAnchorShardId; + subqueryTask->taskPlacementList = selectPlacementList; + subqueryTask->upsertQuery = false; + subqueryTask->relationShardList = relationShardList; - ExtractRangeTableRelationWalker((Node *) taskQuery, &taskRangeTableList); - UpdateRangeTableAlias(taskRangeTableList, fragmentCombination); - - /* transform the updated task query to a SQL query string */ - sqlQueryString = makeStringInfo(); - pg_get_query_def(taskQuery, sqlQueryString); - - sqlTask = CreateBasicTask(jobId, taskIdIndex, SQL_TASK, sqlQueryString->data); - sqlTask->dependedTaskList = dataFetchTaskList; - - /* log the query string we generated */ - ereport(DEBUG4, (errmsg("generated sql query for task %d", sqlTask->taskId), - errdetail("query string: \"%s\"", sqlQueryString->data))); - - sqlTask->anchorShardId = AnchorShardId(fragmentCombination, anchorRangeTableId); - - taskIdIndex++; - sqlTaskList = lappend(sqlTaskList, sqlTask); - } - - return sqlTaskList; + return subqueryTask; } @@ -3483,54 +3671,6 @@ FragmentIntervalString(ShardInterval *fragmentInterval) } -/* - * UniqueFragmentList walks over the given relation fragment list, compares - * shard ids, eliminate duplicates and returns a new fragment list of unique - * shard ids. Note that this is a helper function for subquery pushdown, and it - * is used to prevent creating multiple data fetch tasks for same shards. - */ -static List * -UniqueFragmentList(List *fragmentList) -{ - List *uniqueFragmentList = NIL; - ListCell *fragmentCell = NULL; - - foreach(fragmentCell, fragmentList) - { - ShardInterval *shardInterval = NULL; - bool shardIdAlreadyAdded = false; - ListCell *uniqueFragmentCell = NULL; - - RangeTableFragment *fragment = (RangeTableFragment *) lfirst(fragmentCell); - Assert(fragment->fragmentType == CITUS_RTE_RELATION); - - Assert(CitusIsA(fragment->fragmentReference, ShardInterval)); - shardInterval = (ShardInterval *) fragment->fragmentReference; - - foreach(uniqueFragmentCell, uniqueFragmentList) - { - RangeTableFragment *uniqueFragment = - (RangeTableFragment *) lfirst(uniqueFragmentCell); - ShardInterval *uniqueShardInterval = - (ShardInterval *) uniqueFragment->fragmentReference; - - if (shardInterval->shardId == uniqueShardInterval->shardId) - { - shardIdAlreadyAdded = true; - break; - } - } - - if (!shardIdAlreadyAdded) - { - uniqueFragmentList = lappend(uniqueFragmentList, fragment); - } - } - - return uniqueFragmentList; -} - - /* * DataFetchTaskList builds a data fetch task for every shard in the given shard * list, appends these data fetch tasks into a list, and returns this list. diff --git a/src/backend/distributed/planner/multi_planner.c b/src/backend/distributed/planner/multi_planner.c index 71cd14a15..0b5d03d3c 100644 --- a/src/backend/distributed/planner/multi_planner.c +++ b/src/backend/distributed/planner/multi_planner.c @@ -309,8 +309,8 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query */ if ((!distributedPlan || distributedPlan->planningError) && !hasUnresolvedParams) { - /* Create and optimize logical plan */ - MultiTreeRoot *logicalPlan = MultiLogicalPlanCreate(query); + MultiTreeRoot *logicalPlan = MultiLogicalPlanCreate(originalQuery, query, + plannerRestrictionContext); MultiLogicalPlanOptimize(logicalPlan); /* @@ -323,7 +323,8 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query CheckNodeIsDumpable((Node *) logicalPlan); /* Create the physical plan */ - distributedPlan = MultiPhysicalPlanCreate(logicalPlan); + distributedPlan = MultiPhysicalPlanCreate(logicalPlan, + plannerRestrictionContext); /* distributed plan currently should always succeed or error out */ Assert(distributedPlan && distributedPlan->planningError == NULL); @@ -796,11 +797,10 @@ CopyPlanParamList(List *originalPlanParamList) /* - * CreateAndPushPlannerRestrictionContext creates a new planner restriction context. - * Later, it creates a relation restriction context and a join restriction - * context, and sets those contexts in the planner restriction context. Finally, - * the planner restriction context is inserted to the beginning of the - * plannerRestrictionContextList and it is returned. + * CreateAndPushPlannerRestrictionContext creates a new relation restriction context + * and a new join context, inserts it to the beginning of the + * plannerRestrictionContextList. Finally, the planner restriction context is + * inserted to the beginning of the plannerRestrictionContextList and it is returned. */ static PlannerRestrictionContext * CreateAndPushPlannerRestrictionContext(void) diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index e094ad8d0..5fe534e9c 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -89,13 +89,14 @@ static MultiPlan * CreateSingleTaskRouterPlan(Query *originalQuery, static MultiPlan * CreateInsertSelectRouterPlan(Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext); +static bool SafeToPushDownSubquery(PlannerRestrictionContext *plannerRestrictionContext, + Query *originalQuery); static Task * RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInterval, RelationRestrictionContext * restrictionContext, uint32 taskIdIndex, bool allRelationsJoinedOnPartitionKey); -static List * ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex); static bool MasterIrreducibleExpression(Node *expression, bool *varArgument, bool *badCoalesce); static bool MasterIrreducibleExpressionWalker(Node *expression, WalkerState *state); @@ -113,10 +114,6 @@ static Expr * ExtractInsertPartitionValue(Query *query, Var *partitionColumn); static Task * RouterSelectTask(Query *originalQuery, RelationRestrictionContext *restrictionContext, List **placementList); -static bool RouterSelectQuery(Query *originalQuery, - RelationRestrictionContext *restrictionContext, - List **placementList, uint64 *anchorShardId, - List **relationShardList, bool replacePrunedQueryWithDummy); static bool RelationPrunesToMultipleShards(List *relationShardList); static List * TargetShardIntervalsForSelect(Query *query, RelationRestrictionContext *restrictionContext); @@ -125,8 +122,6 @@ static List * IntersectPlacementList(List *lhsPlacementList, List *rhsPlacementL static Job * RouterQueryJob(Query *query, Task *task, List *placementList); static bool MultiRouterPlannableQuery(Query *query, RelationRestrictionContext *restrictionContext); -static RelationRestrictionContext * CopyRelationRestrictionContext( - RelationRestrictionContext *oldContext); static DeferredErrorMessage * InsertSelectQuerySupported(Query *queryTree, RangeTblEntry *insertRte, RangeTblEntry *subqueryRte, @@ -299,7 +294,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery, RelationRestrictionContext *relationRestrictionContext = plannerRestrictionContext->relationRestrictionContext; bool allReferenceTables = relationRestrictionContext->allReferenceTables; - bool restrictionEquivalenceForPartitionKeys = false; + bool safeToPushDownSubquery = false; multiPlan->operation = originalQuery->commandType; @@ -315,8 +310,8 @@ CreateInsertSelectRouterPlan(Query *originalQuery, return multiPlan; } - restrictionEquivalenceForPartitionKeys = - RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext); + safeToPushDownSubquery = SafeToPushDownSubquery(plannerRestrictionContext, + originalQuery); /* * Plan select query for each shard in the target table. Do so by replacing the @@ -336,7 +331,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery, modifyTask = RouterModifyTaskForShardInterval(originalQuery, targetShardInterval, relationRestrictionContext, taskIdIndex, - restrictionEquivalenceForPartitionKeys); + safeToPushDownSubquery); /* add the task if it could be created */ if (modifyTask != NULL) @@ -382,6 +377,36 @@ CreateInsertSelectRouterPlan(Query *originalQuery, } +/* + * SafeToPushDownSubquery returns true if either + * (i) there exists join in the query and all relations joined on their + * partition keys + * (ii) there exists only union set operations and all relations has + * partition keys in the same ordinal position in the query + */ +static bool +SafeToPushDownSubquery(PlannerRestrictionContext *plannerRestrictionContext, + Query *originalQuery) +{ + RelationRestrictionContext *relationRestrictionContext = + plannerRestrictionContext->relationRestrictionContext; + bool restrictionEquivalenceForPartitionKeys = + RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext); + + if (restrictionEquivalenceForPartitionKeys) + { + return true; + } + + if (ContainsUnionSubquery(originalQuery)) + { + return SafeToPushdownUnionSubquery(relationRestrictionContext); + } + + return false; +} + + /* * RouterModifyTaskForShardInterval creates a modify task by * replacing the partitioning qual parameter added in multi_planner() @@ -397,7 +422,7 @@ static Task * RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInterval, RelationRestrictionContext *restrictionContext, uint32 taskIdIndex, - bool allRelationsJoinedOnPartitionKey) + bool safeToPushdownSubquery) { Query *copiedQuery = copyObject(originalQuery); RangeTblEntry *copiedInsertRte = ExtractInsertRangeTableEntry(copiedQuery); @@ -424,8 +449,8 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter bool upsertQuery = false; bool replacePrunedQueryWithDummy = false; bool allReferenceTables = restrictionContext->allReferenceTables; - List *hashedOpExpressions = NIL; - RestrictInfo *hashedRestrictInfo = NULL; + List *shardOpExpressions = NIL; + RestrictInfo *shardRestrictionList = NULL; /* grab shared metadata lock to stop concurrent placement additions */ LockShardDistributionMetadata(shardId, ShareLock); @@ -437,20 +462,21 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter foreach(restrictionCell, copiedRestrictionContext->relationRestrictionList) { RelationRestriction *restriction = lfirst(restrictionCell); - List *originalBaserestrictInfo = restriction->relOptInfo->baserestrictinfo; + List *originalBaseRestrictInfo = restriction->relOptInfo->baserestrictinfo; + List *extendedBaseRestrictInfo = originalBaseRestrictInfo; Index rteIndex = restriction->index; - if (!allRelationsJoinedOnPartitionKey || allReferenceTables) + if (!safeToPushdownSubquery || allReferenceTables) { continue; } - hashedOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex); + shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex); + shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions); + extendedBaseRestrictInfo = lappend(extendedBaseRestrictInfo, + shardRestrictionList); - hashedRestrictInfo = make_simple_restrictinfo((Expr *) hashedOpExpressions); - originalBaserestrictInfo = lappend(originalBaserestrictInfo, hashedRestrictInfo); - - restriction->relOptInfo->baserestrictinfo = originalBaserestrictInfo; + restriction->relOptInfo->baserestrictinfo = extendedBaseRestrictInfo; } /* @@ -562,7 +588,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter * * NB: If you update this, also look at PrunableExpressionsWalker(). */ -static List * +List * ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex) { Oid relationId = shardInterval->relationId; @@ -578,7 +604,6 @@ ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex) DISTRIBUTE_BY_APPEND) { Assert(rteIndex > 0); - partitionColumn = PartitionColumn(relationId, rteIndex); } else @@ -876,6 +901,15 @@ MultiTaskRouterSelectQuerySupported(Query *query) Assert(subquery->commandType == CMD_SELECT); + /* pushing down rtes without relations yields (shardCount * expectedRows) */ + if (subquery->rtable == NIL) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "Subqueries without relations are not allowed in " + "INSERT ... SELECT queries", + NULL, NULL); + } + /* pushing down limit per shard would yield wrong results */ if (subquery->limitCount != NULL) { @@ -2262,7 +2296,7 @@ RouterSelectTask(Query *originalQuery, RelationRestrictionContext *restrictionCo * relationShardList is filled with the list of relation-to-shard mappings for * the query. */ -static bool +bool RouterSelectQuery(Query *originalQuery, RelationRestrictionContext *restrictionContext, List **placementList, uint64 *anchorShardId, List **relationShardList, bool replacePrunedQueryWithDummy) @@ -2910,7 +2944,7 @@ InsertSelectQuery(Query *query) * plannerInfo which is read-only. All other parts of the relOptInfo is also shallowly * copied. */ -static RelationRestrictionContext * +RelationRestrictionContext * CopyRelationRestrictionContext(RelationRestrictionContext *oldContext) { RelationRestrictionContext *newContext = (RelationRestrictionContext *) diff --git a/src/backend/distributed/planner/relation_restriction_equivalence.c b/src/backend/distributed/planner/relation_restriction_equivalence.c index d7dbc75ae..5fb340e17 100644 --- a/src/backend/distributed/planner/relation_restriction_equivalence.c +++ b/src/backend/distributed/planner/relation_restriction_equivalence.c @@ -12,6 +12,7 @@ #include "distributed/multi_planner.h" #include "distributed/multi_logical_planner.h" +#include "distributed/multi_logical_optimizer.h" #include "distributed/pg_dist_partition.h" #include "distributed/relation_restriction_equivalence.h" #include "nodes/nodeFuncs.h" @@ -61,6 +62,11 @@ typedef struct AttributeEquivalenceClassMember } AttributeEquivalenceClassMember; +static Var * FindTranslatedVar(List *appendRelList, Oid relationOid, + Index relationRteIndex, Index *partitionKeyIndex); +static bool EquivalenceListContainsRelationsEquality(List *attributeEquivalenceList, + RelationRestrictionContext * + restrictionContext); static uint32 ReferenceRelationCount(RelationRestrictionContext *restrictionContext); static List * GenerateAttributeEquivalencesForRelationRestrictions( RelationRestrictionContext *restrictionContext); @@ -69,6 +75,29 @@ static AttributeEquivalenceClass * AttributeEquivalenceClassForEquivalenceClass( static void AddToAttributeEquivalenceClass(AttributeEquivalenceClass ** attributeEquivalanceClass, PlannerInfo *root, Var *varToBeAdded); +static void AddRteSubqueryToAttributeEquivalenceClass(AttributeEquivalenceClass * + *attributeEquivalanceClass, + RangeTblEntry * + rangeTableEntry, + PlannerInfo *root, + Var *varToBeAdded); +static Query * GetTargetSubquery(PlannerInfo *root, RangeTblEntry *rangeTableEntry, + Var *varToBeAdded); +static void AddUnionAllSetOperationsToAttributeEquivalenceClass( + AttributeEquivalenceClass ** + attributeEquivalanceClass, + PlannerInfo *root, + Var *varToBeAdded); +static void AddUnionSetOperationsToAttributeEquivalenceClass(AttributeEquivalenceClass ** + attributeEquivalenceClass, + PlannerInfo *root, + SetOperationStmt * + setOperation, + Var *varToBeAdded); +static void AddRteRelationToAttributeEquivalenceClass(AttributeEquivalenceClass ** + attrEquivalenceClass, + RangeTblEntry *rangeTableEntry, + Var *varToBeAdded); static Var * GetVarFromAssignedParam(List *parentPlannerParamList, Param *plannerParam); static List * GenerateAttributeEquivalencesForJoinRestrictions(JoinRestrictionContext @@ -90,6 +119,193 @@ static void ListConcatUniqueAttributeClassMemberLists(AttributeEquivalenceClass firstClass, AttributeEquivalenceClass * secondClass); +static Index RelationRestrictionPartitionKeyIndex(RelationRestriction * + relationRestriction); + + +/* + * SafeToPushdownUnionSubquery returns true if all the relations are returns + * partition keys in the same ordinal position. + * + * Note that the function expects (and asserts) the input query to be a top + * level union query defined by TopLevelUnionQuery(). + * + * Lastly, the function fails to produce correct output if the target lists contains + * multiple partition keys on the target list such as the following: + * + * select count(*) from ( + * select user_id, user_id from users_table + * union + * select 2, user_id from users_table) u; + * + * For the above query, although the second item in the target list make this query + * safe to push down, the function would fail to return true. + */ +bool +SafeToPushdownUnionSubquery(RelationRestrictionContext *restrictionContext) +{ + Index unionQueryPartitionKeyIndex = 0; + AttributeEquivalenceClass *attributeEquivalance = + palloc0(sizeof(AttributeEquivalenceClass)); + ListCell *relationRestrictionCell = NULL; + + attributeEquivalance->equivalenceId = attributeEquivalenceId++; + + foreach(relationRestrictionCell, restrictionContext->relationRestrictionList) + { + RelationRestriction *relationRestriction = lfirst(relationRestrictionCell); + Index partitionKeyIndex = InvalidAttrNumber; + PlannerInfo *relationPlannerRoot = relationRestriction->plannerInfo; + List *targetList = relationPlannerRoot->parse->targetList; + List *appendRelList = relationPlannerRoot->append_rel_list; + Var *varToBeAdded = NULL; + TargetEntry *targetEntryToAdd = NULL; + + /* + * We first check whether UNION ALLs are pulled up or not. Note that Postgres + * planner creates AppendRelInfos per each UNION ALL query that is pulled up. + * Then, postgres stores the related information in the append_rel_list on the + * plannerInfo struct. + */ + if (appendRelList != NULL) + { + varToBeAdded = FindTranslatedVar(appendRelList, + relationRestriction->relationId, + relationRestriction->index, + &partitionKeyIndex); + + /* union does not have partition key in the target list */ + if (partitionKeyIndex == 0) + { + return false; + } + } + else + { + partitionKeyIndex = + RelationRestrictionPartitionKeyIndex(relationRestriction); + + /* union does not have partition key in the target list */ + if (partitionKeyIndex == 0) + { + return false; + } + + targetEntryToAdd = list_nth(targetList, partitionKeyIndex - 1); + if (!IsA(targetEntryToAdd->expr, Var)) + { + return false; + } + + varToBeAdded = (Var *) targetEntryToAdd->expr; + } + + /* + * If the first relation doesn't have partition key on the target + * list of the query that the relation in, simply not allow to push down + * the query. + */ + if (partitionKeyIndex == InvalidAttrNumber) + { + return false; + } + + /* + * We find the first relations partition key index in the target list. Later, + * we check whether all the relations have partition keys in the + * same position. + */ + if (unionQueryPartitionKeyIndex == InvalidAttrNumber) + { + unionQueryPartitionKeyIndex = partitionKeyIndex; + } + else if (unionQueryPartitionKeyIndex != partitionKeyIndex) + { + return false; + } + + AddToAttributeEquivalenceClass(&attributeEquivalance, relationPlannerRoot, + varToBeAdded); + } + + return EquivalenceListContainsRelationsEquality(list_make1(attributeEquivalance), + restrictionContext); +} + + +/* + * FindTranslatedVar iterates on the appendRelList and tries to find a translated + * child var identified by the relation id and the relation rte index. + * + * Note that postgres translates UNION ALL target list elements into translated_vars + * list on the corresponding AppendRelInfo struct. For details, see the related + * structs. + * + * The function returns NULL if it cannot find a translated var. + */ +static Var * +FindTranslatedVar(List *appendRelList, Oid relationOid, Index relationRteIndex, + Index *partitionKeyIndex) +{ + ListCell *appendRelCell = NULL; + AppendRelInfo *targetAppendRelInfo = NULL; + ListCell *translatedVarCell = NULL; + AttrNumber childAttrNumber = 0; + Var *relationPartitionKey = NULL; + List *translaterVars = NULL; + + *partitionKeyIndex = 0; + + /* iterate on the queries that are part of UNION ALL subselects */ + foreach(appendRelCell, appendRelList) + { + AppendRelInfo *appendRelInfo = (AppendRelInfo *) lfirst(appendRelCell); + + /* + * We're only interested in the child rel that is equal to the + * relation we're investigating. + */ + if (appendRelInfo->child_relid == relationRteIndex) + { + targetAppendRelInfo = appendRelInfo; + break; + } + } + + /* we couldn't find the necessary append rel info */ + if (targetAppendRelInfo == NULL) + { + return NULL; + } + + relationPartitionKey = PartitionKey(relationOid); + + translaterVars = targetAppendRelInfo->translated_vars; + foreach(translatedVarCell, translaterVars) + { + Node *targetNode = (Node *) lfirst(translatedVarCell); + Var *targetVar = NULL; + + childAttrNumber++; + + if (!IsA(targetNode, Var)) + { + continue; + } + + targetVar = (Var *) lfirst(translatedVarCell); + if (targetVar->varno == relationRteIndex && + targetVar->varattno == relationPartitionKey->varattno) + { + *partitionKeyIndex = childAttrNumber; + + return targetVar; + } + } + + return NULL; +} + /* * RestrictionEquivalenceForPartitionKeys aims to deduce whether each of the RTE_RELATION @@ -117,14 +333,14 @@ static void ListConcatUniqueAttributeClassMemberLists(AttributeEquivalenceClass * step, we try generate a common attribute equivalence class that holds as much as * AttributeEquivalenceMembers whose attributes are a partition keys. * - * AllRelationsJoinedOnPartitionKey uses both relation restrictions and join restrictions + * RestrictionEquivalenceForPartitionKeys uses both relation restrictions and join restrictions * to find as much as information that Postgres planner provides to extensions. For the * details of the usage, please see GenerateAttributeEquivalencesForRelationRestrictions() * and GenerateAttributeEquivalencesForJoinRestrictions() */ bool -RestrictionEquivalenceForPartitionKeys( - PlannerRestrictionContext *plannerRestrictionContext) +RestrictionEquivalenceForPartitionKeys(PlannerRestrictionContext * + plannerRestrictionContext) { RelationRestrictionContext *restrictionContext = plannerRestrictionContext->relationRestrictionContext; @@ -134,13 +350,9 @@ RestrictionEquivalenceForPartitionKeys( List *relationRestrictionAttributeEquivalenceList = NIL; List *joinRestrictionAttributeEquivalenceList = NIL; List *allAttributeEquivalenceList = NIL; - AttributeEquivalenceClass *commonEquivalenceClass = NULL; uint32 referenceRelationCount = ReferenceRelationCount(restrictionContext); uint32 totalRelationCount = list_length(restrictionContext->relationRestrictionList); uint32 nonReferenceRelationCount = totalRelationCount - referenceRelationCount; - ListCell *commonEqClassCell = NULL; - ListCell *relationRestrictionCell = NULL; - Relids commonRteIdentities = NULL; /* * If the query includes a single relation which is not a reference table, @@ -172,12 +384,33 @@ RestrictionEquivalenceForPartitionKeys( list_concat(relationRestrictionAttributeEquivalenceList, joinRestrictionAttributeEquivalenceList); + return EquivalenceListContainsRelationsEquality(allAttributeEquivalenceList, + restrictionContext); +} + + +/* + * EquivalenceListContainsRelationsEquality gets a list of attributed equivalence + * list and a relation restriction context. The function first generates a common + * equivalence class out of the attributeEquivalenceList. Later, the function checks + * whether all the relations exists in the common equivalence class. + * + */ +static bool +EquivalenceListContainsRelationsEquality(List *attributeEquivalenceList, + RelationRestrictionContext *restrictionContext) +{ + AttributeEquivalenceClass *commonEquivalenceClass = NULL; + ListCell *commonEqClassCell = NULL; + ListCell *relationRestrictionCell = NULL; + Relids commonRteIdentities = NULL; + /* * In general we're trying to expand existing the equivalence classes to find a * common equivalence class. The main goal is to test whether this main class * contains all partition keys of the existing relations. */ - commonEquivalenceClass = GenerateCommonEquivalence(allAttributeEquivalenceList); + commonEquivalenceClass = GenerateCommonEquivalence(attributeEquivalenceList); /* add the rte indexes of relations to a bitmap */ foreach(commonEqClassCell, commonEquivalenceClass->equivalentAttributes) @@ -632,106 +865,271 @@ GenerateAttributeEquivalencesForJoinRestrictions(JoinRestrictionContext * * - Generate an AttributeEquivalenceMember and add to the input * AttributeEquivalenceClass * - If the RTE that corresponds to a subquery - * - Find the corresponding target entry via varno - * - if subquery entry is a set operation (i.e., only UNION/UNION ALL allowed) - * - recursively add both left and right sides of the set operation's + * - If the RTE that corresponds to a UNION ALL subquery + * - Iterate on each of the appendRels (i.e., each of the UNION ALL query) + * - Recursively add all children of the set operation's + * corresponding target entries + * - If the corresponding subquery entry is a UNION set operation + * - Recursively add all children of the set operation's * corresponding target entries - * - if subquery is not a set operation - * - recursively try to add the corresponding target entry to the + * - If the corresponding subquery is a regular subquery (i.e., No set operations) + * - Recursively try to add the corresponding target entry to the * equivalence class - * - * Note that this function only adds partition keys to the attributeEquivalanceClass. - * This implies that there wouldn't be any columns for reference tables. */ static void AddToAttributeEquivalenceClass(AttributeEquivalenceClass **attributeEquivalanceClass, PlannerInfo *root, Var *varToBeAdded) { - RangeTblEntry *rangeTableEntry = root->simple_rte_array[varToBeAdded->varno]; + RangeTblEntry *rangeTableEntry = NULL; + /* punt if it's a whole-row var rather than a plain column reference */ + if (varToBeAdded->varattno == InvalidAttrNumber) + { + return; + } + + /* we also don't want to process ctid, tableoid etc */ + if (varToBeAdded->varattno < InvalidAttrNumber) + { + return; + } + + rangeTableEntry = root->simple_rte_array[varToBeAdded->varno]; if (rangeTableEntry->rtekind == RTE_RELATION) { - AttributeEquivalenceClassMember *attributeEqMember = NULL; - Oid relationId = rangeTableEntry->relid; - Var *relationPartitionKey = NULL; - - if (PartitionMethod(relationId) == DISTRIBUTE_BY_NONE) - { - return; - } - - relationPartitionKey = PartitionKey(relationId); - if (relationPartitionKey->varattno != varToBeAdded->varattno) - { - return; - } - - attributeEqMember = palloc0(sizeof(AttributeEquivalenceClassMember)); - - attributeEqMember->varattno = varToBeAdded->varattno; - attributeEqMember->varno = varToBeAdded->varno; - attributeEqMember->rteIdentity = GetRTEIdentity(rangeTableEntry); - attributeEqMember->relationId = rangeTableEntry->relid; - - (*attributeEquivalanceClass)->equivalentAttributes = - lappend((*attributeEquivalanceClass)->equivalentAttributes, - attributeEqMember); + AddRteRelationToAttributeEquivalenceClass(attributeEquivalanceClass, + rangeTableEntry, + varToBeAdded); } - else if (rangeTableEntry->rtekind == RTE_SUBQUERY && !rangeTableEntry->inh) + else if (rangeTableEntry->rtekind == RTE_SUBQUERY) { - Query *subquery = rangeTableEntry->subquery; - RelOptInfo *baseRelOptInfo = NULL; - TargetEntry *subqueryTargetEntry = NULL; + AddRteSubqueryToAttributeEquivalenceClass(attributeEquivalanceClass, + rangeTableEntry, root, + varToBeAdded); + } +} - /* punt if it's a whole-row var rather than a plain column reference */ - if (varToBeAdded->varattno == InvalidAttrNumber) - { - return; - } - /* we also don't want to process ctid, tableoid etc */ - if (varToBeAdded->varattno < InvalidAttrNumber) - { - return; - } +/* + * AddRteSubqueryToAttributeEquivalenceClass adds the given var to the given + * attribute equivalence class. + * + * The main algorithm is outlined in AddToAttributeEquivalenceClass(). + */ +static void +AddRteSubqueryToAttributeEquivalenceClass(AttributeEquivalenceClass + **attributeEquivalanceClass, + RangeTblEntry *rangeTableEntry, + PlannerInfo *root, Var *varToBeAdded) +{ + RelOptInfo *baseRelOptInfo = find_base_rel(root, varToBeAdded->varno); + TargetEntry *subqueryTargetEntry = NULL; + Query *targetSubquery = GetTargetSubquery(root, rangeTableEntry, varToBeAdded); - baseRelOptInfo = find_base_rel(root, varToBeAdded->varno); + subqueryTargetEntry = get_tle_by_resno(targetSubquery->targetList, + varToBeAdded->varattno); - /* If the subquery hasn't been planned yet, we have to punt */ + /* if we fail to find corresponding target entry, do not proceed */ + if (subqueryTargetEntry == NULL || subqueryTargetEntry->resjunk) + { + return; + } + + /* we're only interested in Vars */ + if (!IsA(subqueryTargetEntry->expr, Var)) + { + return; + } + + varToBeAdded = (Var *) subqueryTargetEntry->expr; + + /* + * "inh" flag is set either when inheritance or "UNION ALL" exists in the + * subquery. Here we're only interested in the "UNION ALL" case. + * + * Else, we check one more thing: Does the subquery contain a "UNION" query. + * If so, we recursively traverse all "UNION" tree and add the corresponding + * target list elements to the attribute equivalence. + * + * Finally, if it is a regular subquery (i.e., does not contain UNION or UNION ALL), + * we simply recurse to find the corresponding RTE_RELATION to add to the + * equivalence class. + * + * Note that we're treating "UNION" and "UNION ALL" clauses differently given + * that postgres planner process/plans them separately. + */ + if (rangeTableEntry->inh) + { + AddUnionAllSetOperationsToAttributeEquivalenceClass(attributeEquivalanceClass, + root, varToBeAdded); + } + else if (targetSubquery->setOperations) + { + AddUnionSetOperationsToAttributeEquivalenceClass(attributeEquivalanceClass, + baseRelOptInfo->subroot, + (SetOperationStmt *) + targetSubquery->setOperations, + varToBeAdded); + } + else if (varToBeAdded && IsA(varToBeAdded, Var) && varToBeAdded->varlevelsup == 0) + { + AddToAttributeEquivalenceClass(attributeEquivalanceClass, + baseRelOptInfo->subroot, varToBeAdded); + } +} + + +/* + * GetTargetSubquery returns the corresponding subquery for the given planner root, + * range table entry and the var. + * + * The aim of this function is to simplify extracting the subquery in case of "UNION ALL" + * queries. + */ +static Query * +GetTargetSubquery(PlannerInfo *root, RangeTblEntry *rangeTableEntry, Var *varToBeAdded) +{ + Query *targetSubquery = NULL; + + /* + * For subqueries other than "UNION ALL", find the corresponding targetSubquery. See + * the details of how we process subqueries in the below comments. + */ + if (!rangeTableEntry->inh) + { + RelOptInfo *baseRelOptInfo = find_base_rel(root, varToBeAdded->varno); + + /* If the targetSubquery hasn't been planned yet, we have to punt */ if (baseRelOptInfo->subroot == NULL) { - return; + return NULL; } Assert(IsA(baseRelOptInfo->subroot, PlannerInfo)); - subquery = baseRelOptInfo->subroot->parse; - Assert(IsA(subquery, Query)); - - /* Get the subquery output expression referenced by the upper Var */ - subqueryTargetEntry = get_tle_by_resno(subquery->targetList, - varToBeAdded->varattno); - if (subqueryTargetEntry == NULL || subqueryTargetEntry->resjunk) - { - ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - errmsg("subquery %s does not have attribute %d", - rangeTableEntry->eref->aliasname, - varToBeAdded->varattno))); - } - - if (!IsA(subqueryTargetEntry->expr, Var)) - { - return; - } - - varToBeAdded = (Var *) subqueryTargetEntry->expr; - - if (varToBeAdded && IsA(varToBeAdded, Var) && varToBeAdded->varlevelsup == 0) - { - AddToAttributeEquivalenceClass(attributeEquivalanceClass, - baseRelOptInfo->subroot, varToBeAdded); - } + targetSubquery = baseRelOptInfo->subroot->parse; + Assert(IsA(targetSubquery, Query)); } + else + { + targetSubquery = rangeTableEntry->subquery; + } + + return targetSubquery; +} + + +/* + * AddUnionAllSetOperationsToAttributeEquivalenceClass recursively iterates on all the + * append rels, sets the varno's accordingly and adds the + * var the given equivalence class. + */ +static void +AddUnionAllSetOperationsToAttributeEquivalenceClass(AttributeEquivalenceClass ** + attributeEquivalanceClass, + PlannerInfo *root, + Var *varToBeAdded) +{ + List *appendRelList = root->append_rel_list; + ListCell *appendRelCell = NULL; + + /* iterate on the queries that are part of UNION ALL subqueries */ + foreach(appendRelCell, appendRelList) + { + AppendRelInfo *appendRelInfo = (AppendRelInfo *) lfirst(appendRelCell); + + /* + * We're only interested in UNION ALL clauses and parent_reloid is invalid + * only for UNION ALL (i.e., equals to a legitimate Oid for inheritance) + */ + if (appendRelInfo->parent_reloid != InvalidOid) + { + continue; + } + + /* set the varno accordingly for this specific child */ + varToBeAdded->varno = appendRelInfo->child_relid; + + AddToAttributeEquivalenceClass(attributeEquivalanceClass, root, + varToBeAdded); + } +} + + +/* + * AddUnionSetOperationsToAttributeEquivalenceClass recursively iterates on all the + * setOperations and adds each corresponding target entry to the given equivalence + * class. + * + * Although the function silently accepts INTERSECT and EXPECT set operations, they are + * rejected later in the planning. We prefer this behavior to provide better error + * messages. + */ +static void +AddUnionSetOperationsToAttributeEquivalenceClass(AttributeEquivalenceClass ** + attributeEquivalenceClass, + PlannerInfo *root, + SetOperationStmt *setOperation, + Var *varToBeAdded) +{ + List *rangeTableIndexList = NIL; + ListCell *rangeTableIndexCell = NULL; + + ExtractRangeTableIndexWalker((Node *) setOperation, &rangeTableIndexList); + + foreach(rangeTableIndexCell, rangeTableIndexList) + { + int rangeTableIndex = lfirst_int(rangeTableIndexCell); + + varToBeAdded->varno = rangeTableIndex; + AddToAttributeEquivalenceClass(attributeEquivalenceClass, root, varToBeAdded); + } +} + + +/* + * AddRteRelationToAttributeEquivalenceClass adds the given var to the given equivalence + * class using the rteIdentity provided by the rangeTableEntry. Note that + * rteIdentities are only assigned to RTE_RELATIONs and this function asserts + * the input rte to be an RTE_RELATION. + * + * Note that this function only adds partition keys to the attributeEquivalanceClass. + * This implies that there wouldn't be any columns for reference tables. + */ +static void +AddRteRelationToAttributeEquivalenceClass(AttributeEquivalenceClass ** + attrEquivalenceClass, + RangeTblEntry *rangeTableEntry, + Var *varToBeAdded) +{ + AttributeEquivalenceClassMember *attributeEqMember = NULL; + Oid relationId = InvalidOid; + Var *relationPartitionKey = NULL; + + Assert(rangeTableEntry->rtekind == RTE_RELATION); + + relationId = rangeTableEntry->relid; + if (PartitionMethod(relationId) == DISTRIBUTE_BY_NONE) + { + return; + } + + relationPartitionKey = PartitionKey(relationId); + if (relationPartitionKey->varattno != varToBeAdded->varattno) + { + return; + } + + attributeEqMember = palloc0(sizeof(AttributeEquivalenceClassMember)); + + attributeEqMember->varattno = varToBeAdded->varattno; + attributeEqMember->varno = varToBeAdded->varno; + attributeEqMember->rteIdentity = GetRTEIdentity(rangeTableEntry); + attributeEqMember->relationId = rangeTableEntry->relid; + + (*attrEquivalenceClass)->equivalentAttributes = + lappend((*attrEquivalenceClass)->equivalentAttributes, + attributeEqMember); } @@ -821,8 +1219,10 @@ static bool AttributeEquivalancesAreEqual(AttributeEquivalenceClass *firstAttributeEquivalance, AttributeEquivalenceClass *secondAttributeEquivalance) { - List *firstEquivalenceMemberList = firstAttributeEquivalance->equivalentAttributes; - List *secondEquivalenceMemberList = secondAttributeEquivalance->equivalentAttributes; + List *firstEquivalenceMemberList = + firstAttributeEquivalance->equivalentAttributes; + List *secondEquivalenceMemberList = + secondAttributeEquivalance->equivalentAttributes; ListCell *firstAttributeEquivalanceCell = NULL; ListCell *secondAttributeEquivalanceCell = NULL; @@ -832,7 +1232,6 @@ AttributeEquivalancesAreEqual(AttributeEquivalenceClass *firstAttributeEquivalan return false; } - foreach(firstAttributeEquivalanceCell, firstEquivalenceMemberList) { AttributeEquivalenceClassMember *firstEqMember = @@ -862,3 +1261,134 @@ AttributeEquivalancesAreEqual(AttributeEquivalenceClass *firstAttributeEquivalan return true; } + + +/* + * ContainsUnionSubquery gets a queryTree and returns true if the query + * contains + * - a subquery with UNION set operation + * - no joins above the UNION set operation in the query tree + * + * Note that the function allows top level unions being wrapped into aggregations + * queries and/or simple projection queries that only selects some fields from + * the lower level queries. + * + * If there exists joins before the set operations, the function returns false. + * Similarly, if the query does not contain any union set operations, the + * function returns false. + */ +bool +ContainsUnionSubquery(Query *queryTree) +{ + List *rangeTableList = queryTree->rtable; + Node *setOperations = queryTree->setOperations; + List *joinTreeTableIndexList = NIL; + Index subqueryRteIndex = 0; + uint32 joiningRangeTableCount = 0; + RangeTblEntry *rangeTableEntry = NULL; + Query *subqueryTree = NULL; + + ExtractRangeTableIndexWalker((Node *) queryTree->jointree, &joinTreeTableIndexList); + joiningRangeTableCount = list_length(joinTreeTableIndexList); + + /* don't allow joins on top of unions */ + if (joiningRangeTableCount > 1) + { + return false; + } + + subqueryRteIndex = linitial_int(joinTreeTableIndexList); + rangeTableEntry = rt_fetch(subqueryRteIndex, rangeTableList); + if (rangeTableEntry->rtekind != RTE_SUBQUERY) + { + return false; + } + + subqueryTree = rangeTableEntry->subquery; + setOperations = subqueryTree->setOperations; + if (setOperations != NULL) + { + SetOperationStmt *setOperationStatement = (SetOperationStmt *) setOperations; + + /* + * Note that the set operation tree is traversed elsewhere for ensuring + * that we only support UNIONs. + */ + if (setOperationStatement->op != SETOP_UNION) + { + return false; + } + + return true; + } + + return ContainsUnionSubquery(subqueryTree); +} + + +/* + * RelationRestrictionPartitionKeyIndex gets a relation restriction and finds the + * index that the partition key of the relation exists in the query. The query is + * found in the planner info of the relation restriction. + */ +static Index +RelationRestrictionPartitionKeyIndex(RelationRestriction *relationRestriction) +{ + PlannerInfo *relationPlannerRoot = NULL; + Query *relationPlannerParseQuery = NULL; + List *relationTargetList = NIL; + ListCell *targetEntryCell = NULL; + Index partitionKeyTargetAttrIndex = 0; + + relationPlannerRoot = relationRestriction->plannerInfo; + relationPlannerParseQuery = relationPlannerRoot->parse; + relationTargetList = relationPlannerParseQuery->targetList; + + foreach(targetEntryCell, relationTargetList) + { + TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); + Expr *targetExpression = targetEntry->expr; + + partitionKeyTargetAttrIndex++; + + if (!targetEntry->resjunk && + IsPartitionColumn(targetExpression, relationPlannerParseQuery) && + IsA(targetExpression, Var)) + { + Var *targetColumn = (Var *) targetExpression; + + if (targetColumn->varno == relationRestriction->index) + { + return partitionKeyTargetAttrIndex; + } + } + } + + return InvalidAttrNumber; +} + + +/* + * RelationIdList returns list of unique relation ids in query tree. + */ +List * +RelationIdList(Query *query) +{ + List *rangeTableList = NIL; + List *tableEntryList = NIL; + List *relationIdList = NIL; + ListCell *tableEntryCell = NULL; + + ExtractRangeTableRelationWalker((Node *) query, &rangeTableList); + tableEntryList = TableEntryList(rangeTableList); + + foreach(tableEntryCell, tableEntryList) + { + TableEntry *tableEntry = (TableEntry *) lfirst(tableEntryCell); + Oid relationId = tableEntry->relationId; + + relationIdList = list_append_unique_oid(relationIdList, relationId); + } + + return relationIdList; +} diff --git a/src/backend/distributed/utils/citus_clauses.c b/src/backend/distributed/utils/citus_clauses.c index 91f2a0db2..0016499a3 100644 --- a/src/backend/distributed/utils/citus_clauses.c +++ b/src/backend/distributed/utils/citus_clauses.c @@ -31,7 +31,6 @@ typedef struct FunctionEvaluationContext /* private function declarations */ -static Node * PartiallyEvaluateExpression(Node *expression, PlanState *planState); static Node * EvaluateNodeIfReferencesFunction(Node *expression, PlanState *planState); static Node * PartiallyEvaluateExpressionMutator(Node *expression, FunctionEvaluationContext *context); @@ -162,7 +161,7 @@ ExecuteMasterEvaluableFunctions(Query *query, PlanState *planState) * Walks the expression evaluating any node which invokes a function as long as a Var * doesn't show up in the parameter list. */ -static Node * +Node * PartiallyEvaluateExpression(Node *expression, PlanState *planState) { FunctionEvaluationContext globalContext = { planState, false }; diff --git a/src/include/distributed/citus_clauses.h b/src/include/distributed/citus_clauses.h index 5612fc3b2..4bd619000 100644 --- a/src/include/distributed/citus_clauses.h +++ b/src/include/distributed/citus_clauses.h @@ -17,5 +17,6 @@ extern bool RequiresMasterEvaluation(Query *query); extern void ExecuteMasterEvaluableFunctions(Query *query, PlanState *planState); +extern Node * PartiallyEvaluateExpression(Node *expression, PlanState *planState); #endif /* CITUS_CLAUSES_H */ diff --git a/src/include/distributed/multi_logical_optimizer.h b/src/include/distributed/multi_logical_optimizer.h index 31fd56408..9c3c137fe 100644 --- a/src/include/distributed/multi_logical_optimizer.h +++ b/src/include/distributed/multi_logical_optimizer.h @@ -16,6 +16,7 @@ #include "distributed/master_metadata_utility.h" #include "distributed/multi_logical_planner.h" +#include "distributed/relation_restriction_equivalence.h" /* Definitions local to logical plan optimizer */ diff --git a/src/include/distributed/multi_logical_planner.h b/src/include/distributed/multi_logical_planner.h index 48dba38ca..52c676b4b 100644 --- a/src/include/distributed/multi_logical_planner.h +++ b/src/include/distributed/multi_logical_planner.h @@ -16,6 +16,7 @@ #include "distributed/citus_nodes.h" #include "distributed/multi_join_order.h" +#include "distributed/relation_restriction_equivalence.h" #include "nodes/nodes.h" #include "nodes/primnodes.h" #include "nodes/parsenodes.h" @@ -24,7 +25,7 @@ #define SUBQUERY_RANGE_TABLE_ID -1 #define SUBQUERY_RELATION_ID 10000 -#define HEAP_ANALYTICS_SUBQUERY_RELATION_ID 10001 +#define SUBQUERY_PUSHDOWN_RELATION_ID 10001 /* @@ -180,7 +181,9 @@ extern bool SubqueryPushdown; /* Function declarations for building logical plans */ -extern MultiTreeRoot * MultiLogicalPlanCreate(Query *queryTree); +extern MultiTreeRoot * MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree, + PlannerRestrictionContext * + plannerRestrictionContext); extern bool NeedsDistributedPlanning(Query *queryTree); extern MultiNode * ParentNode(MultiNode *multiNode); extern MultiNode * ChildNode(MultiUnaryNode *multiNode); diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 0c701c479..299ba46f7 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -22,6 +22,7 @@ #include "distributed/errormessage.h" #include "distributed/master_metadata_utility.h" #include "distributed/multi_logical_planner.h" +#include "distributed/multi_planner.h" #include "lib/stringinfo.h" #include "nodes/parsenodes.h" #include "utils/array.h" @@ -248,7 +249,9 @@ extern bool EnableUniqueJobIds; /* Function declarations for building physical plans and constructing queries */ -extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree); +extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree, + PlannerRestrictionContext * + plannerRestrictionContext); extern StringInfo ShardFetchQueryString(uint64 shardId); extern Task * CreateBasicTask(uint64 jobId, uint32 taskId, TaskType taskType, char *queryString); diff --git a/src/include/distributed/multi_planner.h b/src/include/distributed/multi_planner.h index 0575834c9..952f2fa53 100644 --- a/src/include/distributed/multi_planner.h +++ b/src/include/distributed/multi_planner.h @@ -71,8 +71,6 @@ typedef struct RelationShard extern PlannedStmt * multi_planner(Query *parse, int cursorOptions, ParamListInfo boundParams); - -struct MultiPlan; extern struct MultiPlan * GetMultiPlan(CustomScan *node); extern void multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo, Index index, RangeTblEntry *rte); diff --git a/src/include/distributed/multi_router_planner.h b/src/include/distributed/multi_router_planner.h index 589724464..0dd0f1e7b 100644 --- a/src/include/distributed/multi_router_planner.h +++ b/src/include/distributed/multi_router_planner.h @@ -31,11 +31,18 @@ extern MultiPlan * CreateRouterPlan(Query *originalQuery, Query *query, extern MultiPlan * CreateModifyPlan(Query *originalQuery, Query *query, PlannerRestrictionContext * plannerRestrictionContext); - +extern bool RouterSelectQuery(Query *originalQuery, + RelationRestrictionContext *restrictionContext, + List **placementList, uint64 *anchorShardId, + List **relationShardList, bool replacePrunedQueryWithDummy); extern DeferredErrorMessage * ModifyQuerySupported(Query *queryTree); extern Query * ReorderInsertSelectTargetLists(Query *originalQuery, RangeTblEntry *insertRte, RangeTblEntry *subqueryRte); +extern List * ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex); +extern RelationRestrictionContext * CopyRelationRestrictionContext( + RelationRestrictionContext *oldContext); + extern bool InsertSelectQuery(Query *query); extern Oid ExtractFirstDistributedTableId(Query *query); extern RangeTblEntry * ExtractSelectRangeTableEntry(Query *query); diff --git a/src/include/distributed/relation_restriction_equivalence.h b/src/include/distributed/relation_restriction_equivalence.h index be7c25674..f0fcd33e9 100644 --- a/src/include/distributed/relation_restriction_equivalence.h +++ b/src/include/distributed/relation_restriction_equivalence.h @@ -15,8 +15,11 @@ #include "distributed/multi_planner.h" +extern bool ContainsUnionSubquery(Query *queryTree); extern bool RestrictionEquivalenceForPartitionKeys(PlannerRestrictionContext * plannerRestrictionContext); +extern bool SafeToPushdownUnionSubquery(RelationRestrictionContext *restrictionContext); +extern List * RelationIdList(Query *query); #endif /* RELATION_RESTRICTION_EQUIVALENCE_H */ diff --git a/src/test/regress/expected/.gitignore b/src/test/regress/expected/.gitignore index 9711f0c5e..7d87f6e93 100644 --- a/src/test/regress/expected/.gitignore +++ b/src/test/regress/expected/.gitignore @@ -17,3 +17,4 @@ /worker_copy.out /multi_complex_count_distinct.out /multi_mx_copy_data.out +/multi_insert_select_behavioral_analytics_create_table.out diff --git a/src/test/regress/expected/multi_complex_expressions.out b/src/test/regress/expected/multi_complex_expressions.out index 3723cff0a..0914adb6f 100644 --- a/src/test/regress/expected/multi_complex_expressions.out +++ b/src/test/regress/expected/multi_complex_expressions.out @@ -348,16 +348,14 @@ ERROR: cannot perform local joins that involve expressions DETAIL: local joins can be performed between columns only -- Check that we can issue limit/offset queries -- OFFSET in subqueries are not supported --- Error in the planner when subquery pushdown is off +-- Error in the planner when single repartition subquery +SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries with offset are not supported yet +-- Error in the optimizer when subquery pushdown is on SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq; ERROR: cannot perform distributed planning on this query DETAIL: Subqueries with offset are not supported yet -SET citus.subquery_pushdown TO true; --- Error in the optimizer when subquery pushdown is on -SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq; -ERROR: cannot push down this subquery -DETAIL: Offset clause is currently unsupported -SET citus.subquery_pushdown TO false; -- Simple LIMIT/OFFSET with ORDER BY SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20; o_orderkey diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index 8038e5834..d099a635c 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -963,6 +963,50 @@ DETAIL: Select query cannot be pushed down to the worker. GROUP BY outer_most.id; ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + -- if the given filter was on value_1 as shown in the above, Citus could + -- push it down. But here the query is refused + INSERT INTO agg_events + (user_id) + SELECT raw_events_first.user_id + FROM raw_events_first, + raw_events_second + WHERE raw_events_second.user_id = raw_events_first.value_1 + AND raw_events_first.value_2 = 12; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + -- lets do some unsupported query tests with subqueries + -- foo is not joined on the partition key so the query is not + -- pushed down + INSERT INTO agg_events + (user_id, value_4_agg) + SELECT + outer_most.id, max(outer_most.value) + FROM + ( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first LEFT JOIN + reference_table + ON (raw_events_first.value_1 = reference_table.user_id)) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 + ON (f.id = f2.id)) as outer_most + GROUP BY + outer_most.id; +ERROR: cannot perform distributed planning for the given modification DETAIL: Select query cannot be pushed down to the worker. INSERT INTO agg_events (value_4_agg, diff --git a/src/test/regress/expected/multi_insert_select_non_pushable_queries.out b/src/test/regress/expected/multi_insert_select_non_pushable_queries.out index e3c9c0d6b..ef0ec218b 100644 --- a/src/test/regress/expected/multi_insert_select_non_pushable_queries.out +++ b/src/test/regress/expected/multi_insert_select_non_pushable_queries.out @@ -660,3 +660,30 @@ FROM WHERE users_table.value_1 < 50; ERROR: cannot perform distributed planning for the given modification DETAIL: Select query cannot be pushed down to the worker. +-- not supported since one of the queries doesn't have a relation +INSERT INTO agg_results (user_id, agg_time, value_2_agg) +SELECT + user_id, + user_lastseen, + array_length(event_array, 1) +FROM ( + SELECT + user_id, + max(u.time) as user_lastseen, + array_agg(event_type ORDER BY u.time) AS event_array + FROM ( + SELECT user_id, time, value_3 as val_3 + FROM users_table + WHERE + user_id >= 10 AND user_id <= 70 AND + users_table.value_1 > 10 AND users_table.value_1 < 12 + ) u LEFT JOIN LATERAL ( + SELECT event_type, time + FROM events_table, (SELECT 1 as x) as f + WHERE user_id = u.user_id AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) t ON true + GROUP BY user_id +) AS shard_union +ORDER BY user_lastseen DESC; +ERROR: Subqueries without relations are not allowed in INSERT ... SELECT queries diff --git a/src/test/regress/expected/multi_mx_router_planner.out b/src/test/regress/expected/multi_mx_router_planner.out index 74b3c0b24..37f3a8718 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -802,8 +802,8 @@ SELECT * FROM ( (SELECT * FROM articles_hash_mx WHERE author_id = 1) UNION (SELECT * FROM articles_hash_mx WHERE author_id = 2)) uu; -ERROR: cannot perform distributed planning on this query -DETAIL: Subqueries without group by clause are not supported yet +ERROR: cannot push down this subquery +DETAIL: Currently all leaf queries need to have same filters on partition column -- error out for queries with repartition jobs SELECT * FROM articles_hash_mx a, articles_hash_mx b diff --git a/src/test/regress/expected/multi_router_planner.out b/src/test/regress/expected/multi_router_planner.out index d12ca62fd..c2996c6cb 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -916,8 +916,8 @@ SELECT * FROM ( (SELECT * FROM articles_hash WHERE author_id = 1) UNION (SELECT * FROM articles_hash WHERE author_id = 2)) uu; -ERROR: cannot perform distributed planning on this query -DETAIL: Subqueries without group by clause are not supported yet +ERROR: cannot push down this subquery +DETAIL: Currently all leaf queries need to have same filters on partition column -- error out for queries with repartition jobs SELECT * FROM articles_hash a, articles_hash b diff --git a/src/test/regress/expected/multi_subquery_behavioral_analytics.out b/src/test/regress/expected/multi_subquery_behavioral_analytics.out new file mode 100644 index 000000000..4340d115b --- /dev/null +++ b/src/test/regress/expected/multi_subquery_behavioral_analytics.out @@ -0,0 +1,1954 @@ +-- +-- multi subquery behavioral analytics queries aims to expand existing subquery pushdown +-- regression tests to cover more cases +-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- +-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests +-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1430000; +-- ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1430000; +SET citus.enable_router_execution TO FALSE; +------------------------------------ +-- Vanilla funnel query +------------------------------------ +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT u.user_id, e.event_type::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1; + user_id | array_length +---------+-------------- + 13 | 172 + 12 | 121 + 23 | 115 + 10 | 114 + 20 | 90 +(5 rows) + +------------------------------------ +-- Funnel grouped by whether or not a user has done an event +-- This has multiple subqueries joinin at the top level +------------------------------------ +SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event +FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + ( + SELECT u.user_id, 'step=>1'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102) + ) + UNION + ( + SELECT u.user_id, 'step=>2'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (103, 104, 105) + ) + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event +) t GROUP BY user_id, hasdone_event +ORDER BY user_id; + user_id | sum | length | hasdone_event +---------+-----+--------+-------------------- + 10 | 1 | 18 | Has not done event + 12 | 1 | 14 | Has done event + 13 | 2 | 18 | Has not done event + 15 | 1 | 18 | Has not done event + 17 | 1 | 18 | Has not done event + 19 | 1 | 14 | Has done event + 20 | 2 | 18 | Has not done event + 23 | 1 | 18 | Has not done event +(8 rows) + +-- same query but multiple joins are one level below, returns count of row instead of actual rows +SELECT count(*) +FROM ( + SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event + FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + ( + SELECT u.user_id, 'step=>1'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102) + ) + UNION + ( + SELECT u.user_id, 'step=>2'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (103, 104, 105) + ) + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event + ) t GROUP BY user_id, hasdone_event + ORDER BY user_id) u; + count +------- + 8 +(1 row) + +-- Same queries written without unions +SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event +FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + SELECT + u.user_id, + CASE WHEN e.event_type IN (100, 101, 102) THEN 'step=>1'::text else 'step==>2'::text END AS event, + e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102, 103, 104, 105) + GROUP BY 1,2,3 + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event +) t GROUP BY user_id, hasdone_event +ORDER BY user_id; + user_id | sum | length | hasdone_event +---------+-----+--------+-------------------- + 10 | 1 | 18 | Has not done event + 12 | 1 | 14 | Has done event + 13 | 2 | 18 | Has not done event + 15 | 1 | 18 | Has not done event + 17 | 1 | 18 | Has not done event + 19 | 1 | 14 | Has done event + 20 | 2 | 18 | Has not done event + 23 | 1 | 18 | Has not done event +(8 rows) + +-- same query but multiple joins are one level below, returns count of row instead of actual rows +SELECT count(*) +FROM ( + SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event + FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + SELECT + u.user_id, + CASE WHEN e.event_type in (100, 101, 102) then 'step=>1'::text else 'step==>2'::text END AS event, + e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102, 103, 104, 105) + GROUP BY 1,2,3 + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event + ) t GROUP BY user_id, hasdone_event + ORDER BY user_id) u; + count +------- + 8 +(1 row) + +------------------------------------ +-- Funnel, grouped by the number of times a user has done an event +------------------------------------ +SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + (SELECT + users_table.user_id, + 'action=>1'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) + UNION + (SELECT + users_table.user_id, + 'action=>2'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 12 AND events_table.event_type < 14 + ) + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + user_id | event_average | count_pay +---------+------------------------+----------- + 69 | 1.00000000000000000000 | 0 + 65 | 1.00000000000000000000 | 0 + 58 | 1.00000000000000000000 | 0 + 49 | 1.00000000000000000000 | 0 + 40 | 1.00000000000000000000 | 0 + 32 | 1.00000000000000000000 | 0 + 29 | 1.00000000000000000000 | 0 + 18 | 1.00000000000000000000 | 0 +(8 rows) + +SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + (SELECT + users_table.user_id, + 'action=>1'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) + UNION + (SELECT + users_table.user_id, + 'action=>2'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 12 AND events_table.event_type < 14 + ) + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +HAVING + avg(array_length(events_table, 1)) > 0 +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + user_id | event_average | count_pay +---------+------------------------+----------- + 69 | 1.00000000000000000000 | 0 + 65 | 1.00000000000000000000 | 0 + 58 | 1.00000000000000000000 | 0 + 49 | 1.00000000000000000000 | 0 + 40 | 1.00000000000000000000 | 0 + 32 | 1.00000000000000000000 | 0 + 29 | 1.00000000000000000000 | 0 + 18 | 1.00000000000000000000 | 0 +(8 rows) + +-- Same queries rewritten without using unions + SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + SELECT + users_table.user_id, + CASE + WHEN + events_table.event_type > 10 AND events_table.event_type < 12 + THEN 'action=>1' + ELSE 'action=>2' + END AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + (events_table.event_type > 10 AND events_table.event_type < 12 + OR + events_table.event_type > 12 AND events_table.event_type < 14) + GROUP BY 1, 2, 3 + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + user_id | event_average | count_pay +---------+------------------------+----------- + 69 | 1.00000000000000000000 | 0 + 65 | 1.00000000000000000000 | 0 + 58 | 1.00000000000000000000 | 0 + 49 | 1.00000000000000000000 | 0 + 40 | 1.00000000000000000000 | 0 + 32 | 1.00000000000000000000 | 0 + 29 | 1.00000000000000000000 | 0 + 18 | 1.00000000000000000000 | 0 +(8 rows) + +SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + SELECT + users_table.user_id, + CASE WHEN events_table.event_type > 10 AND events_table.event_type < 12 THEN 'action=>1' ELSE 'action=>2' END AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + (events_table.event_type > 10 AND events_table.event_type < 12 + OR + events_table.event_type > 12 AND events_table.event_type < 14) + GROUP BY 1, 2, 3 + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +HAVING + avg(array_length(events_table, 1)) > 0 +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + user_id | event_average | count_pay +---------+------------------------+----------- + 69 | 1.00000000000000000000 | 0 + 65 | 1.00000000000000000000 | 0 + 58 | 1.00000000000000000000 | 0 + 49 | 1.00000000000000000000 | 0 + 40 | 1.00000000000000000000 | 0 + 32 | 1.00000000000000000000 | 0 + 29 | 1.00000000000000000000 | 0 + 18 | 1.00000000000000000000 | 0 +(8 rows) + +------------------------------------ +-- Most recently seen users_table events_table +------------------------------------ +-- Note that we don't use ORDER BY/LIMIT yet +------------------------------------ +SELECT + user_id, + user_lastseen, + array_length(event_array, 1) +FROM ( + SELECT + user_id, + max(u.time) as user_lastseen, + array_agg(event_type ORDER BY u.time) AS event_array + FROM ( + SELECT user_id, time + FROM users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 10 AND users_table.value_1 < 12 + ) u LEFT JOIN LATERAL ( + SELECT event_type, time + FROM events_table + WHERE user_id = u.user_id AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) t ON true + GROUP BY user_id +) AS shard_union +ORDER BY user_lastseen DESC, user_id; + user_id | user_lastseen | array_length +---------+---------------------------------+-------------- + 12 | Sun Jan 19 01:49:20.372688 2014 | 1 + 20 | Sat Jan 18 14:25:31.817903 2014 | 1 + 42 | Thu Jan 16 07:08:02.651966 2014 | 1 + 56 | Tue Jan 14 12:11:47.27375 2014 | 1 + 57 | Mon Jan 13 14:53:50.494836 2014 | 1 + 65 | Sun Jan 12 03:14:26.810597 2014 | 1 +(6 rows) + +------------------------------------ +-- Count the number of distinct users_table who are in segment X and Y and Z +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT DISTINCT user_id +FROM users_table +WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20) + AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40) + AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60); +ERROR: could not run distributed query with join types other than INNER or OUTER JOINS +HINT: Consider joining tables on partition column and have equal filter on joining columns. +------------------------------------ +-- Find customers who have done X, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_1 > 101 AND value_1 < 110 + AND value_2 >= 5 + AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 101 AND event_type < 110 AND value_3 > 100 AND user_id = users_table.user_id); +ERROR: could not run distributed query with join types other than INNER or OUTER JOINS +HINT: Consider joining tables on partition column and have equal filter on joining columns. +------------------------------------ +-- Customers who haven’t done X, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_1 = 101 + AND value_2 >= 5 + AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND user_id = users_table.user_id); +ERROR: could not run distributed query with subquery outside the FROM clause +HINT: Consider using an equality filter on the distributed table's partition column. +------------------------------------ +-- Customers who have done X and Y, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_1 > 100 + AND value_2 >= 5 + AND EXISTS (SELECT user_id FROM events_table WHERE event_type != 100 AND value_3 > 100 AND user_id = users_table.user_id) + AND EXISTS (SELECT user_id FROM events_table WHERE event_type = 101 AND value_3 > 100 AND user_id = users_table.user_id); +ERROR: could not run distributed query with join types other than INNER or OUTER JOINS +HINT: Consider joining tables on partition column and have equal filter on joining columns. +------------------------------------ +-- Customers who have done X and haven’t done Y, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_2 >= 5 + AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id) + AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id); +ERROR: could not run distributed query with join types other than INNER or OUTER JOINS +HINT: Consider joining tables on partition column and have equal filter on joining columns. +------------------------------------ +-- Customers who have done X more than 2 times, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, + value_2 + FROM users_table + WHERE value_1 > 100 + AND value_1 < 124 + AND value_2 >= 5 + AND EXISTS (SELECT user_id + FROM events_table + WHERE event_type > 100 + AND event_type < 124 + AND value_3 > 100 + AND user_id = users_table.user_id + GROUP BY user_id + HAVING Count(*) > 2); +ERROR: could not run distributed query with subquery outside the FROM clause +HINT: Consider using an equality filter on the distributed table's partition column. +------------------------------------ +-- Find me all users_table who logged in more than once +------------------------------------ +SELECT user_id, value_1 from +( + SELECT + user_id, value_1 From users_table + WHERE + value_2 > 100 and user_id = 15 + GROUP BY + value_1, user_id + HAVING + count(*) > 1 +) AS a +ORDER BY + user_id ASC, value_1 ASC; + user_id | value_1 +---------+--------- + 15 | 212 + 15 | 230 + 15 | 417 + 15 | 490 + 15 | 529 + 15 | 926 +(6 rows) + +-- same query with additional filter to make it not router plannable +SELECT user_id, value_1 from +( + SELECT + user_id, value_1 From users_table + WHERE + value_2 > 100 and (user_id = 15 OR user_id = 16) + GROUP BY + value_1, user_id + HAVING count(*) > 1 +) AS a +ORDER BY + user_id ASC, value_1 ASC; + user_id | value_1 +---------+--------- + 15 | 212 + 15 | 230 + 15 | 417 + 15 | 490 + 15 | 529 + 15 | 926 + 16 | 339 + 16 | 485 + 16 | 717 + 16 | 903 +(10 rows) + +------------------------------------ +-- Find me all users_table who has done some event and has filters +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id +FROM events_table +WHERE + event_type = 16 AND value_2 > 50 AND + user_id IN + (SELECT + user_id + FROM + users_table + WHERE + value_1 = 15 AND value_2 > 25 + ); +ERROR: could not run distributed query with join types other than INNER or OUTER JOINS +HINT: Consider joining tables on partition column and have equal filter on joining columns. +------------------------------------ +-- Which events_table did people who has done some specific events_table +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, event_type FROM events_table +WHERE user_id in (SELECT user_id from events_table WHERE event_type > 500 and event_type < 505) +GROUP BY user_id, event_type; +ERROR: could not run distributed query with join types other than INNER or OUTER JOINS +HINT: Consider joining tables on partition column and have equal filter on joining columns. +------------------------------------ +-- Find me all the users_table who has done some event more than three times +------------------------------------ +SELECT user_id FROM +( + SELECT + user_id + FROM + events_table + WHERE + event_type = 901 + GROUP BY + user_id + HAVING + count(*) > 3 +) AS a +ORDER BY + user_id; + user_id +--------- + 57 +(1 row) + +------------------------------------ +-- Find my assets that have the highest probability and fetch their metadata +------------------------------------ +CREATE TEMP TABLE assets AS +SELECT + users_table.user_id, users_table.value_1, prob +FROM + users_table + JOIN + (SELECT + ma.user_id, (GREATEST(coalesce(ma.value_4 / 250, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma, events_table as short_list + WHERE + short_list.user_id = ma.user_id and ma.value_1 < 50 and short_list.event_type < 50 + ) temp + ON users_table.user_id = temp.user_id + WHERE + users_table.value_1 < 50; + -- get some statistics from the aggregated results to ensure the results are correct +SELECT count(*), count(DISTINCT user_id), avg(user_id) FROM assets; + count | count | avg +-------+-------+--------------------- + 14371 | 101 | 50.5232064574490293 +(1 row) + +DROP TABLE assets; +-- count number of distinct users who have value_1 equal to 5 or 13 but not 3 +-- original query that fails +SELECT count(*) FROM +( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') AND + user_id NOT IN (select user_id from users_table where value_1 = '3') + GROUP BY + user_id + HAVING + count(distinct value_1) = 2 +) as foo; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- previous push down query +SELECT subquery_count FROM + (SELECT count(*) as subquery_count FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY + user_id + HAVING + count(distinct value_1) = 2) as a + LEFT JOIN + (SELECT + user_id + FROM + users_table + WHERE + (value_1 = '3') + GROUP BY + user_id) as b + ON a.user_id = b.user_id + WHERE + b.user_id IS NULL + GROUP BY + a.user_id + ) AS inner_subquery; + subquery_count +---------------- + 1 +(1 row) + +-- new pushdown query without single range table entry at top requirement +SELECT count(*) as subquery_count +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY + user_id + HAVING + count(distinct value_1) = 2 + ) as a + LEFT JOIN ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '3') + GROUP BY + user_id) AS b + ON a.user_id = b.user_id +WHERE + b.user_id IS NULL +GROUP BY + a.user_id; + subquery_count +---------------- + 1 +(1 row) + +-- most queries below has limit clause +-- therefore setting subquery_pushdown flag for all +SET citus.subquery_pushdown to ON; +-- multi-subquery-join +-- The first query has filters on partion column to make it router plannable +-- but it is processed by logical planner since we disabled router execution +SELECT + e1.user_id, + sum(view_homepage) AS viewed_homepage, + sum(use_demo) AS use_demo, + sum(enter_credit_card) AS entered_credit_card, + sum(submit_card_info) as submit_card_info, + sum(see_bought_screen) as see_bought_screen +FROM ( + -- Get the first time each user viewed the homepage. + SELECT + user_id, + 1 AS view_homepage, + min(time) AS view_homepage_time + FROM events_table + WHERE user_id = 1 and + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90) + GROUP BY user_id +) e1 LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS use_demo, + time AS use_demo_time + FROM events_table + WHERE + user_id = e1.user_id AND user_id = 1 and + event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91) + ORDER BY time + LIMIT 1 +) e2 ON true LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS enter_credit_card, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e2.user_id AND user_id = 1 and + event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92) + ORDER BY time + LIMIT 1 +) e3 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS submit_card_info, + user_id, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e3.user_id AND user_id = 1 and + event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93) + ORDER BY time + LIMIT 1 +) e4 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS see_bought_screen + FROM events_table + WHERE + user_id = e4.user_id AND user_id = 1 and + event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94) + ORDER BY time + LIMIT 1 +) e5 ON true +WHERE + e1.user_id = 1 +GROUP BY + e1.user_id +LIMIT 1; + user_id | viewed_homepage | use_demo | entered_credit_card | submit_card_info | see_bought_screen +---------+-----------------+----------+---------------------+------------------+------------------- + 1 | 1 | | | | +(1 row) + +-- Same query without all limitations +SELECT + e1.user_id, + sum(view_homepage) AS viewed_homepage, + sum(use_demo) AS use_demo, + sum(enter_credit_card) AS entered_credit_card, + sum(submit_card_info) as submit_card_info, + sum(see_bought_screen) as see_bought_screen +FROM ( + -- Get the first time each user viewed the homepage. + SELECT + user_id, + 1 AS view_homepage, + min(time) AS view_homepage_time + FROM events_table + WHERE + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90) + GROUP BY user_id +) e1 LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS use_demo, + time AS use_demo_time + FROM events_table + WHERE + user_id = e1.user_id AND + event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91) + ORDER BY time +) e2 ON true LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS enter_credit_card, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e2.user_id AND + event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92) + ORDER BY time +) e3 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS submit_card_info, + user_id, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e3.user_id AND + event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93) + ORDER BY time +) e4 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS see_bought_screen + FROM events_table + WHERE + user_id = e4.user_id AND + event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94) + ORDER BY time +) e5 ON true +GROUP BY e1.user_id +ORDER BY 6 DESC NULLS LAST, 5 DESC NULLS LAST, 4 DESC NULLS LAST, 3 DESC NULLS LAST, 2 DESC NULLS LAST, 1 +LIMIT 15; + user_id | viewed_homepage | use_demo | entered_credit_card | submit_card_info | see_bought_screen +---------+-----------------+----------+---------------------+------------------+------------------- + 72 | 36 | 36 | 36 | 36 | 36 + 95 | 12 | 12 | 12 | 12 | 12 + 82 | 4 | 4 | 4 | 4 | 4 + 74 | 3 | 3 | 3 | 3 | 3 + 83 | 3 | 3 | 3 | 3 | 3 + 6 | 2 | 2 | 2 | 2 | 2 + 42 | 1 | 1 | 1 | 1 | 1 + 5 | 4 | 4 | 4 | 4 | + 93 | 4 | 4 | 4 | 4 | + 51 | 1 | 1 | 1 | 1 | + 85 | 6 | 6 | 6 | | + 73 | 4 | 4 | 4 | | + 0 | 3 | 3 | 3 | | + 10 | 2 | 2 | 2 | | + 13 | 2 | 2 | 2 | | +(15 rows) + +-- Same query without all limitations but uses having() to show only those submitted their credit card info +SELECT + e1.user_id, + sum(view_homepage) AS viewed_homepage, + sum(use_demo) AS use_demo, + sum(enter_credit_card) AS entered_credit_card, + sum(submit_card_info) as submit_card_info, + sum(see_bought_screen) as see_bought_screen +FROM ( + -- Get the first time each user viewed the homepage. + SELECT + user_id, + 1 AS view_homepage, + min(time) AS view_homepage_time + FROM events_table + WHERE + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90) + GROUP BY user_id +) e1 LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS use_demo, + time AS use_demo_time + FROM events_table + WHERE + user_id = e1.user_id AND + event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91) + ORDER BY time +) e2 ON true LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS enter_credit_card, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e2.user_id AND + event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92) + ORDER BY time +) e3 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS submit_card_info, + user_id, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e3.user_id AND + event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93) + ORDER BY time +) e4 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS see_bought_screen + FROM events_table + WHERE + user_id = e4.user_id AND + event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94) + ORDER BY time +) e5 ON true +group by e1.user_id +HAVING sum(submit_card_info) > 0 +ORDER BY 6 DESC NULLS LAST, 5 DESC NULLS LAST, 4 DESC NULLS LAST, 3 DESC NULLS LAST, 2 DESC NULLS LAST, 1 +LIMIT 15; + user_id | viewed_homepage | use_demo | entered_credit_card | submit_card_info | see_bought_screen +---------+-----------------+----------+---------------------+------------------+------------------- + 72 | 36 | 36 | 36 | 36 | 36 + 95 | 12 | 12 | 12 | 12 | 12 + 82 | 4 | 4 | 4 | 4 | 4 + 74 | 3 | 3 | 3 | 3 | 3 + 83 | 3 | 3 | 3 | 3 | 3 + 6 | 2 | 2 | 2 | 2 | 2 + 42 | 1 | 1 | 1 | 1 | 1 + 5 | 4 | 4 | 4 | 4 | + 93 | 4 | 4 | 4 | 4 | + 51 | 1 | 1 | 1 | 1 | +(10 rows) + +-- Explain analyze on this query fails due to #756 +-- avg expression used on order by +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN ( + SELECT + user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3)) AS b +ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +ORDER BY + avg(b.value_3), 2, 1 +LIMIT 5; + user_id | subquery_avg +---------+---------------------- + 99 | 456.7446808510638298 + 83 | 469.6037735849056604 + 61 | 486.5869565217391304 + 78 | 434.9009009009009009 + 77 | 449.9313725490196078 +(5 rows) + +-- add having to the same query +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY user_id + HAVING count(distinct value_1) > 88 + ) as a + LEFT JOIN ( + SELECT + user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3)) AS b +ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +HAVING + sum(b.value_3) > 50000 +ORDER BY + avg(b.value_3), 2, 1 +LIMIT 5; + user_id | subquery_avg +---------+---------------------- + 78 | 434.9009009009009009 + 29 | 505.0934579439252336 + 17 | 526.9633027522935780 + 91 | 501.4339622641509434 + 24 | 515.1714285714285714 +(5 rows) + +-- avg on the value_3 is not a resjunk +SELECT a.user_id, avg(b.value_2) as subquery_avg, avg(b.value_3) +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + ( + SELECT + user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ) AS b + ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +ORDER BY + avg(b.value_3) DESC, 2, 1 +LIMIT 5; + user_id | subquery_avg | avg +---------+----------------------+------------------ + 6 | 523.8247422680412371 | 569.226804123711 + 62 | 497.1545454545454545 | 567.681818181818 + 8 | 524.5894736842105263 | 565.2 + 10 | 502.2017543859649123 | 561.929824561404 + 16 | 467.5145631067961165 | 561.73786407767 +(5 rows) + +-- a powerful query structure that analyzes users/events +-- using (relation JOIN subquery JOIN relation) +SELECT u.user_id, sub.value_2, sub.value_3, COUNT(e2.user_id) counts +FROM + users_table u + LEFT OUTER JOIN LATERAL + (SELECT + * + FROM + events_table e1 + WHERE + e1.user_id = u.user_id + ORDER BY + e1.value_3 DESC + LIMIT 1 + ) sub + ON true + LEFT OUTER JOIN events_table e2 + ON e2.user_id = sub.user_id +WHERE + e2.value_2 > 10 AND e2.value_2 < 50 AND u.value_2 > 10 AND u.value_2 < 50 +GROUP BY + u.user_id, sub.value_2, sub.value_3 +ORDER BY + 4 DESC, 1 DESC, 2 ASC, 3 ASC +LIMIT 10; + user_id | value_2 | value_3 | counts +---------+---------+---------+-------- + 87 | 807 | 990 | 45 + 25 | 613 | 992 | 40 + 26 | 952 | 982 | 36 + 17 | 277 | 993 | 36 + 83 | 571 | 1000 | 35 + 99 | 309 | 998 | 32 + 96 | 571 | 987 | 30 + 95 | 631 | 997 | 30 + 82 | 444 | 997 | 28 + 57 | 975 | 989 | 25 +(10 rows) + +-- distinct users joined with events +SELECT + avg(events_table.event_type) as avg_type, + count(*) as users_count +FROM events_table + JOIN + (SELECT + DISTINCT user_id + FROM + users_table + ) as distinct_users + ON distinct_users.user_id = events_table.user_id +GROUP BY + distinct_users.user_id +ORDER BY + users_count desc, avg_type DESC +LIMIT 5; + avg_type | users_count +----------------------+------------- + 496.5748031496062992 | 127 + 531.1788617886178862 | 123 + 504.6806722689075630 | 119 + 503.7203389830508475 | 118 + 506.3793103448275862 | 116 +(5 rows) + +-- reduce the data set, aggregate and join +SELECT + events_table.event_type, + users_count.ct +FROM events_table + JOIN + (SELECT distinct_users.user_id, count(1) as ct + FROM + (SELECT + user_id + FROM + users_table + ) as distinct_users + GROUP BY + distinct_users.user_id + ) as users_count + ON users_count.user_id = events_table.user_id +ORDER BY + users_count.ct desc, event_type DESC +LIMIT 5; + event_type | ct +------------+----- + 996 | 121 + 986 | 121 + 979 | 121 + 975 | 121 + 960 | 121 +(5 rows) + +--- now, test (subquery JOIN subquery) +SELECT n1.user_id, count_1, total_count +FROM + (SELECT + user_id, count(1) as count_1 + FROM + users_table + GROUP BY + user_id + ) n1 + INNER JOIN + ( + SELECT + user_id, count(1) as total_count + FROM + events_table + GROUP BY + user_id, event_type + ) n2 + ON (n2.user_id = n1.user_id) +ORDER BY + total_count DESC, count_1 DESC, 1 DESC +LIMIT 10; + user_id | count_1 | total_count +---------+---------+------------- + 57 | 105 | 4 + 78 | 112 | 3 + 45 | 111 | 3 + 40 | 107 | 3 + 36 | 106 | 3 + 25 | 105 | 3 + 86 | 100 | 3 + 80 | 100 | 3 + 60 | 100 | 3 + 35 | 100 | 3 +(10 rows) + +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + (SELECT + DISTINCT ON (user_id) user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ORDER BY + 1,2,3 + ) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NOT NULL +GROUP BY a.user_id +ORDER BY avg(b.value_3), 2, 1 +LIMIT 5; + user_id | subquery_avg +---------+--------------------- + 10 | 5.0000000000000000 + 87 | 12.0000000000000000 + 77 | 28.0000000000000000 + 37 | 17.0000000000000000 + 11 | 3.0000000000000000 +(5 rows) + +-- distinct clause must include partition column +-- when used in target list +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + (SELECT + DISTINCT ON (value_2) value_2 , user_id, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ORDER BY + 1,2,3 + ) AS b + USING (user_id) +GROUP BY user_id; +ERROR: cannot push down this subquery +DETAIL: Distinct on columns without partition column is currently unsupported +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + (SELECT + DISTINCT ON (value_2, user_id) value_2 , user_id, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ORDER BY + 1,2,3 + ) AS b + ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +ORDER BY + avg(b.value_3), 2, 1 +LIMIT 5; + user_id | subquery_avg +---------+---------------------- + 99 | 459.1910112359550562 + 83 | 458.0721649484536082 + 9 | 541.5217391304347826 + 78 | 434.2336448598130841 + 77 | 443.8686868686868687 +(5 rows) + +SELECT user_id, event_type +FROM + (SELECT * + FROM + ( + (SELECT + event_type, user_id as a_user_id + FROM + events_table) AS a + JOIN + (SELECT + ma.user_id AS user_id, ma.value_2 AS value_2, + (GREATEST(coalesce((ma.value_3 * ma.value_2) / 20, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma + WHERE + (ma.value_2 > 100) + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS ma + ON (a.a_user_id = ma.user_id) + ) AS inner_sub + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS outer_sub +ORDER BY + prob DESC, event_type DESC, user_id DESC +LIMIT 10; + user_id | event_type +---------+------------ + 10 | 813 + 10 | 806 + 10 | 805 + 10 | 685 + 10 | 591 + 10 | 442 + 10 | 333 + 10 | 317 + 10 | 244 + 10 | 169 +(10 rows) + +-- very similar query but produces different result due to +-- ordering difference in the previous one's inner query +SELECT user_id, event_type +FROM + (SELECT + event_type, user_id as a_user_id + FROM + events_table) AS a + JOIN + (SELECT + ma.user_id AS user_id, ma.value_2 AS value_2, + (GREATEST(coalesce((ma.value_3 * ma.value_2) / 20, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma + WHERE + (ma.value_2 > 100) + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS ma + ON (a.a_user_id = ma.user_id) +ORDER BY + prob DESC, event_type DESC, user_id DESC +LIMIT 10; + user_id | event_type +---------+------------ + 10 | 998 + 10 | 996 + 10 | 981 + 10 | 975 + 10 | 962 + 10 | 945 + 10 | 945 + 10 | 933 + 10 | 932 + 10 | 915 +(10 rows) + +-- now they produce the same result when ordering fixed in 'outer_sub' +SELECT user_id, event_type +FROM + (SELECT * + FROM + ( + (SELECT + event_type, user_id as a_user_id + FROM + events_table + ) AS a + JOIN + (SELECT + ma.user_id AS user_id, ma.value_2 AS value_2, + (GREATEST(coalesce((ma.value_3 * ma.value_2) / 20, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma + WHERE + (ma.value_2 > 100) + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS ma + ON (a.a_user_id = ma.user_id) + ) AS inner_sub + ORDER BY + prob DESC, event_type DESC, user_id DESC + LIMIT 10 + ) AS outer_sub +ORDER BY + prob DESC, event_type DESC, user_id DESC +LIMIT 10; + user_id | event_type +---------+------------ + 10 | 998 + 10 | 996 + 10 | 981 + 10 | 975 + 10 | 962 + 10 | 945 + 10 | 945 + 10 | 933 + 10 | 932 + 10 | 915 +(10 rows) + +-- this is one complex join query derived from a user's production query +-- first declare the function on workers on master +-- With array_index: +SELECT * FROM run_command_on_workers('CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) + RETURNS INT AS $$ + SELECT i + FROM (SELECT generate_series(array_lower($1, 1), array_upper($1, 1))) g(i) + WHERE $1 [i] = $2 + LIMIT 1; + $$ LANGUAGE sql') +ORDER BY 1,2; + nodename | nodeport | success | result +-----------+----------+---------+----------------- + localhost | 57637 | t | CREATE FUNCTION + localhost | 57638 | t | CREATE FUNCTION +(2 rows) + +CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) + RETURNS INT AS $$ + SELECT i + FROM (SELECT generate_series(array_lower($1, 1), array_upper($1, 1))) g(i) + WHERE $1 [i] = $2 + LIMIT 1; + $$ LANGUAGE sql; +SELECT * +FROM + (SELECT * + FROM ( + (SELECT user_id AS user_id_e, + event_type AS event_type_e + FROM events_table ) AS ma_e + JOIN + (SELECT value_2, + value_3, + user_id + FROM + (SELECT * + FROM ( + (SELECT user_id_p AS user_id + FROM + (SELECT * + FROM ( + (SELECT + user_id AS user_id_p + FROM + events_table + WHERE + (event_type IN (1,2,3,4,5)) ) AS ma_p + JOIN + (SELECT + user_id AS user_id_a + FROM + users_table + WHERE + (value_2 % 5 = 1) ) AS a + ON (a.user_id_a = ma_p.user_id_p) ) ) AS a_ma_p ) AS inner_filter_q + JOIN + (SELECT + value_2, value_3, user_id AS user_id_ck + FROM + events_table + WHERE + event_type = ANY(ARRAY [10, 11, 12]) + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 ) + AS ma_ck ON (ma_ck.user_id_ck = inner_filter_q.user_id) ) + AS inner_sub_q + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 ) + AS outer_sub_q + ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10) + AS inner_search_q + ON (ma_e.user_id_e = inner_search_q.user_id) ) + AS outer_inner_sub_q + ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC, event_type_e DESC + LIMIT 10) +AS outer_outer_sub_q +ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC, event_type_e DESC +LIMIT 10; + user_id_e | event_type_e | value_2 | value_3 | user_id +-----------+--------------+---------+---------+--------- + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 +(10 rows) + +-- top level select * is removed now there is +-- a join at top level. +SELECT * +FROM + ( + (SELECT + user_id AS user_id_e, event_type as event_type_e + FROM + events_table + ) AS ma_e + JOIN + (SELECT + value_2, value_3, user_id + FROM + (SELECT + * + FROM + ( + (SELECT + user_id_p AS user_id + FROM + (SELECT + * + FROM + ( + (SELECT + user_id AS user_id_p + FROM + events_table + WHERE + (event_type IN (1, 2, 3, 4, 5)) + ) AS ma_p + JOIN + (SELECT + user_id AS user_id_a + FROM + users_table + WHERE + (value_2 % 5 = 1) + ) AS a + ON (a.user_id_a = ma_p.user_id_p) + ) + ) AS a_ma_p + ) AS inner_filter_q + JOIN + (SELECT + value_2, value_3, user_id AS user_id_ck + FROM + events_table + WHERE + event_type = ANY(ARRAY [10, 11, 12]) + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 + ) AS ma_ck + ON (ma_ck.user_id_ck = inner_filter_q.user_id) + ) AS inner_sub_q + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 + ) AS outer_sub_q + ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10) AS inner_search_q + ON (ma_e.user_id_e = inner_search_q.user_id) + ) AS outer_inner_sub_q +ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC, event_type_e DESC +LIMIT 10; + user_id_e | event_type_e | value_2 | value_3 | user_id +-----------+--------------+---------+---------+--------- + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 + 65 | 991 | 167 | 108 | 65 +(10 rows) + +-- drop created functions +SELECT * FROM run_command_on_workers('DROP FUNCTION array_index(ANYARRAY, ANYELEMENT)') +ORDER BY 1,2; + nodename | nodeport | success | result +-----------+----------+---------+--------------- + localhost | 57637 | t | DROP FUNCTION + localhost | 57638 | t | DROP FUNCTION +(2 rows) + +DROP FUNCTION array_index(ANYARRAY, ANYELEMENT); +-- a not supported query due to constant range table entry +SELECT count(*) as subquery_count +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY user_id + HAVING count(distinct value_1) = 2 + ) as a + LEFT JOIN ( + SELECT + 1 as user_id + ) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NULL +GROUP BY a.user_id; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +-- same with INNER JOIN +SELECT count(*) as subquery_count +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY user_id + HAVING count(distinct value_1) = 2 + ) as a + INNER JOIN ( + SELECT + 1 as user_id + ) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NULL +GROUP BY a.user_id; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +-- this is slightly different, we use RTE_VALUEs here +SELECT Count(*) AS subquery_count +FROM (SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13' ) + GROUP BY + user_id + HAVING + Count(DISTINCT value_1) = 2) AS a + INNER JOIN + (SELECT + * + FROM + (VALUES (1, 'one'), (2, 'two'), (3, 'three')) AS t (user_id, letter)) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NULL +GROUP BY a.user_id; +ERROR: cannot push down this subquery +DETAIL: Table expressions other than simple relations and subqueries are currently unsupported +-- same query without LIMIT/OFFSET returns 30 rows +SET client_min_messages TO DEBUG1; +-- now, lets use a simple expression on the LIMIT and explicit coercion on the OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT 3+3 OFFSET 5::smallint; +DEBUG: push down of limit count: 11 + user_id | array_length +---------+-------------- + 23 | 115 + 46 | 115 + 10 | 114 + 96 | 113 + 73 | 111 + 91 | 107 +(6 rows) + +-- now, lets use implicit coersion in LIMIT and a simple expressions on OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT '3' OFFSET 27+2; +DEBUG: push down of limit count: 32 + user_id | array_length +---------+-------------- + 0 | 54 +(1 row) + +-- create a test function which is marked as volatile +CREATE OR REPLACE FUNCTION volatile_func_test() + RETURNS INT AS $$ + SELECT 5; + $$ LANGUAGE sql VOLATILE; +-- Citus should be able to evalute functions/row comparisons on the LIMIT/OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT volatile_func_test() + (ROW(1,2,NULL) < ROW(1,3,0))::int OFFSET volatile_func_test() + volatile_func_test(); +DEBUG: push down of limit count: 16 + user_id | array_length +---------+-------------- + 91 | 107 + 69 | 103 + 67 | 101 + 35 | 100 + 80 | 100 + 86 | 100 +(6 rows) + +-- now, lets use expressions on both the LIMIT and OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT (5 > 4)::int OFFSET + CASE + WHEN 5 != 5 THEN 27 + WHEN 1 > 5 THEN 28 + ELSE 29 + END; +DEBUG: push down of limit count: 30 + user_id | array_length +---------+-------------- + 0 | 54 +(1 row) + +-- we don't allow parameters on the LIMIT/OFFSET clauses +PREPARE parametrized_limit AS +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT u.user_id, e.event_type::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id + ) q + ORDER BY 2 DESC, 1 + LIMIT $1 OFFSET $2; + EXECUTE parametrized_limit(3,3); +ERROR: no value found for parameter 1 +PREPARE parametrized_offset AS +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT u.user_id, e.event_type::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id + ) q + ORDER BY 2 DESC, 1 + LIMIT 3 OFFSET $1; + EXECUTE parametrized_offset(3); +ERROR: no value found for parameter 1 +SET client_min_messages TO DEFAULT; +DROP FUNCTION volatile_func_test(); +SET citus.subquery_pushdown to OFF; +SET citus.enable_router_execution TO TRUE; diff --git a/src/test/regress/expected/multi_subquery_complex_queries.out b/src/test/regress/expected/multi_subquery_complex_queries.out new file mode 100644 index 000000000..fb17b1f6e --- /dev/null +++ b/src/test/regress/expected/multi_subquery_complex_queries.out @@ -0,0 +1,2512 @@ +-- +-- multi subquery complex queries aims to expand existing subquery pushdown +-- regression tests to cover more caeses +-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- +-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests +-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1400000; +ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1400000; + +SET citus.enable_router_execution TO FALSE; + -- + -- UNIONs and JOINs mixed + -- +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + types | sumofeventtype +-------+---------------- + 0 | 115 + 1 | 82 + 2 | 160 + 3 | 158 +(4 rows) + +-- same query with target entries shuffled inside UNIONs +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + types | sumofeventtype +-------+---------------- + 0 | 115 + 1 | 82 + 2 | 160 + 3 | 158 +(4 rows) + +-- not supported since events_subquery_2 doesn't have partition key on the target list +-- within the shuffled target list +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" * 2 + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since events_subquery_2 doesn't have partition key on the target list +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."value_2" as user_id + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- we can support arbitrary subqueries within UNIONs +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT + *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + ( + SELECT * FROM + ( + SELECT + max("events"."time"), + 0 AS event, + "events"."user_id" + FROM + events_table as "events", users_table as "users" + WHERE + events.user_id = users.user_id AND + event_type IN (10, 11, 12, 13, 14, 15) + GROUP BY "events"."user_id" + ) as events_subquery_5 + ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + types | sumofeventtype +-------+---------------- + 0 | 115 + 2 | 160 + 3 | 158 +(3 rows) + +-- not supported since events_subquery_5 is not joined on partition key +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT + *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + ( + SELECT * FROM + ( + SELECT + max("events"."time"), + 0 AS event, + "events"."user_id" + FROM + events_table as "events", users_table as "users" + WHERE + events.user_id = users.value_2 AND + event_type IN (10, 11, 12, 13, 14, 15) + GROUP BY "events"."user_id" + ) as events_subquery_5 + ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since the join is not equi join +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id != q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since subquery 3 includes a JOIN with non-equi join +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events", users_table as "users" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) AND users.user_id != events.user_id ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- similar query with more union statements (to enable UNION tree become larger) +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 4 AS event + FROM + events_table as "events" + WHERE + event_type IN (31, 32, 33, 34, 35, 36)) events_subquery_5) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 5 AS event + FROM + events_table as "events" + WHERE + event_type IN (37, 38, 39, 40, 41, 42)) events_subquery_6) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 6 AS event + FROM + events_table as "events" + WHERE + event_type IN (50, 51, 52, 53, 54, 55)) events_subquery_6) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT "users"."user_id" + FROM users_table as "users" + WHERE value_1 > 50 and value_1 < 70) AS t ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 115 + 1 | 82 + 2 | 160 + 3 | 158 + 4 | 117 + 5 | 98 + 6 | 167 +(7 rows) + +-- +-- UNION ALL Queries +-- +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT "users"."user_id" + FROM users_table as "users" + WHERE value_1 > 50 and value_1 < 70) AS t ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 115 + 1 | 82 + 2 | 160 + 3 | 158 +(4 rows) + +-- same query target list entries shuffled +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t +ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 115 + 1 | 82 + 2 | 160 + 3 | 158 +(4 rows) + +-- not supported since subquery 3 does not have partition key +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."value_2", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT "users"."user_id" + FROM users_table as "users" + WHERE value_1 > 50 and value_1 < 70) AS t ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since events_subquery_4 does not have partition key on the +-- target list +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" * 2 + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t +ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- union all with inner and left joins +SELECT user_id, count(*) as cnt +FROM + (SELECT first_query.user_id, random() + FROM + ( SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "first_query" +INNER JOIN + (SELECT "t"."user_id" + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + LEFT OUTER JOIN + ( + SELECT + DISTINCT "events"."user_id" as user_id + FROM + events_table as "events" + WHERE + event_type IN (35, 36, 37, 38) + GROUP BY + user_id + ) as t2 + ON (t2.user_id = t.user_id) WHERE t2.user_id is NULL) as second_query + ON ("first_query".user_id = "second_query".user_id)) as final_query +GROUP BY + user_id ORDER BY cnt DESC, user_id DESC +LIMIT 10; + user_id | cnt +---------+----- + 27 | 35 + 87 | 27 + 74 | 20 + 72 | 16 + 12 | 16 + 66 | 15 + 56 | 15 + 40 | 15 + 23 | 12 + 59 | 10 +(10 rows) + +-- not supported since the join between t and t2 is not equi join +-- union all with inner and left joins +SELECT user_id, count(*) as cnt +FROM + (SELECT first_query.user_id, random() + FROM + ( SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "first_query" +INNER JOIN + (SELECT "t"."user_id" + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + LEFT OUTER JOIN + ( + SELECT + DISTINCT "events"."user_id" as user_id + FROM + events_table as "events" + WHERE + event_type IN (35, 36, 37, 38) + GROUP BY + user_id + ) as t2 + ON (t2.user_id > t.user_id) WHERE t2.user_id is NULL) as second_query + ON ("first_query".user_id = "second_query".user_id)) as final_query +GROUP BY + user_id ORDER BY cnt DESC, user_id DESC +LIMIT 10; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. + -- + -- Union, inner join and left join + -- +SELECT user_id, count(*) as cnt +FROM + (SELECT first_query.user_id, random() + FROM + ( SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "first_query" +INNER JOIN + (SELECT "t"."user_id" + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + LEFT OUTER JOIN + ( + SELECT + DISTINCT "events"."user_id" as user_id + FROM + events_table as "events" + WHERE + event_type IN (35, 36, 37, 38) + GROUP BY + user_id + ) as t2 + ON (t2.user_id = t.user_id) WHERE t2.user_id is NULL) as second_query + ON ("first_query".user_id = "second_query".user_id)) as final_query +GROUP BY + user_id ORDER BY cnt DESC, user_id DESC +LIMIT 10; + user_id | cnt +---------+----- + 27 | 35 + 87 | 27 + 74 | 20 + 72 | 16 + 12 | 16 + 66 | 15 + 56 | 15 + 40 | 15 + 23 | 12 + 59 | 10 +(10 rows) + +-- Simple LATERAL JOINs with GROUP BYs in each side +-- need to set subquery_pushdown due to limit for next 2 queries +SET citus.subquery_pushdown to ON; +SELECT * +FROM + (SELECT "some_users_data".user_id, lastseen + FROM + (SELECT user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(time) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE + ORDER BY + lastseen DESC + LIMIT 50) "some_users" +order BY + user_id +LIMIT 50; + user_id | lastseen +---------+--------------------------------- + 19 | Tue Jan 21 05:23:09.26298 2014 + 22 | Tue Jan 21 05:22:28.223506 2014 + 25 | Tue Jan 21 01:10:29.315788 2014 + 31 | Tue Jan 21 02:43:24.591489 2014 + 33 | Tue Jan 21 04:23:35.623056 2014 + 34 | Tue Jan 21 04:15:03.874341 2014 +(6 rows) + +-- same query with subuqery joins in topmost select +SELECT "some_users_data".user_id, lastseen +FROM + (SELECT user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(TIME) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE +ORDER BY + user_id +limit 50; + user_id | lastseen +---------+--------------------------------- + 19 | Tue Jan 21 05:23:09.26298 2014 + 22 | Tue Jan 21 05:22:28.223506 2014 + 25 | Tue Jan 21 01:10:29.315788 2014 + 31 | Tue Jan 21 02:43:24.591489 2014 + 33 | Tue Jan 21 04:23:35.623056 2014 + 34 | Tue Jan 21 04:15:03.874341 2014 +(6 rows) + +-- reset subquery_pushdown +SET citus.subquery_pushdown to OFF; +-- not supported since JOIN is not on the partition key +SELECT "some_users_data".user_id, lastseen +FROM + (SELECT user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(TIME) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."value_1" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE +ORDER BY + user_id +limit 50; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since JOIN is not on the partition key +-- see (2 * user_id as user_id) target list element +SELECT "some_users_data".user_id, lastseen +FROM + (SELECT 2 * user_id as user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(TIME) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE +ORDER BY + user_id +limit 50; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- LATERAL JOINs used with INNER JOINs +SET citus.subquery_pushdown to ON; +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) + filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 AND + user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" + ON TRUE + ORDER BY + time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" + ON TRUE + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; + user_id | lastseen +---------+--------------------------------- + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 + 15 | Tue Jan 21 02:25:36.136461 2014 +(10 rows) + +-- +-- A similar query with topmost select is dropped +-- and replaced by aggregation. Notice the heavy use of limit +-- +SELECT "some_users_data".user_id, MAX(lastseen), count(*) + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true +GROUP BY 1 +ORDER BY 2, 1 DESC +LIMIT 10; + user_id | max | count +---------+---------------------------------+------- + 15 | Tue Jan 21 02:25:36.136461 2014 | 10 + 13 | Tue Jan 21 05:06:48.989766 2014 | 10 + 14 | Tue Jan 21 05:46:51.286381 2014 | 10 +(3 rows) + +SET citus.subquery_pushdown to OFF; +-- not supported since the inner JOIN is not equi join +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id != "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since the inner JOIN is not on the partition key +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_1" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".value_1)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since upper LATERAL JOIN is not equi join +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_1" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id != filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since lower LATERAL JOIN is not on the partition key +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_1" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."value_1" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- NESTED INNER JOINs +SELECT + count(*) AS value, "generated_group_field" + FROM + (SELECT + DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field" + FROM + (SELECT + "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field" + FROM + (SELECT + * + FROM + (SELECT + "events"."time", "events"."user_id", "events"."value_2" + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40 AND event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries" + INNER JOIN + (SELECT + user_where_1_1.real_user_id + FROM + (SELECT + "users"."user_id" as real_user_id + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1 + ON ("user_where_1_1".real_user_id = "user_where_1_join_1".user_id)) "user_filters_1" + ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery" +GROUP BY + "generated_group_field" +ORDER BY + generated_group_field DESC, value DESC; + value | generated_group_field +-------+----------------------- + 1 | 966 + 1 | 917 + 1 | 905 + 1 | 868 + 1 | 836 + 1 | 791 + 1 | 671 + 1 | 642 + 1 | 358 + 1 | 317 + 1 | 307 + 1 | 302 + 1 | 214 + 1 | 166 + 1 | 116 + 1 | 1 +(16 rows) + +-- not supported since the first inner join is not on the partition key +SELECT + count(*) AS value, "generated_group_field" + FROM + (SELECT + DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field" + FROM + (SELECT + "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field" + FROM + (SELECT + * + FROM + (SELECT + "events"."time", "events"."user_id", "events"."value_2" + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40 AND event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries" + INNER JOIN + (SELECT + user_where_1_1.real_user_id + FROM + (SELECT + "users"."user_id" as real_user_id + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_2" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1 + ON ("user_where_1_1".real_user_id = "user_where_1_join_1".value_2)) "user_filters_1" + ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery" +GROUP BY + "generated_group_field" +ORDER BY + generated_group_field DESC, value DESC; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- not supported since the first inner join is not an equi join +SELECT + count(*) AS value, "generated_group_field" + FROM + (SELECT + DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field" + FROM + (SELECT + "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field" + FROM + (SELECT + * + FROM + (SELECT + "events"."time", "events"."user_id", "events"."value_2" + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40 AND event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries" + INNER JOIN + (SELECT + user_where_1_1.real_user_id + FROM + (SELECT + "users"."user_id" as real_user_id + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_2" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1 + ON ("user_where_1_1".real_user_id >= "user_where_1_join_1".user_id)) "user_filters_1" + ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery" +GROUP BY + "generated_group_field" +ORDER BY + generated_group_field DESC, value DESC; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- single level inner joins +SELECT + "value_3", count(*) AS cnt +FROM + (SELECT + "value_3", "user_id", random() + FROM + (SELECT + users_in_segment_1.user_id, value_3 + FROM + (SELECT + user_id, value_3 * 2 as value_3 + FROM + (SELECT + user_id, value_3 + FROM + (SELECT + "users"."user_id", value_3 + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 30 + ) simple_user_where_1 + ) all_buckets_1 + ) users_in_segment_1 + JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 60 + ) some_users_data + ON ("users_in_segment_1".user_id = "some_users_data".user_id) + ) segmentalias_1) "tempQuery" +GROUP BY "value_3" +ORDER BY cnt, value_3 DESC LIMIT 10; + value_3 | cnt +---------+----- + 556 | 75 + 228 | 75 + 146 | 75 + 70 | 75 + 1442 | 79 + 1232 | 79 + 1090 | 79 + 1012 | 79 + 886 | 79 + 674 | 79 +(10 rows) + +-- not supported since there is no partition column equality at all +SELECT + "value_3", count(*) AS cnt +FROM + (SELECT + "value_3", "user_id", random() + FROM + (SELECT + users_in_segment_1.user_id, value_3 + FROM + (SELECT + user_id, value_3 * 2 as value_3 + FROM + (SELECT + user_id, value_3 + FROM + (SELECT + "users"."user_id", value_3 + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 30 + ) simple_user_where_1 + ) all_buckets_1 + ) users_in_segment_1 + JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 60 + ) some_users_data + ON (true) + ) segmentalias_1) "tempQuery" +GROUP BY "value_3" +ORDER BY cnt, value_3 DESC LIMIT 10; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- nested LATERAL JOINs +SET citus.subquery_pushdown to ON; +SELECT * +FROM + (SELECT "some_users_data".user_id, "some_recent_users".value_3 + FROM + (SELECT + filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1) "last_events_1" ON true + ORDER BY value_3 DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_3 DESC +LIMIT 10) "some_users" +ORDER BY + value_3 DESC +LIMIT 10; + user_id | value_3 +---------+--------- + 44 | 998 + 65 | 996 + 66 | 996 + 37 | 995 + 57 | 989 + 21 | 985 +(6 rows) + +-- nested lateral join at top most level +SELECT "some_users_data".user_id, "some_recent_users".value_3 +FROM + (SELECT + filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200 + ) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1 + ) "last_events_1" ON true + ORDER BY value_3 DESC + LIMIT 10 + ) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1 + ) "some_users_data" ON true +ORDER BY + value_3 DESC, user_id ASC +LIMIT 10; + user_id | value_3 +---------+--------- + 44 | 998 + 65 | 996 + 66 | 996 + 37 | 995 + 57 | 989 + 21 | 985 +(6 rows) + +-- longer nested lateral joins +SELECT * +FROM + (SELECT "some_users_data".user_id, "some_recent_users".value_3 + FROM + (SELECT filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1) "last_events_1" ON true + ORDER BY + value_3 DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_3 DESC + LIMIT 10) "some_users" +ORDER BY + value_3 DESC +LIMIT 10; + user_id | value_3 +---------+--------- + 44 | 998 + 65 | 996 + 66 | 996 + 37 | 995 + 57 | 989 + 21 | 985 +(6 rows) + +-- longer nested lateral join wth top level join +SELECT "some_users_data".user_id, "some_recent_users".value_3 +FROM + (SELECT filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200 + ) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 + AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1 + ) "last_events_1" ON TRUE + ORDER BY value_3 DESC + LIMIT 10 + ) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1 + ) "some_users_data" ON TRUE +ORDER BY value_3 DESC +LIMIT 10; + user_id | value_3 +---------+--------- + 44 | 998 + 65 | 996 + 66 | 996 + 37 | 995 + 57 | 989 + 21 | 985 +(6 rows) + +SET citus.subquery_pushdown to OFF; +-- LEFT JOINs used with INNER JOINs +SELECT +count(*) AS cnt, "generated_group_field" + FROM + (SELECT + "eventQuery"."user_id", random(), generated_group_field + FROM + (SELECT + "multi_group_wrapper_1".*, generated_group_field, random() + FROM + (SELECT * + FROM + (SELECT + "events"."time", "events"."user_id" as event_user_id + FROM + events_table as "events" + WHERE + user_id > 80) "temp_data_queries" + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 80 and value_2 = 5) "user_filters_1" + ON ("temp_data_queries".event_user_id = "user_filters_1".user_id)) AS "multi_group_wrapper_1" + LEFT JOIN + (SELECT + "users"."user_id" AS "user_id", value_2 AS "generated_group_field" + FROM + users_table as "users") "left_group_by_1" + ON ("left_group_by_1".user_id = "multi_group_wrapper_1".event_user_id)) "eventQuery") "pushedDownQuery" + group BY + "generated_group_field" + ORDER BY + cnt DESC, generated_group_field ASC + LIMIT 10; + cnt | generated_group_field +-----+----------------------- + 176 | 551 + 176 | 569 + 176 | 645 + 176 | 713 + 176 | 734 + 88 | 3 + 88 | 5 + 88 | 15 + 88 | 32 + 88 | 68 +(10 rows) + +-- single table subquery, no JOINS involved +SELECT +count(*) AS cnt, user_id +FROM + (SELECT + "eventQuery"."user_id", random() + FROM + (SELECT + "events"."user_id" + FROM + events_table "events" + WHERE + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90)) "eventQuery") "pushedDownQuery" +GROUP BY + "user_id" +ORDER BY + cnt DESC, user_id DESC +LIMIT 10; + cnt | user_id +-----+--------- + 4 | 24 + 3 | 96 + 3 | 93 + 3 | 49 + 3 | 46 + 3 | 38 + 3 | 14 + 3 | 10 + 2 | 99 + 2 | 95 +(10 rows) + +-- lateral joins in the nested manner +SET citus.subquery_pushdown to ON; +SELECT * +FROM + (SELECT + "some_users_data".user_id, value_2 + FROM + (SELECT user_id, max(value_2) AS value_2 + FROM + (SELECT user_id, value_2 + FROM + (SELECT + user_id, value_2 + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 20) "events_1" + ORDER BY + value_2 DESC + LIMIT 10000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(value_2) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + value_2 > 75 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_2 DESC + LIMIT 10) "some_users" +ORDER BY + value_2 DESC, user_id DESC +LIMIT 10; + user_id | value_2 +---------+--------- + 13 | 998 + 18 | 994 + 16 | 993 + 12 | 993 + 11 | 993 + 14 | 991 + 17 | 976 + 15 | 976 + 19 | 966 +(9 rows) + +SET citus.subquery_pushdown to OFF; +-- not supported since join is not on the partition key +SELECT * +FROM + (SELECT + "some_users_data".user_id, value_2 + FROM + (SELECT user_id, max(value_2) AS value_2 + FROM + (SELECT user_id, value_2 + FROM + (SELECT + user_id, value_2 + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 20) "events_1" + ORDER BY + value_2 DESC + LIMIT 10000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(value_2) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."value_2" = "some_recent_users"."user_id" AND + value_2 > 75 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_2 DESC + LIMIT 10) "some_users" +ORDER BY + value_2 DESC, user_id DESC +LIMIT 10; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- lets test some unsupported set operations +-- not supported since we use INTERSECT +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + INTERSECT + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot push down this subquery +DETAIL: Intersect and Except are currently unsupported +-- not supported due to offset +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) OFFSET 3) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot push down this subquery +DETAIL: Offset clause is currently unsupported +-- not supported due to window functions +SELECT user_id, + some_vals +FROM ( + SELECT * , + Row_number() over (PARTITION BY "user_id" ORDER BY "user_id") AS "some_vals", + Random() + FROM users_table + ) user_id +ORDER BY 1, + 2 limit 10; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries without group by clause are not supported yet +-- not supported due to non relation rte +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + 1 as user_id, now(), 3 AS event + ) events_subquery_4) OFFSET 3) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +-- similar to the above, but constant rte is on the right side of the query +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + 1 as user_id, now(), 3 AS event + ) events_subquery_4) OFFSET 3) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT random()::int as user_id) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +SET citus.enable_router_execution TO TRUE; diff --git a/src/test/regress/expected/multi_subquery_union.out b/src/test/regress/expected/multi_subquery_union.out new file mode 100644 index 000000000..0a858baf7 --- /dev/null +++ b/src/test/regress/expected/multi_subquery_union.out @@ -0,0 +1,901 @@ +-- +-- multi subquery toplevel union queries aims to expand existing subquery pushdown +-- regression tests to cover more cases +-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests +-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1400000; +SET citus.enable_router_execution TO false; +-- a very simple union query +SELECT user_id, counter +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +ORDER BY 2 DESC,1 +LIMIT 5; + user_id | counter +---------+--------- + 7 | 9 + 8 | 9 + 15 | 9 + 16 | 9 + 20 | 9 +(5 rows) + +-- the same query with union all +SELECT user_id, counter +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION ALL + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +ORDER BY 2 DESC,1 +LIMIT 5; + user_id | counter +---------+--------- + 7 | 9 + 7 | 9 + 8 | 9 + 15 | 9 + 15 | 9 +(5 rows) + +-- the same query with group by +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +GROUP BY 1 +ORDER BY 2 DESC,1 +LIMIT 5; + user_id | sum +---------+----- + 49 | 22 + 15 | 19 + 26 | 17 + 48 | 17 + 61 | 17 +(5 rows) + +-- the same query with UNION ALL clause +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION ALL + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +GROUP BY 1 +ORDER BY 2 DESC,1 +LIMIT 5; + user_id | sum +---------+----- + 48 | 35 + 61 | 30 + 15 | 28 + 49 | 25 + 80 | 24 +(5 rows) + +-- the same query target list entries shuffled +SELECT user_id, sum(counter) +FROM ( + SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION + SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +GROUP BY 1 +ORDER BY 2 DESC,1 +LIMIT 5; + user_id | sum +---------+----- + 49 | 22 + 15 | 19 + 26 | 17 + 48 | 17 + 61 | 17 +(5 rows) + +-- same query with GROUP BY +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (1, 2) + UNION + SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (5, 6) +) user_id +GROUP BY + user_id +--HAVING sum(counter) > 900 +ORDER BY 1,2 DESC LIMIT 5; + user_id | sum +---------+------ + 1 | 518 + 2 | 637 + 4 | 343 + 6 | 354 + 7 | 1374 +(5 rows) + +-- the same query target list entries shuffled but this time the subqueries target list +-- is shuffled +SELECT user_id, sum(counter) +FROM ( + SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (1, 2) + UNION + SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (5, 6) +) user_id +GROUP BY + user_id +--HAVING sum(counter) > 900 +ORDER BY 1,2 DESC LIMIT 5; + user_id | sum +---------+------ + 1 | 518 + 2 | 637 + 4 | 343 + 6 | 354 + 7 | 1374 +(5 rows) + +-- similar query this time more subqueries and target list contains a resjunk entry +SELECT sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500 +) user_id +GROUP BY user_id ORDER BY 1 DESC LIMIT 5; + sum +------- + 27772 + 25720 + 24993 + 24968 + 23508 +(5 rows) + +-- similar query as above, with UNION ALL +SELECT sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 5000 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500 +) user_id +GROUP BY user_id ORDER BY 1 DESC LIMIT 5; + sum +------- + 27667 + 25080 + 24814 + 24365 + 23508 +(5 rows) + +-- unions within unions +SELECT * +FROM ( + ( SELECT user_id, + sum(counter) + FROM + (SELECT + user_id, sum(value_2) AS counter + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_1 + GROUP BY + user_id) + UNION + (SELECT + user_id, sum(counter) + FROM + (SELECT + user_id, sum(value_2) AS counter + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_2 + GROUP BY + user_id)) AS ftop +ORDER BY 2 DESC, 1 DESC +LIMIT 5; + user_id | sum +---------+-------- + 23 | 126017 + 45 | 117323 + 25 | 116595 + 17 | 116520 + 90 | 115843 +(5 rows) + +-- top level unions are wrapped into top level aggregations +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +) as final_query +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 55 + 1 | 38 + 2 | 70 + 3 | 58 +(4 rows) + +-- exactly the same query +-- but wrapper unions are removed from the inner part of the query +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + (SELECT *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + (SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + GROUP BY "t1"."user_id") AS t) "q" +) as final_query +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 55 + 1 | 38 + 2 | 70 + 3 | 58 +(4 rows) + +-- again excatly the same query with top level wrapper removed +SELECT ("q"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + GROUP BY "t1"."user_id") AS t) "q" +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 55 + 1 | 38 + 2 | 70 + 3 | 58 +(4 rows) + +-- again same query but with only two top level empty queries (i.e., no group bys) +SELECT * +FROM + ( SELECT * + FROM + ( SELECT "t1"."user_id" + FROM ( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + ) AS t) "q" +ORDER BY 1 +LIMIT 5; + user_id +--------- + 0 + 0 + 0 + 1 + 1 +(5 rows) + +-- a very similar query UNION ALL +SELECT ("q"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION ALL + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION ALL + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION ALL + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + GROUP BY "t1"."user_id") AS t) "q" +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 55 + 1 | 38 + 2 | 70 + 3 | 58 +(4 rows) + +-- some UNION ALL queries that are going to be pulled up +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT user_id FROM events_table) +) b; + count +------- + 20002 +(1 row) + +-- similar query without top level agg +SELECT + user_id +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT user_id FROM events_table) +) b +ORDER BY 1 DESC +LIMIT 5; + user_id +--------- + 100 + 100 + 100 + 100 + 100 +(5 rows) + +-- similar query with multiple target list entries +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM users_table) + UNION ALL + (SELECT value_3, user_id FROM events_table) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; + user_id | value_3 +---------+--------- + 100 | 999 + 100 | 997 + 100 | 991 + 100 | 989 + 100 | 988 +(5 rows) + +-- similar query group by inside the subqueries +SELECT + user_id, value_3_sum +FROM +( + (SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id) + UNION ALL + (SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id) +) b +ORDER BY 2 DESC, 1 DESC +LIMIT 5; + user_id | value_3_sum +---------+------------- + 10 | 64060 + 10 | 64060 + 62 | 62445 + 62 | 62445 + 26 | 60536 +(5 rows) + +-- similar query top level group by +SELECT + user_id, sum(value_3) +FROM +( + (SELECT value_3, user_id FROM users_table) + UNION ALL + (SELECT value_3, user_id FROM events_table) +) b +GROUP BY 1 +ORDER BY 2 DESC, 1 DESC +LIMIT 5; + user_id | sum +---------+-------- + 23 | 123923 + 25 | 118087 + 69 | 115828 + 26 | 114705 + 3 | 113915 +(5 rows) + +-- a long set operation list +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30)) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; + user_id | value_3 +---------+--------- + 100 | 951 + 99 | 558 + 99 | 14 + 98 | 987 + 98 | 577 +(5 rows) + +-- no partition key on the top +SELECT + max(value_3) +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30)) +) b +GROUP BY user_id +ORDER BY 1 DESC +LIMIT 5; + max +----- + 997 + 997 + 996 + 995 + 995 +(5 rows) + +-- now lets also have some unsupported queries +-- group by is not on the partition key +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + UNION + SELECT value_1 as user_id, sum(value_2) AS counter FROM users_table GROUP BY value_1 +) user_id +GROUP BY user_id; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- partition key is not selected +SELECT sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT 2 * user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500 +) user_id +GROUP BY user_id ORDER BY 1 DESC LIMIT 5; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- excepts within unions are not supported +SELECT * FROM +( +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + ) user_id_1 + GROUP BY user_id +) +UNION +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + EXCEPT + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id +) user_id_2 + GROUP BY user_id) +) as ftop; +ERROR: cannot push down this subquery +DETAIL: Intersect and Except are currently unsupported +-- joins inside unions are not supported +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE users_table.user_id > events_table.user_id GROUP BY 1 +) user_id +GROUP BY user_id; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- joins inside unions are not supported -- slightly more comlex than the above +SELECT * FROM +( +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + ) user_id_1 + GROUP BY user_id +) +UNION +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE (events_table.user_id = users_table.user_id) GROUP BY events_table.user_id +) user_id_2 + GROUP BY user_id) +) as ftop; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- offset inside the union +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id OFFSET 4 +) user_id +GROUP BY user_id; +ERROR: cannot push down this subquery +DETAIL: Offset clause is currently unsupported +-- lower level union does not return partition key with the other relations +SELECT * +FROM ( + ( SELECT user_id, + sum(counter) + FROM + (SELECT + user_id, sum(value_2) AS counter + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_1 + GROUP BY + user_id) + UNION + (SELECT + user_id, sum(counter) + FROM + (SELECT + sum(value_2) AS counter, user_id + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_2 + GROUP BY + user_id)) AS ftop; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- some UNION all queries that are going to be pulled up +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT 2 * user_id FROM events_table) +) b; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- last query does not have partition key +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT value_3, value_2 FROM events_table where event_type IN (26, 27, 28, 29, 30)) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- we don't allow joins within unions +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT users_table.user_id FROM events_table, users_table WHERE events_table.user_id = users_table.user_id) +) b; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- we don't support subqueries without relations +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT 1) +) b; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +-- we don't support subqueries without relations +SELECT + * +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT (random() * 100)::int) +) b; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +-- we don't support subqueries without relations +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT 1, 2) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT 1, now(), 3 AS event) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +) as final_query +GROUP BY types +ORDER BY types; +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported +SET citus.enable_router_execution TO true; diff --git a/src/test/regress/expected/multi_view.out b/src/test/regress/expected/multi_view.out index f8444c562..a8d4c9b45 100644 --- a/src/test/regress/expected/multi_view.out +++ b/src/test/regress/expected/multi_view.out @@ -230,20 +230,54 @@ SELECT l_suppkey, count(*) FROM GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5; ERROR: cannot perform distributed planning on this query DETAIL: Subqueries without group by clause are not supported yet +-- repartition query on view with single table subquery +CREATE VIEW supp_count_view AS SELECT * FROM (SELECT l_suppkey, count(*) FROM lineitem_hash_part GROUP BY 1) s1; +SELECT * FROM supp_count_view ORDER BY 2 DESC, 1 LIMIT 10; + l_suppkey | count +-----------+------- + 6104 | 8 + 1868 | 6 + 5532 | 6 + 5849 | 6 + 6169 | 6 + 6669 | 6 + 6692 | 6 + 7703 | 6 + 7869 | 6 + 8426 | 6 +(10 rows) + SET citus.task_executor_type to DEFAULT; -- create a view with aggregate CREATE VIEW lineitems_by_shipping_method AS SELECT l_shipmode, count(*) as cnt FROM lineitem_hash_part GROUP BY 1; --- following will fail due to non-flattening of subquery due to GROUP BY +-- following will fail due to non GROUP BY of partition key SELECT * FROM lineitems_by_shipping_method; ERROR: Unrecognized range table id 1 -- create a view with group by on partition column CREATE VIEW lineitems_by_orderkey AS - SELECT l_orderkey, count(*) FROM lineitem_hash_part GROUP BY 1; --- this will also fail due to same reason -SELECT * FROM lineitems_by_orderkey; -ERROR: Unrecognized range table id 1 --- however it would work if it is made router plannable + SELECT + l_orderkey, count(*) + FROM + lineitem_hash_part + GROUP BY 1; +-- this should work since we're able to push down this query +SELECT * FROM lineitems_by_orderkey ORDER BY 2 DESC, 1 ASC LIMIT 10; + l_orderkey | count +------------+------- + 7 | 7 + 68 | 7 + 129 | 7 + 164 | 7 + 194 | 7 + 225 | 7 + 226 | 7 + 322 | 7 + 326 | 7 + 354 | 7 +(10 rows) + +-- it would also work since it is made router plannable SELECT * FROM lineitems_by_orderkey WHERE l_orderkey = 100; l_orderkey | count ------------+------- @@ -251,3 +285,613 @@ SELECT * FROM lineitems_by_orderkey WHERE l_orderkey = 100; (1 row) DROP TABLE temp_lineitem CASCADE; +DROP VIEW supp_count_view; +DROP VIEW lineitems_by_orderkey; +DROP VIEW lineitems_by_shipping_method; +DROP VIEW air_shipped_lineitems; +DROP VIEW priority_lineitem; +DROP VIEW priority_orders; +-- new tests for real time use case including views and subqueries +-- create view to display recent user who has an activity after a timestamp +CREATE VIEW recent_users AS + SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC; +SELECT * FROM recent_users; + user_id | lastseen +---------+--------------------------------- + 87 | Tue Jan 21 05:53:51.866813 2014 + 50 | Tue Jan 21 05:53:44.251016 2014 + 74 | Tue Jan 21 05:54:04.837808 2014 + 6 | Tue Jan 21 05:57:47.118755 2014 + 71 | Tue Jan 21 05:55:52.018461 2014 + 39 | Tue Jan 21 05:55:18.875997 2014 + 66 | Tue Jan 21 05:51:31.681997 2014 + 100 | Tue Jan 21 05:49:04.953009 2014 + 46 | Tue Jan 21 05:49:00.229807 2014 + 86 | Tue Jan 21 05:48:54.381334 2014 + 13 | Tue Jan 21 05:48:45.418146 2014 + 90 | Tue Jan 21 05:48:25.027491 2014 + 58 | Tue Jan 21 05:47:30.418553 2014 + 44 | Tue Jan 21 05:47:01.104523 2014 +(14 rows) + +-- create a view for recent_events +CREATE VIEW recent_events AS + SELECT user_id, time FROM events_table + WHERE time > '2014-01-20 01:45:49.978738'::timestamp; +SELECT count(*) FROM recent_events; + count +------- + 1105 +(1 row) + +-- count number of events of recent_users +SELECT count(*) FROM recent_users ru JOIN events_table et ON (ru.user_id = et.user_id); + count +------- + 1336 +(1 row) + +-- count number of events of per recent users order by count +SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1; + user_id | count +---------+------- + 13 | 118 + 44 | 109 + 90 | 109 + 87 | 105 + 46 | 103 + 86 | 100 + 66 | 98 + 39 | 96 + 71 | 95 + 74 | 93 + 6 | 89 + 58 | 87 + 50 | 79 + 100 | 55 +(14 rows) + +-- the same query with a left join however, it would still generate the same result +SELECT ru.user_id, count(*) + FROM recent_users ru + LEFT JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1; + user_id | count +---------+------- + 13 | 118 + 44 | 109 + 90 | 109 + 87 | 105 + 46 | 103 + 86 | 100 + 66 | 98 + 39 | 96 + 71 | 95 + 74 | 93 + 6 | 89 + 58 | 87 + 50 | 79 + 100 | 55 +(14 rows) + +-- query wrapped inside a subquery, it needs another top level order by +SELECT * FROM + (SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1) s1 +ORDER BY 2 DESC, 1; + user_id | count +---------+------- + 13 | 118 + 44 | 109 + 90 | 109 + 87 | 105 + 46 | 103 + 86 | 100 + 66 | 98 + 39 | 96 + 71 | 95 + 74 | 93 + 6 | 89 + 58 | 87 + 50 | 79 + 100 | 55 +(14 rows) + +-- non-partition key joins are not supported inside subquery +SELECT * FROM + (SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.event_type) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1) s1 +ORDER BY 2 DESC, 1; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- join between views +-- recent users who has an event in recent events +SELECT ru.user_id FROM recent_users ru JOIN recent_events re USING(user_id) GROUP BY ru.user_id ORDER BY ru.user_id; + user_id +--------- + 6 + 13 + 39 + 44 + 46 + 50 + 58 + 66 + 71 + 74 + 86 + 87 + 90 + 100 +(14 rows) + +-- outer join inside a subquery +-- recent_events who are not done by recent users +SELECT count(*) FROM ( + SELECT re.*, ru.user_id AS recent_user + FROM recent_events re LEFT JOIN recent_users ru USING(user_id)) reu + WHERE recent_user IS NULL; + count +------- + 957 +(1 row) + +-- same query with anti-join +SELECT count(*) + FROM recent_events re LEFT JOIN recent_users ru ON(ru.user_id = re.user_id) + WHERE ru.user_id IS NULL; + count +------- + 957 +(1 row) + +-- join between view and table +-- users who has recent activity and they have an entry with value_1 is less than 15 +SELECT ut.* FROM recent_users ru JOIN users_table ut USING (user_id) WHERE ut.value_1 < 15 ORDER BY 1,2; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 6 | Mon Jan 13 05:30:08.289267 2014 | 12 | 140 | 618 | + 6 | Thu Jan 16 15:17:16.779695 2014 | 6 | 978 | 430 | + 6 | Sun Jan 19 06:09:39.900888 2014 | 3 | 908 | 688 | + 13 | Sun Jan 19 22:09:26.256209 2014 | 2 | 755 | 584 | + 39 | Wed Jan 15 05:46:51.48765 2014 | 14 | 657 | 137 | + 39 | Sun Jan 19 11:26:47.45937 2014 | 12 | 118 | 165 | + 44 | Wed Jan 15 14:23:52.532426 2014 | 8 | 204 | 735 | + 44 | Sun Jan 19 05:53:34.829093 2014 | 4 | 758 | 205 | + 46 | Mon Jan 13 20:39:11.211169 2014 | 0 | 235 | 475 | + 46 | Wed Jan 15 09:14:57.471944 2014 | 2 | 407 | 664 | + 50 | Sat Jan 11 11:07:13.089216 2014 | 6 | 292 | 425 | + 58 | Sun Jan 19 22:36:14.795396 2014 | 2 | 86 | 311 | + 66 | Tue Jan 14 20:16:31.219213 2014 | 14 | 347 | 655 | + 74 | Tue Jan 21 01:38:39.570986 2014 | 9 | 334 | 642 | + 86 | Sun Jan 19 06:18:51.466578 2014 | 14 | 712 | 490 | + 87 | Sat Jan 11 20:46:28.439073 2014 | 2 | 528 | 311 | + 90 | Sun Jan 12 21:37:30.778206 2014 | 11 | 458 | 377 | + 100 | Sun Jan 19 22:32:08.284043 2014 | 2 | 384 | 149 | +(18 rows) + +-- determine if a recent user has done a given event type or not +SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.user_id AND et.event_type = 625) + ORDER BY 2 DESC, 1; + user_id | done_event +---------+------------ + 6 | YES + 13 | NO + 39 | NO + 44 | NO + 46 | NO + 50 | NO + 58 | NO + 66 | NO + 71 | NO + 74 | NO + 86 | NO + 87 | NO + 90 | NO + 100 | NO +(14 rows) + +-- view vs table join wrapped inside a subquery +SELECT * FROM + (SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.user_id AND et.event_type = 625) + ) s1 +ORDER BY 2 DESC, 1; + user_id | done_event +---------+------------ + 6 | YES + 13 | NO + 39 | NO + 44 | NO + 46 | NO + 50 | NO + 58 | NO + 66 | NO + 71 | NO + 74 | NO + 86 | NO + 87 | NO + 90 | NO + 100 | NO +(14 rows) + +-- event vs table non-partition-key join is not supported +SELECT * FROM + (SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.event_type) + ) s1 +ORDER BY 2 DESC, 1; +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- create a select only view +CREATE VIEW selected_users AS SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150; +CREATE VIEW recent_selected_users AS SELECT su.* FROM selected_users su JOIN recent_users ru USING(user_id); +SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; + user_id +--------- + 6 + 13 + 39 + 44 + 46 + 50 + 58 + 66 + 71 + 74 + 86 + 90 +(12 rows) + +-- this would be supported when we implement where partition_key in (subquery) support +SELECT et.* FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users); +ERROR: could not run distributed query with subquery outside the FROM clause +HINT: Consider using an equality filter on the distributed table's partition column. +-- it is supported when it is a router query +SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90); + count +------- + 109 +(1 row) + +-- expected this to work but it did not +(SELECT user_id FROM recent_users) +UNION +(SELECT user_id FROM selected_users); +ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT +HINT: Consider using an equality filter on the distributed table's partition column. +-- wrapping it inside a SELECT * works +SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10 + ORDER BY user_id; + user_id +--------- + 11 + 12 + 13 + 14 +(4 rows) + +-- union all also works for views +SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION ALL + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10 + ORDER BY user_id; + user_id +--------- + 11 + 11 + 11 + 12 + 12 + 12 + 12 + 12 + 12 + 13 + 13 + 13 + 13 + 13 + 14 +(15 rows) + +SELECT count(*) + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10; + count +------- + 4 +(1 row) + +-- expected this to work but it does not +SELECT count(*) + FROM ( + (SELECT user_id FROM recent_users) + UNION ALL + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- expand view definitions and re-run last 2 queries +SELECT count(*) + FROM ( + (SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa + ) + UNION + (SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u + WHERE user_id < 15 AND user_id > 10; + count +------- + 4 +(1 row) + +SELECT count(*) + FROM ( + (SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa + ) + UNION ALL + (SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u + WHERE user_id < 15 AND user_id > 10; +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. +-- test distinct +-- distinct is supported if it is on a partition key +CREATE VIEW distinct_user_with_value_1_15 AS SELECT DISTINCT user_id FROM users_table WHERE value_1 = 15; +SELECT * FROM distinct_user_with_value_1_15 ORDER BY user_id; + user_id +--------- + 7 + 8 + 35 + 42 + 46 + 53 + 70 + 82 + 87 + 88 + 96 +(11 rows) + +-- distinct is not supported if it is on a non-partition key +CREATE VIEW distinct_value_1 AS SELECT DISTINCT value_1 FROM users_table WHERE value_2 = 15; +SELECT * FROM distinct_value_1; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries without group by clause are not supported yet +-- CTEs are not supported even if they are on views +CREATE VIEW cte_view_1 AS +WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 15) SELECT * FROM c1 WHERE value_2 < 500; +SELECT * FROM cte_view_1; +ERROR: cannot push down this subquery +DETAIL: Table expressions other than simple relations and subqueries are currently unsupported +-- this is single shard query but still not supported since it has view + cte +-- router planner can't detect it +SELECT * FROM cte_view_1 WHERE user_id = 8; +ERROR: cannot push down this subquery +DETAIL: Table expressions other than simple relations and subqueries are currently unsupported +-- if CTE itself prunes down to a single shard than the view is supported (router plannable) +CREATE VIEW cte_view_2 AS +WITH c1 AS (SELECT * FROM users_table WHERE user_id = 8) SELECT * FROM c1 WHERE value_1 = 15; +SELECT * FROM cte_view_2; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 8 | Tue Jan 21 00:52:36.967785 2014 | 15 | 10 | 868 | +(1 row) + +CREATE VIEW router_view AS SELECT * FROM users_table WHERE user_id = 2; +-- router plannable +SELECT user_id FROM router_view GROUP BY 1; + user_id +--------- + 2 +(1 row) + +-- There is a known issue with router plannable subqueries joined with non-router +-- plannable subqueries. Following tests should be uncommented when we fix it +-- join a router view (not implement error) +-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id); +-- it still does not work when converted to 2 subquery join +-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id); +-- views are completely removed and still it does not work +-- SELECT * FROM +-- (SELECT user_id FROM (SELECT * FROM users_table WHERE user_id = 2) rv1 GROUP BY 1) rv2 +-- JOIN (SELECT user_id, time FROM events_table +-- WHERE time > '2014-01-20 01:45:49.978738'::timestamp) re +-- USING (user_id); +-- views with limits +CREATE VIEW recent_10_users AS + SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + ORDER BY lastseen DESC + LIMIT 10; +-- this is not supported since it has limit in it and subquery_pushdown is not set +SELECT * FROM recent_10_users; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries with limit are not supported yet +SET citus.subquery_pushdown to ON; +-- still not supported since outer query does not have limit +-- it shows a different (subquery with single relation) error message +SELECT * FROM recent_10_users; +ERROR: cannot perform distributed planning on this query +DETAIL: Subqueries with limit are not supported yet +-- now it displays more correct error message +SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id); +ERROR: cannot push down this subquery +DETAIL: Limit in subquery without limit in the outermost query is unsupported +-- now both are supported when there is a limit on the outer most query +SELECT * FROM recent_10_users ORDER BY lastseen DESC LIMIT 10; + user_id | lastseen +---------+--------------------------------- + 6 | Tue Jan 21 05:57:47.118755 2014 + 71 | Tue Jan 21 05:55:52.018461 2014 + 39 | Tue Jan 21 05:55:18.875997 2014 + 74 | Tue Jan 21 05:54:04.837808 2014 + 87 | Tue Jan 21 05:53:51.866813 2014 + 50 | Tue Jan 21 05:53:44.251016 2014 + 66 | Tue Jan 21 05:51:31.681997 2014 + 100 | Tue Jan 21 05:49:04.953009 2014 + 46 | Tue Jan 21 05:49:00.229807 2014 + 86 | Tue Jan 21 05:48:54.381334 2014 +(10 rows) + +SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; + user_id | time | event_type | value_2 | value_3 | value_4 +---------+---------------------------------+------------+---------+---------+--------- + 65 | Tue Jan 21 05:56:52.624231 2014 | 241 | 30 | 543 | + 42 | Tue Jan 21 05:46:35.158342 2014 | 761 | 877 | 335 | + 54 | Tue Jan 21 05:46:19.103645 2014 | 595 | 477 | 996 | + 44 | Tue Jan 21 05:43:00.838945 2014 | 682 | 641 | 448 | + 27 | Tue Jan 21 05:34:10.935865 2014 | 912 | 605 | 989 | + 61 | Tue Jan 21 05:25:27.452065 2014 | 392 | 472 | 925 | + 19 | Tue Jan 21 05:23:09.26298 2014 | 202 | 888 | 640 | + 65 | Tue Jan 21 05:22:56.725329 2014 | 519 | 457 | 259 | + 27 | Tue Jan 21 05:19:14.38026 2014 | 19 | 19 | 205 | + 11 | Tue Jan 21 05:15:14.879531 2014 | 459 | 545 | 80 | +(10 rows) + +RESET citus.subquery_pushdown; +-- explain tests +EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Sort Key: remote_scan.user_id + -> HashAggregate + Group Key: remote_scan.user_id + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> HashAggregate + Group Key: users_table.user_id + -> Hash Join + Hash Cond: (users_table.user_id = ru.user_id) + -> Bitmap Heap Scan on users_table_1400000 users_table + Recheck Cond: ((value_1 >= 120) AND (value_1 < 150)) + -> Bitmap Index Scan on is_index3_1400000 + Index Cond: ((value_1 >= 120) AND (value_1 < 150)) + -> Hash + -> Subquery Scan on ru + -> Sort + Sort Key: (max(users_table_1."time")) DESC + -> HashAggregate + Group Key: users_table_1.user_id + Filter: (max(users_table_1."time") > '2014-01-21 05:45:49.978738'::timestamp without time zone) + -> Seq Scan on users_table_1400000 users_table_1 +(25 rows) + +EXPLAIN (COSTS FALSE) SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10 + ORDER BY user_id; + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Sort Key: remote_scan.user_id + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Unique + -> Sort + Sort Key: recent_users.user_id + -> Append + -> Subquery Scan on recent_users + -> Sort + Sort Key: (max(users_table."time")) DESC + -> GroupAggregate + Group Key: users_table.user_id + Filter: (max(users_table."time") > '2014-01-21 05:45:49.978738'::timestamp without time zone) + -> Index Scan using is_index1_1400000 on users_table_1400000 users_table + Index Cond: ((user_id < 15) AND (user_id > 10)) + -> Index Scan using is_index1_1400000 on users_table_1400000 users_table_1 + Index Cond: ((user_id < 15) AND (user_id > 10)) + Filter: ((value_1 >= 120) AND (value_1 < 150)) +(22 rows) + +EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; +ERROR: cannot push down this subquery +DETAIL: Limit in subquery is currently unsupported +SET citus.subquery_pushdown to ON; +EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------- + Limit + -> Sort + Sort Key: remote_scan."time" DESC + -> Custom Scan (Citus Real-Time) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Limit + -> Sort + Sort Key: et."time" DESC + -> Hash Join + Hash Cond: (et.user_id = recent_10_users.user_id) + -> Seq Scan on events_table_1400004 et + -> Hash + -> Subquery Scan on recent_10_users + -> Limit + -> Sort + Sort Key: (max(users_table."time")) DESC + -> HashAggregate + Group Key: users_table.user_id + -> Seq Scan on users_table_1400000 users_table +(22 rows) + +RESET citus.subquery_pushdown; +DROP VIEW recent_10_users; +DROP VIEW router_view; +DROP VIEW cte_view_2; +DROP VIEW cte_view_1; +DROP VIEW distinct_value_1; +DROP VIEW distinct_user_with_value_1_15; +DROP VIEW recent_selected_users; +DROP VIEW selected_users; +DROP VIEW recent_events; +DROP VIEW recent_users; diff --git a/src/test/regress/input/multi_complex_count_distinct.source b/src/test/regress/input/multi_complex_count_distinct.source index ffd4438a1..1ca2cc617 100644 --- a/src/test/regress/input/multi_complex_count_distinct.source +++ b/src/test/regress/input/multi_complex_count_distinct.source @@ -130,9 +130,9 @@ SELECT * SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT 1) + l_linenumber, count(DISTINCT 1) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; @@ -140,9 +140,9 @@ SELECT * SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT (random() * 5)::int) + l_linenumber, count(DISTINCT (random() * 5)::int) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; @@ -277,18 +277,18 @@ SELECT * SELECT * FROM ( SELECT - l_orderkey, sum(DISTINCT l_partkey) + l_linenumber, sum(DISTINCT l_partkey) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; SELECT * FROM ( SELECT - l_orderkey, avg(DISTINCT l_partkey) + l_linenumber, avg(DISTINCT l_partkey) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; @@ -297,18 +297,18 @@ SELECT * SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT lineitem_hash) + l_linenumber, count(DISTINCT lineitem_hash) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT lineitem_hash.*) + l_linenumber, count(DISTINCT lineitem_hash.*) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; diff --git a/src/test/regress/input/multi_subquery.source b/src/test/regress/input/multi_subquery.source index 9880c4f4b..ed2dae668 100644 --- a/src/test/regress/input/multi_subquery.source +++ b/src/test/regress/input/multi_subquery.source @@ -46,7 +46,7 @@ SELECT master_create_distributed_table('orders_subquery', 'o_orderkey', 'range') SET citus.enable_router_execution TO 'false'; --- Check that we don't allow subquery pushdown in default settings. +-- Check that we allow subquery pushdown in default settings. SELECT avg(unit_price) @@ -62,8 +62,6 @@ FROM GROUP BY l_orderkey) AS unit_prices; -SET citus.subquery_pushdown to TRUE; - -- Check that we don't crash if there are not any shards. SELECT @@ -130,7 +128,8 @@ FROM UPDATE pg_dist_shard SET shardmaxvalue = '14947' WHERE shardid = 270003; --- If group by is not on partition column then we error out. +-- If group by is not on partition column then we error out from single table +-- repartition code path SELECT avg(order_count) @@ -171,14 +170,14 @@ FROM GROUP BY l_orderkey) AS unit_prices; --- Check that we error out if there is union all. - +-- Check that we error out if there is non relation subqueries SELECT count(*) FROM ( (SELECT l_orderkey FROM lineitem_subquery) UNION ALL (SELECT 1::bigint) ) b; + -- Check that we error out if queries in union do not include partition columns. SELECT count(*) FROM @@ -194,9 +193,7 @@ SELECT count(*) FROM (SELECT l_orderkey FROM lineitem_subquery) UNION (SELECT l_orderkey FROM lineitem_subquery) ) b; - --- Check that we error out if the outermost query has subquery join. - +-- Check that we error out if inner query has Limit but subquery_pushdown is not set SELECT avg(o_totalprice/l_quantity) FROM @@ -217,6 +214,32 @@ FROM WHERE lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true; +-- Limit is only supported when subquery_pushdown is set +-- Check that we error out if inner query has limit but outer query has not. +SET citus.subquery_pushdown to ON; +SELECT + avg(o_totalprice/l_quantity) +FROM + (SELECT + l_orderkey, + l_quantity + FROM + lineitem_subquery + ORDER BY + l_quantity + LIMIT 10 + ) lineitem_quantities + JOIN LATERAL + (SELECT + o_totalprice + FROM + orders_subquery + WHERE + lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true; + +-- reset the flag for next query +SET citus.subquery_pushdown to OFF; + -- Check that we error out if the outermost query is a distinct clause. SELECT @@ -226,6 +249,8 @@ FROM ( count(*) a FROM lineitem_subquery + GROUP BY + l_orderkey ) z; -- Check supported subquery types. @@ -336,7 +361,6 @@ CREATE TABLE subquery_pruning_varchar_test_table SELECT master_create_distributed_table('subquery_pruning_varchar_test_table', 'a', 'hash'); SELECT master_create_worker_shards('subquery_pruning_varchar_test_table', 4, 1); -SET citus.subquery_pushdown TO TRUE; SET client_min_messages TO DEBUG2; SELECT * FROM @@ -388,6 +412,163 @@ CREATE TYPE user_composite_type AS user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); + +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); + +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); + +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); + +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); + +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; + + + +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); + + +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); + + + \c - - - :worker_1_port CREATE TYPE user_composite_type AS @@ -396,6 +577,161 @@ CREATE TYPE user_composite_type AS user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); + +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); + +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); + +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); + +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); + +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; + + + +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); + + +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); + + \c - - - :worker_2_port CREATE TYPE user_composite_type AS @@ -404,6 +740,161 @@ CREATE TYPE user_composite_type AS user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + + +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; + +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); + +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); + +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); + +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); + +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); + +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; + + + +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); + + +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); + + \c - - - :master_port @@ -482,8 +973,6 @@ WHERE shardid = :new_shard_id; \. -SET citus.subquery_pushdown TO TRUE; - -- Simple join subquery pushdown SELECT avg(array_length(events, 1)) AS event_average @@ -502,8 +991,7 @@ FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type IN ('click', 'submit', 'pay')) AS subquery @@ -526,14 +1014,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -541,14 +1029,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -566,8 +1054,7 @@ FROM events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay') AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -590,14 +1077,14 @@ SELECT (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -605,14 +1092,14 @@ SELECT (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -621,6 +1108,7 @@ SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -629,13 +1117,11 @@ SELECT events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay' GROUP BY - tenant_id, - user_id + composite_id HAVING COUNT(*) > 2) AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -646,8 +1132,10 @@ GROUP BY count_pay ORDER BY count_pay; - + -- Lateral join subquery pushdown +-- set subquery_pushdown since there is limit in the query +SET citus.subquery_pushdown to ON; SELECT tenant_id, user_id, @@ -663,6 +1151,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -681,8 +1170,7 @@ FROM FROM events WHERE - (composite_id).tenant_id = subquery_top.tenant_id AND - (composite_id).user_id = subquery_top.user_id + (composite_id) = subquery_top.composite_id ORDER BY event_time DESC LIMIT @@ -697,7 +1185,8 @@ ORDER BY user_lastseen DESC LIMIT 10; - +-- reset the flag for next query +SET citus.subquery_pushdown to OFF; -- Same queries above with explain -- Simple join subquery pushdown @@ -720,8 +1209,7 @@ FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type IN ('click', 'submit', 'pay')) AS subquery @@ -745,14 +1233,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -760,14 +1248,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -785,8 +1273,7 @@ FROM events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay') AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -810,14 +1297,14 @@ EXPLAIN (COSTS OFF) (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -825,14 +1312,14 @@ EXPLAIN (COSTS OFF) (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -841,6 +1328,7 @@ EXPLAIN (COSTS OFF) (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -849,13 +1337,11 @@ EXPLAIN (COSTS OFF) events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay' GROUP BY - tenant_id, - user_id + composite_id HAVING COUNT(*) > 2) AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -868,6 +1354,8 @@ ORDER BY count_pay; -- Lateral join subquery pushdown +-- set subquery_pushdown due to limit in the query +SET citus.subquery_pushdown to ON; EXPLAIN (COSTS OFF) SELECT tenant_id, @@ -884,6 +1372,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -902,8 +1391,7 @@ FROM FROM events WHERE - (composite_id).tenant_id = subquery_top.tenant_id AND - (composite_id).user_id = subquery_top.user_id + (composite_id) = subquery_top.composite_id ORDER BY event_time DESC LIMIT @@ -918,5 +1406,5 @@ ORDER BY user_lastseen DESC LIMIT 10; - +SET citus.subquery_pushdown to OFF; SET citus.enable_router_execution TO 'true'; diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index d54b4078a..1ca71eec6 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -40,7 +40,8 @@ test: multi_insert_select test: multi_deparse_shard_query test: multi_basic_queries multi_complex_expressions multi_verify_no_subquery test: multi_explain -test: multi_subquery +test: multi_subquery multi_subquery_complex_queries multi_subquery_behavioral_analytics +test: multi_subquery_union test: multi_reference_table test: multi_outer_join_reference test: multi_single_relation_subquery diff --git a/src/test/regress/output/multi_complex_count_distinct.source b/src/test/regress/output/multi_complex_count_distinct.source index 707438761..59b4045ad 100644 --- a/src/test/regress/output/multi_complex_count_distinct.source +++ b/src/test/regress/output/multi_complex_count_distinct.source @@ -241,9 +241,9 @@ SELECT * SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT 1) + l_linenumber, count(DISTINCT 1) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute aggregate (distinct) @@ -253,9 +253,9 @@ HINT: You can load the hll extension from contrib packages and enable distinct SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT (random() * 5)::int) + l_linenumber, count(DISTINCT (random() * 5)::int) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute aggregate (distinct) @@ -464,9 +464,9 @@ SELECT * SELECT * FROM ( SELECT - l_orderkey, sum(DISTINCT l_partkey) + l_linenumber, sum(DISTINCT l_partkey) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute aggregate (distinct) @@ -474,9 +474,9 @@ DETAIL: Only count(distinct) aggregate is supported in subqueries SELECT * FROM ( SELECT - l_orderkey, avg(DISTINCT l_partkey) + l_linenumber, avg(DISTINCT l_partkey) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute aggregate (distinct) @@ -486,9 +486,9 @@ DETAIL: Only count(distinct) aggregate is supported in subqueries SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT lineitem_hash) + l_linenumber, count(DISTINCT lineitem_hash) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute count (distinct) @@ -496,9 +496,9 @@ DETAIL: Non-column references are not supported yet SELECT * FROM ( SELECT - l_orderkey, count(DISTINCT lineitem_hash.*) + l_linenumber, count(DISTINCT lineitem_hash.*) FROM lineitem_hash - GROUP BY l_orderkey) sub + GROUP BY l_linenumber) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute count (distinct) diff --git a/src/test/regress/output/multi_subquery.source b/src/test/regress/output/multi_subquery.source index fb101d174..040ca4e7f 100644 --- a/src/test/regress/output/multi_subquery.source +++ b/src/test/regress/output/multi_subquery.source @@ -53,7 +53,7 @@ SELECT master_create_distributed_table('orders_subquery', 'o_orderkey', 'range') (1 row) SET citus.enable_router_execution TO 'false'; --- Check that we don't allow subquery pushdown in default settings. +-- Check that we allow subquery pushdown in default settings. SELECT avg(unit_price) FROM @@ -67,9 +67,11 @@ FROM l_orderkey = o_orderkey GROUP BY l_orderkey) AS unit_prices; -ERROR: cannot perform distributed planning on this query -DETAIL: Join in subqueries is not supported yet -SET citus.subquery_pushdown to TRUE; + avg +----- + +(1 row) + -- Check that we don't crash if there are not any shards. SELECT avg(unit_price) @@ -129,7 +131,8 @@ ERROR: cannot push down this subquery DETAIL: Shards of relations in subquery need to have 1-to-1 shard partitioning -- Update metadata in order to make all shards equal. UPDATE pg_dist_shard SET shardmaxvalue = '14947' WHERE shardid = 270003; --- If group by is not on partition column then we error out. +-- If group by is not on partition column then we error out from single table +-- repartition code path SELECT avg(order_count) FROM @@ -140,8 +143,8 @@ FROM lineitem_subquery GROUP BY l_suppkey) AS order_counts; -ERROR: cannot push down this subquery -DETAIL: Group by list without partition column is currently unsupported +ERROR: cannot use real time executor with repartition jobs +HINT: Set citus.task_executor_type to "task-tracker". -- Check that we error out if join is not on partition columns. SELECT avg(unit_price) @@ -154,8 +157,8 @@ FROM orders_subquery GROUP BY l_orderkey) AS unit_prices; -ERROR: cannot push down this subquery -DETAIL: Relations need to be joining on partition columns +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. SELECT avg(unit_price) FROM @@ -169,23 +172,24 @@ FROM l_orderkey = o_custkey GROUP BY l_orderkey) AS unit_prices; -ERROR: cannot push down this subquery -DETAIL: Relations need to be joining on partition columns --- Check that we error out if there is union all. +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- Check that we error out if there is non relation subqueries SELECT count(*) FROM ( (SELECT l_orderkey FROM lineitem_subquery) UNION ALL (SELECT 1::bigint) ) b; -ERROR: could not run distributed query with complex table expressions +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported -- Check that we error out if queries in union do not include partition columns. SELECT count(*) FROM ( (SELECT l_orderkey FROM lineitem_subquery) UNION (SELECT l_partkey FROM lineitem_subquery) ) b; -ERROR: cannot push down this subquery -DETAIL: Union clauses need to select partition columns +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. -- Check that we run union queries if partition column is selected. SELECT count(*) FROM ( @@ -197,7 +201,7 @@ SELECT count(*) FROM 2985 (1 row) --- Check that we error out if the outermost query has subquery join. +-- Check that we error out if inner query has Limit but subquery_pushdown is not set SELECT avg(o_totalprice/l_quantity) FROM @@ -217,8 +221,34 @@ FROM orders_subquery WHERE lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true; -ERROR: cannot perform distributed planning on this query -DETAIL: Join in subqueries is not supported yet +ERROR: cannot push down this subquery +DETAIL: Limit in subquery is currently unsupported +-- Limit is only supported when subquery_pushdown is set +-- Check that we error out if inner query has limit but outer query has not. +SET citus.subquery_pushdown to ON; +SELECT + avg(o_totalprice/l_quantity) +FROM + (SELECT + l_orderkey, + l_quantity + FROM + lineitem_subquery + ORDER BY + l_quantity + LIMIT 10 + ) lineitem_quantities + JOIN LATERAL + (SELECT + o_totalprice + FROM + orders_subquery + WHERE + lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true; +ERROR: cannot push down this subquery +DETAIL: Limit in subquery without limit in the outermost query is unsupported +-- reset the flag for next query +SET citus.subquery_pushdown to OFF; -- Check that we error out if the outermost query is a distinct clause. SELECT count(DISTINCT a) @@ -227,6 +257,8 @@ FROM ( count(*) a FROM lineitem_subquery + GROUP BY + l_orderkey ) z; ERROR: cannot push down this subquery DETAIL: distinct in the outermost query is unsupported @@ -371,11 +403,13 @@ SELECT master_create_worker_shards('subquery_pruning_varchar_test_table', 4, 1); (1 row) -SET citus.subquery_pushdown TO TRUE; SET client_min_messages TO DEBUG2; SELECT * FROM (SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE a = 'onder' GROUP BY a) AS foo; +DEBUG: Skipping the target shard interval 270005 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270006 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270008 because SELECT query is pruned away for the interval count ------- (0 rows) @@ -383,6 +417,9 @@ AS foo; SELECT * FROM (SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE 'eren' = a GROUP BY a) AS foo; +DEBUG: Skipping the target shard interval 270005 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270007 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270008 because SELECT query is pruned away for the interval count ------- (0 rows) @@ -427,18 +464,415 @@ CREATE TYPE user_composite_type AS tenant_id BIGINT, user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); \c - - - :worker_1_port CREATE TYPE user_composite_type AS ( tenant_id BIGINT, user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); \c - - - :worker_2_port CREATE TYPE user_composite_type AS ( tenant_id BIGINT, user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); \c - - - :master_port CREATE TABLE events ( composite_id user_composite_type, @@ -496,7 +930,6 @@ SELECT master_create_empty_shard('users') AS new_shard_id UPDATE pg_dist_shard SET shardminvalue = '(2,2000000001)', shardmaxvalue = '(2,4300000000)' WHERE shardid = :new_shard_id; \COPY users FROM STDIN WITH CSV -SET citus.subquery_pushdown TO TRUE; -- Simple join subquery pushdown SELECT avg(array_length(events, 1)) AS event_average @@ -515,8 +948,7 @@ FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type IN ('click', 'submit', 'pay')) AS subquery @@ -543,14 +975,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -558,14 +990,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -583,8 +1015,7 @@ FROM events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay') AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -612,14 +1043,14 @@ SELECT (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -627,14 +1058,14 @@ SELECT (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -643,6 +1074,7 @@ SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -651,13 +1083,11 @@ SELECT events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay' GROUP BY - tenant_id, - user_id + composite_id HAVING COUNT(*) > 2) AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -673,7 +1103,10 @@ ORDER BY 3.0000000000000000 | 0 (1 row) + -- Lateral join subquery pushdown +-- set subquery_pushdown since there is limit in the query +SET citus.subquery_pushdown to ON; SELECT tenant_id, user_id, @@ -689,6 +1122,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -707,8 +1141,7 @@ FROM FROM events WHERE - (composite_id).tenant_id = subquery_top.tenant_id AND - (composite_id).user_id = subquery_top.user_id + (composite_id) = subquery_top.composite_id ORDER BY event_time DESC LIMIT @@ -730,6 +1163,8 @@ LIMIT 1 | 1001 | 1472807115 | {click,submit,pay} (3 rows) +-- reset the flag for next query +SET citus.subquery_pushdown to OFF; -- Same queries above with explain -- Simple join subquery pushdown EXPLAIN (COSTS OFF) @@ -750,36 +1185,34 @@ FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type IN ('click', 'submit', 'pay')) AS subquery GROUP BY tenant_id, user_id) AS subquery; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------ Aggregate -> Custom Scan (Citus Real-Time) - Task Count: 2 - Tasks Shown: One of 2 + Task Count: 4 + Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression -> Aggregate -> GroupAggregate - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Merge Join - Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) - -> Sort - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Seq Scan on users_270013 users - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Sort - Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) - -> Seq Scan on events_270009 events - Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) -(19 rows) + Group Key: (((NULL::user_composite_type)).tenant_id), (((NULL::user_composite_type)).user_id) + -> Sort + Sort Key: (((NULL::user_composite_type)).tenant_id), (((NULL::user_composite_type)).user_id) + -> Hash Join + Hash Cond: ((NULL::user_composite_type) = events.composite_id) + -> Result + One-Time Filter: false + -> Hash + -> Seq Scan on events_270011 events + Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) +(18 rows) -- Union and left join subquery pushdown EXPLAIN (COSTS OFF) @@ -797,14 +1230,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -812,14 +1245,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -837,21 +1270,20 @@ FROM events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay') AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, hasdone) AS subquery_top GROUP BY hasdone; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- HashAggregate Group Key: remote_scan.hasdone -> Custom Scan (Citus Real-Time) - Task Count: 2 - Tasks Shown: One of 2 + Task Count: 4 + Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression -> GroupAggregate @@ -860,34 +1292,35 @@ GROUP BY Sort Key: subquery_top.hasdone -> Subquery Scan on subquery_top -> GroupAggregate - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) + Group Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) -> Sort - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Merge Left Join - Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id))) + Sort Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) + -> Hash Right Join + Hash Cond: (events.composite_id = subquery_1.composite_id) -> Unique -> Sort - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time - -> Append - -> Nested Loop - Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id)) - -> Seq Scan on events_270009 events - Filter: ((event_type)::text = 'click'::text) - -> Seq Scan on users_270013 users - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Nested Loop - Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id)) - -> Seq Scan on events_270009 events_1 - Filter: ((event_type)::text = 'submit'::text) - -> Seq Scan on users_270013 users_1 - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Materialize - -> Unique - -> Sort - Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) - -> Seq Scan on events_270009 events_2 - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) -(40 rows) + Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) + -> Seq Scan on events_270011 events + Filter: (((event_type)::text = 'pay'::text) AND (composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Hash + -> Subquery Scan on subquery_1 + -> Unique + -> Sort + Sort Key: (((NULL::user_composite_type)).tenant_id), (((NULL::user_composite_type)).user_id), (NULL::user_composite_type), ('action=>1'::text), events_1.event_time + -> Append + -> Nested Loop + Join Filter: ((NULL::user_composite_type) = events_1.composite_id) + -> Result + One-Time Filter: false + -> Seq Scan on events_270011 events_1 + Filter: ((event_type)::text = 'click'::text) + -> Nested Loop + Join Filter: ((NULL::user_composite_type) = events_2.composite_id) + -> Result + One-Time Filter: false + -> Seq Scan on events_270011 events_2 + Filter: ((event_type)::text = 'submit'::text) +(41 rows) -- Union, left join and having subquery pushdown EXPLAIN (COSTS OFF) @@ -905,14 +1338,14 @@ EXPLAIN (COSTS OFF) (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -920,14 +1353,14 @@ EXPLAIN (COSTS OFF) (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -936,6 +1369,7 @@ EXPLAIN (COSTS OFF) (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -944,13 +1378,11 @@ EXPLAIN (COSTS OFF) events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay' GROUP BY - tenant_id, - user_id + composite_id HAVING COUNT(*) > 2) AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -963,6 +1395,8 @@ ORDER BY count_pay; ERROR: bogus varattno for OUTER_VAR var: 3 -- Lateral join subquery pushdown +-- set subquery_pushdown due to limit in the query +SET citus.subquery_pushdown to ON; EXPLAIN (COSTS OFF) SELECT tenant_id, @@ -979,6 +1413,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -997,8 +1432,7 @@ FROM FROM events WHERE - (composite_id).tenant_id = subquery_top.tenant_id AND - (composite_id).user_id = subquery_top.user_id + (composite_id) = subquery_top.composite_id ORDER BY event_time DESC LIMIT @@ -1013,14 +1447,14 @@ ORDER BY user_lastseen DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------ Limit -> Sort Sort Key: remote_scan.user_lastseen DESC -> Custom Scan (Citus Real-Time) - Task Count: 2 - Tasks Shown: One of 2 + Task Count: 4 + Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression -> Limit @@ -1034,13 +1468,15 @@ LIMIT -> Limit -> Sort Sort Key: users.lastseen DESC - -> Seq Scan on users_270013 users - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Subquery Scan on users + -> Result + One-Time Filter: false -> Limit -> Sort Sort Key: events.event_time DESC - -> Seq Scan on events_270009 events - Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id))) -(26 rows) + -> Seq Scan on events_270011 events + Filter: (composite_id = users.composite_id) +(27 rows) +SET citus.subquery_pushdown to OFF; SET citus.enable_router_execution TO 'true'; diff --git a/src/test/regress/output/multi_subquery_0.source b/src/test/regress/output/multi_subquery_0.source index 625e1e799..205379549 100644 --- a/src/test/regress/output/multi_subquery_0.source +++ b/src/test/regress/output/multi_subquery_0.source @@ -53,7 +53,7 @@ SELECT master_create_distributed_table('orders_subquery', 'o_orderkey', 'range') (1 row) SET citus.enable_router_execution TO 'false'; --- Check that we don't allow subquery pushdown in default settings. +-- Check that we allow subquery pushdown in default settings. SELECT avg(unit_price) FROM @@ -67,9 +67,11 @@ FROM l_orderkey = o_orderkey GROUP BY l_orderkey) AS unit_prices; -ERROR: cannot perform distributed planning on this query -DETAIL: Join in subqueries is not supported yet -SET citus.subquery_pushdown to TRUE; + avg +----- + +(1 row) + -- Check that we don't crash if there are not any shards. SELECT avg(unit_price) @@ -129,7 +131,8 @@ ERROR: cannot push down this subquery DETAIL: Shards of relations in subquery need to have 1-to-1 shard partitioning -- Update metadata in order to make all shards equal. UPDATE pg_dist_shard SET shardmaxvalue = '14947' WHERE shardid = 270003; --- If group by is not on partition column then we error out. +-- If group by is not on partition column then we error out from single table +-- repartition code path SELECT avg(order_count) FROM @@ -140,8 +143,8 @@ FROM lineitem_subquery GROUP BY l_suppkey) AS order_counts; -ERROR: cannot push down this subquery -DETAIL: Group by list without partition column is currently unsupported +ERROR: cannot use real time executor with repartition jobs +HINT: Set citus.task_executor_type to "task-tracker". -- Check that we error out if join is not on partition columns. SELECT avg(unit_price) @@ -154,8 +157,8 @@ FROM orders_subquery GROUP BY l_orderkey) AS unit_prices; -ERROR: cannot push down this subquery -DETAIL: Relations need to be joining on partition columns +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. SELECT avg(unit_price) FROM @@ -169,23 +172,24 @@ FROM l_orderkey = o_custkey GROUP BY l_orderkey) AS unit_prices; -ERROR: cannot push down this subquery -DETAIL: Relations need to be joining on partition columns --- Check that we error out if there is union all. +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- Check that we error out if there is non relation subqueries SELECT count(*) FROM ( (SELECT l_orderkey FROM lineitem_subquery) UNION ALL (SELECT 1::bigint) ) b; -ERROR: could not run distributed query with complex table expressions +ERROR: cannot push down this subquery +DETAIL: Subqueries without relations are unsupported -- Check that we error out if queries in union do not include partition columns. SELECT count(*) FROM ( (SELECT l_orderkey FROM lineitem_subquery) UNION (SELECT l_partkey FROM lineitem_subquery) ) b; -ERROR: cannot push down this subquery -DETAIL: Union clauses need to select partition columns +ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position +DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list. -- Check that we run union queries if partition column is selected. SELECT count(*) FROM ( @@ -197,7 +201,7 @@ SELECT count(*) FROM 2985 (1 row) --- Check that we error out if the outermost query has subquery join. +-- Check that we error out if inner query has Limit but subquery_pushdown is not set SELECT avg(o_totalprice/l_quantity) FROM @@ -217,8 +221,34 @@ FROM orders_subquery WHERE lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true; -ERROR: cannot perform distributed planning on this query -DETAIL: Join in subqueries is not supported yet +ERROR: cannot push down this subquery +DETAIL: Limit in subquery is currently unsupported +-- Limit is only supported when subquery_pushdown is set +-- Check that we error out if inner query has limit but outer query has not. +SET citus.subquery_pushdown to ON; +SELECT + avg(o_totalprice/l_quantity) +FROM + (SELECT + l_orderkey, + l_quantity + FROM + lineitem_subquery + ORDER BY + l_quantity + LIMIT 10 + ) lineitem_quantities + JOIN LATERAL + (SELECT + o_totalprice + FROM + orders_subquery + WHERE + lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true; +ERROR: cannot push down this subquery +DETAIL: Limit in subquery without limit in the outermost query is unsupported +-- reset the flag for next query +SET citus.subquery_pushdown to OFF; -- Check that we error out if the outermost query is a distinct clause. SELECT count(DISTINCT a) @@ -227,6 +257,8 @@ FROM ( count(*) a FROM lineitem_subquery + GROUP BY + l_orderkey ) z; ERROR: cannot push down this subquery DETAIL: distinct in the outermost query is unsupported @@ -371,11 +403,13 @@ SELECT master_create_worker_shards('subquery_pruning_varchar_test_table', 4, 1); (1 row) -SET citus.subquery_pushdown TO TRUE; SET client_min_messages TO DEBUG2; SELECT * FROM (SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE a = 'onder' GROUP BY a) AS foo; +DEBUG: Skipping the target shard interval 270005 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270006 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270008 because SELECT query is pruned away for the interval count ------- (0 rows) @@ -383,6 +417,9 @@ AS foo; SELECT * FROM (SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE 'eren' = a GROUP BY a) AS foo; +DEBUG: Skipping the target shard interval 270005 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270007 because SELECT query is pruned away for the interval +DEBUG: Skipping the target shard interval 270008 because SELECT query is pruned away for the interval count ------- (0 rows) @@ -427,18 +464,415 @@ CREATE TYPE user_composite_type AS tenant_id BIGINT, user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); \c - - - :worker_1_port CREATE TYPE user_composite_type AS ( tenant_id BIGINT, user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); \c - - - :worker_2_port CREATE TYPE user_composite_type AS ( tenant_id BIGINT, user_id BIGINT ); +CREATE FUNCTION user_composite_type_equal(user_composite_type, +user_composite_type) +returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id < v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS +$$ +BEGIN + if v1.tenant_id > v2.tenant_id THEN + return true; + end if; + + if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN + RETURN true; + end if; + + RETURN false; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS +$$ +BEGIN + if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN + return 0; + end if; + + if v1.tenant_id > v2.tenant_id THEN + RETURN 1; + end if; + + if v1.tenant_id < v2.tenant_id THEN + RETURN -1; + end if; + + if v1.user_id > v2.user_id THEN + RETURN 1; + end if; + + if v1.user_id < v2.user_id THEN + RETURN -1; + end if; + + RETURN 0; +END; +$$ language 'plpgsql' immutable +returns NULL ON NULL input; +CREATE operator = +( leftarg = user_composite_type, rightarg = user_composite_type, +PROCEDURE = user_composite_type_equal, +commutator = =, +RESTRICT = eqsel, +JOIN = eqjoinsel, +merges, +hashes); +CREATE operator < +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_lt +); +CREATE operator > +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_gt +); +CREATE operator <= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_le +); +CREATE operator >= +( + leftarg = user_composite_type, rightarg = user_composite_type, + PROCEDURE = user_composite_type_ge +); +CREATE FUNCTION user_composite_type_hash(user_composite_type) +returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable +returns NULL ON NULL input; +CREATE OPERATOR CLASS btree_user_composite_ops +DEFAULT FOR TYPE user_composite_type USING btree +AS + OPERATOR 1 < , + OPERATOR 2 <= , + OPERATOR 3 = , + OPERATOR 4 >= , + OPERATOR 5 > , + FUNCTION 1 btree_comparison(user_composite_type, user_composite_type); +create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type); \c - - - :master_port CREATE TABLE events ( composite_id user_composite_type, @@ -496,7 +930,6 @@ SELECT master_create_empty_shard('users') AS new_shard_id UPDATE pg_dist_shard SET shardminvalue = '(2,2000000001)', shardmaxvalue = '(2,4300000000)' WHERE shardid = :new_shard_id; \COPY users FROM STDIN WITH CSV -SET citus.subquery_pushdown TO TRUE; -- Simple join subquery pushdown SELECT avg(array_length(events, 1)) AS event_average @@ -515,8 +948,7 @@ FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type IN ('click', 'submit', 'pay')) AS subquery @@ -543,14 +975,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -558,14 +990,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -583,8 +1015,7 @@ FROM events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay') AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -612,14 +1043,14 @@ SELECT (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -627,14 +1058,14 @@ SELECT (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -643,6 +1074,7 @@ SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -651,13 +1083,11 @@ SELECT events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay' GROUP BY - tenant_id, - user_id + composite_id HAVING COUNT(*) > 2) AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -673,7 +1103,10 @@ ORDER BY 3.0000000000000000 | 0 (1 row) + -- Lateral join subquery pushdown +-- set subquery_pushdown since there is limit in the query +SET citus.subquery_pushdown to ON; SELECT tenant_id, user_id, @@ -689,6 +1122,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -707,8 +1141,7 @@ FROM FROM events WHERE - (composite_id).tenant_id = subquery_top.tenant_id AND - (composite_id).user_id = subquery_top.user_id + (composite_id) = subquery_top.composite_id ORDER BY event_time DESC LIMIT @@ -730,6 +1163,8 @@ LIMIT 1 | 1001 | 1472807115 | {click,submit,pay} (3 rows) +-- reset the flag for next query +SET citus.subquery_pushdown to OFF; -- Same queries above with explain -- Simple join subquery pushdown EXPLAIN (COSTS OFF) @@ -750,36 +1185,29 @@ FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type IN ('click', 'submit', 'pay')) AS subquery GROUP BY tenant_id, user_id) AS subquery; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------------------------------ Aggregate -> Custom Scan (Citus Real-Time) - Task Count: 2 - Tasks Shown: One of 2 + Task Count: 4 + Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression -> Aggregate -> GroupAggregate Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Merge Join - Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) - -> Sort - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Seq Scan on users_270013 users - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Sort - Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) - -> Seq Scan on events_270009 events - Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) -(19 rows) + -> Sort + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Result + One-Time Filter: false +(13 rows) -- Union and left join subquery pushdown EXPLAIN (COSTS OFF) @@ -797,14 +1225,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -812,14 +1240,14 @@ FROM (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id) as composite_id, 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -837,54 +1265,46 @@ FROM events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay') AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, hasdone) AS subquery_top GROUP BY hasdone; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- HashAggregate Group Key: remote_scan.hasdone -> Custom Scan (Citus Real-Time) - Task Count: 2 - Tasks Shown: One of 2 + Task Count: 4 + Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression -> HashAggregate Group Key: COALESCE(('Has done paying'::text), 'Has not done paying'::text) -> GroupAggregate - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) + Group Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) -> Sort - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Merge Left Join - Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id))) + Sort Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) + -> Hash Right Join + Hash Cond: (events.composite_id = subquery_1.composite_id) -> Unique -> Sort - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time - -> Append - -> Nested Loop - Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id)) - -> Seq Scan on events_270009 events - Filter: ((event_type)::text = 'click'::text) - -> Seq Scan on users_270013 users - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Nested Loop - Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id)) - -> Seq Scan on events_270009 events_1 - Filter: ((event_type)::text = 'submit'::text) - -> Seq Scan on users_270013 users_1 - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Materialize - -> Unique - -> Sort - Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) - -> Seq Scan on events_270009 events_2 - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) -(37 rows) + Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) + -> Seq Scan on events_270011 events + Filter: (((event_type)::text = 'pay'::text) AND (composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Hash + -> Subquery Scan on subquery_1 + -> Unique + -> Sort + Sort Key: ((composite_id).tenant_id), ((composite_id).user_id), composite_id, ('action=>1'::text), event_time + -> Append + -> Result + One-Time Filter: false + -> Result + One-Time Filter: false +(30 rows) -- Union, left join and having subquery pushdown EXPLAIN (COSTS OFF) @@ -902,14 +1322,14 @@ EXPLAIN (COSTS OFF) (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>1'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'click') @@ -917,14 +1337,14 @@ EXPLAIN (COSTS OFF) (SELECT (users.composite_id).tenant_id, (users.composite_id).user_id, + (users.composite_id), 'action=>2'AS event, events.event_time FROM users, events WHERE - (users.composite_id).tenant_id = (events.composite_id).tenant_id AND - (users.composite_id).user_id = (events.composite_id).user_id AND + (users.composite_id) = (events.composite_id) AND users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'submit') @@ -933,6 +1353,7 @@ EXPLAIN (COSTS OFF) (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -941,13 +1362,11 @@ EXPLAIN (COSTS OFF) events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND event_type = 'pay' GROUP BY - tenant_id, - user_id + composite_id HAVING COUNT(*) > 2) AS subquery_2 ON - subquery_1.tenant_id = subquery_2.tenant_id AND - subquery_1.user_id = subquery_2.user_id + subquery_1.composite_id = subquery_2.composite_id GROUP BY subquery_1.tenant_id, subquery_1.user_id, @@ -960,6 +1379,8 @@ ORDER BY count_pay; ERROR: bogus varattno for OUTER_VAR var: 3 -- Lateral join subquery pushdown +-- set subquery_pushdown due to limit in the query +SET citus.subquery_pushdown to ON; EXPLAIN (COSTS OFF) SELECT tenant_id, @@ -976,6 +1397,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -994,8 +1416,7 @@ FROM FROM events WHERE - (composite_id).tenant_id = subquery_top.tenant_id AND - (composite_id).user_id = subquery_top.user_id + (composite_id) = subquery_top.composite_id ORDER BY event_time DESC LIMIT @@ -1010,34 +1431,35 @@ ORDER BY user_lastseen DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +------------------------------------------------------------------------------------------------------------ Limit -> Sort Sort Key: remote_scan.user_lastseen DESC -> Custom Scan (Citus Real-Time) - Task Count: 2 - Tasks Shown: One of 2 + Task Count: 4 + Tasks Shown: One of 4 -> Task Node: host=localhost port=57637 dbname=regression -> Limit -> Sort - Sort Key: (max(users.lastseen)) DESC + Sort Key: (max(lastseen)) DESC -> GroupAggregate - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + Group Key: ((composite_id).tenant_id), ((composite_id).user_id) -> Sort - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + Sort Key: ((composite_id).tenant_id), ((composite_id).user_id) -> Nested Loop Left Join -> Limit -> Sort - Sort Key: users.lastseen DESC - -> Seq Scan on users_270013 users - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + Sort Key: lastseen DESC + -> Result + One-Time Filter: false -> Limit -> Sort Sort Key: events.event_time DESC - -> Seq Scan on events_270009 events - Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id))) + -> Seq Scan on events_270011 events + Filter: (composite_id = composite_id) (26 rows) +SET citus.subquery_pushdown to OFF; SET citus.enable_router_execution TO 'true'; diff --git a/src/test/regress/sql/.gitignore b/src/test/regress/sql/.gitignore index a41eca963..98052d0c7 100644 --- a/src/test/regress/sql/.gitignore +++ b/src/test/regress/sql/.gitignore @@ -16,3 +16,4 @@ /worker_copy.sql /multi_complex_count_distinct.sql /multi_mx_copy_data.sql +/multi_insert_select_behavioral_analytics_create_table.sql diff --git a/src/test/regress/sql/multi_complex_expressions.sql b/src/test/regress/sql/multi_complex_expressions.sql index 236e8f4c6..4891b2103 100644 --- a/src/test/regress/sql/multi_complex_expressions.sql +++ b/src/test/regress/sql/multi_complex_expressions.sql @@ -161,13 +161,11 @@ SELECT count(*) FROM lineitem, orders WHERE l_orderkey + 1 = o_orderkey; -- Check that we can issue limit/offset queries -- OFFSET in subqueries are not supported --- Error in the planner when subquery pushdown is off -SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq; -SET citus.subquery_pushdown TO true; +-- Error in the planner when single repartition subquery +SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq; -- Error in the optimizer when subquery pushdown is on SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq; -SET citus.subquery_pushdown TO false; -- Simple LIMIT/OFFSET with ORDER BY SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20; diff --git a/src/test/regress/sql/multi_insert_select.sql b/src/test/regress/sql/multi_insert_select.sql index fd9154399..000dc1755 100644 --- a/src/test/regress/sql/multi_insert_select.sql +++ b/src/test/regress/sql/multi_insert_select.sql @@ -733,6 +733,47 @@ WHERE GROUP BY outer_most.id; + -- if the given filter was on value_1 as shown in the above, Citus could + -- push it down. But here the query is refused + INSERT INTO agg_events + (user_id) + SELECT raw_events_first.user_id + FROM raw_events_first, + raw_events_second + WHERE raw_events_second.user_id = raw_events_first.value_1 + AND raw_events_first.value_2 = 12; + + -- lets do some unsupported query tests with subqueries + -- foo is not joined on the partition key so the query is not + -- pushed down + INSERT INTO agg_events + (user_id, value_4_agg) + SELECT + outer_most.id, max(outer_most.value) + FROM + ( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first LEFT JOIN + reference_table + ON (raw_events_first.value_1 = reference_table.user_id)) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 + ON (f.id = f2.id)) as outer_most + GROUP BY + outer_most.id; INSERT INTO agg_events (value_4_agg, diff --git a/src/test/regress/sql/multi_insert_select_non_pushable_queries.sql b/src/test/regress/sql/multi_insert_select_non_pushable_queries.sql index c7089903b..3982900a8 100644 --- a/src/test/regress/sql/multi_insert_select_non_pushable_queries.sql +++ b/src/test/regress/sql/multi_insert_select_non_pushable_queries.sql @@ -649,3 +649,30 @@ FROM ) temp ON users_table.user_id = temp.user_id WHERE users_table.value_1 < 50; + +-- not supported since one of the queries doesn't have a relation +INSERT INTO agg_results (user_id, agg_time, value_2_agg) +SELECT + user_id, + user_lastseen, + array_length(event_array, 1) +FROM ( + SELECT + user_id, + max(u.time) as user_lastseen, + array_agg(event_type ORDER BY u.time) AS event_array + FROM ( + SELECT user_id, time, value_3 as val_3 + FROM users_table + WHERE + user_id >= 10 AND user_id <= 70 AND + users_table.value_1 > 10 AND users_table.value_1 < 12 + ) u LEFT JOIN LATERAL ( + SELECT event_type, time + FROM events_table, (SELECT 1 as x) as f + WHERE user_id = u.user_id AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) t ON true + GROUP BY user_id +) AS shard_union +ORDER BY user_lastseen DESC; diff --git a/src/test/regress/sql/multi_subquery_behavioral_analytics.sql b/src/test/regress/sql/multi_subquery_behavioral_analytics.sql new file mode 100644 index 000000000..734bfd784 --- /dev/null +++ b/src/test/regress/sql/multi_subquery_behavioral_analytics.sql @@ -0,0 +1,1604 @@ +-- +-- multi subquery behavioral analytics queries aims to expand existing subquery pushdown +-- regression tests to cover more cases +-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- + +-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests +-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1430000; +-- ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1430000; + +SET citus.enable_router_execution TO FALSE; + +------------------------------------ +-- Vanilla funnel query +------------------------------------ +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT u.user_id, e.event_type::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1; + +------------------------------------ +-- Funnel grouped by whether or not a user has done an event +-- This has multiple subqueries joinin at the top level +------------------------------------ +SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event +FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + ( + SELECT u.user_id, 'step=>1'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102) + ) + UNION + ( + SELECT u.user_id, 'step=>2'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (103, 104, 105) + ) + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event +) t GROUP BY user_id, hasdone_event +ORDER BY user_id; + +-- same query but multiple joins are one level below, returns count of row instead of actual rows +SELECT count(*) +FROM ( + SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event + FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + ( + SELECT u.user_id, 'step=>1'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102) + ) + UNION + ( + SELECT u.user_id, 'step=>2'::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (103, 104, 105) + ) + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event + ) t GROUP BY user_id, hasdone_event + ORDER BY user_id) u; + +-- Same queries written without unions +SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event +FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + SELECT + u.user_id, + CASE WHEN e.event_type IN (100, 101, 102) THEN 'step=>1'::text else 'step==>2'::text END AS event, + e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102, 103, 104, 105) + GROUP BY 1,2,3 + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event +) t GROUP BY user_id, hasdone_event +ORDER BY user_id; + +-- same query but multiple joins are one level below, returns count of row instead of actual rows +SELECT count(*) +FROM ( + SELECT user_id, sum(array_length(events_table, 1)), length(hasdone_event), hasdone_event + FROM ( + SELECT + t1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(hasdone_event, 'Has not done event') AS hasdone_event + FROM ( + SELECT + u.user_id, + CASE WHEN e.event_type in (100, 101, 102) then 'step=>1'::text else 'step==>2'::text END AS event, + e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND u.user_id >= 10 + AND u.user_id <= 25 + AND e.event_type IN (100, 101, 102, 103, 104, 105) + GROUP BY 1,2,3 + ) t1 LEFT JOIN ( + SELECT DISTINCT user_id, + 'Has done event'::TEXT AS hasdone_event + FROM events_table AS e + WHERE e.user_id >= 10 + AND e.user_id <= 25 + AND e.event_type IN (106, 107, 108) + ) t2 ON (t1.user_id = t2.user_id) + GROUP BY t1.user_id, hasdone_event + ) t GROUP BY user_id, hasdone_event + ORDER BY user_id) u; + +------------------------------------ +-- Funnel, grouped by the number of times a user has done an event +------------------------------------ +SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + (SELECT + users_table.user_id, + 'action=>1'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) + UNION + (SELECT + users_table.user_id, + 'action=>2'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 12 AND events_table.event_type < 14 + ) + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + +SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + (SELECT + users_table.user_id, + 'action=>1'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) + UNION + (SELECT + users_table.user_id, + 'action=>2'AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + events_table.event_type > 12 AND events_table.event_type < 14 + ) + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +HAVING + avg(array_length(events_table, 1)) > 0 +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + +-- Same queries rewritten without using unions + SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + SELECT + users_table.user_id, + CASE + WHEN + events_table.event_type > 10 AND events_table.event_type < 12 + THEN 'action=>1' + ELSE 'action=>2' + END AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + (events_table.event_type > 10 AND events_table.event_type < 12 + OR + events_table.event_type > 12 AND events_table.event_type < 14) + GROUP BY 1, 2, 3 + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + +SELECT + user_id, + avg(array_length(events_table, 1)) AS event_average, + count_pay + FROM ( + SELECT + subquery_1.user_id, + array_agg(event ORDER BY time) AS events_table, + COALESCE(count_pay, 0) AS count_pay + FROM + ( + SELECT + users_table.user_id, + CASE WHEN events_table.event_type > 10 AND events_table.event_type < 12 THEN 'action=>1' ELSE 'action=>2' END AS event, + events_table.time + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id AND + users_table.user_id >= 10 AND + users_table.user_id <= 70 AND + (events_table.event_type > 10 AND events_table.event_type < 12 + OR + events_table.event_type > 12 AND events_table.event_type < 14) + GROUP BY 1, 2, 3 + ) AS subquery_1 + LEFT JOIN + (SELECT + user_id, + COUNT(*) AS count_pay + FROM + users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 15 AND users_table.value_1 < 17 + GROUP BY + user_id + HAVING + COUNT(*) > 1) AS subquery_2 + ON + subquery_1.user_id = subquery_2.user_id + GROUP BY + subquery_1.user_id, + count_pay) AS subquery_top +WHERE + array_ndims(events_table) > 0 +GROUP BY + count_pay, user_id +HAVING + avg(array_length(events_table, 1)) > 0 +ORDER BY + event_average DESC, count_pay DESC, user_id DESC; + +------------------------------------ +-- Most recently seen users_table events_table +------------------------------------ +-- Note that we don't use ORDER BY/LIMIT yet +------------------------------------ +SELECT + user_id, + user_lastseen, + array_length(event_array, 1) +FROM ( + SELECT + user_id, + max(u.time) as user_lastseen, + array_agg(event_type ORDER BY u.time) AS event_array + FROM ( + SELECT user_id, time + FROM users_table + WHERE + user_id >= 10 AND + user_id <= 70 AND + users_table.value_1 > 10 AND users_table.value_1 < 12 + ) u LEFT JOIN LATERAL ( + SELECT event_type, time + FROM events_table + WHERE user_id = u.user_id AND + events_table.event_type > 10 AND events_table.event_type < 12 + ) t ON true + GROUP BY user_id +) AS shard_union +ORDER BY user_lastseen DESC, user_id; + +------------------------------------ +-- Count the number of distinct users_table who are in segment X and Y and Z +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT DISTINCT user_id +FROM users_table +WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20) + AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40) + AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60); + +------------------------------------ +-- Find customers who have done X, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_1 > 101 AND value_1 < 110 + AND value_2 >= 5 + AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 101 AND event_type < 110 AND value_3 > 100 AND user_id = users_table.user_id); + +------------------------------------ +-- Customers who haven’t done X, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_1 = 101 + AND value_2 >= 5 + AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND user_id = users_table.user_id); + +------------------------------------ +-- Customers who have done X and Y, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_1 > 100 + AND value_2 >= 5 + AND EXISTS (SELECT user_id FROM events_table WHERE event_type != 100 AND value_3 > 100 AND user_id = users_table.user_id) + AND EXISTS (SELECT user_id FROM events_table WHERE event_type = 101 AND value_3 > 100 AND user_id = users_table.user_id); + +------------------------------------ +-- Customers who have done X and haven’t done Y, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, value_2 FROM users_table WHERE + value_2 >= 5 + AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id) + AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id); + +------------------------------------ +-- Customers who have done X more than 2 times, and satisfy other customer specific criteria +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, + value_2 + FROM users_table + WHERE value_1 > 100 + AND value_1 < 124 + AND value_2 >= 5 + AND EXISTS (SELECT user_id + FROM events_table + WHERE event_type > 100 + AND event_type < 124 + AND value_3 > 100 + AND user_id = users_table.user_id + GROUP BY user_id + HAVING Count(*) > 2); + +------------------------------------ +-- Find me all users_table who logged in more than once +------------------------------------ +SELECT user_id, value_1 from +( + SELECT + user_id, value_1 From users_table + WHERE + value_2 > 100 and user_id = 15 + GROUP BY + value_1, user_id + HAVING + count(*) > 1 +) AS a +ORDER BY + user_id ASC, value_1 ASC; + +-- same query with additional filter to make it not router plannable +SELECT user_id, value_1 from +( + SELECT + user_id, value_1 From users_table + WHERE + value_2 > 100 and (user_id = 15 OR user_id = 16) + GROUP BY + value_1, user_id + HAVING count(*) > 1 +) AS a +ORDER BY + user_id ASC, value_1 ASC; + +------------------------------------ +-- Find me all users_table who has done some event and has filters +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id +FROM events_table +WHERE + event_type = 16 AND value_2 > 50 AND + user_id IN + (SELECT + user_id + FROM + users_table + WHERE + value_1 = 15 AND value_2 > 25 + ); + +------------------------------------ +-- Which events_table did people who has done some specific events_table +-- This query will be supported when we have subqueries in where clauses. +------------------------------------ +SELECT user_id, event_type FROM events_table +WHERE user_id in (SELECT user_id from events_table WHERE event_type > 500 and event_type < 505) +GROUP BY user_id, event_type; + +------------------------------------ +-- Find me all the users_table who has done some event more than three times +------------------------------------ +SELECT user_id FROM +( + SELECT + user_id + FROM + events_table + WHERE + event_type = 901 + GROUP BY + user_id + HAVING + count(*) > 3 +) AS a +ORDER BY + user_id; + +------------------------------------ +-- Find my assets that have the highest probability and fetch their metadata +------------------------------------ +CREATE TEMP TABLE assets AS +SELECT + users_table.user_id, users_table.value_1, prob +FROM + users_table + JOIN + (SELECT + ma.user_id, (GREATEST(coalesce(ma.value_4 / 250, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma, events_table as short_list + WHERE + short_list.user_id = ma.user_id and ma.value_1 < 50 and short_list.event_type < 50 + ) temp + ON users_table.user_id = temp.user_id + WHERE + users_table.value_1 < 50; + + -- get some statistics from the aggregated results to ensure the results are correct +SELECT count(*), count(DISTINCT user_id), avg(user_id) FROM assets; + +DROP TABLE assets; + +-- count number of distinct users who have value_1 equal to 5 or 13 but not 3 +-- original query that fails +SELECT count(*) FROM +( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') AND + user_id NOT IN (select user_id from users_table where value_1 = '3') + GROUP BY + user_id + HAVING + count(distinct value_1) = 2 +) as foo; + +-- previous push down query +SELECT subquery_count FROM + (SELECT count(*) as subquery_count FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY + user_id + HAVING + count(distinct value_1) = 2) as a + LEFT JOIN + (SELECT + user_id + FROM + users_table + WHERE + (value_1 = '3') + GROUP BY + user_id) as b + ON a.user_id = b.user_id + WHERE + b.user_id IS NULL + GROUP BY + a.user_id + ) AS inner_subquery; + +-- new pushdown query without single range table entry at top requirement +SELECT count(*) as subquery_count +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY + user_id + HAVING + count(distinct value_1) = 2 + ) as a + LEFT JOIN ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '3') + GROUP BY + user_id) AS b + ON a.user_id = b.user_id +WHERE + b.user_id IS NULL +GROUP BY + a.user_id; + +-- most queries below has limit clause +-- therefore setting subquery_pushdown flag for all +SET citus.subquery_pushdown to ON; + +-- multi-subquery-join +-- The first query has filters on partion column to make it router plannable +-- but it is processed by logical planner since we disabled router execution +SELECT + e1.user_id, + sum(view_homepage) AS viewed_homepage, + sum(use_demo) AS use_demo, + sum(enter_credit_card) AS entered_credit_card, + sum(submit_card_info) as submit_card_info, + sum(see_bought_screen) as see_bought_screen +FROM ( + -- Get the first time each user viewed the homepage. + SELECT + user_id, + 1 AS view_homepage, + min(time) AS view_homepage_time + FROM events_table + WHERE user_id = 1 and + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90) + GROUP BY user_id +) e1 LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS use_demo, + time AS use_demo_time + FROM events_table + WHERE + user_id = e1.user_id AND user_id = 1 and + event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91) + ORDER BY time + LIMIT 1 +) e2 ON true LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS enter_credit_card, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e2.user_id AND user_id = 1 and + event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92) + ORDER BY time + LIMIT 1 +) e3 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS submit_card_info, + user_id, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e3.user_id AND user_id = 1 and + event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93) + ORDER BY time + LIMIT 1 +) e4 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS see_bought_screen + FROM events_table + WHERE + user_id = e4.user_id AND user_id = 1 and + event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94) + ORDER BY time + LIMIT 1 +) e5 ON true +WHERE + e1.user_id = 1 +GROUP BY + e1.user_id +LIMIT 1; + +-- Same query without all limitations +SELECT + e1.user_id, + sum(view_homepage) AS viewed_homepage, + sum(use_demo) AS use_demo, + sum(enter_credit_card) AS entered_credit_card, + sum(submit_card_info) as submit_card_info, + sum(see_bought_screen) as see_bought_screen +FROM ( + -- Get the first time each user viewed the homepage. + SELECT + user_id, + 1 AS view_homepage, + min(time) AS view_homepage_time + FROM events_table + WHERE + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90) + GROUP BY user_id +) e1 LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS use_demo, + time AS use_demo_time + FROM events_table + WHERE + user_id = e1.user_id AND + event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91) + ORDER BY time +) e2 ON true LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS enter_credit_card, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e2.user_id AND + event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92) + ORDER BY time +) e3 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS submit_card_info, + user_id, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e3.user_id AND + event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93) + ORDER BY time +) e4 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS see_bought_screen + FROM events_table + WHERE + user_id = e4.user_id AND + event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94) + ORDER BY time +) e5 ON true +GROUP BY e1.user_id +ORDER BY 6 DESC NULLS LAST, 5 DESC NULLS LAST, 4 DESC NULLS LAST, 3 DESC NULLS LAST, 2 DESC NULLS LAST, 1 +LIMIT 15; + +-- Same query without all limitations but uses having() to show only those submitted their credit card info +SELECT + e1.user_id, + sum(view_homepage) AS viewed_homepage, + sum(use_demo) AS use_demo, + sum(enter_credit_card) AS entered_credit_card, + sum(submit_card_info) as submit_card_info, + sum(see_bought_screen) as see_bought_screen +FROM ( + -- Get the first time each user viewed the homepage. + SELECT + user_id, + 1 AS view_homepage, + min(time) AS view_homepage_time + FROM events_table + WHERE + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90) + GROUP BY user_id +) e1 LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS use_demo, + time AS use_demo_time + FROM events_table + WHERE + user_id = e1.user_id AND + event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91) + ORDER BY time +) e2 ON true LEFT JOIN LATERAL ( + SELECT + user_id, + 1 AS enter_credit_card, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e2.user_id AND + event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92) + ORDER BY time +) e3 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS submit_card_info, + user_id, + time AS enter_credit_card_time + FROM events_table + WHERE + user_id = e3.user_id AND + event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93) + ORDER BY time +) e4 ON true LEFT JOIN LATERAL ( + SELECT + 1 AS see_bought_screen + FROM events_table + WHERE + user_id = e4.user_id AND + event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94) + ORDER BY time +) e5 ON true +group by e1.user_id +HAVING sum(submit_card_info) > 0 +ORDER BY 6 DESC NULLS LAST, 5 DESC NULLS LAST, 4 DESC NULLS LAST, 3 DESC NULLS LAST, 2 DESC NULLS LAST, 1 +LIMIT 15; + +-- Explain analyze on this query fails due to #756 +-- avg expression used on order by +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN ( + SELECT + user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3)) AS b +ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +ORDER BY + avg(b.value_3), 2, 1 +LIMIT 5; + +-- add having to the same query +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY user_id + HAVING count(distinct value_1) > 88 + ) as a + LEFT JOIN ( + SELECT + user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3)) AS b +ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +HAVING + sum(b.value_3) > 50000 +ORDER BY + avg(b.value_3), 2, 1 +LIMIT 5; + +-- avg on the value_3 is not a resjunk +SELECT a.user_id, avg(b.value_2) as subquery_avg, avg(b.value_3) +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + ( + SELECT + user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ) AS b + ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +ORDER BY + avg(b.value_3) DESC, 2, 1 +LIMIT 5; + +-- a powerful query structure that analyzes users/events +-- using (relation JOIN subquery JOIN relation) +SELECT u.user_id, sub.value_2, sub.value_3, COUNT(e2.user_id) counts +FROM + users_table u + LEFT OUTER JOIN LATERAL + (SELECT + * + FROM + events_table e1 + WHERE + e1.user_id = u.user_id + ORDER BY + e1.value_3 DESC + LIMIT 1 + ) sub + ON true + LEFT OUTER JOIN events_table e2 + ON e2.user_id = sub.user_id +WHERE + e2.value_2 > 10 AND e2.value_2 < 50 AND u.value_2 > 10 AND u.value_2 < 50 +GROUP BY + u.user_id, sub.value_2, sub.value_3 +ORDER BY + 4 DESC, 1 DESC, 2 ASC, 3 ASC +LIMIT 10; + +-- distinct users joined with events +SELECT + avg(events_table.event_type) as avg_type, + count(*) as users_count +FROM events_table + JOIN + (SELECT + DISTINCT user_id + FROM + users_table + ) as distinct_users + ON distinct_users.user_id = events_table.user_id +GROUP BY + distinct_users.user_id +ORDER BY + users_count desc, avg_type DESC +LIMIT 5; + +-- reduce the data set, aggregate and join +SELECT + events_table.event_type, + users_count.ct +FROM events_table + JOIN + (SELECT distinct_users.user_id, count(1) as ct + FROM + (SELECT + user_id + FROM + users_table + ) as distinct_users + GROUP BY + distinct_users.user_id + ) as users_count + ON users_count.user_id = events_table.user_id +ORDER BY + users_count.ct desc, event_type DESC +LIMIT 5; + +--- now, test (subquery JOIN subquery) +SELECT n1.user_id, count_1, total_count +FROM + (SELECT + user_id, count(1) as count_1 + FROM + users_table + GROUP BY + user_id + ) n1 + INNER JOIN + ( + SELECT + user_id, count(1) as total_count + FROM + events_table + GROUP BY + user_id, event_type + ) n2 + ON (n2.user_id = n1.user_id) +ORDER BY + total_count DESC, count_1 DESC, 1 DESC +LIMIT 10; + +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + (SELECT + DISTINCT ON (user_id) user_id, value_2, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ORDER BY + 1,2,3 + ) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NOT NULL +GROUP BY a.user_id +ORDER BY avg(b.value_3), 2, 1 +LIMIT 5; + +-- distinct clause must include partition column +-- when used in target list +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + (SELECT + DISTINCT ON (value_2) value_2 , user_id, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ORDER BY + 1,2,3 + ) AS b + USING (user_id) +GROUP BY user_id; + +SELECT a.user_id, avg(b.value_2) as subquery_avg +FROM + (SELECT + user_id + FROM + users_table + WHERE + (value_1 > 5) + GROUP BY + user_id + HAVING + count(distinct value_1) > 88 + ) as a + LEFT JOIN + (SELECT + DISTINCT ON (value_2, user_id) value_2 , user_id, value_3 + FROM + users_table + WHERE + (value_1 > 3) + ORDER BY + 1,2,3 + ) AS b + ON a.user_id = b.user_id +WHERE + b.user_id IS NOT NULL +GROUP BY + a.user_id +ORDER BY + avg(b.value_3), 2, 1 +LIMIT 5; + +SELECT user_id, event_type +FROM + (SELECT * + FROM + ( + (SELECT + event_type, user_id as a_user_id + FROM + events_table) AS a + JOIN + (SELECT + ma.user_id AS user_id, ma.value_2 AS value_2, + (GREATEST(coalesce((ma.value_3 * ma.value_2) / 20, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma + WHERE + (ma.value_2 > 100) + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS ma + ON (a.a_user_id = ma.user_id) + ) AS inner_sub + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS outer_sub +ORDER BY + prob DESC, event_type DESC, user_id DESC +LIMIT 10; + +-- very similar query but produces different result due to +-- ordering difference in the previous one's inner query +SELECT user_id, event_type +FROM + (SELECT + event_type, user_id as a_user_id + FROM + events_table) AS a + JOIN + (SELECT + ma.user_id AS user_id, ma.value_2 AS value_2, + (GREATEST(coalesce((ma.value_3 * ma.value_2) / 20, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma + WHERE + (ma.value_2 > 100) + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS ma + ON (a.a_user_id = ma.user_id) +ORDER BY + prob DESC, event_type DESC, user_id DESC +LIMIT 10; + +-- now they produce the same result when ordering fixed in 'outer_sub' +SELECT user_id, event_type +FROM + (SELECT * + FROM + ( + (SELECT + event_type, user_id as a_user_id + FROM + events_table + ) AS a + JOIN + (SELECT + ma.user_id AS user_id, ma.value_2 AS value_2, + (GREATEST(coalesce((ma.value_3 * ma.value_2) / 20, 0.0) + GREATEST(1.0))) / 2 AS prob + FROM + users_table AS ma + WHERE + (ma.value_2 > 100) + ORDER BY + prob DESC, user_id DESC + LIMIT 10 + ) AS ma + ON (a.a_user_id = ma.user_id) + ) AS inner_sub + ORDER BY + prob DESC, event_type DESC, user_id DESC + LIMIT 10 + ) AS outer_sub +ORDER BY + prob DESC, event_type DESC, user_id DESC +LIMIT 10; + +-- this is one complex join query derived from a user's production query +-- first declare the function on workers on master +-- With array_index: +SELECT * FROM run_command_on_workers('CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) + RETURNS INT AS $$ + SELECT i + FROM (SELECT generate_series(array_lower($1, 1), array_upper($1, 1))) g(i) + WHERE $1 [i] = $2 + LIMIT 1; + $$ LANGUAGE sql') +ORDER BY 1,2; + +CREATE OR REPLACE FUNCTION array_index(ANYARRAY, ANYELEMENT) + RETURNS INT AS $$ + SELECT i + FROM (SELECT generate_series(array_lower($1, 1), array_upper($1, 1))) g(i) + WHERE $1 [i] = $2 + LIMIT 1; + $$ LANGUAGE sql; + +SELECT * +FROM + (SELECT * + FROM ( + (SELECT user_id AS user_id_e, + event_type AS event_type_e + FROM events_table ) AS ma_e + JOIN + (SELECT value_2, + value_3, + user_id + FROM + (SELECT * + FROM ( + (SELECT user_id_p AS user_id + FROM + (SELECT * + FROM ( + (SELECT + user_id AS user_id_p + FROM + events_table + WHERE + (event_type IN (1,2,3,4,5)) ) AS ma_p + JOIN + (SELECT + user_id AS user_id_a + FROM + users_table + WHERE + (value_2 % 5 = 1) ) AS a + ON (a.user_id_a = ma_p.user_id_p) ) ) AS a_ma_p ) AS inner_filter_q + JOIN + (SELECT + value_2, value_3, user_id AS user_id_ck + FROM + events_table + WHERE + event_type = ANY(ARRAY [10, 11, 12]) + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 ) + AS ma_ck ON (ma_ck.user_id_ck = inner_filter_q.user_id) ) + AS inner_sub_q + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 ) + AS outer_sub_q + ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10) + AS inner_search_q + ON (ma_e.user_id_e = inner_search_q.user_id) ) + AS outer_inner_sub_q + ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC, event_type_e DESC + LIMIT 10) +AS outer_outer_sub_q +ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC, event_type_e DESC +LIMIT 10; + +-- top level select * is removed now there is +-- a join at top level. +SELECT * +FROM + ( + (SELECT + user_id AS user_id_e, event_type as event_type_e + FROM + events_table + ) AS ma_e + JOIN + (SELECT + value_2, value_3, user_id + FROM + (SELECT + * + FROM + ( + (SELECT + user_id_p AS user_id + FROM + (SELECT + * + FROM + ( + (SELECT + user_id AS user_id_p + FROM + events_table + WHERE + (event_type IN (1, 2, 3, 4, 5)) + ) AS ma_p + JOIN + (SELECT + user_id AS user_id_a + FROM + users_table + WHERE + (value_2 % 5 = 1) + ) AS a + ON (a.user_id_a = ma_p.user_id_p) + ) + ) AS a_ma_p + ) AS inner_filter_q + JOIN + (SELECT + value_2, value_3, user_id AS user_id_ck + FROM + events_table + WHERE + event_type = ANY(ARRAY [10, 11, 12]) + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 + ) AS ma_ck + ON (ma_ck.user_id_ck = inner_filter_q.user_id) + ) AS inner_sub_q + ORDER BY + value_3 ASC, user_id_ck DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10 + ) AS outer_sub_q + ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC + LIMIT 10) AS inner_search_q + ON (ma_e.user_id_e = inner_search_q.user_id) + ) AS outer_inner_sub_q +ORDER BY + value_3 ASC, user_id DESC, array_index(ARRAY [1, 2, 3], (value_2 % 3)) ASC, event_type_e DESC +LIMIT 10; + + +-- drop created functions +SELECT * FROM run_command_on_workers('DROP FUNCTION array_index(ANYARRAY, ANYELEMENT)') +ORDER BY 1,2; +DROP FUNCTION array_index(ANYARRAY, ANYELEMENT); + +-- a not supported query due to constant range table entry +SELECT count(*) as subquery_count +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY user_id + HAVING count(distinct value_1) = 2 + ) as a + LEFT JOIN ( + SELECT + 1 as user_id + ) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NULL +GROUP BY a.user_id; + +-- same with INNER JOIN +SELECT count(*) as subquery_count +FROM ( + SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13') + GROUP BY user_id + HAVING count(distinct value_1) = 2 + ) as a + INNER JOIN ( + SELECT + 1 as user_id + ) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NULL +GROUP BY a.user_id; + +-- this is slightly different, we use RTE_VALUEs here +SELECT Count(*) AS subquery_count +FROM (SELECT + user_id + FROM + users_table + WHERE + (value_1 = '5' OR value_1 = '13' ) + GROUP BY + user_id + HAVING + Count(DISTINCT value_1) = 2) AS a + INNER JOIN + (SELECT + * + FROM + (VALUES (1, 'one'), (2, 'two'), (3, 'three')) AS t (user_id, letter)) AS b + ON a.user_id = b.user_id +WHERE b.user_id IS NULL +GROUP BY a.user_id; + + +-- same query without LIMIT/OFFSET returns 30 rows + +SET client_min_messages TO DEBUG1; +-- now, lets use a simple expression on the LIMIT and explicit coercion on the OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT 3+3 OFFSET 5::smallint; + +-- now, lets use implicit coersion in LIMIT and a simple expressions on OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT '3' OFFSET 27+2; + +-- create a test function which is marked as volatile +CREATE OR REPLACE FUNCTION volatile_func_test() + RETURNS INT AS $$ + SELECT 5; + $$ LANGUAGE sql VOLATILE; + +-- Citus should be able to evalute functions/row comparisons on the LIMIT/OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT volatile_func_test() + (ROW(1,2,NULL) < ROW(1,3,0))::int OFFSET volatile_func_test() + volatile_func_test(); + +-- now, lets use expressions on both the LIMIT and OFFSET +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT + u.user_id, e.event_type::text AS event, e.time + FROM + users_table AS u, + events_table AS e + WHERE + u.user_id = e.user_id AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id +) q +ORDER BY 2 DESC, 1 +LIMIT (5 > 4)::int OFFSET + CASE + WHEN 5 != 5 THEN 27 + WHEN 1 > 5 THEN 28 + ELSE 29 + END; + +-- we don't allow parameters on the LIMIT/OFFSET clauses +PREPARE parametrized_limit AS +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT u.user_id, e.event_type::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id + ) q + ORDER BY 2 DESC, 1 + LIMIT $1 OFFSET $2; + + EXECUTE parametrized_limit(3,3); + +PREPARE parametrized_offset AS +SELECT user_id, array_length(events_table, 1) +FROM ( + SELECT user_id, array_agg(event ORDER BY time) AS events_table + FROM ( + SELECT u.user_id, e.event_type::text AS event, e.time + FROM users_table AS u, + events_table AS e + WHERE u.user_id = e.user_id + AND e.event_type IN (100, 101, 102) + ) t + GROUP BY user_id + ) q + ORDER BY 2 DESC, 1 + LIMIT 3 OFFSET $1; + + EXECUTE parametrized_offset(3); + +SET client_min_messages TO DEFAULT; +DROP FUNCTION volatile_func_test(); + +SET citus.subquery_pushdown to OFF; +SET citus.enable_router_execution TO TRUE; diff --git a/src/test/regress/sql/multi_subquery_complex_queries.sql b/src/test/regress/sql/multi_subquery_complex_queries.sql new file mode 100644 index 000000000..656dc7e4e --- /dev/null +++ b/src/test/regress/sql/multi_subquery_complex_queries.sql @@ -0,0 +1,2288 @@ +-- +-- multi subquery complex queries aims to expand existing subquery pushdown +-- regression tests to cover more caeses +-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql +-- + +-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests +-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1400000; +ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1400000; + +SET citus.enable_router_execution TO FALSE; + + -- + -- UNIONs and JOINs mixed + -- +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- same query with target entries shuffled inside UNIONs +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- not supported since events_subquery_2 doesn't have partition key on the target list +-- within the shuffled target list +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" * 2 + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- not supported since events_subquery_2 doesn't have partition key on the target list +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."value_2" as user_id + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- we can support arbitrary subqueries within UNIONs +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT + *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + ( + SELECT * FROM + ( + SELECT + max("events"."time"), + 0 AS event, + "events"."user_id" + FROM + events_table as "events", users_table as "users" + WHERE + events.user_id = users.user_id AND + event_type IN (10, 11, 12, 13, 14, 15) + GROUP BY "events"."user_id" + ) as events_subquery_5 + ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- not supported since events_subquery_5 is not joined on partition key +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT + *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + ( + SELECT * FROM + ( + SELECT + max("events"."time"), + 0 AS event, + "events"."user_id" + FROM + events_table as "events", users_table as "users" + WHERE + events.user_id = users.value_2 AND + event_type IN (10, 11, 12, 13, 14, 15) + GROUP BY "events"."user_id" + ) as events_subquery_5 + ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- not supported since the join is not equi join +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id != q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- not supported since subquery 3 includes a JOIN with non-equi join +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events", users_table as "users" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) AND users.user_id != events.user_id ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- similar query with more union statements (to enable UNION tree become larger) +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) + + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 4 AS event + FROM + events_table as "events" + WHERE + event_type IN (31, 32, 33, 34, 35, 36)) events_subquery_5) + + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 5 AS event + FROM + events_table as "events" + WHERE + event_type IN (37, 38, 39, 40, 41, 42)) events_subquery_6) + + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 6 AS event + FROM + events_table as "events" + WHERE + event_type IN (50, 51, 52, 53, 54, 55)) events_subquery_6) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT "users"."user_id" + FROM users_table as "users" + WHERE value_1 > 50 and value_1 < 70) AS t ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + + +-- +-- UNION ALL Queries +-- +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT "users"."user_id" + FROM users_table as "users" + WHERE value_1 > 50 and value_1 < 70) AS t ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + +-- same query target list entries shuffled +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t +ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + +-- not supported since subquery 3 does not have partition key +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."value_2", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT "users"."user_id" + FROM users_table as "users" + WHERE value_1 > 50 and value_1 < 70) AS t ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + +-- not supported since events_subquery_4 does not have partition key on the +-- target list +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 1 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" * 2 + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t +ON (t.user_id = q.user_id)) as final_query +GROUP BY types +ORDER BY types; + +-- union all with inner and left joins +SELECT user_id, count(*) as cnt +FROM + (SELECT first_query.user_id, random() + FROM + ( SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "first_query" +INNER JOIN + (SELECT "t"."user_id" + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + LEFT OUTER JOIN + ( + SELECT + DISTINCT "events"."user_id" as user_id + FROM + events_table as "events" + WHERE + event_type IN (35, 36, 37, 38) + GROUP BY + user_id + ) as t2 + ON (t2.user_id = t.user_id) WHERE t2.user_id is NULL) as second_query + ON ("first_query".user_id = "second_query".user_id)) as final_query +GROUP BY + user_id ORDER BY cnt DESC, user_id DESC +LIMIT 10; + +-- not supported since the join between t and t2 is not equi join +-- union all with inner and left joins +SELECT user_id, count(*) as cnt +FROM + (SELECT first_query.user_id, random() + FROM + ( SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION ALL + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "first_query" +INNER JOIN + (SELECT "t"."user_id" + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + LEFT OUTER JOIN + ( + SELECT + DISTINCT "events"."user_id" as user_id + FROM + events_table as "events" + WHERE + event_type IN (35, 36, 37, 38) + GROUP BY + user_id + ) as t2 + ON (t2.user_id > t.user_id) WHERE t2.user_id is NULL) as second_query + ON ("first_query".user_id = "second_query".user_id)) as final_query +GROUP BY + user_id ORDER BY cnt DESC, user_id DESC +LIMIT 10; + + -- + -- Union, inner join and left join + -- +SELECT user_id, count(*) as cnt +FROM + (SELECT first_query.user_id, random() + FROM + ( SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "first_query" +INNER JOIN + (SELECT "t"."user_id" + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + LEFT OUTER JOIN + ( + SELECT + DISTINCT "events"."user_id" as user_id + FROM + events_table as "events" + WHERE + event_type IN (35, 36, 37, 38) + GROUP BY + user_id + ) as t2 + ON (t2.user_id = t.user_id) WHERE t2.user_id is NULL) as second_query + ON ("first_query".user_id = "second_query".user_id)) as final_query +GROUP BY + user_id ORDER BY cnt DESC, user_id DESC +LIMIT 10; + +-- Simple LATERAL JOINs with GROUP BYs in each side +-- need to set subquery_pushdown due to limit for next 2 queries +SET citus.subquery_pushdown to ON; +SELECT * +FROM + (SELECT "some_users_data".user_id, lastseen + FROM + (SELECT user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(time) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE + ORDER BY + lastseen DESC + LIMIT 50) "some_users" +order BY + user_id +LIMIT 50; + +-- same query with subuqery joins in topmost select +SELECT "some_users_data".user_id, lastseen +FROM + (SELECT user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(TIME) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE +ORDER BY + user_id +limit 50; + +-- reset subquery_pushdown +SET citus.subquery_pushdown to OFF; + +-- not supported since JOIN is not on the partition key +SELECT "some_users_data".user_id, lastseen +FROM + (SELECT user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(TIME) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."value_1" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE +ORDER BY + user_id +limit 50; + +-- not supported since JOIN is not on the partition key +-- see (2 * user_id as user_id) target list element +SELECT "some_users_data".user_id, lastseen +FROM + (SELECT 2 * user_id as user_id, max(time) AS lastseen + FROM + (SELECT user_id, time + FROM + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40) "events_1" + ORDER BY + time DESC + LIMIT 1000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(TIME) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 50 and users.value_2 < 55 + LIMIT 1) "some_users_data" + ON TRUE +ORDER BY + user_id +limit 50; + +-- LATERAL JOINs used with INNER JOINs +SET citus.subquery_pushdown to ON; +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) + filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 AND + user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" + ON TRUE + ORDER BY + time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" + ON TRUE + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; + +-- +-- A similar query with topmost select is dropped +-- and replaced by aggregation. Notice the heavy use of limit +-- +SELECT "some_users_data".user_id, MAX(lastseen), count(*) + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true +GROUP BY 1 +ORDER BY 2, 1 DESC +LIMIT 10; + +SET citus.subquery_pushdown to OFF; + +-- not supported since the inner JOIN is not equi join +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id != "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; + +-- not supported since the inner JOIN is not on the partition key +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_1" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".value_1)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; + + +-- not supported since upper LATERAL JOIN is not equi join +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_1" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id != filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; + +-- not supported since lower LATERAL JOIN is not on the partition key +SELECT user_id, lastseen +FROM + (SELECT + "some_users_data".user_id, lastseen + FROM + (SELECT + filter_users_1.user_id, time AS lastseen + FROM + (SELECT + user_where_1_1.user_id + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_1" + FROM + users_table as "users" + WHERE + user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1 + ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1 + JOIN LATERAL + (SELECT + user_id, time + FROM + events_table as "events" + WHERE + user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id + ORDER BY + time DESC + LIMIT 1) "last_events_1" ON true + ORDER BY time DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."value_1" = "some_recent_users"."user_id" AND + "users"."value_2" > 70 + LIMIT 1) "some_users_data" ON true + ORDER BY + lastseen DESC + LIMIT 10) "some_users" +ORDER BY + user_id DESC +LIMIT 10; + +-- NESTED INNER JOINs +SELECT + count(*) AS value, "generated_group_field" + FROM + (SELECT + DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field" + FROM + (SELECT + "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field" + FROM + (SELECT + * + FROM + (SELECT + "events"."time", "events"."user_id", "events"."value_2" + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40 AND event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries" + INNER JOIN + (SELECT + user_where_1_1.real_user_id + FROM + (SELECT + "users"."user_id" as real_user_id + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1 + ON ("user_where_1_1".real_user_id = "user_where_1_join_1".user_id)) "user_filters_1" + ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery" +GROUP BY + "generated_group_field" +ORDER BY + generated_group_field DESC, value DESC; + +-- not supported since the first inner join is not on the partition key +SELECT + count(*) AS value, "generated_group_field" + FROM + (SELECT + DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field" + FROM + (SELECT + "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field" + FROM + (SELECT + * + FROM + (SELECT + "events"."time", "events"."user_id", "events"."value_2" + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40 AND event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries" + INNER JOIN + (SELECT + user_where_1_1.real_user_id + FROM + (SELECT + "users"."user_id" as real_user_id + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_2" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1 + ON ("user_where_1_1".real_user_id = "user_where_1_join_1".value_2)) "user_filters_1" + ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery" +GROUP BY + "generated_group_field" +ORDER BY + generated_group_field DESC, value DESC; + +-- not supported since the first inner join is not an equi join +SELECT + count(*) AS value, "generated_group_field" + FROM + (SELECT + DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field" + FROM + (SELECT + "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field" + FROM + (SELECT + * + FROM + (SELECT + "events"."time", "events"."user_id", "events"."value_2" + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 40 AND event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries" + INNER JOIN + (SELECT + user_where_1_1.real_user_id + FROM + (SELECT + "users"."user_id" as real_user_id + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1 + INNER JOIN + (SELECT + "users"."user_id", "users"."value_2" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1 + ON ("user_where_1_1".real_user_id >= "user_where_1_join_1".user_id)) "user_filters_1" + ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery" +GROUP BY + "generated_group_field" +ORDER BY + generated_group_field DESC, value DESC; + +-- single level inner joins +SELECT + "value_3", count(*) AS cnt +FROM + (SELECT + "value_3", "user_id", random() + FROM + (SELECT + users_in_segment_1.user_id, value_3 + FROM + (SELECT + user_id, value_3 * 2 as value_3 + FROM + (SELECT + user_id, value_3 + FROM + (SELECT + "users"."user_id", value_3 + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 30 + ) simple_user_where_1 + ) all_buckets_1 + ) users_in_segment_1 + JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 60 + ) some_users_data + ON ("users_in_segment_1".user_id = "some_users_data".user_id) + ) segmentalias_1) "tempQuery" +GROUP BY "value_3" +ORDER BY cnt, value_3 DESC LIMIT 10; + + +-- not supported since there is no partition column equality at all +SELECT + "value_3", count(*) AS cnt +FROM + (SELECT + "value_3", "user_id", random() + FROM + (SELECT + users_in_segment_1.user_id, value_3 + FROM + (SELECT + user_id, value_3 * 2 as value_3 + FROM + (SELECT + user_id, value_3 + FROM + (SELECT + "users"."user_id", value_3 + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 30 + ) simple_user_where_1 + ) all_buckets_1 + ) users_in_segment_1 + JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 10 and user_id < 40 and value_2 > 60 + ) some_users_data + ON (true) + ) segmentalias_1) "tempQuery" +GROUP BY "value_3" +ORDER BY cnt, value_3 DESC LIMIT 10; + +-- nested LATERAL JOINs +SET citus.subquery_pushdown to ON; +SELECT * +FROM + (SELECT "some_users_data".user_id, "some_recent_users".value_3 + FROM + (SELECT + filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1) "last_events_1" ON true + ORDER BY value_3 DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_3 DESC +LIMIT 10) "some_users" +ORDER BY + value_3 DESC +LIMIT 10; + +-- nested lateral join at top most level +SELECT "some_users_data".user_id, "some_recent_users".value_3 +FROM + (SELECT + filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200 + ) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1 + ) "last_events_1" ON true + ORDER BY value_3 DESC + LIMIT 10 + ) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1 + ) "some_users_data" ON true +ORDER BY + value_3 DESC, user_id ASC +LIMIT 10; + +-- longer nested lateral joins +SELECT * +FROM + (SELECT "some_users_data".user_id, "some_recent_users".value_3 + FROM + (SELECT filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1) "last_events_1" ON true + ORDER BY + value_3 DESC + LIMIT 10) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_3 DESC + LIMIT 10) "some_users" +ORDER BY + value_3 DESC +LIMIT 10; + +-- longer nested lateral join wth top level join +SELECT "some_users_data".user_id, "some_recent_users".value_3 +FROM + (SELECT filter_users_1.user_id, value_3 + FROM + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 20 and user_id < 70 and users.value_2 = 200 + ) filter_users_1 + JOIN LATERAL + (SELECT + user_id, value_3 + FROM + events_table as "events" + WHERE + user_id > 20 and user_id < 70 + AND + ("events".user_id = "filter_users_1".user_id) + ORDER BY + value_3 DESC + LIMIT 1 + ) "last_events_1" ON TRUE + ORDER BY value_3 DESC + LIMIT 10 + ) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + users.value_2 > 200 + LIMIT 1 + ) "some_users_data" ON TRUE +ORDER BY value_3 DESC +LIMIT 10; + +SET citus.subquery_pushdown to OFF; + +-- LEFT JOINs used with INNER JOINs +SELECT +count(*) AS cnt, "generated_group_field" + FROM + (SELECT + "eventQuery"."user_id", random(), generated_group_field + FROM + (SELECT + "multi_group_wrapper_1".*, generated_group_field, random() + FROM + (SELECT * + FROM + (SELECT + "events"."time", "events"."user_id" as event_user_id + FROM + events_table as "events" + WHERE + user_id > 80) "temp_data_queries" + INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + user_id > 80 and value_2 = 5) "user_filters_1" + ON ("temp_data_queries".event_user_id = "user_filters_1".user_id)) AS "multi_group_wrapper_1" + LEFT JOIN + (SELECT + "users"."user_id" AS "user_id", value_2 AS "generated_group_field" + FROM + users_table as "users") "left_group_by_1" + ON ("left_group_by_1".user_id = "multi_group_wrapper_1".event_user_id)) "eventQuery") "pushedDownQuery" + group BY + "generated_group_field" + ORDER BY + cnt DESC, generated_group_field ASC + LIMIT 10; + +-- single table subquery, no JOINS involved +SELECT +count(*) AS cnt, user_id +FROM + (SELECT + "eventQuery"."user_id", random() + FROM + (SELECT + "events"."user_id" + FROM + events_table "events" + WHERE + event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90)) "eventQuery") "pushedDownQuery" +GROUP BY + "user_id" +ORDER BY + cnt DESC, user_id DESC +LIMIT 10; + +-- lateral joins in the nested manner +SET citus.subquery_pushdown to ON; +SELECT * +FROM + (SELECT + "some_users_data".user_id, value_2 + FROM + (SELECT user_id, max(value_2) AS value_2 + FROM + (SELECT user_id, value_2 + FROM + (SELECT + user_id, value_2 + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 20) "events_1" + ORDER BY + value_2 DESC + LIMIT 10000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(value_2) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."user_id" = "some_recent_users"."user_id" AND + value_2 > 75 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_2 DESC + LIMIT 10) "some_users" +ORDER BY + value_2 DESC, user_id DESC +LIMIT 10; +SET citus.subquery_pushdown to OFF; + +-- not supported since join is not on the partition key +SELECT * +FROM + (SELECT + "some_users_data".user_id, value_2 + FROM + (SELECT user_id, max(value_2) AS value_2 + FROM + (SELECT user_id, value_2 + FROM + (SELECT + user_id, value_2 + FROM + events_table as "events" + WHERE + user_id > 10 and user_id < 20) "events_1" + ORDER BY + value_2 DESC + LIMIT 10000) "recent_events_1" + GROUP BY + user_id + ORDER BY + max(value_2) DESC) "some_recent_users" + JOIN LATERAL + (SELECT + "users".user_id + FROM + users_table as "users" + WHERE + "users"."value_2" = "some_recent_users"."user_id" AND + value_2 > 75 + LIMIT 1) "some_users_data" ON true + ORDER BY + value_2 DESC + LIMIT 10) "some_users" +ORDER BY + value_2 DESC, user_id DESC +LIMIT 10; + +-- lets test some unsupported set operations + +-- not supported since we use INTERSECT +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + INTERSECT + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- not supported due to offset +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4) OFFSET 3) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- not supported due to window functions +SELECT user_id, + some_vals +FROM ( + SELECT * , + Row_number() over (PARTITION BY "user_id" ORDER BY "user_id") AS "some_vals", + Random() + FROM users_table + ) user_id +ORDER BY 1, + 2 limit 10; + +-- not supported due to non relation rte +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + 1 as user_id, now(), 3 AS event + ) events_subquery_4) OFFSET 3) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + users_table as "users" + WHERE + value_1 > 50 and value_1 < 70) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +-- similar to the above, but constant rte is on the right side of the query +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15) ) events_subquery_1) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT + * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT + * + FROM + (SELECT + 1 as user_id, now(), 3 AS event + ) events_subquery_4) OFFSET 3) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT random()::int as user_id) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + +SET citus.enable_router_execution TO TRUE; \ No newline at end of file diff --git a/src/test/regress/sql/multi_subquery_union.sql b/src/test/regress/sql/multi_subquery_union.sql new file mode 100644 index 000000000..ef960e49e --- /dev/null +++ b/src/test/regress/sql/multi_subquery_union.sql @@ -0,0 +1,726 @@ +-- +-- multi subquery toplevel union queries aims to expand existing subquery pushdown +-- regression tests to cover more cases +-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql + +-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests +-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1400000; + +SET citus.enable_router_execution TO false; +-- a very simple union query +SELECT user_id, counter +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +ORDER BY 2 DESC,1 +LIMIT 5; + +-- the same query with union all +SELECT user_id, counter +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION ALL + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +ORDER BY 2 DESC,1 +LIMIT 5; + +-- the same query with group by +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +GROUP BY 1 +ORDER BY 2 DESC,1 +LIMIT 5; + +-- the same query with UNION ALL clause +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION ALL + SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +GROUP BY 1 +ORDER BY 2 DESC,1 +LIMIT 5; + +-- the same query target list entries shuffled +SELECT user_id, sum(counter) +FROM ( + SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (1, 2, 3, 4, 5) + UNION + SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10) +) user_id +GROUP BY 1 +ORDER BY 2 DESC,1 +LIMIT 5; + +-- same query with GROUP BY +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (1, 2) + UNION + SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (5, 6) +) user_id +GROUP BY + user_id +--HAVING sum(counter) > 900 +ORDER BY 1,2 DESC LIMIT 5; + + +-- the same query target list entries shuffled but this time the subqueries target list +-- is shuffled +SELECT user_id, sum(counter) +FROM ( + SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (1, 2) + UNION + SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (5, 6) +) user_id +GROUP BY + user_id +--HAVING sum(counter) > 900 +ORDER BY 1,2 DESC LIMIT 5; + + +-- similar query this time more subqueries and target list contains a resjunk entry +SELECT sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500 +) user_id +GROUP BY user_id ORDER BY 1 DESC LIMIT 5; + +-- similar query as above, with UNION ALL +SELECT sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 5000 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500 + UNION ALL + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500 +) user_id +GROUP BY user_id ORDER BY 1 DESC LIMIT 5; + +-- unions within unions +SELECT * +FROM ( + ( SELECT user_id, + sum(counter) + FROM + (SELECT + user_id, sum(value_2) AS counter + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_1 + GROUP BY + user_id) + UNION + (SELECT + user_id, sum(counter) + FROM + (SELECT + user_id, sum(value_2) AS counter + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_2 + GROUP BY + user_id)) AS ftop +ORDER BY 2 DESC, 1 DESC +LIMIT 5; + +-- top level unions are wrapped into top level aggregations +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +) as final_query +GROUP BY types +ORDER BY types; + +-- exactly the same query +-- but wrapper unions are removed from the inner part of the query +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + (SELECT *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + (SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + GROUP BY "t1"."user_id") AS t) "q" +) as final_query +GROUP BY types +ORDER BY types; + +-- again excatly the same query with top level wrapper removed +SELECT ("q"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + GROUP BY "t1"."user_id") AS t) "q" +GROUP BY types +ORDER BY types; + +-- again same query but with only two top level empty queries (i.e., no group bys) +SELECT * +FROM + ( SELECT * + FROM + ( SELECT "t1"."user_id" + FROM ( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + ) AS t) "q" +ORDER BY 1 +LIMIT 5; + +-- a very similar query UNION ALL +SELECT ("q"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) + UNION ALL + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) + UNION ALL + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) + UNION ALL + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + events_table as "events" + WHERE + event_type IN (26, 27, 28, 29, 30, 13))) t1 + GROUP BY "t1"."user_id") AS t) "q" +GROUP BY types +ORDER BY types; + +-- some UNION ALL queries that are going to be pulled up +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT user_id FROM events_table) +) b; + +-- similar query without top level agg +SELECT + user_id +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT user_id FROM events_table) +) b +ORDER BY 1 DESC +LIMIT 5; + +-- similar query with multiple target list entries +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM users_table) + UNION ALL + (SELECT value_3, user_id FROM events_table) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; + +-- similar query group by inside the subqueries +SELECT + user_id, value_3_sum +FROM +( + (SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id) + UNION ALL + (SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id) +) b +ORDER BY 2 DESC, 1 DESC +LIMIT 5; + +-- similar query top level group by +SELECT + user_id, sum(value_3) +FROM +( + (SELECT value_3, user_id FROM users_table) + UNION ALL + (SELECT value_3, user_id FROM events_table) +) b +GROUP BY 1 +ORDER BY 2 DESC, 1 DESC +LIMIT 5; + +-- a long set operation list +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30)) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; + +-- no partition key on the top +SELECT + max(value_3) +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30)) +) b +GROUP BY user_id +ORDER BY 1 DESC +LIMIT 5; + + +-- now lets also have some unsupported queries + +-- group by is not on the partition key +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + UNION + SELECT value_1 as user_id, sum(value_2) AS counter FROM users_table GROUP BY value_1 +) user_id +GROUP BY user_id; + +-- partition key is not selected +SELECT sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500 + UNION + SELECT 2 * user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500 +) user_id +GROUP BY user_id ORDER BY 1 DESC LIMIT 5; + +-- excepts within unions are not supported +SELECT * FROM +( +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + ) user_id_1 + GROUP BY user_id +) +UNION +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + EXCEPT + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id +) user_id_2 + GROUP BY user_id) +) as ftop; + +-- joins inside unions are not supported +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE users_table.user_id > events_table.user_id GROUP BY 1 +) user_id +GROUP BY user_id; + +-- joins inside unions are not supported -- slightly more comlex than the above +SELECT * FROM +( +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + ) user_id_1 + GROUP BY user_id +) +UNION +( + SELECT user_id, sum(counter) + FROM ( + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id + UNION + SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE (events_table.user_id = users_table.user_id) GROUP BY events_table.user_id +) user_id_2 + GROUP BY user_id) +) as ftop; + +-- offset inside the union +SELECT user_id, sum(counter) +FROM ( + SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id + UNION + SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id OFFSET 4 +) user_id +GROUP BY user_id; + +-- lower level union does not return partition key with the other relations +SELECT * +FROM ( + ( SELECT user_id, + sum(counter) + FROM + (SELECT + user_id, sum(value_2) AS counter + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_1 + GROUP BY + user_id) + UNION + (SELECT + user_id, sum(counter) + FROM + (SELECT + sum(value_2) AS counter, user_id + FROM + users_table + GROUP BY + user_id + UNION + SELECT + user_id, sum(value_2) AS counter + FROM + events_table + GROUP BY + user_id) user_id_2 + GROUP BY + user_id)) AS ftop; + + +-- some UNION all queries that are going to be pulled up +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT 2 * user_id FROM events_table) +) b; + +-- last query does not have partition key +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT value_3, value_2 FROM events_table where event_type IN (26, 27, 28, 29, 30)) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; + +-- we don't allow joins within unions +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT users_table.user_id FROM events_table, users_table WHERE events_table.user_id = users_table.user_id) +) b; + +-- we don't support subqueries without relations +SELECT + count(*) +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT 1) +) b; + +-- we don't support subqueries without relations +SELECT + * +FROM +( + (SELECT user_id FROM users_table) + UNION ALL + (SELECT (random() * 100)::int) +) b; + +-- we don't support subqueries without relations +SELECT + user_id, value_3 +FROM +( + (SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20)) + UNION ALL + (SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25)) + UNION ALL + (SELECT 1, 2) +) b +ORDER BY 1 DESC, 2 DESC +LIMIT 5; + +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + ( SELECT *, random() + FROM + ( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + ( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + events_table as "events" + WHERE + event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + events_table as "events" + WHERE + event_type IN (15, 16, 17, 18, 19) ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + events_table as "events" + WHERE + event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT 1, now(), 3 AS event) events_subquery_4)) t1 + GROUP BY "t1"."user_id") AS t) "q" +) as final_query +GROUP BY types +ORDER BY types; + +SET citus.enable_router_execution TO true; diff --git a/src/test/regress/sql/multi_view.sql b/src/test/regress/sql/multi_view.sql index 164e5a48f..26619b439 100644 --- a/src/test/regress/sql/multi_view.sql +++ b/src/test/regress/sql/multi_view.sql @@ -117,23 +117,303 @@ SELECT l_suppkey, count(*) FROM GROUP BY l_suppkey, l_shipdate) supps GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5; +-- repartition query on view with single table subquery +CREATE VIEW supp_count_view AS SELECT * FROM (SELECT l_suppkey, count(*) FROM lineitem_hash_part GROUP BY 1) s1; +SELECT * FROM supp_count_view ORDER BY 2 DESC, 1 LIMIT 10; + SET citus.task_executor_type to DEFAULT; -- create a view with aggregate CREATE VIEW lineitems_by_shipping_method AS SELECT l_shipmode, count(*) as cnt FROM lineitem_hash_part GROUP BY 1; --- following will fail due to non-flattening of subquery due to GROUP BY +-- following will fail due to non GROUP BY of partition key SELECT * FROM lineitems_by_shipping_method; -- create a view with group by on partition column CREATE VIEW lineitems_by_orderkey AS - SELECT l_orderkey, count(*) FROM lineitem_hash_part GROUP BY 1; + SELECT + l_orderkey, count(*) + FROM + lineitem_hash_part + GROUP BY 1; --- this will also fail due to same reason -SELECT * FROM lineitems_by_orderkey; +-- this should work since we're able to push down this query +SELECT * FROM lineitems_by_orderkey ORDER BY 2 DESC, 1 ASC LIMIT 10; --- however it would work if it is made router plannable +-- it would also work since it is made router plannable SELECT * FROM lineitems_by_orderkey WHERE l_orderkey = 100; DROP TABLE temp_lineitem CASCADE; + +DROP VIEW supp_count_view; +DROP VIEW lineitems_by_orderkey; +DROP VIEW lineitems_by_shipping_method; +DROP VIEW air_shipped_lineitems; +DROP VIEW priority_lineitem; +DROP VIEW priority_orders; + +-- new tests for real time use case including views and subqueries + +-- create view to display recent user who has an activity after a timestamp +CREATE VIEW recent_users AS + SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC; +SELECT * FROM recent_users; + +-- create a view for recent_events +CREATE VIEW recent_events AS + SELECT user_id, time FROM events_table + WHERE time > '2014-01-20 01:45:49.978738'::timestamp; + +SELECT count(*) FROM recent_events; + +-- count number of events of recent_users +SELECT count(*) FROM recent_users ru JOIN events_table et ON (ru.user_id = et.user_id); +-- count number of events of per recent users order by count +SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1; + +-- the same query with a left join however, it would still generate the same result +SELECT ru.user_id, count(*) + FROM recent_users ru + LEFT JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1; + +-- query wrapped inside a subquery, it needs another top level order by +SELECT * FROM + (SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.user_id) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1) s1 +ORDER BY 2 DESC, 1; + +-- non-partition key joins are not supported inside subquery +SELECT * FROM + (SELECT ru.user_id, count(*) + FROM recent_users ru + JOIN events_table et + ON (ru.user_id = et.event_type) + GROUP BY ru.user_id + ORDER BY 2 DESC, 1) s1 +ORDER BY 2 DESC, 1; + +-- join between views +-- recent users who has an event in recent events +SELECT ru.user_id FROM recent_users ru JOIN recent_events re USING(user_id) GROUP BY ru.user_id ORDER BY ru.user_id; + +-- outer join inside a subquery +-- recent_events who are not done by recent users +SELECT count(*) FROM ( + SELECT re.*, ru.user_id AS recent_user + FROM recent_events re LEFT JOIN recent_users ru USING(user_id)) reu + WHERE recent_user IS NULL; + +-- same query with anti-join +SELECT count(*) + FROM recent_events re LEFT JOIN recent_users ru ON(ru.user_id = re.user_id) + WHERE ru.user_id IS NULL; + +-- join between view and table +-- users who has recent activity and they have an entry with value_1 is less than 15 +SELECT ut.* FROM recent_users ru JOIN users_table ut USING (user_id) WHERE ut.value_1 < 15 ORDER BY 1,2; + +-- determine if a recent user has done a given event type or not +SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.user_id AND et.event_type = 625) + ORDER BY 2 DESC, 1; + +-- view vs table join wrapped inside a subquery +SELECT * FROM + (SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.user_id AND et.event_type = 625) + ) s1 +ORDER BY 2 DESC, 1; + +-- event vs table non-partition-key join is not supported +SELECT * FROM + (SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event + FROM recent_users ru + LEFT JOIN events_table et + ON(ru.user_id = et.event_type) + ) s1 +ORDER BY 2 DESC, 1; + +-- create a select only view +CREATE VIEW selected_users AS SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150; +CREATE VIEW recent_selected_users AS SELECT su.* FROM selected_users su JOIN recent_users ru USING(user_id); + +SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; + +-- this would be supported when we implement where partition_key in (subquery) support +SELECT et.* FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users); + +-- it is supported when it is a router query +SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90); + +-- expected this to work but it did not +(SELECT user_id FROM recent_users) +UNION +(SELECT user_id FROM selected_users); + +-- wrapping it inside a SELECT * works +SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10 + ORDER BY user_id; + +-- union all also works for views +SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION ALL + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10 + ORDER BY user_id; + +SELECT count(*) + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10; + +-- expected this to work but it does not +SELECT count(*) + FROM ( + (SELECT user_id FROM recent_users) + UNION ALL + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10; + +-- expand view definitions and re-run last 2 queries +SELECT count(*) + FROM ( + (SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa + ) + UNION + (SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u + WHERE user_id < 15 AND user_id > 10; + +SELECT count(*) + FROM ( + (SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa + ) + UNION ALL + (SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u + WHERE user_id < 15 AND user_id > 10; + +-- test distinct +-- distinct is supported if it is on a partition key +CREATE VIEW distinct_user_with_value_1_15 AS SELECT DISTINCT user_id FROM users_table WHERE value_1 = 15; +SELECT * FROM distinct_user_with_value_1_15 ORDER BY user_id; + +-- distinct is not supported if it is on a non-partition key +CREATE VIEW distinct_value_1 AS SELECT DISTINCT value_1 FROM users_table WHERE value_2 = 15; +SELECT * FROM distinct_value_1; + +-- CTEs are not supported even if they are on views +CREATE VIEW cte_view_1 AS +WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 15) SELECT * FROM c1 WHERE value_2 < 500; + +SELECT * FROM cte_view_1; + +-- this is single shard query but still not supported since it has view + cte +-- router planner can't detect it +SELECT * FROM cte_view_1 WHERE user_id = 8; + +-- if CTE itself prunes down to a single shard than the view is supported (router plannable) +CREATE VIEW cte_view_2 AS +WITH c1 AS (SELECT * FROM users_table WHERE user_id = 8) SELECT * FROM c1 WHERE value_1 = 15; +SELECT * FROM cte_view_2; + +CREATE VIEW router_view AS SELECT * FROM users_table WHERE user_id = 2; +-- router plannable +SELECT user_id FROM router_view GROUP BY 1; + +-- There is a known issue with router plannable subqueries joined with non-router +-- plannable subqueries. Following tests should be uncommented when we fix it + +-- join a router view (not implement error) +-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id); + +-- it still does not work when converted to 2 subquery join +-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id); + +-- views are completely removed and still it does not work +-- SELECT * FROM +-- (SELECT user_id FROM (SELECT * FROM users_table WHERE user_id = 2) rv1 GROUP BY 1) rv2 +-- JOIN (SELECT user_id, time FROM events_table +-- WHERE time > '2014-01-20 01:45:49.978738'::timestamp) re +-- USING (user_id); + +-- views with limits +CREATE VIEW recent_10_users AS + SELECT user_id, max(time) as lastseen FROM users_table + GROUP BY user_id + ORDER BY lastseen DESC + LIMIT 10; + +-- this is not supported since it has limit in it and subquery_pushdown is not set +SELECT * FROM recent_10_users; + +SET citus.subquery_pushdown to ON; +-- still not supported since outer query does not have limit +-- it shows a different (subquery with single relation) error message +SELECT * FROM recent_10_users; +-- now it displays more correct error message +SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id); + +-- now both are supported when there is a limit on the outer most query +SELECT * FROM recent_10_users ORDER BY lastseen DESC LIMIT 10; +SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; + +RESET citus.subquery_pushdown; + +-- explain tests +EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; + +EXPLAIN (COSTS FALSE) SELECT * + FROM ( + (SELECT user_id FROM recent_users) + UNION + (SELECT user_id FROM selected_users) ) u + WHERE user_id < 15 AND user_id > 10 + ORDER BY user_id; + +EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; +SET citus.subquery_pushdown to ON; +EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10; + +RESET citus.subquery_pushdown; + +DROP VIEW recent_10_users; +DROP VIEW router_view; +DROP VIEW cte_view_2; +DROP VIEW cte_view_1; +DROP VIEW distinct_value_1; +DROP VIEW distinct_user_with_value_1_15; +DROP VIEW recent_selected_users; +DROP VIEW selected_users; +DROP VIEW recent_events; +DROP VIEW recent_users;