mirror of https://github.com/citusdata/citus.git
Merge pull request #1282 from citusdata/subquery_physical_planner
Physical planner changes for subquery pushdownpull/1297/head
commit
3c41f9abff
|
@ -147,16 +147,15 @@ static bool TablePartitioningSupportsDistinct(List *tableNodeList,
|
|||
static bool GroupedByColumn(List *groupClauseList, List *targetList, Var *column);
|
||||
|
||||
/* Local functions forward declarations for subquery pushdown checks */
|
||||
static void ErrorIfContainsUnsupportedSubquery(MultiNode *logicalPlanNode);
|
||||
static void ErrorIfContainsUnsupportedSubquery(MultiNode *logicalPlanNode,
|
||||
PlannerRestrictionContext *
|
||||
plannerRestrictionContext);
|
||||
static void ErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerQueryHasLimit);
|
||||
static void ErrorIfUnsupportedTableCombination(Query *queryTree);
|
||||
static void ErrorIfUnsupportedUnionQuery(Query *unionQuery);
|
||||
static bool TargetListOnPartitionColumn(Query *query, List *targetEntryList);
|
||||
static FieldSelect * CompositeFieldRecursive(Expr *expression, Query *query);
|
||||
static bool FullCompositeFieldList(List *compositeFieldList);
|
||||
static Query * LateralQuery(Query *query);
|
||||
static bool SupportedLateralQuery(Query *parentQuery, Query *lateralQuery);
|
||||
static bool JoinOnPartitionColumn(Query *query);
|
||||
static void ErrorIfUnsupportedShardDistribution(Query *query);
|
||||
static List * RelationIdList(Query *query);
|
||||
static bool CoPartitionedTables(Oid firstRelationId, Oid secondRelationId);
|
||||
|
@ -189,9 +188,15 @@ static bool HasOrderByHllType(List *sortClauseList, List *targetList);
|
|||
* Third, the function pulls up the collect operators in the tree. Fourth, the
|
||||
* function finds the extended operator node, and splits this node into master
|
||||
* and worker extended operator nodes.
|
||||
*
|
||||
* We also pass plannerRestrictionContext to the optimizer. The context
|
||||
* is primarily used to decide whether the subquery is safe to pushdown.
|
||||
* If not, it helps to produce meaningful error messages for subquery
|
||||
* pushdown planning.
|
||||
*/
|
||||
void
|
||||
MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan)
|
||||
MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan,
|
||||
PlannerRestrictionContext *plannerRestrictionContext)
|
||||
{
|
||||
bool hasOrderByHllType = false;
|
||||
List *selectNodeList = NIL;
|
||||
|
@ -211,7 +216,7 @@ MultiLogicalPlanOptimize(MultiTreeRoot *multiLogicalPlan)
|
|||
ErrorIfContainsUnsupportedAggregate(logicalPlanNode);
|
||||
|
||||
/* check that we can pushdown subquery in the plan */
|
||||
ErrorIfContainsUnsupportedSubquery(logicalPlanNode);
|
||||
ErrorIfContainsUnsupportedSubquery(logicalPlanNode, plannerRestrictionContext);
|
||||
|
||||
/*
|
||||
* If a select node exists, we use the idempower property to split the node
|
||||
|
@ -2805,13 +2810,15 @@ GroupedByColumn(List *groupClauseList, List *targetList, Var *column)
|
|||
* the subquery.
|
||||
*/
|
||||
static void
|
||||
ErrorIfContainsUnsupportedSubquery(MultiNode *logicalPlanNode)
|
||||
ErrorIfContainsUnsupportedSubquery(MultiNode *logicalPlanNode,
|
||||
PlannerRestrictionContext *plannerRestrictionContext)
|
||||
{
|
||||
Query *subquery = NULL;
|
||||
List *extendedOpNodeList = NIL;
|
||||
MultiTable *multiTable = NULL;
|
||||
MultiExtendedOp *extendedOpNode = NULL;
|
||||
bool outerQueryHasLimit = false;
|
||||
bool restrictionEquivalenceForPartitionKeys = false;
|
||||
|
||||
/* check if logical plan includes a subquery */
|
||||
List *subqueryMultiTableList = SubqueryMultiTableList(logicalPlanNode);
|
||||
|
@ -2823,6 +2830,18 @@ ErrorIfContainsUnsupportedSubquery(MultiNode *logicalPlanNode)
|
|||
/* currently in the planner we only allow one subquery in from-clause*/
|
||||
Assert(list_length(subqueryMultiTableList) == 1);
|
||||
|
||||
restrictionEquivalenceForPartitionKeys =
|
||||
RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext);
|
||||
if (!restrictionEquivalenceForPartitionKeys)
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("cannot pushdown the subquery since all relations are not "
|
||||
"joined using distribution keys"),
|
||||
errdetail("Each relation should be joined with at least "
|
||||
"one another relation using distribution keys and "
|
||||
"equality operator.")));
|
||||
}
|
||||
|
||||
multiTable = (MultiTable *) linitial(subqueryMultiTableList);
|
||||
subquery = multiTable->subquery;
|
||||
|
||||
|
@ -2895,7 +2914,6 @@ ErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerQueryHasLimit)
|
|||
{
|
||||
bool preconditionsSatisfied = true;
|
||||
char *errorDetail = NULL;
|
||||
Query *lateralQuery = NULL;
|
||||
List *subqueryEntryList = NIL;
|
||||
ListCell *rangeTableEntryCell = NULL;
|
||||
|
||||
|
@ -2991,43 +3009,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 +3570,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.
|
||||
|
|
|
@ -118,8 +118,13 @@ static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray,
|
|||
|
||||
/* Local functions forward declarations for task list creation and helper functions */
|
||||
static bool MultiPlanRouterExecutable(MultiPlan *multiPlan);
|
||||
static Job * BuildJobTreeTaskList(Job *jobTree);
|
||||
static List * SubquerySqlTaskList(Job *job);
|
||||
static Job * BuildJobTreeTaskList(Job *jobTree,
|
||||
PlannerRestrictionContext *plannerRestrictionContext);
|
||||
static List * SubquerySqlTaskList(Job *job,
|
||||
PlannerRestrictionContext *plannerRestrictionContext);
|
||||
static Task * SubqueryTaskCreate(Query *originalQuery, ShardInterval *shardInterval,
|
||||
RelationRestrictionContext *restrictionContext,
|
||||
uint32 taskId);
|
||||
static List * SqlTaskList(Job *job);
|
||||
static bool DependsOnHashPartitionJob(Job *job);
|
||||
static uint32 AnchorRangeTableId(List *rangeTableList);
|
||||
|
@ -145,7 +150,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 +197,8 @@ static uint32 FinalTargetEntryCount(List *targetEntryList);
|
|||
* executed on worker nodes, and the final query to run on the master node.
|
||||
*/
|
||||
MultiPlan *
|
||||
MultiPhysicalPlanCreate(MultiTreeRoot *multiTree)
|
||||
MultiPhysicalPlanCreate(MultiTreeRoot *multiTree,
|
||||
PlannerRestrictionContext *plannerRestrictionContext)
|
||||
{
|
||||
MultiPlan *multiPlan = NULL;
|
||||
Job *workerJob = NULL;
|
||||
|
@ -204,7 +209,7 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree)
|
|||
workerJob = BuildJobTree(multiTree);
|
||||
|
||||
/* create the tree of executable tasks for the worker job */
|
||||
workerJob = BuildJobTreeTaskList(workerJob);
|
||||
workerJob = BuildJobTreeTaskList(workerJob, plannerRestrictionContext);
|
||||
|
||||
/* build the final merge query to execute on the master */
|
||||
masterDependedJobList = list_make1(workerJob);
|
||||
|
@ -1889,7 +1894,7 @@ SplitPointObject(ShardInterval **shardIntervalArray, uint32 shardIntervalCount)
|
|||
* tasks to worker nodes.
|
||||
*/
|
||||
static Job *
|
||||
BuildJobTreeTaskList(Job *jobTree)
|
||||
BuildJobTreeTaskList(Job *jobTree, PlannerRestrictionContext *plannerRestrictionContext)
|
||||
{
|
||||
List *flattenedJobList = NIL;
|
||||
uint32 flattenedJobCount = 0;
|
||||
|
@ -1927,7 +1932,7 @@ BuildJobTreeTaskList(Job *jobTree)
|
|||
/* create sql tasks for the job, and prune redundant data fetch tasks */
|
||||
if (job->subqueryPushdown)
|
||||
{
|
||||
sqlTaskList = SubquerySqlTaskList(job);
|
||||
sqlTaskList = SubquerySqlTaskList(job, plannerRestrictionContext);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -1979,133 +1984,170 @@ BuildJobTreeTaskList(Job *jobTree)
|
|||
|
||||
/*
|
||||
* SubquerySqlTaskList creates a list of SQL tasks to execute the given subquery
|
||||
* pushdown job. For this, it gets all range tables in the subquery tree, then
|
||||
* walks over each range table in the list, gets shards for each range table,
|
||||
* and prunes unneeded shards. Then for remaining shards, fragments are created
|
||||
* and merged to create fragment combinations. For each created combination, the
|
||||
* function builds a SQL task, and appends this task to a task list.
|
||||
* pushdown job. For this, the it is being checked whether the query is router
|
||||
* plannable per target shard interval. For those router plannable worker
|
||||
* queries, we create a SQL task and append the task to the task list that is going
|
||||
* to be executed.
|
||||
*/
|
||||
static List *
|
||||
SubquerySqlTaskList(Job *job)
|
||||
SubquerySqlTaskList(Job *job, PlannerRestrictionContext *plannerRestrictionContext)
|
||||
{
|
||||
Query *subquery = job->jobQuery;
|
||||
uint64 jobId = job->jobId;
|
||||
List *sqlTaskList = NIL;
|
||||
List *fragmentCombinationList = NIL;
|
||||
List *opExpressionList = NIL;
|
||||
List *queryList = NIL;
|
||||
List *rangeTableList = NIL;
|
||||
ListCell *fragmentCombinationCell = NULL;
|
||||
ListCell *rangeTableCell = NULL;
|
||||
ListCell *queryCell = NULL;
|
||||
Node *whereClauseTree = NULL;
|
||||
uint32 taskIdIndex = 1; /* 0 is reserved for invalid taskId */
|
||||
uint32 anchorRangeTableId = 0;
|
||||
uint32 rangeTableIndex = 0;
|
||||
const uint32 fragmentSize = sizeof(RangeTableFragment);
|
||||
uint64 largestTableSize = 0;
|
||||
Oid relationId = 0;
|
||||
int shardCount = 0;
|
||||
int shardOffset = 0;
|
||||
DistTableCacheEntry *targetCacheEntry = NULL;
|
||||
RelationRestrictionContext *relationRestrictionContext =
|
||||
plannerRestrictionContext->relationRestrictionContext;
|
||||
bool restrictionEquivalenceForPartitionKeys PG_USED_FOR_ASSERTS_ONLY =
|
||||
RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext);
|
||||
|
||||
/* 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;
|
||||
}
|
||||
}
|
||||
/*
|
||||
* If we're going to create tasks for subquery pushdown, all
|
||||
* relations needs to be joined on the partition key.
|
||||
*/
|
||||
Assert(restrictionEquivalenceForPartitionKeys);
|
||||
|
||||
/* 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)
|
||||
targetCacheEntry = DistributedTableCacheEntry(relationId);
|
||||
break;
|
||||
}
|
||||
|
||||
Assert(targetCacheEntry != NULL);
|
||||
|
||||
shardCount = targetCacheEntry->shardIntervalArrayLength;
|
||||
for (shardOffset = 0; shardOffset < shardCount; shardOffset++)
|
||||
{
|
||||
ShardInterval *targetShardInterval =
|
||||
targetCacheEntry->sortedShardIntervalArray[shardOffset];
|
||||
Task *subqueryTask = NULL;
|
||||
|
||||
subqueryTask = SubqueryTaskCreate(subquery, targetShardInterval,
|
||||
relationRestrictionContext, taskIdIndex);
|
||||
|
||||
/* add the task if it could be created */
|
||||
if (subqueryTask != NULL)
|
||||
{
|
||||
return NIL;
|
||||
subqueryTask->jobId = jobId;
|
||||
sqlTaskList = lappend(sqlTaskList, subqueryTask);
|
||||
|
||||
++taskIdIndex;
|
||||
}
|
||||
}
|
||||
|
||||
fragmentCombinationCell = list_head(fragmentCombinationList);
|
||||
return sqlTaskList;
|
||||
}
|
||||
|
||||
foreach(shardIntervalCell, finalShardIntervalList)
|
||||
{
|
||||
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
|
||||
|
||||
RangeTableFragment *shardFragment = palloc0(fragmentSize);
|
||||
shardFragment->fragmentReference = shardInterval;
|
||||
shardFragment->fragmentType = CITUS_RTE_RELATION;
|
||||
shardFragment->rangeTableId = tableId;
|
||||
/*
|
||||
* SubqueryTaskCreate creates a sql task by replacing the target
|
||||
* shardInterval's boundary value.. Then performs the normal
|
||||
* shard pruning on the subquery via RouterSelectQuery().
|
||||
*
|
||||
* The function errors out if the subquery is not router select query (i.e.,
|
||||
* subqueries with non equi-joins.).
|
||||
*/
|
||||
static Task *
|
||||
SubqueryTaskCreate(Query *originalQuery, ShardInterval *shardInterval,
|
||||
RelationRestrictionContext *restrictionContext,
|
||||
uint32 taskId)
|
||||
{
|
||||
Query *taskQuery = copyObject(originalQuery);
|
||||
|
||||
tableSize += ShardLength(shardInterval->shardId);
|
||||
uint64 shardId = shardInterval->shardId;
|
||||
Oid distributedTableId = shardInterval->relationId;
|
||||
StringInfo queryString = makeStringInfo();
|
||||
ListCell *restrictionCell = NULL;
|
||||
Task *subqueryTask = NULL;
|
||||
List *selectPlacementList = NIL;
|
||||
uint64 selectAnchorShardId = INVALID_SHARD_ID;
|
||||
List *relationShardList = NIL;
|
||||
uint64 jobId = INVALID_JOB_ID;
|
||||
bool routerPlannable = false;
|
||||
bool replacePrunedQueryWithDummy = false;
|
||||
RelationRestrictionContext *copiedRestrictionContext =
|
||||
CopyRelationRestrictionContext(restrictionContext);
|
||||
List *shardOpExpressions = NIL;
|
||||
RestrictInfo *shardRestrictionList = NULL;
|
||||
|
||||
if (tableId == 1)
|
||||
{
|
||||
List *fragmentCombination = list_make1(shardFragment);
|
||||
fragmentCombinationList = lappend(fragmentCombinationList,
|
||||
fragmentCombination);
|
||||
}
|
||||
else
|
||||
{
|
||||
List *fragmentCombination = (List *) lfirst(fragmentCombinationCell);
|
||||
fragmentCombination = lappend(fragmentCombination, shardFragment);
|
||||
/* such queries should go through router planner */
|
||||
Assert(!restrictionContext->allReferenceTables);
|
||||
|
||||
/* get next fragment for the first relation list */
|
||||
fragmentCombinationCell = lnext(fragmentCombinationCell);
|
||||
}
|
||||
}
|
||||
/*
|
||||
* Add the restriction qual parameter value in all baserestrictinfos.
|
||||
* Note that this has to be done on a copy, as the originals are needed
|
||||
* per target shard interval.
|
||||
*/
|
||||
foreach(restrictionCell, copiedRestrictionContext->relationRestrictionList)
|
||||
{
|
||||
RelationRestriction *restriction = lfirst(restrictionCell);
|
||||
Index rteIndex = restriction->index;
|
||||
List *originalBaseRestrictInfo = restriction->relOptInfo->baserestrictinfo;
|
||||
List *extendedBaseRestrictInfo = originalBaseRestrictInfo;
|
||||
|
||||
/*
|
||||
* Determine anchor table using shards which survive pruning instead of calling
|
||||
* AnchorRangeTableId
|
||||
*/
|
||||
if (anchorRangeTableId == 0 || tableSize > largestTableSize)
|
||||
{
|
||||
largestTableSize = tableSize;
|
||||
anchorRangeTableId = tableId;
|
||||
}
|
||||
shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex);
|
||||
|
||||
rangeTableIndex++;
|
||||
shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions);
|
||||
extendedBaseRestrictInfo = lappend(extendedBaseRestrictInfo,
|
||||
shardRestrictionList);
|
||||
|
||||
restriction->relOptInfo->baserestrictinfo = extendedBaseRestrictInfo;
|
||||
}
|
||||
|
||||
|
||||
/* mark that we don't want the router planner to generate dummy hosts/queries */
|
||||
replacePrunedQueryWithDummy = false;
|
||||
|
||||
/*
|
||||
* Use router select planner to decide on whether we can push down the query
|
||||
* or not. If we can, we also rely on the side-effects that all RTEs have been
|
||||
* updated to point to the relevant nodes and selectPlacementList is determined.
|
||||
*/
|
||||
routerPlannable = RouterSelectQuery(taskQuery, copiedRestrictionContext,
|
||||
&selectPlacementList, &selectAnchorShardId,
|
||||
&relationShardList, replacePrunedQueryWithDummy);
|
||||
|
||||
/* we don't expect to this this error but keeping it as a precaution for future changes */
|
||||
if (!routerPlannable)
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("cannot perform distributed planning for the given "
|
||||
"query"),
|
||||
errdetail("Select query cannot be pushed down to the worker.")));
|
||||
}
|
||||
|
||||
/* ensure that we do not send queries where select is pruned away completely */
|
||||
if (list_length(selectPlacementList) == 0)
|
||||
{
|
||||
ereport(DEBUG2, (errmsg("Skipping the target shard interval %ld because "
|
||||
"SELECT query is pruned away for the interval",
|
||||
shardId)));
|
||||
|
||||
return NULL;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -2114,47 +2156,22 @@ SubquerySqlTaskList(Job *job)
|
|||
* that the query string is generated as (...) AND (...) as opposed to
|
||||
* (...), (...).
|
||||
*/
|
||||
whereClauseTree = (Node *) make_ands_explicit((List *) subquery->jointree->quals);
|
||||
subquery->jointree->quals = whereClauseTree;
|
||||
taskQuery->jointree->quals =
|
||||
(Node *) make_ands_explicit((List *) taskQuery->jointree->quals);
|
||||
|
||||
/* create tasks from every fragment combination */
|
||||
foreach(fragmentCombinationCell, fragmentCombinationList)
|
||||
{
|
||||
List *fragmentCombination = (List *) lfirst(fragmentCombinationCell);
|
||||
List *taskRangeTableList = NIL;
|
||||
Query *taskQuery = copyObject(subquery);
|
||||
Task *sqlTask = NULL;
|
||||
StringInfo sqlQueryString = NULL;
|
||||
/* and generate the full query string */
|
||||
deparse_shard_query(taskQuery, distributedTableId, shardInterval->shardId,
|
||||
queryString);
|
||||
ereport(DEBUG4, (errmsg("distributed statement: %s", queryString->data)));
|
||||
|
||||
/* create tasks to fetch fragments required for the sql task */
|
||||
List *uniqueFragmentList = UniqueFragmentList(fragmentCombination);
|
||||
List *dataFetchTaskList = DataFetchTaskList(jobId, taskIdIndex,
|
||||
uniqueFragmentList);
|
||||
int32 dataFetchTaskCount = list_length(dataFetchTaskList);
|
||||
taskIdIndex += dataFetchTaskCount;
|
||||
subqueryTask = CreateBasicTask(jobId, taskId, SQL_TASK, queryString->data);
|
||||
subqueryTask->dependedTaskList = NULL;
|
||||
subqueryTask->anchorShardId = selectAnchorShardId;
|
||||
subqueryTask->taskPlacementList = selectPlacementList;
|
||||
subqueryTask->upsertQuery = false;
|
||||
subqueryTask->relationShardList = relationShardList;
|
||||
|
||||
ExtractRangeTableRelationWalker((Node *) taskQuery, &taskRangeTableList);
|
||||
UpdateRangeTableAlias(taskRangeTableList, fragmentCombination);
|
||||
|
||||
/* transform the updated task query to a SQL query string */
|
||||
sqlQueryString = makeStringInfo();
|
||||
pg_get_query_def(taskQuery, sqlQueryString);
|
||||
|
||||
sqlTask = CreateBasicTask(jobId, taskIdIndex, SQL_TASK, sqlQueryString->data);
|
||||
sqlTask->dependedTaskList = dataFetchTaskList;
|
||||
|
||||
/* log the query string we generated */
|
||||
ereport(DEBUG4, (errmsg("generated sql query for 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 +3776,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.
|
||||
|
|
|
@ -296,7 +296,7 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query
|
|||
{
|
||||
/* Create and optimize logical plan */
|
||||
MultiTreeRoot *logicalPlan = MultiLogicalPlanCreate(query);
|
||||
MultiLogicalPlanOptimize(logicalPlan);
|
||||
MultiLogicalPlanOptimize(logicalPlan, plannerRestrictionContext);
|
||||
|
||||
/*
|
||||
* This check is here to make it likely that all node types used in
|
||||
|
@ -308,7 +308,8 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query
|
|||
CheckNodeIsDumpable((Node *) logicalPlan);
|
||||
|
||||
/* Create the physical plan */
|
||||
distributedPlan = MultiPhysicalPlanCreate(logicalPlan);
|
||||
distributedPlan = MultiPhysicalPlanCreate(logicalPlan,
|
||||
plannerRestrictionContext);
|
||||
|
||||
/* distributed plan currently should always succeed or error out */
|
||||
Assert(distributedPlan && distributedPlan->planningError == NULL);
|
||||
|
|
|
@ -93,7 +93,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);
|
||||
|
@ -108,10 +107,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);
|
||||
|
@ -120,8 +115,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,
|
||||
|
@ -396,8 +389,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);
|
||||
|
@ -409,7 +402,8 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter
|
|||
foreach(restrictionCell, copiedRestrictionContext->relationRestrictionList)
|
||||
{
|
||||
RelationRestriction *restriction = lfirst(restrictionCell);
|
||||
List *originalBaserestrictInfo = restriction->relOptInfo->baserestrictinfo;
|
||||
List *originalBaseRestrictInfo = restriction->relOptInfo->baserestrictinfo;
|
||||
List *extendedBaseRestrictInfo = originalBaseRestrictInfo;
|
||||
Index rteIndex = restriction->index;
|
||||
|
||||
if (!allRelationsJoinedOnPartitionKey || allReferenceTables)
|
||||
|
@ -417,12 +411,12 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter
|
|||
continue;
|
||||
}
|
||||
|
||||
hashedOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex);
|
||||
shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex);
|
||||
shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions);
|
||||
extendedBaseRestrictInfo = lappend(extendedBaseRestrictInfo,
|
||||
shardRestrictionList);
|
||||
|
||||
hashedRestrictInfo = make_simple_restrictinfo((Expr *) hashedOpExpressions);
|
||||
originalBaserestrictInfo = lappend(originalBaserestrictInfo, hashedRestrictInfo);
|
||||
|
||||
restriction->relOptInfo->baserestrictinfo = originalBaserestrictInfo;
|
||||
restriction->relOptInfo->baserestrictinfo = extendedBaseRestrictInfo;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -532,7 +526,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;
|
||||
|
@ -548,7 +542,6 @@ ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex)
|
|||
DISTRIBUTE_BY_APPEND)
|
||||
{
|
||||
Assert(rteIndex > 0);
|
||||
|
||||
partitionColumn = PartitionColumn(relationId, rteIndex);
|
||||
}
|
||||
else
|
||||
|
@ -2218,7 +2211,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)
|
||||
|
@ -2858,7 +2851,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 *)
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
|
||||
#include "distributed/master_metadata_utility.h"
|
||||
#include "distributed/multi_logical_planner.h"
|
||||
#include "distributed/relation_restriction_equivalence.h"
|
||||
|
||||
|
||||
/* Definitions local to logical plan optimizer */
|
||||
|
@ -106,7 +107,8 @@ extern double CountDistinctErrorRate;
|
|||
|
||||
|
||||
/* Function declaration for optimizing logical plans */
|
||||
extern void MultiLogicalPlanOptimize(MultiTreeRoot *multiTree);
|
||||
extern void MultiLogicalPlanOptimize(MultiTreeRoot *multiTree,
|
||||
PlannerRestrictionContext *plannerRestrictionContext);
|
||||
|
||||
/* Function declaration for getting partition method for the given relation */
|
||||
extern char PartitionMethod(Oid relationId);
|
||||
|
|
|
@ -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,
|
||||
PlannerRestrictionContext *
|
||||
plannerRestrictionContext);
|
||||
extern StringInfo ShardFetchQueryString(uint64 shardId);
|
||||
extern Task * CreateBasicTask(uint64 jobId, uint32 taskId, TaskType taskType,
|
||||
char *queryString);
|
||||
|
|
|
@ -70,8 +70,6 @@ typedef struct RelationShard
|
|||
|
||||
extern PlannedStmt * multi_planner(Query *parse, int cursorOptions,
|
||||
ParamListInfo boundParams);
|
||||
|
||||
struct MultiPlan;
|
||||
extern struct MultiPlan * GetMultiPlan(CustomScan *node);
|
||||
extern void multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo,
|
||||
Index index, RangeTblEntry *rte);
|
||||
|
|
|
@ -31,11 +31,18 @@ extern MultiPlan * CreateRouterPlan(Query *originalQuery, Query *query,
|
|||
extern MultiPlan * CreateModifyPlan(Query *originalQuery, Query *query,
|
||||
PlannerRestrictionContext *
|
||||
plannerRestrictionContext);
|
||||
|
||||
extern bool RouterSelectQuery(Query *originalQuery,
|
||||
RelationRestrictionContext *restrictionContext,
|
||||
List **placementList, uint64 *anchorShardId,
|
||||
List **relationShardList, bool replacePrunedQueryWithDummy);
|
||||
extern DeferredErrorMessage * ModifyQuerySupported(Query *queryTree);
|
||||
extern Query * ReorderInsertSelectTargetLists(Query *originalQuery,
|
||||
RangeTblEntry *insertRte,
|
||||
RangeTblEntry *subqueryRte);
|
||||
extern List * ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex);
|
||||
extern RelationRestrictionContext * CopyRelationRestrictionContext(
|
||||
RelationRestrictionContext *oldContext);
|
||||
|
||||
extern bool InsertSelectQuery(Query *query);
|
||||
extern Oid ExtractFirstDistributedTableId(Query *query);
|
||||
extern RangeTblEntry * ExtractSelectRangeTableEntry(Query *query);
|
||||
|
|
|
@ -1556,6 +1556,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,
|
||||
|
|
|
@ -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 | Tue Jan 21 05:23:09.26298 2014
|
||||
22 | Tue Jan 21 05:22:28.223506 2014
|
||||
25 | Tue Jan 21 01:10:29.315788 2014
|
||||
31 | Tue Jan 21 02:43:24.591489 2014
|
||||
33 | Tue Jan 21 04:23:35.623056 2014
|
||||
34 | Tue Jan 21 04:15:03.874341 2014
|
||||
(6 rows)
|
||||
|
||||
-- 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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- not supported since JOIN is not on the partition key
|
||||
-- see (2 * user_id as user_id) target list element
|
||||
SELECT *
|
||||
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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- LATERAL JOINs used with INNER JOINs
|
||||
SELECT user_id, lastseen
|
||||
FROM
|
||||
(SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT filter_users_1.user_id,
|
||||
time AS lastseen
|
||||
FROM
|
||||
(SELECT user_where_1_1.user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1
|
||||
ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1
|
||||
JOIN LATERAL
|
||||
(SELECT user_id,
|
||||
time
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id
|
||||
ORDER BY time DESC
|
||||
LIMIT 1) "last_events_1" ON TRUE
|
||||
ORDER BY time DESC
|
||||
LIMIT 10) "some_recent_users"
|
||||
JOIN LATERAL
|
||||
(SELECT "users".user_id
|
||||
FROM users_table as "users"
|
||||
WHERE "users"."user_id" = "some_recent_users"."user_id"
|
||||
AND "users"."value_2" > 70
|
||||
LIMIT 1) "some_users_data" ON TRUE
|
||||
ORDER BY lastseen DESC
|
||||
LIMIT 10) "some_users"
|
||||
order BY user_id DESC
|
||||
limit 10;
|
||||
user_id | lastseen
|
||||
---------+---------------------------------
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
15 | Tue Jan 21 02:25:36.136461 2014
|
||||
(10 rows)
|
||||
|
||||
-- not supported since the inner JOIN is not equi join
|
||||
SELECT user_id, lastseen
|
||||
FROM
|
||||
(SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT filter_users_1.user_id,
|
||||
time AS lastseen
|
||||
FROM
|
||||
(SELECT user_where_1_1.user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1
|
||||
ON ("user_where_1_1".user_id != "user_where_1_join_1".user_id)) filter_users_1
|
||||
JOIN LATERAL
|
||||
(SELECT user_id,
|
||||
time
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id
|
||||
ORDER BY time DESC
|
||||
LIMIT 1) "last_events_1" ON TRUE
|
||||
ORDER BY time DESC
|
||||
LIMIT 10) "some_recent_users"
|
||||
JOIN LATERAL
|
||||
(SELECT "users".user_id
|
||||
FROM users_table as "users"
|
||||
WHERE "users"."user_id" = "some_recent_users"."user_id"
|
||||
AND "users"."value_2" > 70
|
||||
LIMIT 1) "some_users_data" ON TRUE
|
||||
ORDER BY lastseen DESC
|
||||
LIMIT 10) "some_users"
|
||||
order BY user_id DESC
|
||||
limit 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- not supported since the inner JOIN is not on the partition key
|
||||
SELECT user_id, lastseen
|
||||
FROM
|
||||
(SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT filter_users_1.user_id,
|
||||
time AS lastseen
|
||||
FROM
|
||||
(SELECT user_where_1_1.user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id", "users"."value_1"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1
|
||||
ON ("user_where_1_1".user_id = "user_where_1_join_1".value_1)) filter_users_1
|
||||
JOIN LATERAL
|
||||
(SELECT user_id,
|
||||
time
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id
|
||||
ORDER BY time DESC
|
||||
LIMIT 1) "last_events_1" ON TRUE
|
||||
ORDER BY time DESC
|
||||
LIMIT 10) "some_recent_users"
|
||||
JOIN LATERAL
|
||||
(SELECT "users".user_id
|
||||
FROM users_table as "users"
|
||||
WHERE "users"."user_id" = "some_recent_users"."user_id"
|
||||
AND "users"."value_2" > 70
|
||||
LIMIT 1) "some_users_data" ON TRUE
|
||||
ORDER BY lastseen DESC
|
||||
LIMIT 10) "some_users"
|
||||
order BY user_id DESC
|
||||
limit 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- not supported since upper LATERAL JOIN is not equi join
|
||||
SELECT user_id, lastseen
|
||||
FROM
|
||||
(SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT filter_users_1.user_id,
|
||||
time AS lastseen
|
||||
FROM
|
||||
(SELECT user_where_1_1.user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id", "users"."value_1"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1
|
||||
ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1
|
||||
JOIN LATERAL
|
||||
(SELECT user_id,
|
||||
time
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 12 and user_id < 16 and user_id != filter_users_1.user_id
|
||||
ORDER BY time DESC
|
||||
LIMIT 1) "last_events_1" ON TRUE
|
||||
ORDER BY time DESC
|
||||
LIMIT 10) "some_recent_users"
|
||||
JOIN LATERAL
|
||||
(SELECT "users".user_id
|
||||
FROM users_table as "users"
|
||||
WHERE "users"."user_id" = "some_recent_users"."user_id"
|
||||
AND "users"."value_2" > 70
|
||||
LIMIT 1) "some_users_data" ON TRUE
|
||||
ORDER BY lastseen DESC
|
||||
LIMIT 10) "some_users"
|
||||
order BY user_id DESC
|
||||
limit 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- not supported since lower LATERAL JOIN is not on the partition key
|
||||
SELECT user_id, lastseen
|
||||
FROM
|
||||
(SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT filter_users_1.user_id,
|
||||
time AS lastseen
|
||||
FROM
|
||||
(SELECT user_where_1_1.user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_1 > 20) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id", "users"."value_1"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 12 and user_id < 16 and value_2 > 60) user_where_1_join_1
|
||||
ON ("user_where_1_1".user_id = "user_where_1_join_1".user_id)) filter_users_1
|
||||
JOIN LATERAL
|
||||
(SELECT user_id,
|
||||
time
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 12 and user_id < 16 and user_id = filter_users_1.user_id
|
||||
ORDER BY time DESC
|
||||
LIMIT 1) "last_events_1" ON TRUE
|
||||
ORDER BY time DESC
|
||||
LIMIT 10) "some_recent_users"
|
||||
JOIN LATERAL
|
||||
(SELECT "users".user_id
|
||||
FROM users_table as "users"
|
||||
WHERE "users"."value_1" = "some_recent_users"."user_id"
|
||||
AND "users"."value_2" > 70
|
||||
LIMIT 1) "some_users_data" ON TRUE
|
||||
ORDER BY lastseen DESC
|
||||
LIMIT 10) "some_users"
|
||||
order BY user_id DESC
|
||||
limit 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- NESTED INNER JOINs
|
||||
SELECT
|
||||
count(*) AS value, "generated_group_field"
|
||||
FROM
|
||||
(SELECT DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field"
|
||||
FROM
|
||||
(SELECT "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field"
|
||||
FROM
|
||||
(SELECT *
|
||||
FROM
|
||||
(SELECT "events"."time", "events"."user_id", "events"."value_2"
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 10 and user_id < 40 and event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries"
|
||||
inner JOIN
|
||||
(SELECT
|
||||
user_where_1_1.real_user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id" as real_user_id
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1
|
||||
ON ("user_where_1_1".real_user_id = "user_where_1_join_1".user_id)) "user_filters_1"
|
||||
ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery"
|
||||
GROUP BY "generated_group_field" ORDER BY generated_group_field DESC, value DESC;
|
||||
value | generated_group_field
|
||||
-------+-----------------------
|
||||
1 | 966
|
||||
1 | 917
|
||||
1 | 905
|
||||
1 | 868
|
||||
1 | 836
|
||||
1 | 791
|
||||
1 | 671
|
||||
1 | 642
|
||||
1 | 358
|
||||
1 | 317
|
||||
1 | 307
|
||||
1 | 302
|
||||
1 | 214
|
||||
1 | 166
|
||||
1 | 116
|
||||
1 | 1
|
||||
(16 rows)
|
||||
|
||||
-- not supported since the first inner join is not on the partition key
|
||||
SELECT
|
||||
count(*) AS value, "generated_group_field"
|
||||
FROM
|
||||
(SELECT DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field"
|
||||
FROM
|
||||
(SELECT "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field"
|
||||
FROM
|
||||
(SELECT *
|
||||
FROM
|
||||
(SELECT "events"."time", "events"."user_id", "events"."value_2"
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 10 and user_id < 40 and event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries"
|
||||
inner JOIN
|
||||
(SELECT
|
||||
user_where_1_1.real_user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id" as real_user_id
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id"
|
||||
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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- not supported since the first inner join is not an equi join
|
||||
SELECT
|
||||
count(*) AS value, "generated_group_field"
|
||||
FROM
|
||||
(SELECT DISTINCT "pushedDownQuery"."real_user_id", "generated_group_field"
|
||||
FROM
|
||||
(SELECT "eventQuery"."real_user_id", "eventQuery"."time", random(), ("eventQuery"."value_2") AS "generated_group_field"
|
||||
FROM
|
||||
(SELECT *
|
||||
FROM
|
||||
(SELECT "events"."time", "events"."user_id", "events"."value_2"
|
||||
FROM events_table as "events"
|
||||
WHERE user_id > 10 and user_id < 40 and event_type IN (40, 41, 42, 43, 44, 45) ) "temp_data_queries"
|
||||
inner JOIN
|
||||
(SELECT
|
||||
user_where_1_1.real_user_id
|
||||
FROM
|
||||
(SELECT "users"."user_id" as real_user_id
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_2 > 50 ) user_where_1_1
|
||||
INNER JOIN
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_3 > 50 ) user_where_1_join_1
|
||||
ON ("user_where_1_1".real_user_id >= "user_where_1_join_1".user_id)) "user_filters_1"
|
||||
ON ("temp_data_queries".user_id = "user_filters_1".real_user_id)) "eventQuery") "pushedDownQuery") "pushedDownQuery"
|
||||
GROUP BY "generated_group_field" ORDER BY generated_group_field DESC, value DESC;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- single level inner joins
|
||||
SELECT
|
||||
"value_3", count(*) AS cnt
|
||||
FROM
|
||||
(SELECT "value_3", "user_id", random()
|
||||
FROM
|
||||
(SELECT users_in_segment_1.user_id, value_3
|
||||
FROM
|
||||
(SELECT user_id, value_3 * 2 as value_3
|
||||
FROM
|
||||
(SELECT user_id, value_3
|
||||
FROM
|
||||
(SELECT "users"."user_id", value_3
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_2 > 30) simple_user_where_1) all_buckets_1) users_in_segment_1
|
||||
JOIN
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_2 > 60) some_users_data
|
||||
ON ("users_in_segment_1".user_id = "some_users_data".user_id)) segmentalias_1) "tempQuery"
|
||||
GROUP BY "value_3" ORDER BY cnt, value_3 DESC LIMIT 10;
|
||||
value_3 | cnt
|
||||
---------+-----
|
||||
556 | 75
|
||||
228 | 75
|
||||
146 | 75
|
||||
70 | 75
|
||||
1442 | 79
|
||||
1232 | 79
|
||||
1090 | 79
|
||||
1012 | 79
|
||||
886 | 79
|
||||
674 | 79
|
||||
(10 rows)
|
||||
|
||||
-- not supported since there is no partition column equality at all
|
||||
SELECT
|
||||
"value_3", count(*) AS cnt
|
||||
FROM
|
||||
(SELECT "value_3", "user_id", random()
|
||||
FROM
|
||||
(SELECT users_in_segment_1.user_id, value_3
|
||||
FROM
|
||||
(SELECT user_id, value_3 * 2 as value_3
|
||||
FROM
|
||||
(SELECT user_id, value_3
|
||||
FROM
|
||||
(SELECT "users"."user_id", value_3
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_2 > 30) simple_user_where_1) all_buckets_1) users_in_segment_1
|
||||
JOIN
|
||||
(SELECT "users"."user_id"
|
||||
FROM users_table as "users"
|
||||
WHERE user_id > 10 and user_id < 40 and value_2 > 60) some_users_data
|
||||
ON (true)) segmentalias_1) "tempQuery"
|
||||
GROUP BY "value_3" ORDER BY cnt, value_3 DESC LIMIT 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- nested LATERAL JOINs
|
||||
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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
SET citus.subquery_pushdown TO FALSE;
|
||||
SET citus.enable_router_execution TO TRUE;
|
|
@ -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';
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
SELECT
|
||||
avg(unit_price)
|
||||
FROM
|
||||
|
@ -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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Check that we error out if there is 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 the target shard interval 270005 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270006 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270008 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270005 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270007 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270008 because SELECT query is pruned away for the interval
|
||||
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,20 +977,15 @@ 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;
|
||||
event_average | hasdone
|
||||
--------------------+---------------------
|
||||
4.0000000000000000 | Has not done paying
|
||||
2.5000000000000000 | Has done paying
|
||||
(2 rows)
|
||||
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Union, left join and having subquery pushdown
|
||||
SELECT
|
||||
avg(array_length(events, 1)) AS event_average,
|
||||
|
@ -619,14 +1001,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 +1016,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 +1032,7 @@ SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -658,13 +1041,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,
|
||||
|
@ -675,11 +1056,9 @@ GROUP BY
|
|||
count_pay
|
||||
ORDER BY
|
||||
count_pay;
|
||||
event_average | count_pay
|
||||
--------------------+-----------
|
||||
3.0000000000000000 | 0
|
||||
(1 row)
|
||||
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
|
||||
-- Lateral join subquery pushdown
|
||||
SELECT
|
||||
tenant_id,
|
||||
|
@ -696,6 +1075,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -714,8 +1094,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 +1135,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 +1179,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 +1194,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,58 +1219,15 @@ 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
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
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
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> GroupAggregate (cost=91.93..91.98 rows=2 width=48)
|
||||
Group Key: subquery_top.hasdone
|
||||
-> Sort (cost=91.93..91.93 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)
|
||||
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Union, left join and having subquery pushdown
|
||||
EXPLAIN SELECT
|
||||
avg(array_length(events, 1)) AS event_average,
|
||||
|
@ -909,14 +1243,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 +1258,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 +1274,7 @@ EXPLAIN SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -948,13 +1283,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,
|
||||
|
@ -965,7 +1298,8 @@ GROUP BY
|
|||
count_pay
|
||||
ORDER BY
|
||||
count_pay;
|
||||
ERROR: bogus varattno for OUTER_VAR var: 3
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Lateral join subquery pushdown
|
||||
EXPLAIN SELECT
|
||||
tenant_id,
|
||||
|
@ -982,6 +1316,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -1000,8 +1335,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 +1350,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';
|
||||
|
|
|
@ -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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
SELECT
|
||||
avg(unit_price)
|
||||
FROM
|
||||
|
@ -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 pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Check that we error out if there is 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 the target shard interval 270005 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270006 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270008 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270005 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270007 because SELECT query is pruned away for the interval
|
||||
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 the target shard interval 270008 because SELECT query is pruned away for the interval
|
||||
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,20 +977,15 @@ 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;
|
||||
event_average | hasdone
|
||||
--------------------+---------------------
|
||||
4.0000000000000000 | Has not done paying
|
||||
2.5000000000000000 | Has done paying
|
||||
(2 rows)
|
||||
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Union, left join and having subquery pushdown
|
||||
SELECT
|
||||
avg(array_length(events, 1)) AS event_average,
|
||||
|
@ -619,14 +1001,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 +1016,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 +1032,7 @@ SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -658,13 +1041,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,
|
||||
|
@ -675,11 +1056,9 @@ GROUP BY
|
|||
count_pay
|
||||
ORDER BY
|
||||
count_pay;
|
||||
event_average | count_pay
|
||||
--------------------+-----------
|
||||
3.0000000000000000 | 0
|
||||
(1 row)
|
||||
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
|
||||
-- Lateral join subquery pushdown
|
||||
SELECT
|
||||
tenant_id,
|
||||
|
@ -696,6 +1075,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -714,8 +1094,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 +1135,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 +1174,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 +1189,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,55 +1214,15 @@ 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
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
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
|
||||
Node: host=localhost port=57637 dbname=regression
|
||||
-> HashAggregate (cost=91.94..91.96 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)
|
||||
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Union, left join and having subquery pushdown
|
||||
EXPLAIN SELECT
|
||||
avg(array_length(events, 1)) AS event_average,
|
||||
|
@ -906,14 +1238,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 +1253,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 +1269,7 @@ EXPLAIN SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -945,13 +1278,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,
|
||||
|
@ -962,7 +1293,8 @@ GROUP BY
|
|||
count_pay
|
||||
ORDER BY
|
||||
count_pay;
|
||||
ERROR: bogus varattno for OUTER_VAR var: 3
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- Lateral join subquery pushdown
|
||||
EXPLAIN SELECT
|
||||
tenant_id,
|
||||
|
@ -979,6 +1311,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -997,8 +1330,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 +1345,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';
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
Loading…
Reference in New Issue