diff --git a/src/backend/distributed/planner/multi_logical_optimizer.c b/src/backend/distributed/planner/multi_logical_optimizer.c index c5028ba9c..c10eaf487 100644 --- a/src/backend/distributed/planner/multi_logical_optimizer.c +++ b/src/backend/distributed/planner/multi_logical_optimizer.c @@ -154,9 +154,6 @@ 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); @@ -2895,7 +2892,6 @@ ErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerQueryHasLimit) { bool preconditionsSatisfied = true; char *errorDetail = NULL; - Query *lateralQuery = NULL; List *subqueryEntryList = NIL; ListCell *rangeTableEntryCell = NULL; @@ -2991,43 +2987,6 @@ ErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerQueryHasLimit) "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) { @@ -3589,273 +3548,6 @@ FullCompositeFieldList(List *compositeFieldList) } -/* - * 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. diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index a33808d5b..37cafcc63 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -37,6 +37,8 @@ #include "distributed/multi_logical_optimizer.h" #include "distributed/multi_logical_planner.h" #include "distributed/multi_physical_planner.h" +#include "distributed/multi_planner.h" +#include "distributed/multi_router_planner.h" #include "distributed/pg_dist_partition.h" #include "distributed/pg_dist_shard.h" #include "distributed/shardinterval_utils.h" @@ -118,8 +120,16 @@ 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, + RelationRestrictionContext *restrictionContext, + JoinRestrictionContext *joinRestrictionContext); +static List * SubquerySqlTaskList(Job *job, + RelationRestrictionContext *restrictionContext, + JoinRestrictionContext *joinRestrictionContext); +static Task * SubqueryTaskCreate(Query *originalQuery, ShardInterval *shardInterval, + bool allRelationsJoinedOnPartitionKey, + RelationRestrictionContext *restrictionContext, + uint32 taskIdIndex); 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,9 @@ 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, + RelationRestrictionContext *restrictionContext, + JoinRestrictionContext *joinRestrictionContext) { MultiPlan *multiPlan = NULL; Job *workerJob = NULL; @@ -204,7 +215,8 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree) workerJob = BuildJobTree(multiTree); /* create the tree of executable tasks for the worker job */ - workerJob = BuildJobTreeTaskList(workerJob); + workerJob = BuildJobTreeTaskList(workerJob, restrictionContext, + joinRestrictionContext); /* build the final merge query to execute on the master */ masterDependedJobList = list_make1(workerJob); @@ -1889,7 +1901,8 @@ SplitPointObject(ShardInterval **shardIntervalArray, uint32 shardIntervalCount) * tasks to worker nodes. */ static Job * -BuildJobTreeTaskList(Job *jobTree) +BuildJobTreeTaskList(Job *jobTree, RelationRestrictionContext *restrictionContext, + JoinRestrictionContext *joinRestrictionContext) { List *flattenedJobList = NIL; uint32 flattenedJobCount = 0; @@ -1927,7 +1940,8 @@ BuildJobTreeTaskList(Job *jobTree) /* create sql tasks for the job, and prune redundant data fetch tasks */ if (job->subqueryPushdown) { - sqlTaskList = SubquerySqlTaskList(job); + sqlTaskList = SubquerySqlTaskList(job, restrictionContext, + joinRestrictionContext); } else { @@ -1978,183 +1992,204 @@ BuildJobTreeTaskList(Job *jobTree) /* + * TODO: update comment * 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, it first checks whether the query is safe to push-down. + * In other words, the function first checks whether all the relations that appear + * in the query are JOINed on their partition column. Note that the behaviour is + * slightly different for reference tables and queries that involve UNIONs. For the + * details please see AllRelationsJoinedOnPartitionKey(). + * + * After the above decision is taken, the query 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, RelationRestrictionContext *restrictionContext, + JoinRestrictionContext *joinRestrictionContext) { Query *subquery = job->jobQuery; uint64 jobId = job->jobId; + List *targetShardIntervalList = NIL; + 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; - - /* 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; - } - } + Oid relationId = 0; + int shardCount = 0; + int shardOffset = 0; + DistTableCacheEntry *targetCacheEntry = NULL; + bool allRelationsJoinedOnPartitionKey = + AllRelationsJoinedOnPartitionKey(restrictionContext, joinRestrictionContext); /* 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 *shardIntervalList = LoadShardIntervalList(relationId); - 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 = PruneShardList(relationId, tableId, whereClauseList, - shardIntervalList); - } - else - { - finalShardIntervalList = shardIntervalList; + continue; } - /* if all shards are pruned away, we return an empty task list */ - finalShardCount = list_length(finalShardIntervalList); - if (finalShardCount == 0) + if (targetShardIntervalList == NIL) { - return NIL; + targetCacheEntry = DistributedTableCacheEntry(relationId); + shardCount = targetCacheEntry->shardIntervalArrayLength; + break; } - - fragmentCombinationCell = list_head(fragmentCombinationList); - - foreach(shardIntervalCell, finalShardIntervalList) - { - 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); - } - } - - /* - * Determine anchor table using shards which survive pruning instead of calling - * AnchorRangeTableId - */ - if (anchorRangeTableId == 0 || tableSize > largestTableSize) - { - largestTableSize = tableSize; - anchorRangeTableId = tableId; - } - - rangeTableIndex++; } + for (shardOffset = 0; shardOffset < shardCount; shardOffset++) + { + ShardInterval *targetShardInterval = + targetCacheEntry->sortedShardIntervalArray[shardOffset]; + Task *subqueryTask = NULL; + + subqueryTask = SubqueryTaskCreate(subquery, targetShardInterval, + allRelationsJoinedOnPartitionKey, + restrictionContext, taskIdIndex); + + + /* add the task if it could be created */ + if (subqueryTask != NULL) + { + subqueryTask->jobId = jobId; + sqlTaskList = lappend(sqlTaskList, subqueryTask); + } + + ++taskIdIndex; + } + + return sqlTaskList; +} + + +/* + * SubqueryTaskCreate creates a modify task by replacing the target + * shardInterval's boundary value when allRelationsJoinedOnPartitionKey + * is true. Then performs the normal shard pruning on the subquery. + * + * 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, + bool allRelationsJoinedOnPartitionKey, + RelationRestrictionContext *restrictionContext, + uint32 taskIdIndex) +{ + Query *copiedQuery = 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 upsertQuery = false; + bool replacePrunedQueryWithDummy = false; + bool allReferenceTables = restrictionContext->allReferenceTables; + RelationRestrictionContext *copiedRestrictionContext = + CopyRelationRestrictionContext(restrictionContext); + List *shardOpExpressions = NIL; + RestrictInfo *shardRestrictionList = NULL; + + /* + * 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; + + if (!allRelationsJoinedOnPartitionKey || allReferenceTables) + { + continue; + } + + shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex); + + shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions); + originalBaserestrictInfo = lappend(originalBaserestrictInfo, + shardRestrictionList); + + restriction->relOptInfo->baserestrictinfo = originalBaserestrictInfo; + } + + + /* 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(copiedQuery, copiedRestrictionContext, + &selectPlacementList, &selectAnchorShardId, + &relationShardList, replacePrunedQueryWithDummy); + + 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 target shard interval %ld since " + "SELECT query for it pruned away", shardId))); + + return NULL; + } + + /* * Ands are made implicit during shard pruning, as predicate comparison and * refutation depend on it being so. We need to make them explicit again so * that the query string is generated as (...) AND (...) as opposed to * (...), (...). */ - whereClauseTree = (Node *) make_ands_explicit((List *) subquery->jointree->quals); - subquery->jointree->quals = whereClauseTree; + copiedQuery->jointree->quals = + (Node *) make_ands_explicit((List *) copiedQuery->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(copiedQuery, 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, taskIdIndex, SQL_TASK, queryString->data); + subqueryTask->dependedTaskList = NULL; + subqueryTask->anchorShardId = selectAnchorShardId; + subqueryTask->taskPlacementList = selectPlacementList; + subqueryTask->upsertQuery = upsertQuery; + 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 job " UINT64_FORMAT - " and task %d", sqlTask->jobId, sqlTask->taskId), - errdetail("query string: \"%s\"", sqlQueryString->data))); - - sqlTask->anchorShardId = AnchorShardId(fragmentCombination, anchorRangeTableId); - - taskIdIndex++; - sqlTaskList = lappend(sqlTaskList, sqlTask); - } - - return sqlTaskList; + return subqueryTask; } @@ -3759,54 +3794,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 0ba89afce..6c7506a7f 100644 --- a/src/backend/distributed/planner/multi_planner.c +++ b/src/backend/distributed/planner/multi_planner.c @@ -293,7 +293,8 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query CheckNodeIsDumpable((Node *) logicalPlan); /* Create the physical plan */ - distributedPlan = MultiPhysicalPlanCreate(logicalPlan); + distributedPlan = MultiPhysicalPlanCreate(logicalPlan, restrictionContext, + joinRestrictionContext); /* distributed plan currently should always succeed or error out */ Assert(distributedPlan && distributedPlan->planningError == NULL); diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 8ec97b52a..b0204813f 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -89,10 +89,6 @@ static MultiPlan * CreateInsertSelectRouterPlan(Query *originalQuery, restrictionContext, JoinRestrictionContext * joinRestrictionContext); -static bool AllRelationsJoinedOnPartitionKey(RelationRestrictionContext - *restrictionContext, - JoinRestrictionContext * - joinRestrictionContext); static List * PartitionKeyEquivalenceClassList(List *attributeEquivalenceClassList); static uint32 ReferenceRelationCount(RelationRestrictionContext *restrictionContext); static List * GenerateAttributeEquivalencesForRelationRestrictions( @@ -122,7 +118,6 @@ static Task * RouterModifyTaskForShardInterval(Query *originalQuery, 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); @@ -137,10 +132,6 @@ static Const * 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); @@ -149,8 +140,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, @@ -408,7 +397,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery, * Finally, as the name of the function reveals, the function returns true if all relations * are joined on their partition keys. Otherwise, the function returns false. */ -static bool +bool AllRelationsJoinedOnPartitionKey(RelationRestrictionContext *restrictionContext, JoinRestrictionContext *joinRestrictionContext) { @@ -1122,8 +1111,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); @@ -1143,10 +1132,10 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter continue; } - hashedOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex); - - hashedRestrictInfo = make_simple_restrictinfo((Expr *) hashedOpExpressions); - originalBaserestrictInfo = lappend(originalBaserestrictInfo, hashedRestrictInfo); + shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex); + shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions); + originalBaserestrictInfo = lappend(originalBaserestrictInfo, + shardRestrictionList); restriction->relOptInfo->baserestrictinfo = originalBaserestrictInfo; } @@ -1258,7 +1247,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter * The function errors out if the given shard interval does not belong to a hash, * range and append distributed tables. */ -static List * +List * ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex) { Oid relationId = shardInterval->relationId; @@ -1274,7 +1263,6 @@ ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex) DISTRIBUTE_BY_APPEND) { Assert(rteIndex > 0); - partitionColumn = PartitionColumn(relationId, rteIndex); } else @@ -2950,7 +2938,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) @@ -3590,7 +3578,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/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 0cf340899..9d2556ffe 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" @@ -246,7 +247,9 @@ typedef struct OperatorCacheEntry extern int TaskAssignmentPolicy; /* Function declarations for building physical plans and constructing queries */ -extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree); +extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree, + RelationRestrictionContext *restrictionContext, + JoinRestrictionContext *joinRestrictionContext); 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 b62b80ec4..3f26b758d 100644 --- a/src/include/distributed/multi_planner.h +++ b/src/include/distributed/multi_planner.h @@ -100,8 +100,6 @@ typedef struct AttributeEquivalenceClassMember 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 61f2c074b..2d09ceb92 100644 --- a/src/include/distributed/multi_router_planner.h +++ b/src/include/distributed/multi_router_planner.h @@ -31,11 +31,26 @@ extern MultiPlan * CreateRouterPlan(Query *originalQuery, Query *query, extern MultiPlan * CreateModifyPlan(Query *originalQuery, Query *query, RelationRestrictionContext *restrictionContext, JoinRestrictionContext *joinRestrictionContext); - +extern bool AllRelationsJoinedOnPartitionKey(RelationRestrictionContext + *restrictionContext, + JoinRestrictionContext * + joinRestrictionContext); +extern bool RouterSelectQuery(Query *originalQuery, + RelationRestrictionContext *restrictionContext, + List **placementList, uint64 *anchorShardId, + List **relationShardList, bool replacePrunedQueryWithDummy); +extern bool AllRelationsJoinedOnPartitionKey( + RelationRestrictionContext *restrictionContext, + JoinRestrictionContext * + joinRestrictionContext); 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/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index ed6dc72b2..53c45f66b 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -1557,6 +1557,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_subquery_complex_queries.out b/src/test/regress/expected/multi_subquery_complex_queries.out new file mode 100644 index 000000000..02f526edc --- /dev/null +++ b/src/test/regress/expected/multi_subquery_complex_queries.out @@ -0,0 +1,870 @@ +-- +-- 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.subquery_pushdown TO TRUE; +SET citus.enable_router_execution TO FALSE; + -- + -- NOT SUPPORTED YET BUT SHOULD BE ADDED + -- +--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 YET BUT SHOULD BE ADDED + -- +--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") +-- FROM events_table as "events" +-- WHERE event_type IN (26, 27, 28, 29, 30, 13) +-- GROUP BY "events"."user_id") as t2 on t2.user_id = t.user_id ) 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 YET BUT SHOULD BE ADDED + -- +--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") +-- FROM events_table as "events" +-- WHERE event_type IN (26, 27, 28, 29, 30, 13) +-- GROUP BY "events"."user_id") as t2 on t2.user_id = t.user_id ) 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 +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 | Sat Mar 11 14:43:29.525317 2017 + 22 | Sat Mar 11 14:43:29.525317 2017 + 25 | Sat Mar 11 14:43:29.525317 2017 + 31 | Sat Mar 11 14:43:29.525317 2017 + 33 | Sat Mar 11 14:43:29.525317 2017 + 34 | Sat Mar 11 14:43:29.525317 2017 +(6 rows) + +-- not supported since JOIN is not on the partition key +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"."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 lastseen DESC + LIMIT 50) "some_users" + order BY user_id + limit 50; +ERROR: cannot perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- not supported since JOIN is not on the partition key +-- see (2 * user_id as user_id) target list element +SELECT * +FROM + (SELECT "some_users_data".user_id, lastseen + FROM + (SELECT 2 * user_id as user_id, + (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" + ) "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; +ERROR: cannot perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- LATERAL JOINs used with INNER JOINs +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 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 + 15 | Sat Mar 11 14:43:29.525317 2017 +(10 rows) + +-- 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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- 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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- 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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- 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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- 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" + 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".value_2 = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery" + GROUP BY "generated_group_field" ORDER BY generated_group_field DESC, value DESC; +ERROR: cannot perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- 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" + 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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- 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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +-- 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 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) + +-- 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 +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) + +-- 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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. +SET citus.subquery_pushdown TO FALSE; +SET citus.enable_router_execution TO TRUE; diff --git a/src/test/regress/input/multi_subquery.source b/src/test/regress/input/multi_subquery.source index 4d8b1b5fa..cfb2e43a4 100644 --- a/src/test/regress/input/multi_subquery.source +++ b/src/test/regress/input/multi_subquery.source @@ -180,21 +180,24 @@ SELECT count(*) FROM (SELECT 1::bigint) ) b; +--- +-- TEMPORARLY DISABLE UNIONS WITHOUT JOINS +--- -- 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; +--SELECT count(*) FROM +--( + -- (SELECT l_orderkey FROM lineitem_subquery) UNION + -- (SELECT l_partkey FROM lineitem_subquery) +--) b; -- Check that we run union queries if partition column is selected. -SELECT count(*) FROM -( - (SELECT l_orderkey FROM lineitem_subquery) UNION - (SELECT l_orderkey FROM lineitem_subquery) -) b; +--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. @@ -272,23 +275,23 @@ WHERE -- Check that if subquery is pulled, we don't error and run query properly. -SELECT count(*) FROM -( - SELECT l_orderkey FROM ( - (SELECT l_orderkey FROM lineitem_subquery) UNION - (SELECT l_orderkey FROM lineitem_subquery) - ) a - WHERE l_orderkey = 1 -) b; +--SELECT count(*) FROM +--( + --SELECT l_orderkey FROM ( + --(SELECT l_orderkey FROM lineitem_subquery) UNION + --(SELECT l_orderkey FROM lineitem_subquery) + --) a + --WHERE l_orderkey = 1 +--) b; -SELECT count(*) FROM -( - SELECT * FROM ( - (SELECT * FROM lineitem_subquery) UNION - (SELECT * FROM lineitem_subquery) - ) a - WHERE l_orderkey = 1 -) b; +--SELECT count(*) FROM +--( + --SELECT * FROM ( + --(SELECT * FROM lineitem_subquery) UNION + --(SELECT * FROM lineitem_subquery) + --) a + --WHERE l_orderkey = 1 +--) b; SELECT max(l_orderkey) FROM ( @@ -389,6 +392,154 @@ 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; + + 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 @@ -397,6 +548,153 @@ 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; + + 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 @@ -405,6 +703,153 @@ 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; + + 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 @@ -503,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 @@ -527,14 +971,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') @@ -542,14 +986,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') @@ -567,8 +1011,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, @@ -591,14 +1034,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') @@ -606,14 +1049,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') @@ -622,6 +1065,7 @@ SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -630,13 +1074,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, @@ -647,7 +1089,7 @@ GROUP BY count_pay ORDER BY count_pay; - + -- Lateral join subquery pushdown SELECT tenant_id, @@ -664,6 +1106,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -682,8 +1125,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 @@ -719,8 +1161,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 @@ -743,14 +1184,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') @@ -758,14 +1199,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') @@ -783,8 +1224,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, @@ -807,14 +1247,14 @@ EXPLAIN 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') @@ -822,14 +1262,14 @@ EXPLAIN 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') @@ -838,6 +1278,7 @@ EXPLAIN SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -846,13 +1287,11 @@ EXPLAIN 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, @@ -880,6 +1319,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -898,8 +1338,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 @@ -914,5 +1353,5 @@ ORDER BY user_lastseen DESC LIMIT 10; - + SET citus.enable_router_execution TO 'true'; diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index 3ffb820be..0145ead8c 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -40,7 +40,7 @@ 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 test: multi_reference_table test: multi_outer_join_reference test: multi_single_relation_subquery diff --git a/src/test/regress/output/multi_subquery.source b/src/test/regress/output/multi_subquery.source index 3f892b2fa..0e1a0ec86 100644 --- a/src/test/regress/output/multi_subquery.source +++ b/src/test/regress/output/multi_subquery.source @@ -155,8 +155,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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. SELECT avg(unit_price) FROM @@ -170,8 +170,8 @@ 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 +ERROR: cannot perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. -- Check that we error out if there is union all. SELECT count(*) FROM ( @@ -179,25 +179,21 @@ SELECT count(*) FROM (SELECT 1::bigint) ) b; ERROR: could not run distributed query with complex table expressions +--- +-- TEMPORARLY DISABLE UNIONS WITHOUT JOINS +--- -- 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 +--SELECT count(*) FROM +--( + -- (SELECT l_orderkey FROM lineitem_subquery) UNION + -- (SELECT l_partkey FROM lineitem_subquery) +--) b; -- Check that we run union queries if partition column is selected. -SELECT count(*) FROM -( - (SELECT l_orderkey FROM lineitem_subquery) UNION - (SELECT l_orderkey FROM lineitem_subquery) -) b; - count -------- - 2985 -(1 row) - +--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. SELECT avg(o_totalprice/l_quantity) @@ -289,32 +285,22 @@ WHERE (1 row) -- Check that if subquery is pulled, we don't error and run query properly. -SELECT count(*) FROM -( - SELECT l_orderkey FROM ( - (SELECT l_orderkey FROM lineitem_subquery) UNION - (SELECT l_orderkey FROM lineitem_subquery) - ) a - WHERE l_orderkey = 1 -) b; - count -------- - 1 -(1 row) - -SELECT count(*) FROM -( - SELECT * FROM ( - (SELECT * FROM lineitem_subquery) UNION - (SELECT * FROM lineitem_subquery) - ) a - WHERE l_orderkey = 1 -) b; - count -------- - 6 -(1 row) - +--SELECT count(*) FROM +--( + --SELECT l_orderkey FROM ( + --(SELECT l_orderkey FROM lineitem_subquery) UNION + --(SELECT l_orderkey FROM lineitem_subquery) + --) a + --WHERE l_orderkey = 1 +--) b; +--SELECT count(*) FROM +--( + --SELECT * FROM ( + --(SELECT * FROM lineitem_subquery) UNION + --(SELECT * FROM lineitem_subquery) + --) a + --WHERE l_orderkey = 1 +--) b; SELECT max(l_orderkey) FROM ( SELECT l_orderkey FROM ( @@ -379,7 +365,22 @@ SELECT * FROM AS foo; DEBUG: predicate pruning for shardId 270005 DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270005 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270006 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270008 +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270008 since SELECT query for it pruned away count ------- (0 rows) @@ -388,8 +389,23 @@ SELECT * FROM (SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE 'eren' = a GROUP BY a) AS foo; DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 DEBUG: predicate pruning for shardId 270007 DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270005 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270007 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270008 since SELECT query for it pruned away count ------- (0 rows) @@ -434,18 +450,390 @@ 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; + + 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; + + 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; + + 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, @@ -522,8 +910,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 @@ -550,14 +937,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') @@ -565,14 +952,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') @@ -590,8 +977,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, @@ -619,14 +1005,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') @@ -634,14 +1020,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') @@ -650,6 +1036,7 @@ SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -658,13 +1045,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, @@ -680,6 +1065,7 @@ ORDER BY 3.0000000000000000 | 0 (1 row) + -- Lateral join subquery pushdown SELECT tenant_id, @@ -696,6 +1082,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -714,8 +1101,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 @@ -756,36 +1142,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 (cost=0.00..0.00 rows=0 width=0) -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) - 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 (cost=40.01..40.02 rows=1 width=16) - -> GroupAggregate (cost=39.89..39.99 rows=1 width=48) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Merge Join (cost=39.89..39.97 rows=1 width=540) - Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) - -> Sort (cost=28.08..28.09 rows=6 width=32) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Sort (cost=11.81..11.82 rows=3 width=556) - Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) - -> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556) - Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) -(19 rows) + -> Aggregate (cost=12.04..12.05 rows=1 width=16) + -> GroupAggregate (cost=11.99..12.02 rows=1 width=48) + Group Key: (((NULL::user_composite_type)).tenant_id), (((NULL::user_composite_type)).user_id) + -> Sort (cost=11.99..12.00 rows=1 width=540) + Sort Key: (((NULL::user_composite_type)).tenant_id), (((NULL::user_composite_type)).user_id) + -> Hash Join (cost=11.82..11.98 rows=1 width=540) + Hash Cond: ((NULL::user_composite_type) = events.composite_id) + -> Result (cost=0.00..0.01 rows=1 width=40) + One-Time Filter: false + -> Hash (cost=11.79..11.79 rows=3 width=556) + -> Seq Scan on events_270011 events (cost=0.00..11.79 rows=3 width=556) + Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) +(18 rows) -- Union and left join subquery pushdown EXPLAIN SELECT @@ -802,14 +1186,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') @@ -817,14 +1201,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') @@ -842,57 +1226,57 @@ 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 (cost=0.00..0.00 rows=0 width=0) Group Key: remote_scan.hasdone -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) - 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 (cost=91.93..91.98 rows=2 width=48) + -> GroupAggregate (cost=100.80..100.85 rows=2 width=48) Group Key: subquery_top.hasdone - -> Sort (cost=91.93..91.93 rows=2 width=64) + -> Sort (cost=100.80..100.80 rows=2 width=64) Sort Key: subquery_top.hasdone - -> Subquery Scan on subquery_top (cost=91.85..91.92 rows=2 width=64) - -> GroupAggregate (cost=91.85..91.90 rows=2 width=112) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Sort (cost=91.85..91.85 rows=2 width=88) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Merge Left Join (cost=91.75..91.84 rows=2 width=88) - 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))) - -> Unique (cost=79.46..79.48 rows=2 width=56) - -> Sort (cost=79.46..79.47 rows=2 width=56) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time - -> Append (cost=0.00..79.45 rows=2 width=56) - -> Nested Loop (cost=0.00..39.72 rows=1 width=56) - 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 (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'click'::text) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Nested Loop (cost=0.00..39.72 rows=1 width=56) - 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 (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'submit'::text) - -> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Materialize (cost=12.29..12.31 rows=1 width=48) - -> Unique (cost=12.29..12.30 rows=1 width=80) - -> Sort (cost=12.29..12.29 rows=1 width=80) - Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) - -> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=80) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) -(40 rows) + -> Subquery Scan on subquery_top (cost=100.72..100.79 rows=2 width=64) + -> GroupAggregate (cost=100.72..100.77 rows=2 width=112) + Group Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) + -> Sort (cost=100.72..100.72 rows=2 width=88) + Sort Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) + -> Hash Right Join (cost=100.55..100.71 rows=2 width=88) + Hash Cond: (events.composite_id = subquery_1.composite_id) + -> Unique (cost=76.64..76.64 rows=1 width=80) + -> Sort (cost=76.64..76.64 rows=1 width=80) + Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) + -> Seq Scan on events_270011 events (cost=0.00..76.62 rows=1 width=80) + Filter: (((event_type)::text = 'pay'::text) AND (composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Hash (cost=23.89..23.89 rows=2 width=88) + -> Subquery Scan on subquery_1 (cost=23.84..23.89 rows=2 width=88) + -> Unique (cost=23.84..23.87 rows=2 width=88) + -> Sort (cost=23.84..23.84 rows=2 width=88) + 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 (cost=0.00..23.83 rows=2 width=88) + -> Nested Loop (cost=0.00..11.90 rows=1 width=88) + Join Filter: ((NULL::user_composite_type) = events_1.composite_id) + -> Result (cost=0.00..0.01 rows=1 width=40) + One-Time Filter: false + -> Seq Scan on events_270011 events_1 (cost=0.00..11.62 rows=1 width=40) + Filter: ((event_type)::text = 'click'::text) + -> Nested Loop (cost=0.00..11.90 rows=1 width=88) + Join Filter: ((NULL::user_composite_type) = events_2.composite_id) + -> Result (cost=0.00..0.01 rows=1 width=40) + One-Time Filter: false + -> Seq Scan on events_270011 events_2 (cost=0.00..11.62 rows=1 width=40) + Filter: ((event_type)::text = 'submit'::text) +(41 rows) -- Union, left join and having subquery pushdown EXPLAIN SELECT @@ -909,14 +1293,14 @@ EXPLAIN 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') @@ -924,14 +1308,14 @@ EXPLAIN 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') @@ -940,6 +1324,7 @@ EXPLAIN SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -948,13 +1333,11 @@ EXPLAIN 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, @@ -982,6 +1365,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -1000,8 +1384,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 @@ -1016,34 +1399,36 @@ ORDER BY user_lastseen DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------- Limit (cost=0.00..0.00 rows=0 width=0) -> Sort (cost=0.00..0.00 rows=0 width=0) Sort Key: remote_scan.user_lastseen DESC -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) - 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 (cost=100.43..100.44 rows=6 width=56) - -> Sort (cost=100.43..100.44 rows=6 width=56) + -> Limit (cost=43.94..43.94 rows=1 width=56) + -> Sort (cost=43.94..43.94 rows=1 width=56) Sort Key: (max(users.lastseen)) DESC - -> GroupAggregate (cost=100.14..100.29 rows=6 width=56) + -> GroupAggregate (cost=43.89..43.92 rows=1 width=56) Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Sort (cost=100.14..100.16 rows=6 width=548) + -> Sort (cost=43.89..43.90 rows=1 width=548) Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548) - -> Limit (cost=28.08..28.09 rows=6 width=24) - -> Sort (cost=28.08..28.09 rows=6 width=24) + -> Nested Loop Left Join (cost=43.84..43.88 rows=1 width=548) + -> Limit (cost=0.03..0.04 rows=1 width=56) + -> Sort (cost=0.03..0.04 rows=1 width=56) Sort Key: users.lastseen DESC - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=24) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Limit (cost=11.96..11.96 rows=1 width=524) - -> Sort (cost=11.96..11.96 rows=1 width=524) + -> Subquery Scan on users (cost=0.00..0.02 rows=1 width=56) + -> Result (cost=0.00..0.01 rows=1 width=40) + One-Time Filter: false + -> Limit (cost=43.81..43.82 rows=1 width=524) + -> Sort (cost=43.81..43.82 rows=1 width=524) Sort Key: events.event_time DESC - -> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524) - 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 (cost=0.00..43.80 rows=1 width=524) + Filter: (composite_id = users.composite_id) +(27 rows) + 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 5d5eb53e9..c65697a94 100644 --- a/src/test/regress/output/multi_subquery_0.source +++ b/src/test/regress/output/multi_subquery_0.source @@ -155,8 +155,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 perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. SELECT avg(unit_price) FROM @@ -170,8 +170,8 @@ 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 +ERROR: cannot perform distributed planning for the given query +DETAIL: Select query cannot be pushed down to the worker. -- Check that we error out if there is union all. SELECT count(*) FROM ( @@ -179,25 +179,21 @@ SELECT count(*) FROM (SELECT 1::bigint) ) b; ERROR: could not run distributed query with complex table expressions +--- +-- TEMPORARLY DISABLE UNIONS WITHOUT JOINS +--- -- 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 +--SELECT count(*) FROM +--( + -- (SELECT l_orderkey FROM lineitem_subquery) UNION + -- (SELECT l_partkey FROM lineitem_subquery) +--) b; -- Check that we run union queries if partition column is selected. -SELECT count(*) FROM -( - (SELECT l_orderkey FROM lineitem_subquery) UNION - (SELECT l_orderkey FROM lineitem_subquery) -) b; - count -------- - 2985 -(1 row) - +--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. SELECT avg(o_totalprice/l_quantity) @@ -289,32 +285,22 @@ WHERE (1 row) -- Check that if subquery is pulled, we don't error and run query properly. -SELECT count(*) FROM -( - SELECT l_orderkey FROM ( - (SELECT l_orderkey FROM lineitem_subquery) UNION - (SELECT l_orderkey FROM lineitem_subquery) - ) a - WHERE l_orderkey = 1 -) b; - count -------- - 1 -(1 row) - -SELECT count(*) FROM -( - SELECT * FROM ( - (SELECT * FROM lineitem_subquery) UNION - (SELECT * FROM lineitem_subquery) - ) a - WHERE l_orderkey = 1 -) b; - count -------- - 6 -(1 row) - +--SELECT count(*) FROM +--( + --SELECT l_orderkey FROM ( + --(SELECT l_orderkey FROM lineitem_subquery) UNION + --(SELECT l_orderkey FROM lineitem_subquery) + --) a + --WHERE l_orderkey = 1 +--) b; +--SELECT count(*) FROM +--( + --SELECT * FROM ( + --(SELECT * FROM lineitem_subquery) UNION + --(SELECT * FROM lineitem_subquery) + --) a + --WHERE l_orderkey = 1 +--) b; SELECT max(l_orderkey) FROM ( SELECT l_orderkey FROM ( @@ -379,7 +365,22 @@ SELECT * FROM AS foo; DEBUG: predicate pruning for shardId 270005 DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270005 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270006 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270008 +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270008 since SELECT query for it pruned away count ------- (0 rows) @@ -388,8 +389,23 @@ SELECT * FROM (SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE 'eren' = a GROUP BY a) AS foo; DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 DEBUG: predicate pruning for shardId 270007 DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270005 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270007 since SELECT query for it pruned away +DEBUG: predicate pruning for shardId 270005 +DEBUG: predicate pruning for shardId 270006 +DEBUG: predicate pruning for shardId 270007 +DEBUG: predicate pruning for shardId 270008 +DEBUG: Skipping target shard interval 270008 since SELECT query for it pruned away count ------- (0 rows) @@ -434,18 +450,390 @@ 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; + + 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; + + 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; + + 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, @@ -522,8 +910,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 @@ -550,14 +937,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') @@ -565,14 +952,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') @@ -590,8 +977,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, @@ -619,14 +1005,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') @@ -634,14 +1020,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') @@ -650,6 +1036,7 @@ SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -658,13 +1045,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, @@ -680,6 +1065,7 @@ ORDER BY 3.0000000000000000 | 0 (1 row) + -- Lateral join subquery pushdown SELECT tenant_id, @@ -696,6 +1082,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -714,8 +1101,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 @@ -756,36 +1142,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 (cost=0.00..0.00 rows=0 width=0) -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) - 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 (cost=40.01..40.02 rows=1 width=32) - -> GroupAggregate (cost=39.89..39.99 rows=1 width=556) + -> Aggregate (cost=0.07..0.07 rows=1 width=32) + -> GroupAggregate (cost=0.02..0.05 rows=1 width=0) Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Merge Join (cost=39.89..39.97 rows=1 width=556) - Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id))) - -> Sort (cost=28.08..28.09 rows=6 width=32) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Sort (cost=11.81..11.82 rows=3 width=556) - Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) - -> Seq Scan on events_270009 events (cost=0.00..11.79 rows=3 width=556) - Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[])) -(19 rows) + -> Sort (cost=0.02..0.03 rows=1 width=0) + Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) + -> Result (cost=0.00..0.01 rows=1 width=0) + One-Time Filter: false +(13 rows) -- Union and left join subquery pushdown EXPLAIN SELECT @@ -802,14 +1181,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') @@ -817,14 +1196,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') @@ -842,54 +1221,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 (cost=0.00..0.00 rows=0 width=0) Group Key: remote_scan.hasdone -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) - 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 (cost=91.94..91.96 rows=2 width=64) + -> HashAggregate (cost=77.02..77.04 rows=2 width=64) Group Key: COALESCE(('Has done paying'::text), 'Has not done paying'::text) - -> GroupAggregate (cost=91.85..91.90 rows=2 width=88) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Sort (cost=91.85..91.85 rows=2 width=88) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text) - -> Merge Left Join (cost=91.75..91.84 rows=2 width=88) - 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))) - -> Unique (cost=79.46..79.48 rows=2 width=40) - -> Sort (cost=79.46..79.47 rows=2 width=40) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time - -> Append (cost=0.00..79.45 rows=2 width=40) - -> Nested Loop (cost=0.00..39.72 rows=1 width=40) - 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 (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'click'::text) - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Nested Loop (cost=0.00..39.72 rows=1 width=40) - 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 (cost=0.00..11.62 rows=1 width=40) - Filter: ((event_type)::text = 'submit'::text) - -> Seq Scan on users_270013 users_1 (cost=0.00..28.00 rows=6 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Materialize (cost=12.29..12.31 rows=1 width=48) - -> Unique (cost=12.29..12.30 rows=1 width=32) - -> Sort (cost=12.29..12.29 rows=1 width=32) - Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id) - -> Seq Scan on events_270009 events_2 (cost=0.00..12.28 rows=1 width=32) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text)) -(37 rows) + -> GroupAggregate (cost=76.93..76.98 rows=2 width=88) + Group Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) + -> Sort (cost=76.93..76.93 rows=2 width=88) + Sort Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text) + -> Hash Right Join (cost=76.76..76.92 rows=2 width=88) + Hash Cond: (events.composite_id = subquery_1.composite_id) + -> Unique (cost=76.64..76.64 rows=1 width=32) + -> Sort (cost=76.64..76.64 rows=1 width=32) + Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id) + -> Seq Scan on events_270011 events (cost=0.00..76.62 rows=1 width=32) + Filter: (((event_type)::text = 'pay'::text) AND (composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) + -> Hash (cost=0.10..0.10 rows=2 width=88) + -> Subquery Scan on subquery_1 (cost=0.05..0.10 rows=2 width=88) + -> Unique (cost=0.05..0.08 rows=2 width=0) + -> Sort (cost=0.05..0.06 rows=2 width=0) + Sort Key: ((composite_id).tenant_id), ((composite_id).user_id), composite_id, ('action=>1'::text), event_time + -> Append (cost=0.00..0.04 rows=2 width=0) + -> Result (cost=0.00..0.01 rows=1 width=0) + One-Time Filter: false + -> Result (cost=0.00..0.01 rows=1 width=0) + One-Time Filter: false +(30 rows) -- Union, left join and having subquery pushdown EXPLAIN SELECT @@ -906,14 +1277,14 @@ EXPLAIN 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') @@ -921,14 +1292,14 @@ EXPLAIN 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') @@ -937,6 +1308,7 @@ EXPLAIN SELECT (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, COUNT(*) AS count_pay FROM events @@ -945,13 +1317,11 @@ EXPLAIN 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, @@ -979,6 +1349,7 @@ FROM (SELECT (composite_id).tenant_id, (composite_id).user_id, + composite_id, lastseen FROM users @@ -997,8 +1368,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,34 +1383,35 @@ ORDER BY user_lastseen DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------- Limit (cost=0.00..0.00 rows=0 width=0) -> Sort (cost=0.00..0.00 rows=0 width=0) Sort Key: remote_scan.user_lastseen DESC -> Custom Scan (Citus Real-Time) (cost=0.00..0.00 rows=0 width=0) - 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 (cost=100.43..100.44 rows=6 width=56) - -> Sort (cost=100.43..100.44 rows=6 width=56) - Sort Key: (max(users.lastseen)) DESC - -> GroupAggregate (cost=100.14..100.29 rows=6 width=548) - Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Sort (cost=100.14..100.16 rows=6 width=548) - Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id) - -> Nested Loop Left Join (cost=40.04..100.06 rows=6 width=548) - -> Limit (cost=28.08..28.09 rows=6 width=40) - -> Sort (cost=28.08..28.09 rows=6 width=40) - Sort Key: users.lastseen DESC - -> Seq Scan on users_270013 users (cost=0.00..28.00 rows=6 width=40) - Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type)) - -> Limit (cost=11.96..11.96 rows=1 width=524) - -> Sort (cost=11.96..11.96 rows=1 width=524) + -> Limit (cost=43.93..43.93 rows=1 width=56) + -> Sort (cost=43.93..43.93 rows=1 width=56) + Sort Key: (max(lastseen)) DESC + -> GroupAggregate (cost=43.88..43.91 rows=1 width=548) + Group Key: ((composite_id).tenant_id), ((composite_id).user_id) + -> Sort (cost=43.88..43.89 rows=1 width=548) + Sort Key: ((composite_id).tenant_id), ((composite_id).user_id) + -> Nested Loop Left Join (cost=43.83..43.87 rows=1 width=548) + -> Limit (cost=0.02..0.03 rows=1 width=0) + -> Sort (cost=0.02..0.03 rows=1 width=0) + Sort Key: lastseen DESC + -> Result (cost=0.00..0.01 rows=1 width=0) + One-Time Filter: false + -> Limit (cost=43.81..43.82 rows=1 width=524) + -> Sort (cost=43.81..43.82 rows=1 width=524) Sort Key: events.event_time DESC - -> Seq Scan on events_270009 events (cost=0.00..11.95 rows=1 width=524) - 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 (cost=0.00..43.80 rows=1 width=524) + Filter: (composite_id = composite_id) (26 rows) + SET citus.enable_router_execution TO 'true'; diff --git a/src/test/regress/sql/multi_insert_select.sql b/src/test/regress/sql/multi_insert_select.sql index f1f68c7fb..2cfc02701 100644 --- a/src/test/regress/sql/multi_insert_select.sql +++ b/src/test/regress/sql/multi_insert_select.sql @@ -734,6 +734,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_subquery_complex_queries.sql b/src/test/regress/sql/multi_subquery_complex_queries.sql new file mode 100644 index 000000000..f6336ac90 --- /dev/null +++ b/src/test/regress/sql/multi_subquery_complex_queries.sql @@ -0,0 +1,763 @@ +-- +-- 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.subquery_pushdown TO TRUE; +SET citus.enable_router_execution TO FALSE; + + -- + -- NOT SUPPORTED YET BUT SHOULD BE ADDED + -- +--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 YET BUT SHOULD BE ADDED + -- +--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") +-- FROM events_table as "events" +-- WHERE event_type IN (26, 27, 28, 29, 30, 13) +-- GROUP BY "events"."user_id") as t2 on t2.user_id = t.user_id ) 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 YET BUT SHOULD BE ADDED + -- +--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") +-- FROM events_table as "events" +-- WHERE event_type IN (26, 27, 28, 29, 30, 13) +-- GROUP BY "events"."user_id") as t2 on t2.user_id = t.user_id ) 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 +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; + +-- not supported since JOIN is not on the partition key +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"."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 lastseen DESC + LIMIT 50) "some_users" + 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 * +FROM + (SELECT "some_users_data".user_id, lastseen + FROM + (SELECT 2 * user_id as user_id, + (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" + ) "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; + +-- LATERAL JOINs used with INNER JOINs +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 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" + 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".value_2 = "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" + 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 +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 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; + + + +-- 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 +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; + +-- 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; + +SET citus.subquery_pushdown TO FALSE; +SET citus.enable_router_execution TO TRUE; \ No newline at end of file