mirror of https://github.com/citusdata/citus.git
Enabling physical planner for subquery pushdown changes
This commit applies the logic that exists in INSERT .. SELECT planning to the subquery pushdown changes. The main algorithm is followed as : - pick an anchor relation (i.e., target relation) - per each target shard interval - add the target shard interval's shard range as a restriction to the relations (if all relations joined on the partition keys) - Check whether the query is router plannable per target shard interval. - If router plannable, create a taskpull/1333/head
parent
eef4ed31cb
commit
f5d7ab60ce
|
@ -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.
|
||||
|
|
|
@ -120,8 +120,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);
|
||||
|
@ -147,7 +152,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);
|
||||
|
@ -195,7 +199,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;
|
||||
|
@ -206,7 +211,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);
|
||||
|
@ -1911,7 +1916,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;
|
||||
|
@ -1949,7 +1954,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
|
||||
{
|
||||
|
@ -2001,133 +2006,169 @@ 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;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -2136,46 +2177,22 @@ SubquerySqlTaskList(Job *job)
|
|||
* that the query string is generated as (...) AND (...) as opposed to
|
||||
* (...), (...).
|
||||
*/
|
||||
whereClauseTree = (Node *) make_ands_explicit((List *) subquery->jointree->quals);
|
||||
subquery->jointree->quals = whereClauseTree;
|
||||
taskQuery->jointree->quals =
|
||||
(Node *) make_ands_explicit((List *) taskQuery->jointree->quals);
|
||||
|
||||
/* create tasks from every fragment combination */
|
||||
foreach(fragmentCombinationCell, fragmentCombinationList)
|
||||
{
|
||||
List *fragmentCombination = (List *) lfirst(fragmentCombinationCell);
|
||||
List *taskRangeTableList = NIL;
|
||||
Query *taskQuery = copyObject(subquery);
|
||||
Task *sqlTask = NULL;
|
||||
StringInfo sqlQueryString = NULL;
|
||||
/* and generate the full query string */
|
||||
deparse_shard_query(taskQuery, distributedTableId, shardInterval->shardId,
|
||||
queryString);
|
||||
ereport(DEBUG4, (errmsg("distributed statement: %s", queryString->data)));
|
||||
|
||||
/* create tasks to fetch fragments required for the sql task */
|
||||
List *uniqueFragmentList = UniqueFragmentList(fragmentCombination);
|
||||
List *dataFetchTaskList = DataFetchTaskList(jobId, taskIdIndex,
|
||||
uniqueFragmentList);
|
||||
int32 dataFetchTaskCount = list_length(dataFetchTaskList);
|
||||
taskIdIndex += dataFetchTaskCount;
|
||||
subqueryTask = CreateBasicTask(jobId, taskId, SQL_TASK, queryString->data);
|
||||
subqueryTask->dependedTaskList = NULL;
|
||||
subqueryTask->anchorShardId = selectAnchorShardId;
|
||||
subqueryTask->taskPlacementList = selectPlacementList;
|
||||
subqueryTask->upsertQuery = false;
|
||||
subqueryTask->relationShardList = relationShardList;
|
||||
|
||||
ExtractRangeTableRelationWalker((Node *) taskQuery, &taskRangeTableList);
|
||||
UpdateRangeTableAlias(taskRangeTableList, fragmentCombination);
|
||||
|
||||
/* transform the updated task query to a SQL query string */
|
||||
sqlQueryString = makeStringInfo();
|
||||
pg_get_query_def(taskQuery, sqlQueryString);
|
||||
|
||||
sqlTask = CreateBasicTask(jobId, taskIdIndex, SQL_TASK, sqlQueryString->data);
|
||||
sqlTask->dependedTaskList = dataFetchTaskList;
|
||||
|
||||
/* log the query string we generated */
|
||||
ereport(DEBUG4, (errmsg("generated sql query for task %d", sqlTask->taskId),
|
||||
errdetail("query string: \"%s\"", sqlQueryString->data)));
|
||||
|
||||
sqlTask->anchorShardId = AnchorShardId(fragmentCombination, anchorRangeTableId);
|
||||
|
||||
taskIdIndex++;
|
||||
sqlTaskList = lappend(sqlTaskList, sqlTask);
|
||||
}
|
||||
|
||||
return sqlTaskList;
|
||||
return subqueryTask;
|
||||
}
|
||||
|
||||
|
||||
|
@ -3779,54 +3796,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.
|
||||
|
|
|
@ -312,7 +312,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
|
||||
|
@ -324,7 +324,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);
|
||||
|
@ -109,10 +108,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);
|
||||
|
@ -121,8 +116,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,
|
||||
|
@ -409,8 +402,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);
|
||||
|
@ -422,7 +415,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)
|
||||
|
@ -430,12 +424,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;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -545,7 +539,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;
|
||||
|
@ -561,7 +555,6 @@ ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex)
|
|||
DISTRIBUTE_BY_APPEND)
|
||||
{
|
||||
Assert(rteIndex > 0);
|
||||
|
||||
partitionColumn = PartitionColumn(relationId, rteIndex);
|
||||
}
|
||||
else
|
||||
|
@ -2236,7 +2229,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)
|
||||
|
@ -2876,7 +2869,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"
|
||||
|
@ -247,7 +248,9 @@ extern bool EnableUniqueJobIds;
|
|||
|
||||
|
||||
/* Function declarations for building physical plans and constructing queries */
|
||||
extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree);
|
||||
extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree,
|
||||
PlannerRestrictionContext *
|
||||
plannerRestrictionContext);
|
||||
extern StringInfo ShardFetchQueryString(uint64 shardId);
|
||||
extern Task * CreateBasicTask(uint64 jobId, uint32 taskId, TaskType taskType,
|
||||
char *queryString);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -1555,6 +1555,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;
|
|
@ -179,21 +179,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.
|
||||
|
||||
|
@ -271,23 +274,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
|
||||
(
|
||||
|
@ -388,6 +391,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
|
||||
|
@ -396,6 +547,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
|
||||
|
@ -404,6 +702,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
|
||||
|
||||
|
||||
|
@ -502,8 +947,7 @@ FROM
|
|||
users,
|
||||
events
|
||||
WHERE
|
||||
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
|
||||
(users.composite_id).user_id = (events.composite_id).user_id AND
|
||||
(users.composite_id) = (events.composite_id) AND
|
||||
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
|
||||
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
|
||||
event_type IN ('click', 'submit', 'pay')) AS subquery
|
||||
|
@ -526,14 +970,14 @@ FROM
|
|||
(SELECT
|
||||
(users.composite_id).tenant_id,
|
||||
(users.composite_id).user_id,
|
||||
(users.composite_id) as composite_id,
|
||||
'action=>1'AS event,
|
||||
events.event_time
|
||||
FROM
|
||||
users,
|
||||
events
|
||||
WHERE
|
||||
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
|
||||
(users.composite_id).user_id = (events.composite_id).user_id AND
|
||||
(users.composite_id) = (events.composite_id) AND
|
||||
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
|
||||
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
|
||||
event_type = 'click')
|
||||
|
@ -541,14 +985,14 @@ FROM
|
|||
(SELECT
|
||||
(users.composite_id).tenant_id,
|
||||
(users.composite_id).user_id,
|
||||
(users.composite_id) as composite_id,
|
||||
'action=>2'AS event,
|
||||
events.event_time
|
||||
FROM
|
||||
users,
|
||||
events
|
||||
WHERE
|
||||
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
|
||||
(users.composite_id).user_id = (events.composite_id).user_id AND
|
||||
(users.composite_id) = (events.composite_id) AND
|
||||
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
|
||||
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
|
||||
event_type = 'submit')
|
||||
|
@ -566,8 +1010,7 @@ FROM
|
|||
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
|
||||
event_type = 'pay') AS subquery_2
|
||||
ON
|
||||
subquery_1.tenant_id = subquery_2.tenant_id AND
|
||||
subquery_1.user_id = subquery_2.user_id
|
||||
subquery_1.composite_id = subquery_2.composite_id
|
||||
GROUP BY
|
||||
subquery_1.tenant_id,
|
||||
subquery_1.user_id,
|
||||
|
@ -590,14 +1033,14 @@ SELECT
|
|||
(SELECT
|
||||
(users.composite_id).tenant_id,
|
||||
(users.composite_id).user_id,
|
||||
(users.composite_id),
|
||||
'action=>1'AS event,
|
||||
events.event_time
|
||||
FROM
|
||||
users,
|
||||
events
|
||||
WHERE
|
||||
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
|
||||
(users.composite_id).user_id = (events.composite_id).user_id AND
|
||||
(users.composite_id) = (events.composite_id) AND
|
||||
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
|
||||
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
|
||||
event_type = 'click')
|
||||
|
@ -605,14 +1048,14 @@ SELECT
|
|||
(SELECT
|
||||
(users.composite_id).tenant_id,
|
||||
(users.composite_id).user_id,
|
||||
(users.composite_id),
|
||||
'action=>2'AS event,
|
||||
events.event_time
|
||||
FROM
|
||||
users,
|
||||
events
|
||||
WHERE
|
||||
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
|
||||
(users.composite_id).user_id = (events.composite_id).user_id AND
|
||||
(users.composite_id) = (events.composite_id) AND
|
||||
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
|
||||
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
|
||||
event_type = 'submit')
|
||||
|
@ -621,6 +1064,7 @@ SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -629,13 +1073,11 @@ SELECT
|
|||
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
|
||||
event_type = 'pay'
|
||||
GROUP BY
|
||||
tenant_id,
|
||||
user_id
|
||||
composite_id
|
||||
HAVING
|
||||
COUNT(*) > 2) AS subquery_2
|
||||
ON
|
||||
subquery_1.tenant_id = subquery_2.tenant_id AND
|
||||
subquery_1.user_id = subquery_2.user_id
|
||||
subquery_1.composite_id = subquery_2.composite_id
|
||||
GROUP BY
|
||||
subquery_1.tenant_id,
|
||||
subquery_1.user_id,
|
||||
|
@ -646,7 +1088,7 @@ GROUP BY
|
|||
count_pay
|
||||
ORDER BY
|
||||
count_pay;
|
||||
|
||||
|
||||
-- Lateral join subquery pushdown
|
||||
SELECT
|
||||
tenant_id,
|
||||
|
@ -663,6 +1105,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -681,8 +1124,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
|
||||
|
|
|
@ -154,8 +154,8 @@ FROM
|
|||
orders_subquery
|
||||
GROUP BY
|
||||
l_orderkey) AS unit_prices;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Relations need to be joining on partition columns
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
SELECT
|
||||
avg(unit_price)
|
||||
FROM
|
||||
|
@ -169,8 +169,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
|
||||
(
|
||||
|
@ -178,25 +178,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)
|
||||
|
@ -288,32 +284,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 (
|
||||
|
@ -378,7 +364,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)
|
||||
|
@ -387,8 +388,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)
|
||||
|
@ -433,18 +449,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,
|
||||
|
@ -521,8 +909,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
|
||||
|
@ -549,14 +936,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')
|
||||
|
@ -564,14 +951,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')
|
||||
|
@ -589,20 +976,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,
|
||||
|
@ -618,14 +1000,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')
|
||||
|
@ -633,14 +1015,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')
|
||||
|
@ -649,6 +1031,7 @@ SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -657,13 +1040,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,
|
||||
|
@ -674,11 +1055,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,
|
||||
|
@ -695,6 +1074,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -713,8 +1093,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
|
||||
|
@ -755,36 +1134,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
|
||||
|
@ -801,14 +1178,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')
|
||||
|
@ -816,14 +1193,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')
|
||||
|
@ -841,58 +1218,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,
|
||||
|
@ -908,14 +1242,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')
|
||||
|
@ -923,14 +1257,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')
|
||||
|
@ -939,6 +1273,7 @@ EXPLAIN SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -947,13 +1282,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,
|
||||
|
@ -964,7 +1297,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,
|
||||
|
@ -981,6 +1315,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -999,8 +1334,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
|
||||
|
@ -1015,34 +1349,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';
|
||||
|
|
|
@ -154,8 +154,8 @@ FROM
|
|||
orders_subquery
|
||||
GROUP BY
|
||||
l_orderkey) AS unit_prices;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Relations need to be joining on partition columns
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
SELECT
|
||||
avg(unit_price)
|
||||
FROM
|
||||
|
@ -169,8 +169,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
|
||||
(
|
||||
|
@ -178,25 +178,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)
|
||||
|
@ -288,32 +284,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 (
|
||||
|
@ -378,7 +364,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)
|
||||
|
@ -387,8 +388,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)
|
||||
|
@ -433,18 +449,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,
|
||||
|
@ -521,8 +909,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
|
||||
|
@ -549,14 +936,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')
|
||||
|
@ -564,14 +951,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')
|
||||
|
@ -589,20 +976,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,
|
||||
|
@ -618,14 +1000,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')
|
||||
|
@ -633,14 +1015,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')
|
||||
|
@ -649,6 +1031,7 @@ SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -657,13 +1040,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,
|
||||
|
@ -674,11 +1055,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,
|
||||
|
@ -695,6 +1074,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -713,8 +1093,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
|
||||
|
@ -755,36 +1134,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
|
||||
|
@ -801,14 +1173,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')
|
||||
|
@ -816,14 +1188,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')
|
||||
|
@ -841,55 +1213,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,
|
||||
|
@ -905,14 +1237,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')
|
||||
|
@ -920,14 +1252,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')
|
||||
|
@ -936,6 +1268,7 @@ EXPLAIN SELECT
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
events
|
||||
|
@ -944,13 +1277,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,
|
||||
|
@ -961,7 +1292,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,
|
||||
|
@ -978,6 +1310,7 @@ FROM
|
|||
(SELECT
|
||||
(composite_id).tenant_id,
|
||||
(composite_id).user_id,
|
||||
composite_id,
|
||||
lastseen
|
||||
FROM
|
||||
users
|
||||
|
@ -996,8 +1329,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
|
||||
|
@ -1012,34 +1344,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';
|
||||
|
|
|
@ -733,6 +733,47 @@ WHERE
|
|||
GROUP BY
|
||||
outer_most.id;
|
||||
|
||||
-- if the given filter was on value_1 as shown in the above, Citus could
|
||||
-- push it down. But here the query is refused
|
||||
INSERT INTO agg_events
|
||||
(user_id)
|
||||
SELECT raw_events_first.user_id
|
||||
FROM raw_events_first,
|
||||
raw_events_second
|
||||
WHERE raw_events_second.user_id = raw_events_first.value_1
|
||||
AND raw_events_first.value_2 = 12;
|
||||
|
||||
-- lets do some unsupported query tests with subqueries
|
||||
-- foo is not joined on the partition key so the query is not
|
||||
-- pushed down
|
||||
INSERT INTO agg_events
|
||||
(user_id, value_4_agg)
|
||||
SELECT
|
||||
outer_most.id, max(outer_most.value)
|
||||
FROM
|
||||
(
|
||||
SELECT f2.id as id, f2.v4 as value FROM
|
||||
(SELECT
|
||||
id
|
||||
FROM (SELECT reference_table.user_id AS id
|
||||
FROM raw_events_first LEFT JOIN
|
||||
reference_table
|
||||
ON (raw_events_first.value_1 = reference_table.user_id)) AS foo) as f
|
||||
INNER JOIN
|
||||
(SELECT v4,
|
||||
v1,
|
||||
id
|
||||
FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
||||
SUM(raw_events_first.value_1) AS v1,
|
||||
raw_events_second.user_id AS id
|
||||
FROM raw_events_first,
|
||||
raw_events_second
|
||||
WHERE raw_events_first.user_id = raw_events_second.user_id
|
||||
GROUP BY raw_events_second.user_id
|
||||
HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2
|
||||
ON (f.id = f2.id)) as outer_most
|
||||
GROUP BY
|
||||
outer_most.id;
|
||||
|
||||
INSERT INTO agg_events
|
||||
(value_4_agg,
|
||||
|
|
|
@ -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