Subquery pushdown - main branch (#1323)

* 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 task

* Add union support within the JOINS

This commit adds support for UNION/UNION ALL subqueries that are
in the following form:

     .... (Q1 UNION Q2 UNION ...) as union_query JOIN (QN) ...

In other words, we currently do NOT support the queries that are
in the following form where union query is not JOINed with
other relations/subqueries :

     .... (Q1 UNION Q2 UNION ...) as union_query ....

* Subquery pushdown planner uses original query

With this commit, we change the input to the logical planner for
subquery pushdown. Before this commit, the planner was relying
on the query tree that is transformed by the postgresql planner.
After this commit, the planner uses the original query. The main
motivation behind this change is the simplify deparsing of
subqueries.

* Enable top level subquery join queries

This work enables
- Top level subquery joins
- Joins between subqueries and relations
- Joins involving more than 2 range table entries

A new regression test file is added to reflect enabled test cases

* Add top level union support

This commit adds support for UNION/UNION ALL subqueries that are
in the following form:

     .... (Q1 UNION Q2 UNION ...) as union_query ....

In other words, Citus supports allow top level
unions being wrapped into aggregations queries
and/or simple projection queries that only selects
some fields from the lower level queries.

* Disallow subqueries without a relation in the range table list for subquery pushdown

This commit disallows subqueries without relation in the range table
list. This commit is only applied for subquery pushdown. In other words,
we do not add this limitation for single table re-partition subqueries.

The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.

* Disallow subqueries without a relation in the range table list for INSERT .. SELECT

This commit disallows subqueries without relation in the range table
list. This commit is only applied for INSERT.. SELECT queries.

The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.

* Change behaviour of subquery pushdown flag (#1315)

This commit changes the behaviour of the citus.subquery_pushdown flag.
Before this commit, the flag is used to enable subquery pushdown logic. But,
with this commit, that behaviour is enabled by default. In other words, the
flag is now useless. We prefer to keep the flag since we don't want to break
the backward compatibility. Also, we may consider using that flag for other
purposes in the next commits.

* Require subquery_pushdown when limit is used in subquery

Using limit in subqueries may cause returning incorrect
results. Therefore we allow limits in subqueries only
if user explicitly set subquery_pushdown flag.

* Evaluate expressions on the LIMIT clause (#1333)

Subquery pushdown uses orignal query, the LIMIT and OFFSET clauses
are not evaluated. However, logical optimizer expects these expressions
are already evaluated by the standard planner. This commit manually
evaluates the functions on the logical planner for subquery pushdown.

* Better format subquery regression tests (#1340)

* Style fix for subquery pushdown regression tests

With this commit we intented a more consistent style for the
regression tests we've added in the
  - multi_subquery_union.sql
  - multi_subquery_complex_queries.sql
  - multi_subquery_behavioral_analytics.sql

* Enable the tests that are temporarily commented

This commit enables some of the regression tests that were commented
out until all the development is done.

* Fix merge conflicts (#1347)

 - Update regression tests to meet the changes in the regression
   test output.
 - Replace Ifs with Asserts given that the check is already done
 - Update shard pruning outputs

* Add view regression tests for increased subquery coverage (#1348)

- joins between views and tables
- joins between views
- union/union all queries involving views
- views with limit
- explain queries with view

* Improve btree operators for the subquery tests

This commit adds the missing comprasion for subquery composite key
btree comparator.
pull/1363/head^2
Önder Kalacı 2017-04-29 04:09:48 +03:00 committed by GitHub
parent 0cc9171984
commit ad5cd326a4
38 changed files with 14872 additions and 1923 deletions

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -121,8 +121,18 @@ static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray,
/* Local functions forward declarations for task list creation and helper functions */
static bool MultiPlanRouterExecutable(MultiPlan *multiPlan);
static Job * BuildJobTreeTaskList(Job *jobTree);
static List * SubquerySqlTaskList(Job *job);
static Job * BuildJobTreeTaskList(Job *jobTree,
PlannerRestrictionContext *plannerRestrictionContext);
static List * SubquerySqlTaskList(Job *job,
PlannerRestrictionContext *plannerRestrictionContext);
static void ErrorIfUnsupportedShardDistribution(Query *query);
static bool CoPartitionedTables(Oid firstRelationId, Oid secondRelationId);
static bool ShardIntervalsEqual(FmgrInfo *comparisonFunction,
ShardInterval *firstInterval,
ShardInterval *secondInterval);
static Task * SubqueryTaskCreate(Query *originalQuery, ShardInterval *shardInterval,
RelationRestrictionContext *restrictionContext,
uint32 taskId);
static List * SqlTaskList(Job *job);
static bool DependsOnHashPartitionJob(Job *job);
static uint32 AnchorRangeTableId(List *rangeTableList);
@ -145,7 +155,6 @@ static bool JoinPrunable(RangeTableFragment *leftFragment,
RangeTableFragment *rightFragment);
static ShardInterval * FragmentInterval(RangeTableFragment *fragment);
static StringInfo FragmentIntervalString(ShardInterval *fragmentInterval);
static List * UniqueFragmentList(List *fragmentList);
static List * DataFetchTaskList(uint64 jobId, uint32 taskIdIndex, List *fragmentList);
static StringInfo NodeNameArrayString(List *workerNodeList);
static StringInfo NodePortArrayString(List *workerNodeList);
@ -193,7 +202,8 @@ static uint32 FinalTargetEntryCount(List *targetEntryList);
* executed on worker nodes, and the final query to run on the master node.
*/
MultiPlan *
MultiPhysicalPlanCreate(MultiTreeRoot *multiTree)
MultiPhysicalPlanCreate(MultiTreeRoot *multiTree,
PlannerRestrictionContext *plannerRestrictionContext)
{
MultiPlan *multiPlan = NULL;
Job *workerJob = NULL;
@ -204,7 +214,7 @@ MultiPhysicalPlanCreate(MultiTreeRoot *multiTree)
workerJob = BuildJobTree(multiTree);
/* create the tree of executable tasks for the worker job */
workerJob = BuildJobTreeTaskList(workerJob);
workerJob = BuildJobTreeTaskList(workerJob, plannerRestrictionContext);
/* build the final merge query to execute on the master */
masterDependedJobList = list_make1(workerJob);
@ -810,7 +820,7 @@ BaseRangeTableList(MultiNode *multiNode)
*/
MultiTable *multiTable = (MultiTable *) multiNode;
if (multiTable->relationId != SUBQUERY_RELATION_ID &&
multiTable->relationId != HEAP_ANALYTICS_SUBQUERY_RELATION_ID)
multiTable->relationId != SUBQUERY_PUSHDOWN_RELATION_ID)
{
RangeTblEntry *rangeTableEntry = makeNode(RangeTblEntry);
rangeTableEntry->inFromCl = true;
@ -1390,6 +1400,7 @@ BuildSubqueryJobQuery(MultiNode *multiNode)
List *sortClauseList = NIL;
List *groupClauseList = NIL;
List *whereClauseList = NIL;
Node *havingQual = NULL;
Node *limitCount = NULL;
Node *limitOffset = NULL;
FromExpr *joinTree = NULL;
@ -1429,7 +1440,7 @@ BuildSubqueryJobQuery(MultiNode *multiNode)
targetList = QueryTargetList(multiNode);
}
/* extract limit count/offset and sort clauses */
/* extract limit count/offset, sort and having clauses */
if (extendedOpNodeList != NIL)
{
MultiExtendedOp *extendedOp = (MultiExtendedOp *) linitial(extendedOpNodeList);
@ -1437,6 +1448,7 @@ BuildSubqueryJobQuery(MultiNode *multiNode)
limitCount = extendedOp->limitCount;
limitOffset = extendedOp->limitOffset;
sortClauseList = extendedOp->sortClauseList;
havingQual = extendedOp->havingQual;
}
/* build group clauses */
@ -1466,7 +1478,9 @@ BuildSubqueryJobQuery(MultiNode *multiNode)
jobQuery->groupClause = groupClauseList;
jobQuery->limitOffset = limitOffset;
jobQuery->limitCount = limitCount;
jobQuery->hasAggs = contain_agg_clause((Node *) targetList);
jobQuery->havingQual = havingQual;
jobQuery->hasAggs = contain_agg_clause((Node *) targetList) ||
contain_agg_clause((Node *) havingQual);
return jobQuery;
}
@ -1909,7 +1923,7 @@ SplitPointObject(ShardInterval **shardIntervalArray, uint32 shardIntervalCount)
* tasks to worker nodes.
*/
static Job *
BuildJobTreeTaskList(Job *jobTree)
BuildJobTreeTaskList(Job *jobTree, PlannerRestrictionContext *plannerRestrictionContext)
{
List *flattenedJobList = NIL;
uint32 flattenedJobCount = 0;
@ -1947,7 +1961,7 @@ BuildJobTreeTaskList(Job *jobTree)
/* create sql tasks for the job, and prune redundant data fetch tasks */
if (job->subqueryPushdown)
{
sqlTaskList = SubquerySqlTaskList(job);
sqlTaskList = SubquerySqlTaskList(job, plannerRestrictionContext);
}
else
{
@ -1999,131 +2013,329 @@ BuildJobTreeTaskList(Job *jobTree)
/*
* SubquerySqlTaskList creates a list of SQL tasks to execute the given subquery
* pushdown job. For this, it gets all range tables in the subquery tree, then
* walks over each range table in the list, gets shards for each range table,
* and prunes unneeded shards. Then for remaining shards, fragments are created
* and merged to create fragment combinations. For each created combination, the
* function builds a SQL task, and appends this task to a task list.
* pushdown job. For this, the it is being checked whether the query is router
* plannable per target shard interval. For those router plannable worker
* queries, we create a SQL task and append the task to the task list that is going
* to be executed.
*/
static List *
SubquerySqlTaskList(Job *job)
SubquerySqlTaskList(Job *job, PlannerRestrictionContext *plannerRestrictionContext)
{
Query *subquery = job->jobQuery;
uint64 jobId = job->jobId;
List *sqlTaskList = NIL;
List *fragmentCombinationList = NIL;
List *opExpressionList = NIL;
List *queryList = NIL;
List *rangeTableList = NIL;
ListCell *fragmentCombinationCell = NULL;
ListCell *rangeTableCell = NULL;
ListCell *queryCell = NULL;
Node *whereClauseTree = NULL;
uint32 taskIdIndex = 1; /* 0 is reserved for invalid taskId */
uint32 anchorRangeTableId = 0;
uint32 rangeTableIndex = 0;
const uint32 fragmentSize = sizeof(RangeTableFragment);
uint64 largestTableSize = 0;
Oid relationId = 0;
int shardCount = 0;
int shardOffset = 0;
DistTableCacheEntry *targetCacheEntry = NULL;
RelationRestrictionContext *relationRestrictionContext =
plannerRestrictionContext->relationRestrictionContext;
/* find filters on partition columns */
ExtractQueryWalker((Node *) subquery, &queryList);
foreach(queryCell, queryList)
{
Query *query = (Query *) lfirst(queryCell);
bool leafQuery = LeafQuery(query);
if (!leafQuery)
{
continue;
}
/* we have some filters on partition column */
opExpressionList = PartitionColumnOpExpressionList(query);
if (opExpressionList != NIL)
{
break;
}
}
/* error if shards are not co-partitioned */
ErrorIfUnsupportedShardDistribution(subquery);
/* get list of all range tables in subquery tree */
ExtractRangeTableRelationWalker((Node *) subquery, &rangeTableList);
/*
* For each range table entry, first we prune shards for the relation
* referenced in the range table. Then we sort remaining shards and create
* fragments in this order and add these fragments to fragment combination
* list.
* Find the first relation that is not a reference table. We'll use the shards
* of that relation as the target shards.
*/
foreach(rangeTableCell, rangeTableList)
{
RangeTblEntry *rangeTableEntry = (RangeTblEntry *) lfirst(rangeTableCell);
Oid relationId = rangeTableEntry->relid;
List *finalShardIntervalList = NIL;
ListCell *fragmentCombinationCell = NULL;
ListCell *shardIntervalCell = NULL;
uint32 tableId = rangeTableIndex + 1; /* tableId starts from 1 */
uint32 finalShardCount = 0;
uint64 tableSize = 0;
DistTableCacheEntry *cacheEntry = NULL;
if (opExpressionList != NIL)
relationId = rangeTableEntry->relid;
cacheEntry = DistributedTableCacheEntry(relationId);
if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE)
{
Var *partitionColumn = PartitionColumn(relationId, tableId);
List *whereClauseList = ReplaceColumnsInOpExpressionList(opExpressionList,
partitionColumn);
finalShardIntervalList = PruneShards(relationId, tableId, whereClauseList);
continue;
}
targetCacheEntry = DistributedTableCacheEntry(relationId);
break;
}
Assert(targetCacheEntry != NULL);
shardCount = targetCacheEntry->shardIntervalArrayLength;
for (shardOffset = 0; shardOffset < shardCount; shardOffset++)
{
ShardInterval *targetShardInterval =
targetCacheEntry->sortedShardIntervalArray[shardOffset];
Task *subqueryTask = NULL;
subqueryTask = SubqueryTaskCreate(subquery, targetShardInterval,
relationRestrictionContext, taskIdIndex);
/* add the task if it could be created */
if (subqueryTask != NULL)
{
subqueryTask->jobId = jobId;
sqlTaskList = lappend(sqlTaskList, subqueryTask);
++taskIdIndex;
}
}
return sqlTaskList;
}
/*
* ErrorIfUnsupportedShardDistribution gets list of relations in the given query
* and checks if two conditions below hold for them, otherwise it errors out.
* a. Every relation is distributed by range or hash. This means shards are
* disjoint based on the partition column.
* b. All relations have 1-to-1 shard partitioning between them. This means
* shard count for every relation is same and for every shard in a relation
* there is exactly one shard in other relations with same min/max values.
*/
static void
ErrorIfUnsupportedShardDistribution(Query *query)
{
Oid firstTableRelationId = InvalidOid;
List *relationIdList = RelationIdList(query);
ListCell *relationIdCell = NULL;
uint32 relationIndex = 0;
uint32 rangeDistributedRelationCount = 0;
uint32 hashDistributedRelationCount = 0;
foreach(relationIdCell, relationIdList)
{
Oid relationId = lfirst_oid(relationIdCell);
char partitionMethod = PartitionMethod(relationId);
if (partitionMethod == DISTRIBUTE_BY_RANGE)
{
rangeDistributedRelationCount++;
}
else if (partitionMethod == DISTRIBUTE_BY_HASH)
{
hashDistributedRelationCount++;
}
else
{
finalShardIntervalList = LoadShardIntervalList(relationId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot push down this subquery"),
errdetail("Currently range and hash partitioned "
"relations are supported")));
}
}
/* if all shards are pruned away, we return an empty task list */
finalShardCount = list_length(finalShardIntervalList);
if (finalShardCount == 0)
if ((rangeDistributedRelationCount > 0) && (hashDistributedRelationCount > 0))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot push down this subquery"),
errdetail("A query including both range and hash "
"partitioned relations are unsupported")));
}
foreach(relationIdCell, relationIdList)
{
Oid relationId = lfirst_oid(relationIdCell);
bool coPartitionedTables = false;
Oid currentRelationId = relationId;
/* get shard list of first relation and continue for the next relation */
if (relationIndex == 0)
{
return NIL;
firstTableRelationId = relationId;
relationIndex++;
continue;
}
fragmentCombinationCell = list_head(fragmentCombinationList);
foreach(shardIntervalCell, finalShardIntervalList)
/* check if this table has 1-1 shard partitioning with first table */
coPartitionedTables = CoPartitionedTables(firstTableRelationId,
currentRelationId);
if (!coPartitionedTables)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
RangeTableFragment *shardFragment = palloc0(fragmentSize);
shardFragment->fragmentReference = shardInterval;
shardFragment->fragmentType = CITUS_RTE_RELATION;
shardFragment->rangeTableId = tableId;
tableSize += ShardLength(shardInterval->shardId);
if (tableId == 1)
{
List *fragmentCombination = list_make1(shardFragment);
fragmentCombinationList = lappend(fragmentCombinationList,
fragmentCombination);
}
else
{
List *fragmentCombination = (List *) lfirst(fragmentCombinationCell);
fragmentCombination = lappend(fragmentCombination, shardFragment);
/* get next fragment for the first relation list */
fragmentCombinationCell = lnext(fragmentCombinationCell);
}
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot push down this subquery"),
errdetail("Shards of relations in subquery need to "
"have 1-to-1 shard partitioning")));
}
}
}
/*
* Determine anchor table using shards which survive pruning instead of calling
* AnchorRangeTableId
*/
if (anchorRangeTableId == 0 || tableSize > largestTableSize)
/*
* CoPartitionedTables checks if given two distributed tables have 1-to-1 shard
* partitioning. It uses shard interval array that are sorted on interval minimum
* values. Then it compares every shard interval in order and if any pair of
* shard intervals are not equal it returns false.
*/
static bool
CoPartitionedTables(Oid firstRelationId, Oid secondRelationId)
{
bool coPartitionedTables = true;
uint32 intervalIndex = 0;
DistTableCacheEntry *firstTableCache = DistributedTableCacheEntry(firstRelationId);
DistTableCacheEntry *secondTableCache = DistributedTableCacheEntry(secondRelationId);
ShardInterval **sortedFirstIntervalArray = firstTableCache->sortedShardIntervalArray;
ShardInterval **sortedSecondIntervalArray =
secondTableCache->sortedShardIntervalArray;
uint32 firstListShardCount = firstTableCache->shardIntervalArrayLength;
uint32 secondListShardCount = secondTableCache->shardIntervalArrayLength;
FmgrInfo *comparisonFunction = firstTableCache->shardIntervalCompareFunction;
if (firstListShardCount != secondListShardCount)
{
return false;
}
/* if there are not any shards just return true */
if (firstListShardCount == 0)
{
return true;
}
Assert(comparisonFunction != NULL);
for (intervalIndex = 0; intervalIndex < firstListShardCount; intervalIndex++)
{
ShardInterval *firstInterval = sortedFirstIntervalArray[intervalIndex];
ShardInterval *secondInterval = sortedSecondIntervalArray[intervalIndex];
bool shardIntervalsEqual = ShardIntervalsEqual(comparisonFunction,
firstInterval,
secondInterval);
if (!shardIntervalsEqual)
{
largestTableSize = tableSize;
anchorRangeTableId = tableId;
coPartitionedTables = false;
break;
}
}
rangeTableIndex++;
return coPartitionedTables;
}
/*
* ShardIntervalsEqual checks if given shard intervals have equal min/max values.
*/
static bool
ShardIntervalsEqual(FmgrInfo *comparisonFunction, ShardInterval *firstInterval,
ShardInterval *secondInterval)
{
bool shardIntervalsEqual = false;
Datum firstMin = 0;
Datum firstMax = 0;
Datum secondMin = 0;
Datum secondMax = 0;
firstMin = firstInterval->minValue;
firstMax = firstInterval->maxValue;
secondMin = secondInterval->minValue;
secondMax = secondInterval->maxValue;
if (firstInterval->minValueExists && firstInterval->maxValueExists &&
secondInterval->minValueExists && secondInterval->maxValueExists)
{
Datum minDatum = CompareCall2(comparisonFunction, firstMin, secondMin);
Datum maxDatum = CompareCall2(comparisonFunction, firstMax, secondMax);
int firstComparison = DatumGetInt32(minDatum);
int secondComparison = DatumGetInt32(maxDatum);
if (firstComparison == 0 && secondComparison == 0)
{
shardIntervalsEqual = true;
}
}
return shardIntervalsEqual;
}
/*
* SubqueryTaskCreate creates a sql task by replacing the target
* shardInterval's boundary value.. Then performs the normal
* shard pruning on the subquery via RouterSelectQuery().
*
* The function errors out if the subquery is not router select query (i.e.,
* subqueries with non equi-joins.).
*/
static Task *
SubqueryTaskCreate(Query *originalQuery, ShardInterval *shardInterval,
RelationRestrictionContext *restrictionContext,
uint32 taskId)
{
Query *taskQuery = copyObject(originalQuery);
uint64 shardId = shardInterval->shardId;
Oid distributedTableId = shardInterval->relationId;
StringInfo queryString = makeStringInfo();
ListCell *restrictionCell = NULL;
Task *subqueryTask = NULL;
List *selectPlacementList = NIL;
uint64 selectAnchorShardId = INVALID_SHARD_ID;
List *relationShardList = NIL;
uint64 jobId = INVALID_JOB_ID;
bool routerPlannable = false;
bool replacePrunedQueryWithDummy = false;
RelationRestrictionContext *copiedRestrictionContext =
CopyRelationRestrictionContext(restrictionContext);
List *shardOpExpressions = NIL;
RestrictInfo *shardRestrictionList = NULL;
/* such queries should go through router planner */
Assert(!restrictionContext->allReferenceTables);
/*
* Add the restriction qual parameter value in all baserestrictinfos.
* Note that this has to be done on a copy, as the originals are needed
* per target shard interval.
*/
foreach(restrictionCell, copiedRestrictionContext->relationRestrictionList)
{
RelationRestriction *restriction = lfirst(restrictionCell);
Index rteIndex = restriction->index;
List *originalBaseRestrictInfo = restriction->relOptInfo->baserestrictinfo;
List *extendedBaseRestrictInfo = originalBaseRestrictInfo;
shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex);
shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions);
extendedBaseRestrictInfo = lappend(extendedBaseRestrictInfo,
shardRestrictionList);
restriction->relOptInfo->baserestrictinfo = extendedBaseRestrictInfo;
}
/* mark that we don't want the router planner to generate dummy hosts/queries */
replacePrunedQueryWithDummy = false;
/*
* Use router select planner to decide on whether we can push down the query
* or not. If we can, we also rely on the side-effects that all RTEs have been
* updated to point to the relevant nodes and selectPlacementList is determined.
*/
routerPlannable = RouterSelectQuery(taskQuery, copiedRestrictionContext,
&selectPlacementList, &selectAnchorShardId,
&relationShardList, replacePrunedQueryWithDummy);
/* we don't expect to this this error but keeping it as a precaution for future changes */
if (!routerPlannable)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot perform distributed planning for the given "
"query"),
errdetail("Select query cannot be pushed down to the worker.")));
}
/* ensure that we do not send queries where select is pruned away completely */
if (list_length(selectPlacementList) == 0)
{
ereport(DEBUG2, (errmsg("Skipping the target shard interval %ld because "
"SELECT query is pruned away for the interval",
shardId)));
return NULL;
}
/*
@ -2132,46 +2344,22 @@ SubquerySqlTaskList(Job *job)
* that the query string is generated as (...) AND (...) as opposed to
* (...), (...).
*/
whereClauseTree = (Node *) make_ands_explicit((List *) subquery->jointree->quals);
subquery->jointree->quals = whereClauseTree;
taskQuery->jointree->quals =
(Node *) make_ands_explicit((List *) taskQuery->jointree->quals);
/* create tasks from every fragment combination */
foreach(fragmentCombinationCell, fragmentCombinationList)
{
List *fragmentCombination = (List *) lfirst(fragmentCombinationCell);
List *taskRangeTableList = NIL;
Query *taskQuery = copyObject(subquery);
Task *sqlTask = NULL;
StringInfo sqlQueryString = NULL;
/* and generate the full query string */
deparse_shard_query(taskQuery, distributedTableId, shardInterval->shardId,
queryString);
ereport(DEBUG4, (errmsg("distributed statement: %s", queryString->data)));
/* create tasks to fetch fragments required for the sql task */
List *uniqueFragmentList = UniqueFragmentList(fragmentCombination);
List *dataFetchTaskList = DataFetchTaskList(jobId, taskIdIndex,
uniqueFragmentList);
int32 dataFetchTaskCount = list_length(dataFetchTaskList);
taskIdIndex += dataFetchTaskCount;
subqueryTask = CreateBasicTask(jobId, taskId, SQL_TASK, queryString->data);
subqueryTask->dependedTaskList = NULL;
subqueryTask->anchorShardId = selectAnchorShardId;
subqueryTask->taskPlacementList = selectPlacementList;
subqueryTask->upsertQuery = false;
subqueryTask->relationShardList = relationShardList;
ExtractRangeTableRelationWalker((Node *) taskQuery, &taskRangeTableList);
UpdateRangeTableAlias(taskRangeTableList, fragmentCombination);
/* transform the updated task query to a SQL query string */
sqlQueryString = makeStringInfo();
pg_get_query_def(taskQuery, sqlQueryString);
sqlTask = CreateBasicTask(jobId, taskIdIndex, SQL_TASK, sqlQueryString->data);
sqlTask->dependedTaskList = dataFetchTaskList;
/* log the query string we generated */
ereport(DEBUG4, (errmsg("generated sql query for task %d", sqlTask->taskId),
errdetail("query string: \"%s\"", sqlQueryString->data)));
sqlTask->anchorShardId = AnchorShardId(fragmentCombination, anchorRangeTableId);
taskIdIndex++;
sqlTaskList = lappend(sqlTaskList, sqlTask);
}
return sqlTaskList;
return subqueryTask;
}
@ -3483,54 +3671,6 @@ FragmentIntervalString(ShardInterval *fragmentInterval)
}
/*
* UniqueFragmentList walks over the given relation fragment list, compares
* shard ids, eliminate duplicates and returns a new fragment list of unique
* shard ids. Note that this is a helper function for subquery pushdown, and it
* is used to prevent creating multiple data fetch tasks for same shards.
*/
static List *
UniqueFragmentList(List *fragmentList)
{
List *uniqueFragmentList = NIL;
ListCell *fragmentCell = NULL;
foreach(fragmentCell, fragmentList)
{
ShardInterval *shardInterval = NULL;
bool shardIdAlreadyAdded = false;
ListCell *uniqueFragmentCell = NULL;
RangeTableFragment *fragment = (RangeTableFragment *) lfirst(fragmentCell);
Assert(fragment->fragmentType == CITUS_RTE_RELATION);
Assert(CitusIsA(fragment->fragmentReference, ShardInterval));
shardInterval = (ShardInterval *) fragment->fragmentReference;
foreach(uniqueFragmentCell, uniqueFragmentList)
{
RangeTableFragment *uniqueFragment =
(RangeTableFragment *) lfirst(uniqueFragmentCell);
ShardInterval *uniqueShardInterval =
(ShardInterval *) uniqueFragment->fragmentReference;
if (shardInterval->shardId == uniqueShardInterval->shardId)
{
shardIdAlreadyAdded = true;
break;
}
}
if (!shardIdAlreadyAdded)
{
uniqueFragmentList = lappend(uniqueFragmentList, fragment);
}
}
return uniqueFragmentList;
}
/*
* DataFetchTaskList builds a data fetch task for every shard in the given shard
* list, appends these data fetch tasks into a list, and returns this list.

View File

@ -309,8 +309,8 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query
*/
if ((!distributedPlan || distributedPlan->planningError) && !hasUnresolvedParams)
{
/* Create and optimize logical plan */
MultiTreeRoot *logicalPlan = MultiLogicalPlanCreate(query);
MultiTreeRoot *logicalPlan = MultiLogicalPlanCreate(originalQuery, query,
plannerRestrictionContext);
MultiLogicalPlanOptimize(logicalPlan);
/*
@ -323,7 +323,8 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query
CheckNodeIsDumpable((Node *) logicalPlan);
/* Create the physical plan */
distributedPlan = MultiPhysicalPlanCreate(logicalPlan);
distributedPlan = MultiPhysicalPlanCreate(logicalPlan,
plannerRestrictionContext);
/* distributed plan currently should always succeed or error out */
Assert(distributedPlan && distributedPlan->planningError == NULL);
@ -796,11 +797,10 @@ CopyPlanParamList(List *originalPlanParamList)
/*
* CreateAndPushPlannerRestrictionContext creates a new planner restriction context.
* Later, it creates a relation restriction context and a join restriction
* context, and sets those contexts in the planner restriction context. Finally,
* the planner restriction context is inserted to the beginning of the
* plannerRestrictionContextList and it is returned.
* CreateAndPushPlannerRestrictionContext creates a new relation restriction context
* and a new join context, inserts it to the beginning of the
* plannerRestrictionContextList. Finally, the planner restriction context is
* inserted to the beginning of the plannerRestrictionContextList and it is returned.
*/
static PlannerRestrictionContext *
CreateAndPushPlannerRestrictionContext(void)

View File

@ -89,13 +89,14 @@ static MultiPlan * CreateSingleTaskRouterPlan(Query *originalQuery,
static MultiPlan * CreateInsertSelectRouterPlan(Query *originalQuery,
PlannerRestrictionContext *
plannerRestrictionContext);
static bool SafeToPushDownSubquery(PlannerRestrictionContext *plannerRestrictionContext,
Query *originalQuery);
static Task * RouterModifyTaskForShardInterval(Query *originalQuery,
ShardInterval *shardInterval,
RelationRestrictionContext *
restrictionContext,
uint32 taskIdIndex,
bool allRelationsJoinedOnPartitionKey);
static List * ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex);
static bool MasterIrreducibleExpression(Node *expression, bool *varArgument,
bool *badCoalesce);
static bool MasterIrreducibleExpressionWalker(Node *expression, WalkerState *state);
@ -113,10 +114,6 @@ static Expr * ExtractInsertPartitionValue(Query *query, Var *partitionColumn);
static Task * RouterSelectTask(Query *originalQuery,
RelationRestrictionContext *restrictionContext,
List **placementList);
static bool RouterSelectQuery(Query *originalQuery,
RelationRestrictionContext *restrictionContext,
List **placementList, uint64 *anchorShardId,
List **relationShardList, bool replacePrunedQueryWithDummy);
static bool RelationPrunesToMultipleShards(List *relationShardList);
static List * TargetShardIntervalsForSelect(Query *query,
RelationRestrictionContext *restrictionContext);
@ -125,8 +122,6 @@ static List * IntersectPlacementList(List *lhsPlacementList, List *rhsPlacementL
static Job * RouterQueryJob(Query *query, Task *task, List *placementList);
static bool MultiRouterPlannableQuery(Query *query,
RelationRestrictionContext *restrictionContext);
static RelationRestrictionContext * CopyRelationRestrictionContext(
RelationRestrictionContext *oldContext);
static DeferredErrorMessage * InsertSelectQuerySupported(Query *queryTree,
RangeTblEntry *insertRte,
RangeTblEntry *subqueryRte,
@ -299,7 +294,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
RelationRestrictionContext *relationRestrictionContext =
plannerRestrictionContext->relationRestrictionContext;
bool allReferenceTables = relationRestrictionContext->allReferenceTables;
bool restrictionEquivalenceForPartitionKeys = false;
bool safeToPushDownSubquery = false;
multiPlan->operation = originalQuery->commandType;
@ -315,8 +310,8 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
return multiPlan;
}
restrictionEquivalenceForPartitionKeys =
RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext);
safeToPushDownSubquery = SafeToPushDownSubquery(plannerRestrictionContext,
originalQuery);
/*
* Plan select query for each shard in the target table. Do so by replacing the
@ -336,7 +331,7 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
modifyTask = RouterModifyTaskForShardInterval(originalQuery, targetShardInterval,
relationRestrictionContext,
taskIdIndex,
restrictionEquivalenceForPartitionKeys);
safeToPushDownSubquery);
/* add the task if it could be created */
if (modifyTask != NULL)
@ -382,6 +377,36 @@ CreateInsertSelectRouterPlan(Query *originalQuery,
}
/*
* SafeToPushDownSubquery returns true if either
* (i) there exists join in the query and all relations joined on their
* partition keys
* (ii) there exists only union set operations and all relations has
* partition keys in the same ordinal position in the query
*/
static bool
SafeToPushDownSubquery(PlannerRestrictionContext *plannerRestrictionContext,
Query *originalQuery)
{
RelationRestrictionContext *relationRestrictionContext =
plannerRestrictionContext->relationRestrictionContext;
bool restrictionEquivalenceForPartitionKeys =
RestrictionEquivalenceForPartitionKeys(plannerRestrictionContext);
if (restrictionEquivalenceForPartitionKeys)
{
return true;
}
if (ContainsUnionSubquery(originalQuery))
{
return SafeToPushdownUnionSubquery(relationRestrictionContext);
}
return false;
}
/*
* RouterModifyTaskForShardInterval creates a modify task by
* replacing the partitioning qual parameter added in multi_planner()
@ -397,7 +422,7 @@ static Task *
RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInterval,
RelationRestrictionContext *restrictionContext,
uint32 taskIdIndex,
bool allRelationsJoinedOnPartitionKey)
bool safeToPushdownSubquery)
{
Query *copiedQuery = copyObject(originalQuery);
RangeTblEntry *copiedInsertRte = ExtractInsertRangeTableEntry(copiedQuery);
@ -424,8 +449,8 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter
bool upsertQuery = false;
bool replacePrunedQueryWithDummy = false;
bool allReferenceTables = restrictionContext->allReferenceTables;
List *hashedOpExpressions = NIL;
RestrictInfo *hashedRestrictInfo = NULL;
List *shardOpExpressions = NIL;
RestrictInfo *shardRestrictionList = NULL;
/* grab shared metadata lock to stop concurrent placement additions */
LockShardDistributionMetadata(shardId, ShareLock);
@ -437,20 +462,21 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter
foreach(restrictionCell, copiedRestrictionContext->relationRestrictionList)
{
RelationRestriction *restriction = lfirst(restrictionCell);
List *originalBaserestrictInfo = restriction->relOptInfo->baserestrictinfo;
List *originalBaseRestrictInfo = restriction->relOptInfo->baserestrictinfo;
List *extendedBaseRestrictInfo = originalBaseRestrictInfo;
Index rteIndex = restriction->index;
if (!allRelationsJoinedOnPartitionKey || allReferenceTables)
if (!safeToPushdownSubquery || allReferenceTables)
{
continue;
}
hashedOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex);
shardOpExpressions = ShardIntervalOpExpressions(shardInterval, rteIndex);
shardRestrictionList = make_simple_restrictinfo((Expr *) shardOpExpressions);
extendedBaseRestrictInfo = lappend(extendedBaseRestrictInfo,
shardRestrictionList);
hashedRestrictInfo = make_simple_restrictinfo((Expr *) hashedOpExpressions);
originalBaserestrictInfo = lappend(originalBaserestrictInfo, hashedRestrictInfo);
restriction->relOptInfo->baserestrictinfo = originalBaserestrictInfo;
restriction->relOptInfo->baserestrictinfo = extendedBaseRestrictInfo;
}
/*
@ -562,7 +588,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter
*
* NB: If you update this, also look at PrunableExpressionsWalker().
*/
static List *
List *
ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex)
{
Oid relationId = shardInterval->relationId;
@ -578,7 +604,6 @@ ShardIntervalOpExpressions(ShardInterval *shardInterval, Index rteIndex)
DISTRIBUTE_BY_APPEND)
{
Assert(rteIndex > 0);
partitionColumn = PartitionColumn(relationId, rteIndex);
}
else
@ -876,6 +901,15 @@ MultiTaskRouterSelectQuerySupported(Query *query)
Assert(subquery->commandType == CMD_SELECT);
/* pushing down rtes without relations yields (shardCount * expectedRows) */
if (subquery->rtable == NIL)
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"Subqueries without relations are not allowed in "
"INSERT ... SELECT queries",
NULL, NULL);
}
/* pushing down limit per shard would yield wrong results */
if (subquery->limitCount != NULL)
{
@ -2262,7 +2296,7 @@ RouterSelectTask(Query *originalQuery, RelationRestrictionContext *restrictionCo
* relationShardList is filled with the list of relation-to-shard mappings for
* the query.
*/
static bool
bool
RouterSelectQuery(Query *originalQuery, RelationRestrictionContext *restrictionContext,
List **placementList, uint64 *anchorShardId, List **relationShardList,
bool replacePrunedQueryWithDummy)
@ -2910,7 +2944,7 @@ InsertSelectQuery(Query *query)
* plannerInfo which is read-only. All other parts of the relOptInfo is also shallowly
* copied.
*/
static RelationRestrictionContext *
RelationRestrictionContext *
CopyRelationRestrictionContext(RelationRestrictionContext *oldContext)
{
RelationRestrictionContext *newContext = (RelationRestrictionContext *)

View File

@ -12,6 +12,7 @@
#include "distributed/multi_planner.h"
#include "distributed/multi_logical_planner.h"
#include "distributed/multi_logical_optimizer.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/relation_restriction_equivalence.h"
#include "nodes/nodeFuncs.h"
@ -61,6 +62,11 @@ typedef struct AttributeEquivalenceClassMember
} AttributeEquivalenceClassMember;
static Var * FindTranslatedVar(List *appendRelList, Oid relationOid,
Index relationRteIndex, Index *partitionKeyIndex);
static bool EquivalenceListContainsRelationsEquality(List *attributeEquivalenceList,
RelationRestrictionContext *
restrictionContext);
static uint32 ReferenceRelationCount(RelationRestrictionContext *restrictionContext);
static List * GenerateAttributeEquivalencesForRelationRestrictions(
RelationRestrictionContext *restrictionContext);
@ -69,6 +75,29 @@ static AttributeEquivalenceClass * AttributeEquivalenceClassForEquivalenceClass(
static void AddToAttributeEquivalenceClass(AttributeEquivalenceClass **
attributeEquivalanceClass,
PlannerInfo *root, Var *varToBeAdded);
static void AddRteSubqueryToAttributeEquivalenceClass(AttributeEquivalenceClass *
*attributeEquivalanceClass,
RangeTblEntry *
rangeTableEntry,
PlannerInfo *root,
Var *varToBeAdded);
static Query * GetTargetSubquery(PlannerInfo *root, RangeTblEntry *rangeTableEntry,
Var *varToBeAdded);
static void AddUnionAllSetOperationsToAttributeEquivalenceClass(
AttributeEquivalenceClass **
attributeEquivalanceClass,
PlannerInfo *root,
Var *varToBeAdded);
static void AddUnionSetOperationsToAttributeEquivalenceClass(AttributeEquivalenceClass **
attributeEquivalenceClass,
PlannerInfo *root,
SetOperationStmt *
setOperation,
Var *varToBeAdded);
static void AddRteRelationToAttributeEquivalenceClass(AttributeEquivalenceClass **
attrEquivalenceClass,
RangeTblEntry *rangeTableEntry,
Var *varToBeAdded);
static Var * GetVarFromAssignedParam(List *parentPlannerParamList,
Param *plannerParam);
static List * GenerateAttributeEquivalencesForJoinRestrictions(JoinRestrictionContext
@ -90,6 +119,193 @@ static void ListConcatUniqueAttributeClassMemberLists(AttributeEquivalenceClass
firstClass,
AttributeEquivalenceClass *
secondClass);
static Index RelationRestrictionPartitionKeyIndex(RelationRestriction *
relationRestriction);
/*
* SafeToPushdownUnionSubquery returns true if all the relations are returns
* partition keys in the same ordinal position.
*
* Note that the function expects (and asserts) the input query to be a top
* level union query defined by TopLevelUnionQuery().
*
* Lastly, the function fails to produce correct output if the target lists contains
* multiple partition keys on the target list such as the following:
*
* select count(*) from (
* select user_id, user_id from users_table
* union
* select 2, user_id from users_table) u;
*
* For the above query, although the second item in the target list make this query
* safe to push down, the function would fail to return true.
*/
bool
SafeToPushdownUnionSubquery(RelationRestrictionContext *restrictionContext)
{
Index unionQueryPartitionKeyIndex = 0;
AttributeEquivalenceClass *attributeEquivalance =
palloc0(sizeof(AttributeEquivalenceClass));
ListCell *relationRestrictionCell = NULL;
attributeEquivalance->equivalenceId = attributeEquivalenceId++;
foreach(relationRestrictionCell, restrictionContext->relationRestrictionList)
{
RelationRestriction *relationRestriction = lfirst(relationRestrictionCell);
Index partitionKeyIndex = InvalidAttrNumber;
PlannerInfo *relationPlannerRoot = relationRestriction->plannerInfo;
List *targetList = relationPlannerRoot->parse->targetList;
List *appendRelList = relationPlannerRoot->append_rel_list;
Var *varToBeAdded = NULL;
TargetEntry *targetEntryToAdd = NULL;
/*
* We first check whether UNION ALLs are pulled up or not. Note that Postgres
* planner creates AppendRelInfos per each UNION ALL query that is pulled up.
* Then, postgres stores the related information in the append_rel_list on the
* plannerInfo struct.
*/
if (appendRelList != NULL)
{
varToBeAdded = FindTranslatedVar(appendRelList,
relationRestriction->relationId,
relationRestriction->index,
&partitionKeyIndex);
/* union does not have partition key in the target list */
if (partitionKeyIndex == 0)
{
return false;
}
}
else
{
partitionKeyIndex =
RelationRestrictionPartitionKeyIndex(relationRestriction);
/* union does not have partition key in the target list */
if (partitionKeyIndex == 0)
{
return false;
}
targetEntryToAdd = list_nth(targetList, partitionKeyIndex - 1);
if (!IsA(targetEntryToAdd->expr, Var))
{
return false;
}
varToBeAdded = (Var *) targetEntryToAdd->expr;
}
/*
* If the first relation doesn't have partition key on the target
* list of the query that the relation in, simply not allow to push down
* the query.
*/
if (partitionKeyIndex == InvalidAttrNumber)
{
return false;
}
/*
* We find the first relations partition key index in the target list. Later,
* we check whether all the relations have partition keys in the
* same position.
*/
if (unionQueryPartitionKeyIndex == InvalidAttrNumber)
{
unionQueryPartitionKeyIndex = partitionKeyIndex;
}
else if (unionQueryPartitionKeyIndex != partitionKeyIndex)
{
return false;
}
AddToAttributeEquivalenceClass(&attributeEquivalance, relationPlannerRoot,
varToBeAdded);
}
return EquivalenceListContainsRelationsEquality(list_make1(attributeEquivalance),
restrictionContext);
}
/*
* FindTranslatedVar iterates on the appendRelList and tries to find a translated
* child var identified by the relation id and the relation rte index.
*
* Note that postgres translates UNION ALL target list elements into translated_vars
* list on the corresponding AppendRelInfo struct. For details, see the related
* structs.
*
* The function returns NULL if it cannot find a translated var.
*/
static Var *
FindTranslatedVar(List *appendRelList, Oid relationOid, Index relationRteIndex,
Index *partitionKeyIndex)
{
ListCell *appendRelCell = NULL;
AppendRelInfo *targetAppendRelInfo = NULL;
ListCell *translatedVarCell = NULL;
AttrNumber childAttrNumber = 0;
Var *relationPartitionKey = NULL;
List *translaterVars = NULL;
*partitionKeyIndex = 0;
/* iterate on the queries that are part of UNION ALL subselects */
foreach(appendRelCell, appendRelList)
{
AppendRelInfo *appendRelInfo = (AppendRelInfo *) lfirst(appendRelCell);
/*
* We're only interested in the child rel that is equal to the
* relation we're investigating.
*/
if (appendRelInfo->child_relid == relationRteIndex)
{
targetAppendRelInfo = appendRelInfo;
break;
}
}
/* we couldn't find the necessary append rel info */
if (targetAppendRelInfo == NULL)
{
return NULL;
}
relationPartitionKey = PartitionKey(relationOid);
translaterVars = targetAppendRelInfo->translated_vars;
foreach(translatedVarCell, translaterVars)
{
Node *targetNode = (Node *) lfirst(translatedVarCell);
Var *targetVar = NULL;
childAttrNumber++;
if (!IsA(targetNode, Var))
{
continue;
}
targetVar = (Var *) lfirst(translatedVarCell);
if (targetVar->varno == relationRteIndex &&
targetVar->varattno == relationPartitionKey->varattno)
{
*partitionKeyIndex = childAttrNumber;
return targetVar;
}
}
return NULL;
}
/*
* RestrictionEquivalenceForPartitionKeys aims to deduce whether each of the RTE_RELATION
@ -117,14 +333,14 @@ static void ListConcatUniqueAttributeClassMemberLists(AttributeEquivalenceClass
* step, we try generate a common attribute equivalence class that holds as much as
* AttributeEquivalenceMembers whose attributes are a partition keys.
*
* AllRelationsJoinedOnPartitionKey uses both relation restrictions and join restrictions
* RestrictionEquivalenceForPartitionKeys uses both relation restrictions and join restrictions
* to find as much as information that Postgres planner provides to extensions. For the
* details of the usage, please see GenerateAttributeEquivalencesForRelationRestrictions()
* and GenerateAttributeEquivalencesForJoinRestrictions()
*/
bool
RestrictionEquivalenceForPartitionKeys(
PlannerRestrictionContext *plannerRestrictionContext)
RestrictionEquivalenceForPartitionKeys(PlannerRestrictionContext *
plannerRestrictionContext)
{
RelationRestrictionContext *restrictionContext =
plannerRestrictionContext->relationRestrictionContext;
@ -134,13 +350,9 @@ RestrictionEquivalenceForPartitionKeys(
List *relationRestrictionAttributeEquivalenceList = NIL;
List *joinRestrictionAttributeEquivalenceList = NIL;
List *allAttributeEquivalenceList = NIL;
AttributeEquivalenceClass *commonEquivalenceClass = NULL;
uint32 referenceRelationCount = ReferenceRelationCount(restrictionContext);
uint32 totalRelationCount = list_length(restrictionContext->relationRestrictionList);
uint32 nonReferenceRelationCount = totalRelationCount - referenceRelationCount;
ListCell *commonEqClassCell = NULL;
ListCell *relationRestrictionCell = NULL;
Relids commonRteIdentities = NULL;
/*
* If the query includes a single relation which is not a reference table,
@ -172,12 +384,33 @@ RestrictionEquivalenceForPartitionKeys(
list_concat(relationRestrictionAttributeEquivalenceList,
joinRestrictionAttributeEquivalenceList);
return EquivalenceListContainsRelationsEquality(allAttributeEquivalenceList,
restrictionContext);
}
/*
* EquivalenceListContainsRelationsEquality gets a list of attributed equivalence
* list and a relation restriction context. The function first generates a common
* equivalence class out of the attributeEquivalenceList. Later, the function checks
* whether all the relations exists in the common equivalence class.
*
*/
static bool
EquivalenceListContainsRelationsEquality(List *attributeEquivalenceList,
RelationRestrictionContext *restrictionContext)
{
AttributeEquivalenceClass *commonEquivalenceClass = NULL;
ListCell *commonEqClassCell = NULL;
ListCell *relationRestrictionCell = NULL;
Relids commonRteIdentities = NULL;
/*
* In general we're trying to expand existing the equivalence classes to find a
* common equivalence class. The main goal is to test whether this main class
* contains all partition keys of the existing relations.
*/
commonEquivalenceClass = GenerateCommonEquivalence(allAttributeEquivalenceList);
commonEquivalenceClass = GenerateCommonEquivalence(attributeEquivalenceList);
/* add the rte indexes of relations to a bitmap */
foreach(commonEqClassCell, commonEquivalenceClass->equivalentAttributes)
@ -632,106 +865,271 @@ GenerateAttributeEquivalencesForJoinRestrictions(JoinRestrictionContext *
* - Generate an AttributeEquivalenceMember and add to the input
* AttributeEquivalenceClass
* - If the RTE that corresponds to a subquery
* - Find the corresponding target entry via varno
* - if subquery entry is a set operation (i.e., only UNION/UNION ALL allowed)
* - recursively add both left and right sides of the set operation's
* - If the RTE that corresponds to a UNION ALL subquery
* - Iterate on each of the appendRels (i.e., each of the UNION ALL query)
* - Recursively add all children of the set operation's
* corresponding target entries
* - If the corresponding subquery entry is a UNION set operation
* - Recursively add all children of the set operation's
* corresponding target entries
* - if subquery is not a set operation
* - recursively try to add the corresponding target entry to the
* - If the corresponding subquery is a regular subquery (i.e., No set operations)
* - Recursively try to add the corresponding target entry to the
* equivalence class
*
* Note that this function only adds partition keys to the attributeEquivalanceClass.
* This implies that there wouldn't be any columns for reference tables.
*/
static void
AddToAttributeEquivalenceClass(AttributeEquivalenceClass **attributeEquivalanceClass,
PlannerInfo *root, Var *varToBeAdded)
{
RangeTblEntry *rangeTableEntry = root->simple_rte_array[varToBeAdded->varno];
RangeTblEntry *rangeTableEntry = NULL;
/* punt if it's a whole-row var rather than a plain column reference */
if (varToBeAdded->varattno == InvalidAttrNumber)
{
return;
}
/* we also don't want to process ctid, tableoid etc */
if (varToBeAdded->varattno < InvalidAttrNumber)
{
return;
}
rangeTableEntry = root->simple_rte_array[varToBeAdded->varno];
if (rangeTableEntry->rtekind == RTE_RELATION)
{
AttributeEquivalenceClassMember *attributeEqMember = NULL;
Oid relationId = rangeTableEntry->relid;
Var *relationPartitionKey = NULL;
if (PartitionMethod(relationId) == DISTRIBUTE_BY_NONE)
{
return;
}
relationPartitionKey = PartitionKey(relationId);
if (relationPartitionKey->varattno != varToBeAdded->varattno)
{
return;
}
attributeEqMember = palloc0(sizeof(AttributeEquivalenceClassMember));
attributeEqMember->varattno = varToBeAdded->varattno;
attributeEqMember->varno = varToBeAdded->varno;
attributeEqMember->rteIdentity = GetRTEIdentity(rangeTableEntry);
attributeEqMember->relationId = rangeTableEntry->relid;
(*attributeEquivalanceClass)->equivalentAttributes =
lappend((*attributeEquivalanceClass)->equivalentAttributes,
attributeEqMember);
AddRteRelationToAttributeEquivalenceClass(attributeEquivalanceClass,
rangeTableEntry,
varToBeAdded);
}
else if (rangeTableEntry->rtekind == RTE_SUBQUERY && !rangeTableEntry->inh)
else if (rangeTableEntry->rtekind == RTE_SUBQUERY)
{
Query *subquery = rangeTableEntry->subquery;
RelOptInfo *baseRelOptInfo = NULL;
TargetEntry *subqueryTargetEntry = NULL;
AddRteSubqueryToAttributeEquivalenceClass(attributeEquivalanceClass,
rangeTableEntry, root,
varToBeAdded);
}
}
/* punt if it's a whole-row var rather than a plain column reference */
if (varToBeAdded->varattno == InvalidAttrNumber)
{
return;
}
/* we also don't want to process ctid, tableoid etc */
if (varToBeAdded->varattno < InvalidAttrNumber)
{
return;
}
/*
* AddRteSubqueryToAttributeEquivalenceClass adds the given var to the given
* attribute equivalence class.
*
* The main algorithm is outlined in AddToAttributeEquivalenceClass().
*/
static void
AddRteSubqueryToAttributeEquivalenceClass(AttributeEquivalenceClass
**attributeEquivalanceClass,
RangeTblEntry *rangeTableEntry,
PlannerInfo *root, Var *varToBeAdded)
{
RelOptInfo *baseRelOptInfo = find_base_rel(root, varToBeAdded->varno);
TargetEntry *subqueryTargetEntry = NULL;
Query *targetSubquery = GetTargetSubquery(root, rangeTableEntry, varToBeAdded);
baseRelOptInfo = find_base_rel(root, varToBeAdded->varno);
subqueryTargetEntry = get_tle_by_resno(targetSubquery->targetList,
varToBeAdded->varattno);
/* If the subquery hasn't been planned yet, we have to punt */
/* if we fail to find corresponding target entry, do not proceed */
if (subqueryTargetEntry == NULL || subqueryTargetEntry->resjunk)
{
return;
}
/* we're only interested in Vars */
if (!IsA(subqueryTargetEntry->expr, Var))
{
return;
}
varToBeAdded = (Var *) subqueryTargetEntry->expr;
/*
* "inh" flag is set either when inheritance or "UNION ALL" exists in the
* subquery. Here we're only interested in the "UNION ALL" case.
*
* Else, we check one more thing: Does the subquery contain a "UNION" query.
* If so, we recursively traverse all "UNION" tree and add the corresponding
* target list elements to the attribute equivalence.
*
* Finally, if it is a regular subquery (i.e., does not contain UNION or UNION ALL),
* we simply recurse to find the corresponding RTE_RELATION to add to the
* equivalence class.
*
* Note that we're treating "UNION" and "UNION ALL" clauses differently given
* that postgres planner process/plans them separately.
*/
if (rangeTableEntry->inh)
{
AddUnionAllSetOperationsToAttributeEquivalenceClass(attributeEquivalanceClass,
root, varToBeAdded);
}
else if (targetSubquery->setOperations)
{
AddUnionSetOperationsToAttributeEquivalenceClass(attributeEquivalanceClass,
baseRelOptInfo->subroot,
(SetOperationStmt *)
targetSubquery->setOperations,
varToBeAdded);
}
else if (varToBeAdded && IsA(varToBeAdded, Var) && varToBeAdded->varlevelsup == 0)
{
AddToAttributeEquivalenceClass(attributeEquivalanceClass,
baseRelOptInfo->subroot, varToBeAdded);
}
}
/*
* GetTargetSubquery returns the corresponding subquery for the given planner root,
* range table entry and the var.
*
* The aim of this function is to simplify extracting the subquery in case of "UNION ALL"
* queries.
*/
static Query *
GetTargetSubquery(PlannerInfo *root, RangeTblEntry *rangeTableEntry, Var *varToBeAdded)
{
Query *targetSubquery = NULL;
/*
* For subqueries other than "UNION ALL", find the corresponding targetSubquery. See
* the details of how we process subqueries in the below comments.
*/
if (!rangeTableEntry->inh)
{
RelOptInfo *baseRelOptInfo = find_base_rel(root, varToBeAdded->varno);
/* If the targetSubquery hasn't been planned yet, we have to punt */
if (baseRelOptInfo->subroot == NULL)
{
return;
return NULL;
}
Assert(IsA(baseRelOptInfo->subroot, PlannerInfo));
subquery = baseRelOptInfo->subroot->parse;
Assert(IsA(subquery, Query));
/* Get the subquery output expression referenced by the upper Var */
subqueryTargetEntry = get_tle_by_resno(subquery->targetList,
varToBeAdded->varattno);
if (subqueryTargetEntry == NULL || subqueryTargetEntry->resjunk)
{
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("subquery %s does not have attribute %d",
rangeTableEntry->eref->aliasname,
varToBeAdded->varattno)));
}
if (!IsA(subqueryTargetEntry->expr, Var))
{
return;
}
varToBeAdded = (Var *) subqueryTargetEntry->expr;
if (varToBeAdded && IsA(varToBeAdded, Var) && varToBeAdded->varlevelsup == 0)
{
AddToAttributeEquivalenceClass(attributeEquivalanceClass,
baseRelOptInfo->subroot, varToBeAdded);
}
targetSubquery = baseRelOptInfo->subroot->parse;
Assert(IsA(targetSubquery, Query));
}
else
{
targetSubquery = rangeTableEntry->subquery;
}
return targetSubquery;
}
/*
* AddUnionAllSetOperationsToAttributeEquivalenceClass recursively iterates on all the
* append rels, sets the varno's accordingly and adds the
* var the given equivalence class.
*/
static void
AddUnionAllSetOperationsToAttributeEquivalenceClass(AttributeEquivalenceClass **
attributeEquivalanceClass,
PlannerInfo *root,
Var *varToBeAdded)
{
List *appendRelList = root->append_rel_list;
ListCell *appendRelCell = NULL;
/* iterate on the queries that are part of UNION ALL subqueries */
foreach(appendRelCell, appendRelList)
{
AppendRelInfo *appendRelInfo = (AppendRelInfo *) lfirst(appendRelCell);
/*
* We're only interested in UNION ALL clauses and parent_reloid is invalid
* only for UNION ALL (i.e., equals to a legitimate Oid for inheritance)
*/
if (appendRelInfo->parent_reloid != InvalidOid)
{
continue;
}
/* set the varno accordingly for this specific child */
varToBeAdded->varno = appendRelInfo->child_relid;
AddToAttributeEquivalenceClass(attributeEquivalanceClass, root,
varToBeAdded);
}
}
/*
* AddUnionSetOperationsToAttributeEquivalenceClass recursively iterates on all the
* setOperations and adds each corresponding target entry to the given equivalence
* class.
*
* Although the function silently accepts INTERSECT and EXPECT set operations, they are
* rejected later in the planning. We prefer this behavior to provide better error
* messages.
*/
static void
AddUnionSetOperationsToAttributeEquivalenceClass(AttributeEquivalenceClass **
attributeEquivalenceClass,
PlannerInfo *root,
SetOperationStmt *setOperation,
Var *varToBeAdded)
{
List *rangeTableIndexList = NIL;
ListCell *rangeTableIndexCell = NULL;
ExtractRangeTableIndexWalker((Node *) setOperation, &rangeTableIndexList);
foreach(rangeTableIndexCell, rangeTableIndexList)
{
int rangeTableIndex = lfirst_int(rangeTableIndexCell);
varToBeAdded->varno = rangeTableIndex;
AddToAttributeEquivalenceClass(attributeEquivalenceClass, root, varToBeAdded);
}
}
/*
* AddRteRelationToAttributeEquivalenceClass adds the given var to the given equivalence
* class using the rteIdentity provided by the rangeTableEntry. Note that
* rteIdentities are only assigned to RTE_RELATIONs and this function asserts
* the input rte to be an RTE_RELATION.
*
* Note that this function only adds partition keys to the attributeEquivalanceClass.
* This implies that there wouldn't be any columns for reference tables.
*/
static void
AddRteRelationToAttributeEquivalenceClass(AttributeEquivalenceClass **
attrEquivalenceClass,
RangeTblEntry *rangeTableEntry,
Var *varToBeAdded)
{
AttributeEquivalenceClassMember *attributeEqMember = NULL;
Oid relationId = InvalidOid;
Var *relationPartitionKey = NULL;
Assert(rangeTableEntry->rtekind == RTE_RELATION);
relationId = rangeTableEntry->relid;
if (PartitionMethod(relationId) == DISTRIBUTE_BY_NONE)
{
return;
}
relationPartitionKey = PartitionKey(relationId);
if (relationPartitionKey->varattno != varToBeAdded->varattno)
{
return;
}
attributeEqMember = palloc0(sizeof(AttributeEquivalenceClassMember));
attributeEqMember->varattno = varToBeAdded->varattno;
attributeEqMember->varno = varToBeAdded->varno;
attributeEqMember->rteIdentity = GetRTEIdentity(rangeTableEntry);
attributeEqMember->relationId = rangeTableEntry->relid;
(*attrEquivalenceClass)->equivalentAttributes =
lappend((*attrEquivalenceClass)->equivalentAttributes,
attributeEqMember);
}
@ -821,8 +1219,10 @@ static bool
AttributeEquivalancesAreEqual(AttributeEquivalenceClass *firstAttributeEquivalance,
AttributeEquivalenceClass *secondAttributeEquivalance)
{
List *firstEquivalenceMemberList = firstAttributeEquivalance->equivalentAttributes;
List *secondEquivalenceMemberList = secondAttributeEquivalance->equivalentAttributes;
List *firstEquivalenceMemberList =
firstAttributeEquivalance->equivalentAttributes;
List *secondEquivalenceMemberList =
secondAttributeEquivalance->equivalentAttributes;
ListCell *firstAttributeEquivalanceCell = NULL;
ListCell *secondAttributeEquivalanceCell = NULL;
@ -832,7 +1232,6 @@ AttributeEquivalancesAreEqual(AttributeEquivalenceClass *firstAttributeEquivalan
return false;
}
foreach(firstAttributeEquivalanceCell, firstEquivalenceMemberList)
{
AttributeEquivalenceClassMember *firstEqMember =
@ -862,3 +1261,134 @@ AttributeEquivalancesAreEqual(AttributeEquivalenceClass *firstAttributeEquivalan
return true;
}
/*
* ContainsUnionSubquery gets a queryTree and returns true if the query
* contains
* - a subquery with UNION set operation
* - no joins above the UNION set operation in the query tree
*
* Note that the function allows top level unions being wrapped into aggregations
* queries and/or simple projection queries that only selects some fields from
* the lower level queries.
*
* If there exists joins before the set operations, the function returns false.
* Similarly, if the query does not contain any union set operations, the
* function returns false.
*/
bool
ContainsUnionSubquery(Query *queryTree)
{
List *rangeTableList = queryTree->rtable;
Node *setOperations = queryTree->setOperations;
List *joinTreeTableIndexList = NIL;
Index subqueryRteIndex = 0;
uint32 joiningRangeTableCount = 0;
RangeTblEntry *rangeTableEntry = NULL;
Query *subqueryTree = NULL;
ExtractRangeTableIndexWalker((Node *) queryTree->jointree, &joinTreeTableIndexList);
joiningRangeTableCount = list_length(joinTreeTableIndexList);
/* don't allow joins on top of unions */
if (joiningRangeTableCount > 1)
{
return false;
}
subqueryRteIndex = linitial_int(joinTreeTableIndexList);
rangeTableEntry = rt_fetch(subqueryRteIndex, rangeTableList);
if (rangeTableEntry->rtekind != RTE_SUBQUERY)
{
return false;
}
subqueryTree = rangeTableEntry->subquery;
setOperations = subqueryTree->setOperations;
if (setOperations != NULL)
{
SetOperationStmt *setOperationStatement = (SetOperationStmt *) setOperations;
/*
* Note that the set operation tree is traversed elsewhere for ensuring
* that we only support UNIONs.
*/
if (setOperationStatement->op != SETOP_UNION)
{
return false;
}
return true;
}
return ContainsUnionSubquery(subqueryTree);
}
/*
* RelationRestrictionPartitionKeyIndex gets a relation restriction and finds the
* index that the partition key of the relation exists in the query. The query is
* found in the planner info of the relation restriction.
*/
static Index
RelationRestrictionPartitionKeyIndex(RelationRestriction *relationRestriction)
{
PlannerInfo *relationPlannerRoot = NULL;
Query *relationPlannerParseQuery = NULL;
List *relationTargetList = NIL;
ListCell *targetEntryCell = NULL;
Index partitionKeyTargetAttrIndex = 0;
relationPlannerRoot = relationRestriction->plannerInfo;
relationPlannerParseQuery = relationPlannerRoot->parse;
relationTargetList = relationPlannerParseQuery->targetList;
foreach(targetEntryCell, relationTargetList)
{
TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell);
Expr *targetExpression = targetEntry->expr;
partitionKeyTargetAttrIndex++;
if (!targetEntry->resjunk &&
IsPartitionColumn(targetExpression, relationPlannerParseQuery) &&
IsA(targetExpression, Var))
{
Var *targetColumn = (Var *) targetExpression;
if (targetColumn->varno == relationRestriction->index)
{
return partitionKeyTargetAttrIndex;
}
}
}
return InvalidAttrNumber;
}
/*
* RelationIdList returns list of unique relation ids in query tree.
*/
List *
RelationIdList(Query *query)
{
List *rangeTableList = NIL;
List *tableEntryList = NIL;
List *relationIdList = NIL;
ListCell *tableEntryCell = NULL;
ExtractRangeTableRelationWalker((Node *) query, &rangeTableList);
tableEntryList = TableEntryList(rangeTableList);
foreach(tableEntryCell, tableEntryList)
{
TableEntry *tableEntry = (TableEntry *) lfirst(tableEntryCell);
Oid relationId = tableEntry->relationId;
relationIdList = list_append_unique_oid(relationIdList, relationId);
}
return relationIdList;
}

View File

@ -31,7 +31,6 @@ typedef struct FunctionEvaluationContext
/* private function declarations */
static Node * PartiallyEvaluateExpression(Node *expression, PlanState *planState);
static Node * EvaluateNodeIfReferencesFunction(Node *expression, PlanState *planState);
static Node * PartiallyEvaluateExpressionMutator(Node *expression,
FunctionEvaluationContext *context);
@ -162,7 +161,7 @@ ExecuteMasterEvaluableFunctions(Query *query, PlanState *planState)
* Walks the expression evaluating any node which invokes a function as long as a Var
* doesn't show up in the parameter list.
*/
static Node *
Node *
PartiallyEvaluateExpression(Node *expression, PlanState *planState)
{
FunctionEvaluationContext globalContext = { planState, false };

View File

@ -17,5 +17,6 @@
extern bool RequiresMasterEvaluation(Query *query);
extern void ExecuteMasterEvaluableFunctions(Query *query, PlanState *planState);
extern Node * PartiallyEvaluateExpression(Node *expression, PlanState *planState);
#endif /* CITUS_CLAUSES_H */

View File

@ -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 */

View File

@ -16,6 +16,7 @@
#include "distributed/citus_nodes.h"
#include "distributed/multi_join_order.h"
#include "distributed/relation_restriction_equivalence.h"
#include "nodes/nodes.h"
#include "nodes/primnodes.h"
#include "nodes/parsenodes.h"
@ -24,7 +25,7 @@
#define SUBQUERY_RANGE_TABLE_ID -1
#define SUBQUERY_RELATION_ID 10000
#define HEAP_ANALYTICS_SUBQUERY_RELATION_ID 10001
#define SUBQUERY_PUSHDOWN_RELATION_ID 10001
/*
@ -180,7 +181,9 @@ extern bool SubqueryPushdown;
/* Function declarations for building logical plans */
extern MultiTreeRoot * MultiLogicalPlanCreate(Query *queryTree);
extern MultiTreeRoot * MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree,
PlannerRestrictionContext *
plannerRestrictionContext);
extern bool NeedsDistributedPlanning(Query *queryTree);
extern MultiNode * ParentNode(MultiNode *multiNode);
extern MultiNode * ChildNode(MultiUnaryNode *multiNode);

View File

@ -22,6 +22,7 @@
#include "distributed/errormessage.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/multi_logical_planner.h"
#include "distributed/multi_planner.h"
#include "lib/stringinfo.h"
#include "nodes/parsenodes.h"
#include "utils/array.h"
@ -248,7 +249,9 @@ extern bool EnableUniqueJobIds;
/* Function declarations for building physical plans and constructing queries */
extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree);
extern MultiPlan * MultiPhysicalPlanCreate(MultiTreeRoot *multiTree,
PlannerRestrictionContext *
plannerRestrictionContext);
extern StringInfo ShardFetchQueryString(uint64 shardId);
extern Task * CreateBasicTask(uint64 jobId, uint32 taskId, TaskType taskType,
char *queryString);

View File

@ -71,8 +71,6 @@ typedef struct RelationShard
extern PlannedStmt * multi_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
struct MultiPlan;
extern struct MultiPlan * GetMultiPlan(CustomScan *node);
extern void multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo,
Index index, RangeTblEntry *rte);

View File

@ -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);

View File

@ -15,8 +15,11 @@
#include "distributed/multi_planner.h"
extern bool ContainsUnionSubquery(Query *queryTree);
extern bool RestrictionEquivalenceForPartitionKeys(PlannerRestrictionContext *
plannerRestrictionContext);
extern bool SafeToPushdownUnionSubquery(RelationRestrictionContext *restrictionContext);
extern List * RelationIdList(Query *query);
#endif /* RELATION_RESTRICTION_EQUIVALENCE_H */

View File

@ -17,3 +17,4 @@
/worker_copy.out
/multi_complex_count_distinct.out
/multi_mx_copy_data.out
/multi_insert_select_behavioral_analytics_create_table.out

View File

@ -348,16 +348,14 @@ ERROR: cannot perform local joins that involve expressions
DETAIL: local joins can be performed between columns only
-- Check that we can issue limit/offset queries
-- OFFSET in subqueries are not supported
-- Error in the planner when subquery pushdown is off
-- Error in the planner when single repartition subquery
SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries with offset are not supported yet
-- Error in the optimizer when subquery pushdown is on
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries with offset are not supported yet
SET citus.subquery_pushdown TO true;
-- Error in the optimizer when subquery pushdown is on
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq;
ERROR: cannot push down this subquery
DETAIL: Offset clause is currently unsupported
SET citus.subquery_pushdown TO false;
-- Simple LIMIT/OFFSET with ORDER BY
SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20;
o_orderkey

View File

@ -963,6 +963,50 @@ DETAIL: Select query cannot be pushed down to the worker.
GROUP BY
outer_most.id;
ERROR: cannot perform distributed planning for the given modification
DETAIL: Select query cannot be pushed down to the worker.
-- if the given filter was on value_1 as shown in the above, Citus could
-- push it down. But here the query is refused
INSERT INTO agg_events
(user_id)
SELECT raw_events_first.user_id
FROM raw_events_first,
raw_events_second
WHERE raw_events_second.user_id = raw_events_first.value_1
AND raw_events_first.value_2 = 12;
ERROR: cannot perform distributed planning for the given modification
DETAIL: Select query cannot be pushed down to the worker.
-- lets do some unsupported query tests with subqueries
-- foo is not joined on the partition key so the query is not
-- pushed down
INSERT INTO agg_events
(user_id, value_4_agg)
SELECT
outer_most.id, max(outer_most.value)
FROM
(
SELECT f2.id as id, f2.v4 as value FROM
(SELECT
id
FROM (SELECT reference_table.user_id AS id
FROM raw_events_first LEFT JOIN
reference_table
ON (raw_events_first.value_1 = reference_table.user_id)) AS foo) as f
INNER JOIN
(SELECT v4,
v1,
id
FROM (SELECT SUM(raw_events_second.value_4) AS v4,
SUM(raw_events_first.value_1) AS v1,
raw_events_second.user_id AS id
FROM raw_events_first,
raw_events_second
WHERE raw_events_first.user_id = raw_events_second.user_id
GROUP BY raw_events_second.user_id
HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2
ON (f.id = f2.id)) as outer_most
GROUP BY
outer_most.id;
ERROR: cannot perform distributed planning for the given modification
DETAIL: Select query cannot be pushed down to the worker.
INSERT INTO agg_events
(value_4_agg,

View File

@ -660,3 +660,30 @@ FROM
WHERE users_table.value_1 < 50;
ERROR: cannot perform distributed planning for the given modification
DETAIL: Select query cannot be pushed down to the worker.
-- not supported since one of the queries doesn't have a relation
INSERT INTO agg_results (user_id, agg_time, value_2_agg)
SELECT
user_id,
user_lastseen,
array_length(event_array, 1)
FROM (
SELECT
user_id,
max(u.time) as user_lastseen,
array_agg(event_type ORDER BY u.time) AS event_array
FROM (
SELECT user_id, time, value_3 as val_3
FROM users_table
WHERE
user_id >= 10 AND user_id <= 70 AND
users_table.value_1 > 10 AND users_table.value_1 < 12
) u LEFT JOIN LATERAL (
SELECT event_type, time
FROM events_table, (SELECT 1 as x) as f
WHERE user_id = u.user_id AND
events_table.event_type > 10 AND events_table.event_type < 12
) t ON true
GROUP BY user_id
) AS shard_union
ORDER BY user_lastseen DESC;
ERROR: Subqueries without relations are not allowed in INSERT ... SELECT queries

View File

@ -802,8 +802,8 @@ SELECT * FROM (
(SELECT * FROM articles_hash_mx WHERE author_id = 1)
UNION
(SELECT * FROM articles_hash_mx WHERE author_id = 2)) uu;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries without group by clause are not supported yet
ERROR: cannot push down this subquery
DETAIL: Currently all leaf queries need to have same filters on partition column
-- error out for queries with repartition jobs
SELECT *
FROM articles_hash_mx a, articles_hash_mx b

View File

@ -916,8 +916,8 @@ SELECT * FROM (
(SELECT * FROM articles_hash WHERE author_id = 1)
UNION
(SELECT * FROM articles_hash WHERE author_id = 2)) uu;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries without group by clause are not supported yet
ERROR: cannot push down this subquery
DETAIL: Currently all leaf queries need to have same filters on partition column
-- error out for queries with repartition jobs
SELECT *
FROM articles_hash a, articles_hash b

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,901 @@
--
-- multi subquery toplevel union queries aims to expand existing subquery pushdown
-- regression tests to cover more cases
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1400000;
SET citus.enable_router_execution TO false;
-- a very simple union query
SELECT user_id, counter
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
ORDER BY 2 DESC,1
LIMIT 5;
user_id | counter
---------+---------
7 | 9
8 | 9
15 | 9
16 | 9
20 | 9
(5 rows)
-- the same query with union all
SELECT user_id, counter
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION ALL
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
ORDER BY 2 DESC,1
LIMIT 5;
user_id | counter
---------+---------
7 | 9
7 | 9
8 | 9
15 | 9
15 | 9
(5 rows)
-- the same query with group by
SELECT user_id, sum(counter)
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
GROUP BY 1
ORDER BY 2 DESC,1
LIMIT 5;
user_id | sum
---------+-----
49 | 22
15 | 19
26 | 17
48 | 17
61 | 17
(5 rows)
-- the same query with UNION ALL clause
SELECT user_id, sum(counter)
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION ALL
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
GROUP BY 1
ORDER BY 2 DESC,1
LIMIT 5;
user_id | sum
---------+-----
48 | 35
61 | 30
15 | 28
49 | 25
80 | 24
(5 rows)
-- the same query target list entries shuffled
SELECT user_id, sum(counter)
FROM (
SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION
SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
GROUP BY 1
ORDER BY 2 DESC,1
LIMIT 5;
user_id | sum
---------+-----
49 | 22
15 | 19
26 | 17
48 | 17
61 | 17
(5 rows)
-- same query with GROUP BY
SELECT user_id, sum(counter)
FROM (
SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (1, 2)
UNION
SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (5, 6)
) user_id
GROUP BY
user_id
--HAVING sum(counter) > 900
ORDER BY 1,2 DESC LIMIT 5;
user_id | sum
---------+------
1 | 518
2 | 637
4 | 343
6 | 354
7 | 1374
(5 rows)
-- the same query target list entries shuffled but this time the subqueries target list
-- is shuffled
SELECT user_id, sum(counter)
FROM (
SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (1, 2)
UNION
SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (5, 6)
) user_id
GROUP BY
user_id
--HAVING sum(counter) > 900
ORDER BY 1,2 DESC LIMIT 5;
user_id | sum
---------+------
1 | 518
2 | 637
4 | 343
6 | 354
7 | 1374
(5 rows)
-- similar query this time more subqueries and target list contains a resjunk entry
SELECT sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500
) user_id
GROUP BY user_id ORDER BY 1 DESC LIMIT 5;
sum
-------
27772
25720
24993
24968
23508
(5 rows)
-- similar query as above, with UNION ALL
SELECT sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 5000
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500
) user_id
GROUP BY user_id ORDER BY 1 DESC LIMIT 5;
sum
-------
27667
25080
24814
24365
23508
(5 rows)
-- unions within unions
SELECT *
FROM (
( SELECT user_id,
sum(counter)
FROM
(SELECT
user_id, sum(value_2) AS counter
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_1
GROUP BY
user_id)
UNION
(SELECT
user_id, sum(counter)
FROM
(SELECT
user_id, sum(value_2) AS counter
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_2
GROUP BY
user_id)) AS ftop
ORDER BY 2 DESC, 1 DESC
LIMIT 5;
user_id | sum
---------+--------
23 | 126017
45 | 117323
25 | 116595
17 | 116520
90 | 115843
(5 rows)
-- top level unions are wrapped into top level aggregations
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT *, random()
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) ) events_subquery_2)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q"
) as final_query
GROUP BY types
ORDER BY types;
types | sumofeventtype
-------+----------------
0 | 55
1 | 38
2 | 70
3 | 58
(4 rows)
-- exactly the same query
-- but wrapper unions are removed from the inner part of the query
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
FROM
(SELECT *, random()
FROM
(SELECT
"t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
(SELECT
"t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM(
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
GROUP BY "t1"."user_id") AS t) "q"
) as final_query
GROUP BY types
ORDER BY types;
types | sumofeventtype
-------+----------------
0 | 55
1 | 38
2 | 70
3 | 58
(4 rows)
-- again excatly the same query with top level wrapper removed
SELECT ("q"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
GROUP BY "t1"."user_id") AS t) "q"
GROUP BY types
ORDER BY types;
types | sumofeventtype
-------+----------------
0 | 55
1 | 38
2 | 70
3 | 58
(4 rows)
-- again same query but with only two top level empty queries (i.e., no group bys)
SELECT *
FROM
( SELECT *
FROM
( SELECT "t1"."user_id"
FROM (
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
) AS t) "q"
ORDER BY 1
LIMIT 5;
user_id
---------
0
0
0
1
1
(5 rows)
-- a very similar query UNION ALL
SELECT ("q"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION ALL
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION ALL
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION ALL
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
GROUP BY "t1"."user_id") AS t) "q"
GROUP BY types
ORDER BY types;
types | sumofeventtype
-------+----------------
0 | 55
1 | 38
2 | 70
3 | 58
(4 rows)
-- some UNION ALL queries that are going to be pulled up
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT user_id FROM events_table)
) b;
count
-------
20002
(1 row)
-- similar query without top level agg
SELECT
user_id
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT user_id FROM events_table)
) b
ORDER BY 1 DESC
LIMIT 5;
user_id
---------
100
100
100
100
100
(5 rows)
-- similar query with multiple target list entries
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM users_table)
UNION ALL
(SELECT value_3, user_id FROM events_table)
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
user_id | value_3
---------+---------
100 | 999
100 | 997
100 | 991
100 | 989
100 | 988
(5 rows)
-- similar query group by inside the subqueries
SELECT
user_id, value_3_sum
FROM
(
(SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id)
UNION ALL
(SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id)
) b
ORDER BY 2 DESC, 1 DESC
LIMIT 5;
user_id | value_3_sum
---------+-------------
10 | 64060
10 | 64060
62 | 62445
62 | 62445
26 | 60536
(5 rows)
-- similar query top level group by
SELECT
user_id, sum(value_3)
FROM
(
(SELECT value_3, user_id FROM users_table)
UNION ALL
(SELECT value_3, user_id FROM events_table)
) b
GROUP BY 1
ORDER BY 2 DESC, 1 DESC
LIMIT 5;
user_id | sum
---------+--------
23 | 123923
25 | 118087
69 | 115828
26 | 114705
3 | 113915
(5 rows)
-- a long set operation list
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30))
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
user_id | value_3
---------+---------
100 | 951
99 | 558
99 | 14
98 | 987
98 | 577
(5 rows)
-- no partition key on the top
SELECT
max(value_3)
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30))
) b
GROUP BY user_id
ORDER BY 1 DESC
LIMIT 5;
max
-----
997
997
996
995
995
(5 rows)
-- now lets also have some unsupported queries
-- group by is not on the partition key
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
UNION
SELECT value_1 as user_id, sum(value_2) AS counter FROM users_table GROUP BY value_1
) user_id
GROUP BY user_id;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- partition key is not selected
SELECT sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT 2 * user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500
) user_id
GROUP BY user_id ORDER BY 1 DESC LIMIT 5;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- excepts within unions are not supported
SELECT * FROM
(
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
) user_id_1
GROUP BY user_id
)
UNION
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
EXCEPT
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
) user_id_2
GROUP BY user_id)
) as ftop;
ERROR: cannot push down this subquery
DETAIL: Intersect and Except are currently unsupported
-- joins inside unions are not supported
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE users_table.user_id > events_table.user_id GROUP BY 1
) user_id
GROUP BY user_id;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- joins inside unions are not supported -- slightly more comlex than the above
SELECT * FROM
(
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
) user_id_1
GROUP BY user_id
)
UNION
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE (events_table.user_id = users_table.user_id) GROUP BY events_table.user_id
) user_id_2
GROUP BY user_id)
) as ftop;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- offset inside the union
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id OFFSET 4
) user_id
GROUP BY user_id;
ERROR: cannot push down this subquery
DETAIL: Offset clause is currently unsupported
-- lower level union does not return partition key with the other relations
SELECT *
FROM (
( SELECT user_id,
sum(counter)
FROM
(SELECT
user_id, sum(value_2) AS counter
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_1
GROUP BY
user_id)
UNION
(SELECT
user_id, sum(counter)
FROM
(SELECT
sum(value_2) AS counter, user_id
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_2
GROUP BY
user_id)) AS ftop;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- some UNION all queries that are going to be pulled up
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT 2 * user_id FROM events_table)
) b;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- last query does not have partition key
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT value_3, value_2 FROM events_table where event_type IN (26, 27, 28, 29, 30))
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- we don't allow joins within unions
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT users_table.user_id FROM events_table, users_table WHERE events_table.user_id = users_table.user_id)
) b;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- we don't support subqueries without relations
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT 1)
) b;
ERROR: cannot push down this subquery
DETAIL: Subqueries without relations are unsupported
-- we don't support subqueries without relations
SELECT
*
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT (random() * 100)::int)
) b;
ERROR: cannot push down this subquery
DETAIL: Subqueries without relations are unsupported
-- we don't support subqueries without relations
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT 1, 2)
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
ERROR: cannot push down this subquery
DETAIL: Subqueries without relations are unsupported
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT *, random()
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) ) events_subquery_2)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3)
UNION
(SELECT *
FROM
(SELECT 1, now(), 3 AS event) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q"
) as final_query
GROUP BY types
ORDER BY types;
ERROR: cannot push down this subquery
DETAIL: Subqueries without relations are unsupported
SET citus.enable_router_execution TO true;

View File

@ -230,20 +230,54 @@ SELECT l_suppkey, count(*) FROM
GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries without group by clause are not supported yet
-- repartition query on view with single table subquery
CREATE VIEW supp_count_view AS SELECT * FROM (SELECT l_suppkey, count(*) FROM lineitem_hash_part GROUP BY 1) s1;
SELECT * FROM supp_count_view ORDER BY 2 DESC, 1 LIMIT 10;
l_suppkey | count
-----------+-------
6104 | 8
1868 | 6
5532 | 6
5849 | 6
6169 | 6
6669 | 6
6692 | 6
7703 | 6
7869 | 6
8426 | 6
(10 rows)
SET citus.task_executor_type to DEFAULT;
-- create a view with aggregate
CREATE VIEW lineitems_by_shipping_method AS
SELECT l_shipmode, count(*) as cnt FROM lineitem_hash_part GROUP BY 1;
-- following will fail due to non-flattening of subquery due to GROUP BY
-- following will fail due to non GROUP BY of partition key
SELECT * FROM lineitems_by_shipping_method;
ERROR: Unrecognized range table id 1
-- create a view with group by on partition column
CREATE VIEW lineitems_by_orderkey AS
SELECT l_orderkey, count(*) FROM lineitem_hash_part GROUP BY 1;
-- this will also fail due to same reason
SELECT * FROM lineitems_by_orderkey;
ERROR: Unrecognized range table id 1
-- however it would work if it is made router plannable
SELECT
l_orderkey, count(*)
FROM
lineitem_hash_part
GROUP BY 1;
-- this should work since we're able to push down this query
SELECT * FROM lineitems_by_orderkey ORDER BY 2 DESC, 1 ASC LIMIT 10;
l_orderkey | count
------------+-------
7 | 7
68 | 7
129 | 7
164 | 7
194 | 7
225 | 7
226 | 7
322 | 7
326 | 7
354 | 7
(10 rows)
-- it would also work since it is made router plannable
SELECT * FROM lineitems_by_orderkey WHERE l_orderkey = 100;
l_orderkey | count
------------+-------
@ -251,3 +285,613 @@ SELECT * FROM lineitems_by_orderkey WHERE l_orderkey = 100;
(1 row)
DROP TABLE temp_lineitem CASCADE;
DROP VIEW supp_count_view;
DROP VIEW lineitems_by_orderkey;
DROP VIEW lineitems_by_shipping_method;
DROP VIEW air_shipped_lineitems;
DROP VIEW priority_lineitem;
DROP VIEW priority_orders;
-- new tests for real time use case including views and subqueries
-- create view to display recent user who has an activity after a timestamp
CREATE VIEW recent_users AS
SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC;
SELECT * FROM recent_users;
user_id | lastseen
---------+---------------------------------
87 | Tue Jan 21 05:53:51.866813 2014
50 | Tue Jan 21 05:53:44.251016 2014
74 | Tue Jan 21 05:54:04.837808 2014
6 | Tue Jan 21 05:57:47.118755 2014
71 | Tue Jan 21 05:55:52.018461 2014
39 | Tue Jan 21 05:55:18.875997 2014
66 | Tue Jan 21 05:51:31.681997 2014
100 | Tue Jan 21 05:49:04.953009 2014
46 | Tue Jan 21 05:49:00.229807 2014
86 | Tue Jan 21 05:48:54.381334 2014
13 | Tue Jan 21 05:48:45.418146 2014
90 | Tue Jan 21 05:48:25.027491 2014
58 | Tue Jan 21 05:47:30.418553 2014
44 | Tue Jan 21 05:47:01.104523 2014
(14 rows)
-- create a view for recent_events
CREATE VIEW recent_events AS
SELECT user_id, time FROM events_table
WHERE time > '2014-01-20 01:45:49.978738'::timestamp;
SELECT count(*) FROM recent_events;
count
-------
1105
(1 row)
-- count number of events of recent_users
SELECT count(*) FROM recent_users ru JOIN events_table et ON (ru.user_id = et.user_id);
count
-------
1336
(1 row)
-- count number of events of per recent users order by count
SELECT ru.user_id, count(*)
FROM recent_users ru
JOIN events_table et
ON (ru.user_id = et.user_id)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1;
user_id | count
---------+-------
13 | 118
44 | 109
90 | 109
87 | 105
46 | 103
86 | 100
66 | 98
39 | 96
71 | 95
74 | 93
6 | 89
58 | 87
50 | 79
100 | 55
(14 rows)
-- the same query with a left join however, it would still generate the same result
SELECT ru.user_id, count(*)
FROM recent_users ru
LEFT JOIN events_table et
ON (ru.user_id = et.user_id)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1;
user_id | count
---------+-------
13 | 118
44 | 109
90 | 109
87 | 105
46 | 103
86 | 100
66 | 98
39 | 96
71 | 95
74 | 93
6 | 89
58 | 87
50 | 79
100 | 55
(14 rows)
-- query wrapped inside a subquery, it needs another top level order by
SELECT * FROM
(SELECT ru.user_id, count(*)
FROM recent_users ru
JOIN events_table et
ON (ru.user_id = et.user_id)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1) s1
ORDER BY 2 DESC, 1;
user_id | count
---------+-------
13 | 118
44 | 109
90 | 109
87 | 105
46 | 103
86 | 100
66 | 98
39 | 96
71 | 95
74 | 93
6 | 89
58 | 87
50 | 79
100 | 55
(14 rows)
-- non-partition key joins are not supported inside subquery
SELECT * FROM
(SELECT ru.user_id, count(*)
FROM recent_users ru
JOIN events_table et
ON (ru.user_id = et.event_type)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1) s1
ORDER BY 2 DESC, 1;
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
-- join between views
-- recent users who has an event in recent events
SELECT ru.user_id FROM recent_users ru JOIN recent_events re USING(user_id) GROUP BY ru.user_id ORDER BY ru.user_id;
user_id
---------
6
13
39
44
46
50
58
66
71
74
86
87
90
100
(14 rows)
-- outer join inside a subquery
-- recent_events who are not done by recent users
SELECT count(*) FROM (
SELECT re.*, ru.user_id AS recent_user
FROM recent_events re LEFT JOIN recent_users ru USING(user_id)) reu
WHERE recent_user IS NULL;
count
-------
957
(1 row)
-- same query with anti-join
SELECT count(*)
FROM recent_events re LEFT JOIN recent_users ru ON(ru.user_id = re.user_id)
WHERE ru.user_id IS NULL;
count
-------
957
(1 row)
-- join between view and table
-- users who has recent activity and they have an entry with value_1 is less than 15
SELECT ut.* FROM recent_users ru JOIN users_table ut USING (user_id) WHERE ut.value_1 < 15 ORDER BY 1,2;
user_id | time | value_1 | value_2 | value_3 | value_4
---------+---------------------------------+---------+---------+---------+---------
6 | Mon Jan 13 05:30:08.289267 2014 | 12 | 140 | 618 |
6 | Thu Jan 16 15:17:16.779695 2014 | 6 | 978 | 430 |
6 | Sun Jan 19 06:09:39.900888 2014 | 3 | 908 | 688 |
13 | Sun Jan 19 22:09:26.256209 2014 | 2 | 755 | 584 |
39 | Wed Jan 15 05:46:51.48765 2014 | 14 | 657 | 137 |
39 | Sun Jan 19 11:26:47.45937 2014 | 12 | 118 | 165 |
44 | Wed Jan 15 14:23:52.532426 2014 | 8 | 204 | 735 |
44 | Sun Jan 19 05:53:34.829093 2014 | 4 | 758 | 205 |
46 | Mon Jan 13 20:39:11.211169 2014 | 0 | 235 | 475 |
46 | Wed Jan 15 09:14:57.471944 2014 | 2 | 407 | 664 |
50 | Sat Jan 11 11:07:13.089216 2014 | 6 | 292 | 425 |
58 | Sun Jan 19 22:36:14.795396 2014 | 2 | 86 | 311 |
66 | Tue Jan 14 20:16:31.219213 2014 | 14 | 347 | 655 |
74 | Tue Jan 21 01:38:39.570986 2014 | 9 | 334 | 642 |
86 | Sun Jan 19 06:18:51.466578 2014 | 14 | 712 | 490 |
87 | Sat Jan 11 20:46:28.439073 2014 | 2 | 528 | 311 |
90 | Sun Jan 12 21:37:30.778206 2014 | 11 | 458 | 377 |
100 | Sun Jan 19 22:32:08.284043 2014 | 2 | 384 | 149 |
(18 rows)
-- determine if a recent user has done a given event type or not
SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event
FROM recent_users ru
LEFT JOIN events_table et
ON(ru.user_id = et.user_id AND et.event_type = 625)
ORDER BY 2 DESC, 1;
user_id | done_event
---------+------------
6 | YES
13 | NO
39 | NO
44 | NO
46 | NO
50 | NO
58 | NO
66 | NO
71 | NO
74 | NO
86 | NO
87 | NO
90 | NO
100 | NO
(14 rows)
-- view vs table join wrapped inside a subquery
SELECT * FROM
(SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event
FROM recent_users ru
LEFT JOIN events_table et
ON(ru.user_id = et.user_id AND et.event_type = 625)
) s1
ORDER BY 2 DESC, 1;
user_id | done_event
---------+------------
6 | YES
13 | NO
39 | NO
44 | NO
46 | NO
50 | NO
58 | NO
66 | NO
71 | NO
74 | NO
86 | NO
87 | NO
90 | NO
100 | NO
(14 rows)
-- event vs table non-partition-key join is not supported
SELECT * FROM
(SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event
FROM recent_users ru
LEFT JOIN events_table et
ON(ru.user_id = et.event_type)
) s1
ORDER BY 2 DESC, 1;
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
-- create a select only view
CREATE VIEW selected_users AS SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150;
CREATE VIEW recent_selected_users AS SELECT su.* FROM selected_users su JOIN recent_users ru USING(user_id);
SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1;
user_id
---------
6
13
39
44
46
50
58
66
71
74
86
90
(12 rows)
-- this would be supported when we implement where partition_key in (subquery) support
SELECT et.* FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users);
ERROR: could not run distributed query with subquery outside the FROM clause
HINT: Consider using an equality filter on the distributed table's partition column.
-- it is supported when it is a router query
SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90);
count
-------
109
(1 row)
-- expected this to work but it did not
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users);
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
HINT: Consider using an equality filter on the distributed table's partition column.
-- wrapping it inside a SELECT * works
SELECT *
FROM (
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10
ORDER BY user_id;
user_id
---------
11
12
13
14
(4 rows)
-- union all also works for views
SELECT *
FROM (
(SELECT user_id FROM recent_users)
UNION ALL
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10
ORDER BY user_id;
user_id
---------
11
11
11
12
12
12
12
12
12
13
13
13
13
13
14
(15 rows)
SELECT count(*)
FROM (
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10;
count
-------
4
(1 row)
-- expected this to work but it does not
SELECT count(*)
FROM (
(SELECT user_id FROM recent_users)
UNION ALL
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- expand view definitions and re-run last 2 queries
SELECT count(*)
FROM (
(SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa
)
UNION
(SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u
WHERE user_id < 15 AND user_id > 10;
count
-------
4
(1 row)
SELECT count(*)
FROM (
(SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa
)
UNION ALL
(SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u
WHERE user_id < 15 AND user_id > 10;
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- test distinct
-- distinct is supported if it is on a partition key
CREATE VIEW distinct_user_with_value_1_15 AS SELECT DISTINCT user_id FROM users_table WHERE value_1 = 15;
SELECT * FROM distinct_user_with_value_1_15 ORDER BY user_id;
user_id
---------
7
8
35
42
46
53
70
82
87
88
96
(11 rows)
-- distinct is not supported if it is on a non-partition key
CREATE VIEW distinct_value_1 AS SELECT DISTINCT value_1 FROM users_table WHERE value_2 = 15;
SELECT * FROM distinct_value_1;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries without group by clause are not supported yet
-- CTEs are not supported even if they are on views
CREATE VIEW cte_view_1 AS
WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 15) SELECT * FROM c1 WHERE value_2 < 500;
SELECT * FROM cte_view_1;
ERROR: cannot push down this subquery
DETAIL: Table expressions other than simple relations and subqueries are currently unsupported
-- this is single shard query but still not supported since it has view + cte
-- router planner can't detect it
SELECT * FROM cte_view_1 WHERE user_id = 8;
ERROR: cannot push down this subquery
DETAIL: Table expressions other than simple relations and subqueries are currently unsupported
-- if CTE itself prunes down to a single shard than the view is supported (router plannable)
CREATE VIEW cte_view_2 AS
WITH c1 AS (SELECT * FROM users_table WHERE user_id = 8) SELECT * FROM c1 WHERE value_1 = 15;
SELECT * FROM cte_view_2;
user_id | time | value_1 | value_2 | value_3 | value_4
---------+---------------------------------+---------+---------+---------+---------
8 | Tue Jan 21 00:52:36.967785 2014 | 15 | 10 | 868 |
(1 row)
CREATE VIEW router_view AS SELECT * FROM users_table WHERE user_id = 2;
-- router plannable
SELECT user_id FROM router_view GROUP BY 1;
user_id
---------
2
(1 row)
-- There is a known issue with router plannable subqueries joined with non-router
-- plannable subqueries. Following tests should be uncommented when we fix it
-- join a router view (not implement error)
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id);
-- it still does not work when converted to 2 subquery join
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id);
-- views are completely removed and still it does not work
-- SELECT * FROM
-- (SELECT user_id FROM (SELECT * FROM users_table WHERE user_id = 2) rv1 GROUP BY 1) rv2
-- JOIN (SELECT user_id, time FROM events_table
-- WHERE time > '2014-01-20 01:45:49.978738'::timestamp) re
-- USING (user_id);
-- views with limits
CREATE VIEW recent_10_users AS
SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
ORDER BY lastseen DESC
LIMIT 10;
-- this is not supported since it has limit in it and subquery_pushdown is not set
SELECT * FROM recent_10_users;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries with limit are not supported yet
SET citus.subquery_pushdown to ON;
-- still not supported since outer query does not have limit
-- it shows a different (subquery with single relation) error message
SELECT * FROM recent_10_users;
ERROR: cannot perform distributed planning on this query
DETAIL: Subqueries with limit are not supported yet
-- now it displays more correct error message
SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id);
ERROR: cannot push down this subquery
DETAIL: Limit in subquery without limit in the outermost query is unsupported
-- now both are supported when there is a limit on the outer most query
SELECT * FROM recent_10_users ORDER BY lastseen DESC LIMIT 10;
user_id | lastseen
---------+---------------------------------
6 | Tue Jan 21 05:57:47.118755 2014
71 | Tue Jan 21 05:55:52.018461 2014
39 | Tue Jan 21 05:55:18.875997 2014
74 | Tue Jan 21 05:54:04.837808 2014
87 | Tue Jan 21 05:53:51.866813 2014
50 | Tue Jan 21 05:53:44.251016 2014
66 | Tue Jan 21 05:51:31.681997 2014
100 | Tue Jan 21 05:49:04.953009 2014
46 | Tue Jan 21 05:49:00.229807 2014
86 | Tue Jan 21 05:48:54.381334 2014
(10 rows)
SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10;
user_id | time | event_type | value_2 | value_3 | value_4
---------+---------------------------------+------------+---------+---------+---------
65 | Tue Jan 21 05:56:52.624231 2014 | 241 | 30 | 543 |
42 | Tue Jan 21 05:46:35.158342 2014 | 761 | 877 | 335 |
54 | Tue Jan 21 05:46:19.103645 2014 | 595 | 477 | 996 |
44 | Tue Jan 21 05:43:00.838945 2014 | 682 | 641 | 448 |
27 | Tue Jan 21 05:34:10.935865 2014 | 912 | 605 | 989 |
61 | Tue Jan 21 05:25:27.452065 2014 | 392 | 472 | 925 |
19 | Tue Jan 21 05:23:09.26298 2014 | 202 | 888 | 640 |
65 | Tue Jan 21 05:22:56.725329 2014 | 519 | 457 | 259 |
27 | Tue Jan 21 05:19:14.38026 2014 | 19 | 19 | 205 |
11 | Tue Jan 21 05:15:14.879531 2014 | 459 | 545 | 80 |
(10 rows)
RESET citus.subquery_pushdown;
-- explain tests
EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1;
QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------------------------------------
Sort
Sort Key: remote_scan.user_id
-> HashAggregate
Group Key: remote_scan.user_id
-> Custom Scan (Citus Real-Time)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: users_table.user_id
-> Hash Join
Hash Cond: (users_table.user_id = ru.user_id)
-> Bitmap Heap Scan on users_table_1400000 users_table
Recheck Cond: ((value_1 >= 120) AND (value_1 < 150))
-> Bitmap Index Scan on is_index3_1400000
Index Cond: ((value_1 >= 120) AND (value_1 < 150))
-> Hash
-> Subquery Scan on ru
-> Sort
Sort Key: (max(users_table_1."time")) DESC
-> HashAggregate
Group Key: users_table_1.user_id
Filter: (max(users_table_1."time") > '2014-01-21 05:45:49.978738'::timestamp without time zone)
-> Seq Scan on users_table_1400000 users_table_1
(25 rows)
EXPLAIN (COSTS FALSE) SELECT *
FROM (
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10
ORDER BY user_id;
QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------------
Sort
Sort Key: remote_scan.user_id
-> Custom Scan (Citus Real-Time)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=57637 dbname=regression
-> Unique
-> Sort
Sort Key: recent_users.user_id
-> Append
-> Subquery Scan on recent_users
-> Sort
Sort Key: (max(users_table."time")) DESC
-> GroupAggregate
Group Key: users_table.user_id
Filter: (max(users_table."time") > '2014-01-21 05:45:49.978738'::timestamp without time zone)
-> Index Scan using is_index1_1400000 on users_table_1400000 users_table
Index Cond: ((user_id < 15) AND (user_id > 10))
-> Index Scan using is_index1_1400000 on users_table_1400000 users_table_1
Index Cond: ((user_id < 15) AND (user_id > 10))
Filter: ((value_1 >= 120) AND (value_1 < 150))
(22 rows)
EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10;
ERROR: cannot push down this subquery
DETAIL: Limit in subquery is currently unsupported
SET citus.subquery_pushdown to ON;
EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10;
QUERY PLAN
---------------------------------------------------------------------------------------------------------------------
Limit
-> Sort
Sort Key: remote_scan."time" DESC
-> Custom Scan (Citus Real-Time)
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=57637 dbname=regression
-> Limit
-> Sort
Sort Key: et."time" DESC
-> Hash Join
Hash Cond: (et.user_id = recent_10_users.user_id)
-> Seq Scan on events_table_1400004 et
-> Hash
-> Subquery Scan on recent_10_users
-> Limit
-> Sort
Sort Key: (max(users_table."time")) DESC
-> HashAggregate
Group Key: users_table.user_id
-> Seq Scan on users_table_1400000 users_table
(22 rows)
RESET citus.subquery_pushdown;
DROP VIEW recent_10_users;
DROP VIEW router_view;
DROP VIEW cte_view_2;
DROP VIEW cte_view_1;
DROP VIEW distinct_value_1;
DROP VIEW distinct_user_with_value_1_15;
DROP VIEW recent_selected_users;
DROP VIEW selected_users;
DROP VIEW recent_events;
DROP VIEW recent_users;

View File

@ -130,9 +130,9 @@ SELECT *
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT 1)
l_linenumber, count(DISTINCT 1)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
@ -140,9 +140,9 @@ SELECT *
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT (random() * 5)::int)
l_linenumber, count(DISTINCT (random() * 5)::int)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
@ -277,18 +277,18 @@ SELECT *
SELECT *
FROM (
SELECT
l_orderkey, sum(DISTINCT l_partkey)
l_linenumber, sum(DISTINCT l_partkey)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
SELECT *
FROM (
SELECT
l_orderkey, avg(DISTINCT l_partkey)
l_linenumber, avg(DISTINCT l_partkey)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
@ -297,18 +297,18 @@ SELECT *
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT lineitem_hash)
l_linenumber, count(DISTINCT lineitem_hash)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT lineitem_hash.*)
l_linenumber, count(DISTINCT lineitem_hash.*)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;

View File

@ -46,7 +46,7 @@ SELECT master_create_distributed_table('orders_subquery', 'o_orderkey', 'range')
SET citus.enable_router_execution TO 'false';
-- Check that we don't allow subquery pushdown in default settings.
-- Check that we allow subquery pushdown in default settings.
SELECT
avg(unit_price)
@ -62,8 +62,6 @@ FROM
GROUP BY
l_orderkey) AS unit_prices;
SET citus.subquery_pushdown to TRUE;
-- Check that we don't crash if there are not any shards.
SELECT
@ -130,7 +128,8 @@ FROM
UPDATE pg_dist_shard SET shardmaxvalue = '14947' WHERE shardid = 270003;
-- If group by is not on partition column then we error out.
-- If group by is not on partition column then we error out from single table
-- repartition code path
SELECT
avg(order_count)
@ -171,14 +170,14 @@ FROM
GROUP BY
l_orderkey) AS unit_prices;
-- Check that we error out if there is union all.
-- Check that we error out if there is non relation subqueries
SELECT count(*) FROM
(
(SELECT l_orderkey FROM lineitem_subquery) UNION ALL
(SELECT 1::bigint)
) b;
-- Check that we error out if queries in union do not include partition columns.
SELECT count(*) FROM
@ -194,9 +193,7 @@ SELECT count(*) FROM
(SELECT l_orderkey FROM lineitem_subquery) UNION
(SELECT l_orderkey FROM lineitem_subquery)
) b;
-- Check that we error out if the outermost query has subquery join.
-- Check that we error out if inner query has Limit but subquery_pushdown is not set
SELECT
avg(o_totalprice/l_quantity)
FROM
@ -217,6 +214,32 @@ FROM
WHERE
lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true;
-- Limit is only supported when subquery_pushdown is set
-- Check that we error out if inner query has limit but outer query has not.
SET citus.subquery_pushdown to ON;
SELECT
avg(o_totalprice/l_quantity)
FROM
(SELECT
l_orderkey,
l_quantity
FROM
lineitem_subquery
ORDER BY
l_quantity
LIMIT 10
) lineitem_quantities
JOIN LATERAL
(SELECT
o_totalprice
FROM
orders_subquery
WHERE
lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true;
-- reset the flag for next query
SET citus.subquery_pushdown to OFF;
-- Check that we error out if the outermost query is a distinct clause.
SELECT
@ -226,6 +249,8 @@ FROM (
count(*) a
FROM
lineitem_subquery
GROUP BY
l_orderkey
) z;
-- Check supported subquery types.
@ -336,7 +361,6 @@ CREATE TABLE subquery_pruning_varchar_test_table
SELECT master_create_distributed_table('subquery_pruning_varchar_test_table', 'a', 'hash');
SELECT master_create_worker_shards('subquery_pruning_varchar_test_table', 4, 1);
SET citus.subquery_pushdown TO TRUE;
SET client_min_messages TO DEBUG2;
SELECT * FROM
@ -388,6 +412,163 @@ CREATE TYPE user_composite_type AS
user_id BIGINT
);
CREATE FUNCTION user_composite_type_equal(user_composite_type,
user_composite_type)
returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS
$$
BEGIN
if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN
return 0;
end if;
if v1.tenant_id > v2.tenant_id THEN
RETURN 1;
end if;
if v1.tenant_id < v2.tenant_id THEN
RETURN -1;
end if;
if v1.user_id > v2.user_id THEN
RETURN 1;
end if;
if v1.user_id < v2.user_id THEN
RETURN -1;
end if;
RETURN 0;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE operator =
( leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_equal,
commutator = =,
RESTRICT = eqsel,
JOIN = eqjoinsel,
merges,
hashes);
CREATE operator <
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_lt
);
CREATE operator >
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_gt
);
CREATE operator <=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_le
);
CREATE operator >=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_ge
);
CREATE FUNCTION user_composite_type_hash(user_composite_type)
returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable
returns NULL ON NULL input;
CREATE OPERATOR CLASS btree_user_composite_ops
DEFAULT FOR TYPE user_composite_type USING btree
AS
OPERATOR 1 < ,
OPERATOR 2 <= ,
OPERATOR 3 = ,
OPERATOR 4 >= ,
OPERATOR 5 > ,
FUNCTION 1 btree_comparison(user_composite_type, user_composite_type);
create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type);
\c - - - :worker_1_port
CREATE TYPE user_composite_type AS
@ -396,6 +577,161 @@ CREATE TYPE user_composite_type AS
user_id BIGINT
);
CREATE FUNCTION user_composite_type_equal(user_composite_type,
user_composite_type)
returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS
$$
BEGIN
if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN
return 0;
end if;
if v1.tenant_id > v2.tenant_id THEN
RETURN 1;
end if;
if v1.tenant_id < v2.tenant_id THEN
RETURN -1;
end if;
if v1.user_id > v2.user_id THEN
RETURN 1;
end if;
if v1.user_id < v2.user_id THEN
RETURN -1;
end if;
RETURN 0;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE operator =
( leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_equal,
commutator = =,
RESTRICT = eqsel,
JOIN = eqjoinsel,
merges,
hashes);
CREATE operator <
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_lt
);
CREATE operator >
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_gt
);
CREATE operator <=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_le
);
CREATE operator >=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_ge
);
CREATE FUNCTION user_composite_type_hash(user_composite_type)
returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable
returns NULL ON NULL input;
CREATE OPERATOR CLASS btree_user_composite_ops
DEFAULT FOR TYPE user_composite_type USING btree
AS
OPERATOR 1 < ,
OPERATOR 2 <= ,
OPERATOR 3 = ,
OPERATOR 4 >= ,
OPERATOR 5 > ,
FUNCTION 1 btree_comparison(user_composite_type, user_composite_type);
create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type);
\c - - - :worker_2_port
CREATE TYPE user_composite_type AS
@ -404,6 +740,161 @@ CREATE TYPE user_composite_type AS
user_id BIGINT
);
CREATE FUNCTION user_composite_type_equal(user_composite_type,
user_composite_type)
returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS
$$
BEGIN
if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN
return 0;
end if;
if v1.tenant_id > v2.tenant_id THEN
RETURN 1;
end if;
if v1.tenant_id < v2.tenant_id THEN
RETURN -1;
end if;
if v1.user_id > v2.user_id THEN
RETURN 1;
end if;
if v1.user_id < v2.user_id THEN
RETURN -1;
end if;
RETURN 0;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE operator =
( leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_equal,
commutator = =,
RESTRICT = eqsel,
JOIN = eqjoinsel,
merges,
hashes);
CREATE operator <
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_lt
);
CREATE operator >
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_gt
);
CREATE operator <=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_le
);
CREATE operator >=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_ge
);
CREATE FUNCTION user_composite_type_hash(user_composite_type)
returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable
returns NULL ON NULL input;
CREATE OPERATOR CLASS btree_user_composite_ops
DEFAULT FOR TYPE user_composite_type USING btree
AS
OPERATOR 1 < ,
OPERATOR 2 <= ,
OPERATOR 3 = ,
OPERATOR 4 >= ,
OPERATOR 5 > ,
FUNCTION 1 btree_comparison(user_composite_type, user_composite_type);
create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type);
\c - - - :master_port
@ -482,8 +973,6 @@ WHERE shardid = :new_shard_id;
\.
SET citus.subquery_pushdown TO TRUE;
-- Simple join subquery pushdown
SELECT
avg(array_length(events, 1)) AS event_average
@ -502,8 +991,7 @@ FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type IN ('click', 'submit', 'pay')) AS subquery
@ -526,14 +1014,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -541,14 +1029,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -566,8 +1054,7 @@ FROM
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay') AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -590,14 +1077,14 @@ SELECT
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -605,14 +1092,14 @@ SELECT
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -621,6 +1108,7 @@ SELECT
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
COUNT(*) AS count_pay
FROM
events
@ -629,13 +1117,11 @@ SELECT
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay'
GROUP BY
tenant_id,
user_id
composite_id
HAVING
COUNT(*) > 2) AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -648,6 +1134,8 @@ ORDER BY
count_pay;
-- Lateral join subquery pushdown
-- set subquery_pushdown since there is limit in the query
SET citus.subquery_pushdown to ON;
SELECT
tenant_id,
user_id,
@ -663,6 +1151,7 @@ FROM
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
lastseen
FROM
users
@ -681,8 +1170,7 @@ FROM
FROM
events
WHERE
(composite_id).tenant_id = subquery_top.tenant_id AND
(composite_id).user_id = subquery_top.user_id
(composite_id) = subquery_top.composite_id
ORDER BY
event_time DESC
LIMIT
@ -697,7 +1185,8 @@ ORDER BY
user_lastseen DESC
LIMIT
10;
-- reset the flag for next query
SET citus.subquery_pushdown to OFF;
-- Same queries above with explain
-- Simple join subquery pushdown
@ -720,8 +1209,7 @@ FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type IN ('click', 'submit', 'pay')) AS subquery
@ -745,14 +1233,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -760,14 +1248,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -785,8 +1273,7 @@ FROM
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay') AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -810,14 +1297,14 @@ EXPLAIN (COSTS OFF)
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -825,14 +1312,14 @@ EXPLAIN (COSTS OFF)
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -841,6 +1328,7 @@ EXPLAIN (COSTS OFF)
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
COUNT(*) AS count_pay
FROM
events
@ -849,13 +1337,11 @@ EXPLAIN (COSTS OFF)
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay'
GROUP BY
tenant_id,
user_id
composite_id
HAVING
COUNT(*) > 2) AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -868,6 +1354,8 @@ ORDER BY
count_pay;
-- Lateral join subquery pushdown
-- set subquery_pushdown due to limit in the query
SET citus.subquery_pushdown to ON;
EXPLAIN (COSTS OFF)
SELECT
tenant_id,
@ -884,6 +1372,7 @@ FROM
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
lastseen
FROM
users
@ -902,8 +1391,7 @@ FROM
FROM
events
WHERE
(composite_id).tenant_id = subquery_top.tenant_id AND
(composite_id).user_id = subquery_top.user_id
(composite_id) = subquery_top.composite_id
ORDER BY
event_time DESC
LIMIT
@ -918,5 +1406,5 @@ ORDER BY
user_lastseen DESC
LIMIT
10;
SET citus.subquery_pushdown to OFF;
SET citus.enable_router_execution TO 'true';

View File

@ -40,7 +40,8 @@ test: multi_insert_select
test: multi_deparse_shard_query
test: multi_basic_queries multi_complex_expressions multi_verify_no_subquery
test: multi_explain
test: multi_subquery
test: multi_subquery multi_subquery_complex_queries multi_subquery_behavioral_analytics
test: multi_subquery_union
test: multi_reference_table
test: multi_outer_join_reference
test: multi_single_relation_subquery

View File

@ -241,9 +241,9 @@ SELECT *
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT 1)
l_linenumber, count(DISTINCT 1)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
ERROR: cannot compute aggregate (distinct)
@ -253,9 +253,9 @@ HINT: You can load the hll extension from contrib packages and enable distinct
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT (random() * 5)::int)
l_linenumber, count(DISTINCT (random() * 5)::int)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
ERROR: cannot compute aggregate (distinct)
@ -464,9 +464,9 @@ SELECT *
SELECT *
FROM (
SELECT
l_orderkey, sum(DISTINCT l_partkey)
l_linenumber, sum(DISTINCT l_partkey)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
ERROR: cannot compute aggregate (distinct)
@ -474,9 +474,9 @@ DETAIL: Only count(distinct) aggregate is supported in subqueries
SELECT *
FROM (
SELECT
l_orderkey, avg(DISTINCT l_partkey)
l_linenumber, avg(DISTINCT l_partkey)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
ERROR: cannot compute aggregate (distinct)
@ -486,9 +486,9 @@ DETAIL: Only count(distinct) aggregate is supported in subqueries
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT lineitem_hash)
l_linenumber, count(DISTINCT lineitem_hash)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
ERROR: cannot compute count (distinct)
@ -496,9 +496,9 @@ DETAIL: Non-column references are not supported yet
SELECT *
FROM (
SELECT
l_orderkey, count(DISTINCT lineitem_hash.*)
l_linenumber, count(DISTINCT lineitem_hash.*)
FROM lineitem_hash
GROUP BY l_orderkey) sub
GROUP BY l_linenumber) sub
ORDER BY 2 DESC, 1 DESC
LIMIT 10;
ERROR: cannot compute count (distinct)

File diff suppressed because it is too large Load Diff

View File

@ -53,7 +53,7 @@ SELECT master_create_distributed_table('orders_subquery', 'o_orderkey', 'range')
(1 row)
SET citus.enable_router_execution TO 'false';
-- Check that we don't allow subquery pushdown in default settings.
-- Check that we allow subquery pushdown in default settings.
SELECT
avg(unit_price)
FROM
@ -67,9 +67,11 @@ FROM
l_orderkey = o_orderkey
GROUP BY
l_orderkey) AS unit_prices;
ERROR: cannot perform distributed planning on this query
DETAIL: Join in subqueries is not supported yet
SET citus.subquery_pushdown to TRUE;
avg
-----
(1 row)
-- Check that we don't crash if there are not any shards.
SELECT
avg(unit_price)
@ -129,7 +131,8 @@ ERROR: cannot push down this subquery
DETAIL: Shards of relations in subquery need to have 1-to-1 shard partitioning
-- Update metadata in order to make all shards equal.
UPDATE pg_dist_shard SET shardmaxvalue = '14947' WHERE shardid = 270003;
-- If group by is not on partition column then we error out.
-- If group by is not on partition column then we error out from single table
-- repartition code path
SELECT
avg(order_count)
FROM
@ -140,8 +143,8 @@ FROM
lineitem_subquery
GROUP BY
l_suppkey) AS order_counts;
ERROR: cannot push down this subquery
DETAIL: Group by list without partition column is currently unsupported
ERROR: cannot use real time executor with repartition jobs
HINT: Set citus.task_executor_type to "task-tracker".
-- Check that we error out if join is not on partition columns.
SELECT
avg(unit_price)
@ -154,8 +157,8 @@ FROM
orders_subquery
GROUP BY
l_orderkey) AS unit_prices;
ERROR: cannot push down this subquery
DETAIL: Relations need to be joining on partition columns
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
SELECT
avg(unit_price)
FROM
@ -169,23 +172,24 @@ FROM
l_orderkey = o_custkey
GROUP BY
l_orderkey) AS unit_prices;
ERROR: cannot push down this subquery
DETAIL: Relations need to be joining on partition columns
-- Check that we error out if there is union all.
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
-- Check that we error out if there is non relation subqueries
SELECT count(*) FROM
(
(SELECT l_orderkey FROM lineitem_subquery) UNION ALL
(SELECT 1::bigint)
) b;
ERROR: could not run distributed query with complex table expressions
ERROR: cannot push down this subquery
DETAIL: Subqueries without relations are unsupported
-- Check that we error out if queries in union do not include partition columns.
SELECT count(*) FROM
(
(SELECT l_orderkey FROM lineitem_subquery) UNION
(SELECT l_partkey FROM lineitem_subquery)
) b;
ERROR: cannot push down this subquery
DETAIL: Union clauses need to select partition columns
ERROR: cannot pushdown the subquery since all leaves of the UNION does not include partition key at the same position
DETAIL: Each leaf query of the UNION should return partition key at the same position on its target list.
-- Check that we run union queries if partition column is selected.
SELECT count(*) FROM
(
@ -197,7 +201,7 @@ SELECT count(*) FROM
2985
(1 row)
-- Check that we error out if the outermost query has subquery join.
-- Check that we error out if inner query has Limit but subquery_pushdown is not set
SELECT
avg(o_totalprice/l_quantity)
FROM
@ -217,8 +221,34 @@ FROM
orders_subquery
WHERE
lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true;
ERROR: cannot perform distributed planning on this query
DETAIL: Join in subqueries is not supported yet
ERROR: cannot push down this subquery
DETAIL: Limit in subquery is currently unsupported
-- Limit is only supported when subquery_pushdown is set
-- Check that we error out if inner query has limit but outer query has not.
SET citus.subquery_pushdown to ON;
SELECT
avg(o_totalprice/l_quantity)
FROM
(SELECT
l_orderkey,
l_quantity
FROM
lineitem_subquery
ORDER BY
l_quantity
LIMIT 10
) lineitem_quantities
JOIN LATERAL
(SELECT
o_totalprice
FROM
orders_subquery
WHERE
lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true;
ERROR: cannot push down this subquery
DETAIL: Limit in subquery without limit in the outermost query is unsupported
-- reset the flag for next query
SET citus.subquery_pushdown to OFF;
-- Check that we error out if the outermost query is a distinct clause.
SELECT
count(DISTINCT a)
@ -227,6 +257,8 @@ FROM (
count(*) a
FROM
lineitem_subquery
GROUP BY
l_orderkey
) z;
ERROR: cannot push down this subquery
DETAIL: distinct in the outermost query is unsupported
@ -371,11 +403,13 @@ SELECT master_create_worker_shards('subquery_pruning_varchar_test_table', 4, 1);
(1 row)
SET citus.subquery_pushdown TO TRUE;
SET client_min_messages TO DEBUG2;
SELECT * FROM
(SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE a = 'onder' GROUP BY a)
AS foo;
DEBUG: Skipping the target shard interval 270005 because SELECT query is pruned away for the interval
DEBUG: Skipping the target shard interval 270006 because SELECT query is pruned away for the interval
DEBUG: Skipping the target shard interval 270008 because SELECT query is pruned away for the interval
count
-------
(0 rows)
@ -383,6 +417,9 @@ AS foo;
SELECT * FROM
(SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE 'eren' = a GROUP BY a)
AS foo;
DEBUG: Skipping the target shard interval 270005 because SELECT query is pruned away for the interval
DEBUG: Skipping the target shard interval 270007 because SELECT query is pruned away for the interval
DEBUG: Skipping the target shard interval 270008 because SELECT query is pruned away for the interval
count
-------
(0 rows)
@ -427,18 +464,415 @@ CREATE TYPE user_composite_type AS
tenant_id BIGINT,
user_id BIGINT
);
CREATE FUNCTION user_composite_type_equal(user_composite_type,
user_composite_type)
returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS
$$
BEGIN
if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN
return 0;
end if;
if v1.tenant_id > v2.tenant_id THEN
RETURN 1;
end if;
if v1.tenant_id < v2.tenant_id THEN
RETURN -1;
end if;
if v1.user_id > v2.user_id THEN
RETURN 1;
end if;
if v1.user_id < v2.user_id THEN
RETURN -1;
end if;
RETURN 0;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE operator =
( leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_equal,
commutator = =,
RESTRICT = eqsel,
JOIN = eqjoinsel,
merges,
hashes);
CREATE operator <
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_lt
);
CREATE operator >
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_gt
);
CREATE operator <=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_le
);
CREATE operator >=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_ge
);
CREATE FUNCTION user_composite_type_hash(user_composite_type)
returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable
returns NULL ON NULL input;
CREATE OPERATOR CLASS btree_user_composite_ops
DEFAULT FOR TYPE user_composite_type USING btree
AS
OPERATOR 1 < ,
OPERATOR 2 <= ,
OPERATOR 3 = ,
OPERATOR 4 >= ,
OPERATOR 5 > ,
FUNCTION 1 btree_comparison(user_composite_type, user_composite_type);
create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type);
\c - - - :worker_1_port
CREATE TYPE user_composite_type AS
(
tenant_id BIGINT,
user_id BIGINT
);
CREATE FUNCTION user_composite_type_equal(user_composite_type,
user_composite_type)
returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS
$$
BEGIN
if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN
return 0;
end if;
if v1.tenant_id > v2.tenant_id THEN
RETURN 1;
end if;
if v1.tenant_id < v2.tenant_id THEN
RETURN -1;
end if;
if v1.user_id > v2.user_id THEN
RETURN 1;
end if;
if v1.user_id < v2.user_id THEN
RETURN -1;
end if;
RETURN 0;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE operator =
( leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_equal,
commutator = =,
RESTRICT = eqsel,
JOIN = eqjoinsel,
merges,
hashes);
CREATE operator <
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_lt
);
CREATE operator >
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_gt
);
CREATE operator <=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_le
);
CREATE operator >=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_ge
);
CREATE FUNCTION user_composite_type_hash(user_composite_type)
returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable
returns NULL ON NULL input;
CREATE OPERATOR CLASS btree_user_composite_ops
DEFAULT FOR TYPE user_composite_type USING btree
AS
OPERATOR 1 < ,
OPERATOR 2 <= ,
OPERATOR 3 = ,
OPERATOR 4 >= ,
OPERATOR 5 > ,
FUNCTION 1 btree_comparison(user_composite_type, user_composite_type);
create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type);
\c - - - :worker_2_port
CREATE TYPE user_composite_type AS
(
tenant_id BIGINT,
user_id BIGINT
);
CREATE FUNCTION user_composite_type_equal(user_composite_type,
user_composite_type)
returns boolean AS 'select $1.tenant_id = $2.tenant_id AND $1.user_id = $2.user_id;' language sql immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_lt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id < v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_le(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id < v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id <= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_gt(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id > v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION user_composite_type_ge(v1 user_composite_type, v2 user_composite_type) returns boolean AS
$$
BEGIN
if v1.tenant_id > v2.tenant_id THEN
return true;
end if;
if v1.tenant_id = v2.tenant_id AND v1.user_id >= v2.user_id THEN
RETURN true;
end if;
RETURN false;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE OR REPLACE FUNCTION btree_comparison(v1 user_composite_type, v2 user_composite_type) returns integer AS
$$
BEGIN
if v1.tenant_id = v2.tenant_id AND v1.user_id = v2.user_id THEN
return 0;
end if;
if v1.tenant_id > v2.tenant_id THEN
RETURN 1;
end if;
if v1.tenant_id < v2.tenant_id THEN
RETURN -1;
end if;
if v1.user_id > v2.user_id THEN
RETURN 1;
end if;
if v1.user_id < v2.user_id THEN
RETURN -1;
end if;
RETURN 0;
END;
$$ language 'plpgsql' immutable
returns NULL ON NULL input;
CREATE operator =
( leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_equal,
commutator = =,
RESTRICT = eqsel,
JOIN = eqjoinsel,
merges,
hashes);
CREATE operator <
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_lt
);
CREATE operator >
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_gt
);
CREATE operator <=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_le
);
CREATE operator >=
(
leftarg = user_composite_type, rightarg = user_composite_type,
PROCEDURE = user_composite_type_ge
);
CREATE FUNCTION user_composite_type_hash(user_composite_type)
returns int AS 'SELECT hashtext( ($1.tenant_id + $1.user_id)::text);' language sql immutable
returns NULL ON NULL input;
CREATE OPERATOR CLASS btree_user_composite_ops
DEFAULT FOR TYPE user_composite_type USING btree
AS
OPERATOR 1 < ,
OPERATOR 2 <= ,
OPERATOR 3 = ,
OPERATOR 4 >= ,
OPERATOR 5 > ,
FUNCTION 1 btree_comparison(user_composite_type, user_composite_type);
create operator class user_composite_type_hash_op_class DEFAULT FOR type user_composite_type using hash AS operator 1 = (user_composite_type, user_composite_type), FUNCTION 1 user_composite_type_hash(user_composite_type);
\c - - - :master_port
CREATE TABLE events (
composite_id user_composite_type,
@ -496,7 +930,6 @@ SELECT master_create_empty_shard('users') AS new_shard_id
UPDATE pg_dist_shard SET shardminvalue = '(2,2000000001)', shardmaxvalue = '(2,4300000000)'
WHERE shardid = :new_shard_id;
\COPY users FROM STDIN WITH CSV
SET citus.subquery_pushdown TO TRUE;
-- Simple join subquery pushdown
SELECT
avg(array_length(events, 1)) AS event_average
@ -515,8 +948,7 @@ FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type IN ('click', 'submit', 'pay')) AS subquery
@ -543,14 +975,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -558,14 +990,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -583,8 +1015,7 @@ FROM
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay') AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -612,14 +1043,14 @@ SELECT
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -627,14 +1058,14 @@ SELECT
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -643,6 +1074,7 @@ SELECT
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
COUNT(*) AS count_pay
FROM
events
@ -651,13 +1083,11 @@ SELECT
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay'
GROUP BY
tenant_id,
user_id
composite_id
HAVING
COUNT(*) > 2) AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -673,7 +1103,10 @@ ORDER BY
3.0000000000000000 | 0
(1 row)
-- Lateral join subquery pushdown
-- set subquery_pushdown since there is limit in the query
SET citus.subquery_pushdown to ON;
SELECT
tenant_id,
user_id,
@ -689,6 +1122,7 @@ FROM
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
lastseen
FROM
users
@ -707,8 +1141,7 @@ FROM
FROM
events
WHERE
(composite_id).tenant_id = subquery_top.tenant_id AND
(composite_id).user_id = subquery_top.user_id
(composite_id) = subquery_top.composite_id
ORDER BY
event_time DESC
LIMIT
@ -730,6 +1163,8 @@ LIMIT
1 | 1001 | 1472807115 | {click,submit,pay}
(3 rows)
-- reset the flag for next query
SET citus.subquery_pushdown to OFF;
-- Same queries above with explain
-- Simple join subquery pushdown
EXPLAIN (COSTS OFF)
@ -750,36 +1185,29 @@ FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type IN ('click', 'submit', 'pay')) AS subquery
GROUP BY
tenant_id,
user_id) AS subquery;
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
QUERY PLAN
------------------------------------------------------------------------------------------------------------
Aggregate
-> Custom Scan (Citus Real-Time)
Task Count: 2
Tasks Shown: One of 2
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=57637 dbname=regression
-> Aggregate
-> GroupAggregate
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
-> Merge Join
Merge Cond: ((((users.composite_id).tenant_id) = ((events.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events.composite_id).user_id)))
-> Sort
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
-> Seq Scan on users_270013 users
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
-> Sort
Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id)
-> Seq Scan on events_270009 events
Filter: ((event_type)::text = ANY ('{click,submit,pay}'::text[]))
(19 rows)
-> Sort
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
-> Result
One-Time Filter: false
(13 rows)
-- Union and left join subquery pushdown
EXPLAIN (COSTS OFF)
@ -797,14 +1225,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -812,14 +1240,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -837,54 +1265,46 @@ FROM
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay') AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
hasdone) AS subquery_top
GROUP BY
hasdone;
QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
QUERY PLAN
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
HashAggregate
Group Key: remote_scan.hasdone
-> Custom Scan (Citus Real-Time)
Task Count: 2
Tasks Shown: One of 2
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: COALESCE(('Has done paying'::text), 'Has not done paying'::text)
-> GroupAggregate
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text)
Group Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text)
-> Sort
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('Has done paying'::text)
-> Merge Left Join
Merge Cond: ((((users.composite_id).tenant_id) = ((events_2.composite_id).tenant_id)) AND (((users.composite_id).user_id) = ((events_2.composite_id).user_id)))
Sort Key: subquery_1.tenant_id, subquery_1.user_id, ('Has done paying'::text)
-> Hash Right Join
Hash Cond: (events.composite_id = subquery_1.composite_id)
-> Unique
-> Sort
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id), ('action=>1'::text), events.event_time
-> Append
-> Nested Loop
Join Filter: (((users.composite_id).tenant_id = (events.composite_id).tenant_id) AND ((users.composite_id).user_id = (events.composite_id).user_id))
-> Seq Scan on events_270009 events
Filter: ((event_type)::text = 'click'::text)
-> Seq Scan on users_270013 users
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
-> Nested Loop
Join Filter: (((users_1.composite_id).tenant_id = (events_1.composite_id).tenant_id) AND ((users_1.composite_id).user_id = (events_1.composite_id).user_id))
-> Seq Scan on events_270009 events_1
Filter: ((event_type)::text = 'submit'::text)
-> Seq Scan on users_270013 users_1
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
-> Materialize
-> Unique
-> Sort
Sort Key: ((events_2.composite_id).tenant_id), ((events_2.composite_id).user_id)
-> Seq Scan on events_270009 events_2
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type) AND ((event_type)::text = 'pay'::text))
(37 rows)
Sort Key: ((events.composite_id).tenant_id), ((events.composite_id).user_id)
-> Seq Scan on events_270011 events
Filter: (((event_type)::text = 'pay'::text) AND (composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
-> Hash
-> Subquery Scan on subquery_1
-> Unique
-> Sort
Sort Key: ((composite_id).tenant_id), ((composite_id).user_id), composite_id, ('action=>1'::text), event_time
-> Append
-> Result
One-Time Filter: false
-> Result
One-Time Filter: false
(30 rows)
-- Union, left join and having subquery pushdown
EXPLAIN (COSTS OFF)
@ -902,14 +1322,14 @@ EXPLAIN (COSTS OFF)
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -917,14 +1337,14 @@ EXPLAIN (COSTS OFF)
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -933,6 +1353,7 @@ EXPLAIN (COSTS OFF)
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
COUNT(*) AS count_pay
FROM
events
@ -941,13 +1362,11 @@ EXPLAIN (COSTS OFF)
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay'
GROUP BY
tenant_id,
user_id
composite_id
HAVING
COUNT(*) > 2) AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -960,6 +1379,8 @@ ORDER BY
count_pay;
ERROR: bogus varattno for OUTER_VAR var: 3
-- Lateral join subquery pushdown
-- set subquery_pushdown due to limit in the query
SET citus.subquery_pushdown to ON;
EXPLAIN (COSTS OFF)
SELECT
tenant_id,
@ -976,6 +1397,7 @@ FROM
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
lastseen
FROM
users
@ -994,8 +1416,7 @@ FROM
FROM
events
WHERE
(composite_id).tenant_id = subquery_top.tenant_id AND
(composite_id).user_id = subquery_top.user_id
(composite_id) = subquery_top.composite_id
ORDER BY
event_time DESC
LIMIT
@ -1010,34 +1431,35 @@ ORDER BY
user_lastseen DESC
LIMIT
10;
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
QUERY PLAN
------------------------------------------------------------------------------------------------------------
Limit
-> Sort
Sort Key: remote_scan.user_lastseen DESC
-> Custom Scan (Citus Real-Time)
Task Count: 2
Tasks Shown: One of 2
Task Count: 4
Tasks Shown: One of 4
-> Task
Node: host=localhost port=57637 dbname=regression
-> Limit
-> Sort
Sort Key: (max(users.lastseen)) DESC
Sort Key: (max(lastseen)) DESC
-> GroupAggregate
Group Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
Group Key: ((composite_id).tenant_id), ((composite_id).user_id)
-> Sort
Sort Key: ((users.composite_id).tenant_id), ((users.composite_id).user_id)
Sort Key: ((composite_id).tenant_id), ((composite_id).user_id)
-> Nested Loop Left Join
-> Limit
-> Sort
Sort Key: users.lastseen DESC
-> Seq Scan on users_270013 users
Filter: ((composite_id >= '(1,-9223372036854775808)'::user_composite_type) AND (composite_id <= '(1,9223372036854775807)'::user_composite_type))
Sort Key: lastseen DESC
-> Result
One-Time Filter: false
-> Limit
-> Sort
Sort Key: events.event_time DESC
-> Seq Scan on events_270009 events
Filter: (((composite_id).tenant_id = ((users.composite_id).tenant_id)) AND ((composite_id).user_id = ((users.composite_id).user_id)))
-> Seq Scan on events_270011 events
Filter: (composite_id = composite_id)
(26 rows)
SET citus.subquery_pushdown to OFF;
SET citus.enable_router_execution TO 'true';

View File

@ -16,3 +16,4 @@
/worker_copy.sql
/multi_complex_count_distinct.sql
/multi_mx_copy_data.sql
/multi_insert_select_behavioral_analytics_create_table.sql

View File

@ -161,13 +161,11 @@ SELECT count(*) FROM lineitem, orders WHERE l_orderkey + 1 = o_orderkey;
-- Check that we can issue limit/offset queries
-- OFFSET in subqueries are not supported
-- Error in the planner when subquery pushdown is off
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq;
SET citus.subquery_pushdown TO true;
-- Error in the planner when single repartition subquery
SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq;
-- Error in the optimizer when subquery pushdown is on
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq;
SET citus.subquery_pushdown TO false;
-- Simple LIMIT/OFFSET with ORDER BY
SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20;

View File

@ -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,

View File

@ -649,3 +649,30 @@ FROM
) temp
ON users_table.user_id = temp.user_id
WHERE users_table.value_1 < 50;
-- not supported since one of the queries doesn't have a relation
INSERT INTO agg_results (user_id, agg_time, value_2_agg)
SELECT
user_id,
user_lastseen,
array_length(event_array, 1)
FROM (
SELECT
user_id,
max(u.time) as user_lastseen,
array_agg(event_type ORDER BY u.time) AS event_array
FROM (
SELECT user_id, time, value_3 as val_3
FROM users_table
WHERE
user_id >= 10 AND user_id <= 70 AND
users_table.value_1 > 10 AND users_table.value_1 < 12
) u LEFT JOIN LATERAL (
SELECT event_type, time
FROM events_table, (SELECT 1 as x) as f
WHERE user_id = u.user_id AND
events_table.event_type > 10 AND events_table.event_type < 12
) t ON true
GROUP BY user_id
) AS shard_union
ORDER BY user_lastseen DESC;

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,726 @@
--
-- multi subquery toplevel union queries aims to expand existing subquery pushdown
-- regression tests to cover more cases
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
-- ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1400000;
SET citus.enable_router_execution TO false;
-- a very simple union query
SELECT user_id, counter
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
ORDER BY 2 DESC,1
LIMIT 5;
-- the same query with union all
SELECT user_id, counter
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION ALL
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
ORDER BY 2 DESC,1
LIMIT 5;
-- the same query with group by
SELECT user_id, sum(counter)
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
GROUP BY 1
ORDER BY 2 DESC,1
LIMIT 5;
-- the same query with UNION ALL clause
SELECT user_id, sum(counter)
FROM (
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION ALL
SELECT user_id, value_2 % 10 AS counter FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
GROUP BY 1
ORDER BY 2 DESC,1
LIMIT 5;
-- the same query target list entries shuffled
SELECT user_id, sum(counter)
FROM (
SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (1, 2, 3, 4, 5)
UNION
SELECT value_2 % 10 AS counter, user_id FROM events_table WHERE event_type IN (5, 6, 7, 8, 9, 10)
) user_id
GROUP BY 1
ORDER BY 2 DESC,1
LIMIT 5;
-- same query with GROUP BY
SELECT user_id, sum(counter)
FROM (
SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (1, 2)
UNION
SELECT user_id, value_2 AS counter FROM events_table WHERE event_type IN (5, 6)
) user_id
GROUP BY
user_id
--HAVING sum(counter) > 900
ORDER BY 1,2 DESC LIMIT 5;
-- the same query target list entries shuffled but this time the subqueries target list
-- is shuffled
SELECT user_id, sum(counter)
FROM (
SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (1, 2)
UNION
SELECT value_2 AS counter, user_id FROM events_table WHERE event_type IN (5, 6)
) user_id
GROUP BY
user_id
--HAVING sum(counter) > 900
ORDER BY 1,2 DESC LIMIT 5;
-- similar query this time more subqueries and target list contains a resjunk entry
SELECT sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500
) user_id
GROUP BY user_id ORDER BY 1 DESC LIMIT 5;
-- similar query as above, with UNION ALL
SELECT sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 5000
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500
UNION ALL
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500
) user_id
GROUP BY user_id ORDER BY 1 DESC LIMIT 5;
-- unions within unions
SELECT *
FROM (
( SELECT user_id,
sum(counter)
FROM
(SELECT
user_id, sum(value_2) AS counter
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_1
GROUP BY
user_id)
UNION
(SELECT
user_id, sum(counter)
FROM
(SELECT
user_id, sum(value_2) AS counter
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_2
GROUP BY
user_id)) AS ftop
ORDER BY 2 DESC, 1 DESC
LIMIT 5;
-- top level unions are wrapped into top level aggregations
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT *, random()
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) ) events_subquery_2)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13)) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q"
) as final_query
GROUP BY types
ORDER BY types;
-- exactly the same query
-- but wrapper unions are removed from the inner part of the query
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
FROM
(SELECT *, random()
FROM
(SELECT
"t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
(SELECT
"t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM(
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
GROUP BY "t1"."user_id") AS t) "q"
) as final_query
GROUP BY types
ORDER BY types;
-- again excatly the same query with top level wrapper removed
SELECT ("q"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
GROUP BY "t1"."user_id") AS t) "q"
GROUP BY types
ORDER BY types;
-- again same query but with only two top level empty queries (i.e., no group bys)
SELECT *
FROM
( SELECT *
FROM
( SELECT "t1"."user_id"
FROM (
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
) AS t) "q"
ORDER BY 1
LIMIT 5;
-- a very similar query UNION ALL
SELECT ("q"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15))
UNION ALL
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) )
UNION ALL
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) )
UNION ALL
(SELECT
"events"."user_id", "events"."time", 3 AS event
FROM
events_table as "events"
WHERE
event_type IN (26, 27, 28, 29, 30, 13))) t1
GROUP BY "t1"."user_id") AS t) "q"
GROUP BY types
ORDER BY types;
-- some UNION ALL queries that are going to be pulled up
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT user_id FROM events_table)
) b;
-- similar query without top level agg
SELECT
user_id
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT user_id FROM events_table)
) b
ORDER BY 1 DESC
LIMIT 5;
-- similar query with multiple target list entries
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM users_table)
UNION ALL
(SELECT value_3, user_id FROM events_table)
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
-- similar query group by inside the subqueries
SELECT
user_id, value_3_sum
FROM
(
(SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id)
UNION ALL
(SELECT sum(value_3) as value_3_sum, user_id FROM users_table GROUP BY user_id)
) b
ORDER BY 2 DESC, 1 DESC
LIMIT 5;
-- similar query top level group by
SELECT
user_id, sum(value_3)
FROM
(
(SELECT value_3, user_id FROM users_table)
UNION ALL
(SELECT value_3, user_id FROM events_table)
) b
GROUP BY 1
ORDER BY 2 DESC, 1 DESC
LIMIT 5;
-- a long set operation list
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30))
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
-- no partition key on the top
SELECT
max(value_3)
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (26, 27, 28, 29, 30))
) b
GROUP BY user_id
ORDER BY 1 DESC
LIMIT 5;
-- now lets also have some unsupported queries
-- group by is not on the partition key
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
UNION
SELECT value_1 as user_id, sum(value_2) AS counter FROM users_table GROUP BY value_1
) user_id
GROUP BY user_id;
-- partition key is not selected
SELECT sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 20 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 40 and value_1 < 60 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 60 and value_1 < 80 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 80 and value_1 < 100 GROUP BY user_id HAVING sum(value_2) > 500
UNION
SELECT 2 * user_id, sum(value_2) AS counter FROM users_table where value_1 < 100 and value_1 < 120 GROUP BY user_id HAVING sum(value_2) > 500
) user_id
GROUP BY user_id ORDER BY 1 DESC LIMIT 5;
-- excepts within unions are not supported
SELECT * FROM
(
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
) user_id_1
GROUP BY user_id
)
UNION
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
EXCEPT
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
) user_id_2
GROUP BY user_id)
) as ftop;
-- joins inside unions are not supported
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE users_table.user_id > events_table.user_id GROUP BY 1
) user_id
GROUP BY user_id;
-- joins inside unions are not supported -- slightly more comlex than the above
SELECT * FROM
(
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
) user_id_1
GROUP BY user_id
)
UNION
(
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id
UNION
SELECT events_table.user_id, sum(events_table.value_2) AS counter FROM events_table, users_table WHERE (events_table.user_id = users_table.user_id) GROUP BY events_table.user_id
) user_id_2
GROUP BY user_id)
) as ftop;
-- offset inside the union
SELECT user_id, sum(counter)
FROM (
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
UNION
SELECT user_id, sum(value_2) AS counter FROM users_table GROUP BY user_id OFFSET 4
) user_id
GROUP BY user_id;
-- lower level union does not return partition key with the other relations
SELECT *
FROM (
( SELECT user_id,
sum(counter)
FROM
(SELECT
user_id, sum(value_2) AS counter
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_1
GROUP BY
user_id)
UNION
(SELECT
user_id, sum(counter)
FROM
(SELECT
sum(value_2) AS counter, user_id
FROM
users_table
GROUP BY
user_id
UNION
SELECT
user_id, sum(value_2) AS counter
FROM
events_table
GROUP BY
user_id) user_id_2
GROUP BY
user_id)) AS ftop;
-- some UNION all queries that are going to be pulled up
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT 2 * user_id FROM events_table)
) b;
-- last query does not have partition key
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT value_3, value_2 FROM events_table where event_type IN (26, 27, 28, 29, 30))
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
-- we don't allow joins within unions
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT users_table.user_id FROM events_table, users_table WHERE events_table.user_id = users_table.user_id)
) b;
-- we don't support subqueries without relations
SELECT
count(*)
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT 1)
) b;
-- we don't support subqueries without relations
SELECT
*
FROM
(
(SELECT user_id FROM users_table)
UNION ALL
(SELECT (random() * 100)::int)
) b;
-- we don't support subqueries without relations
SELECT
user_id, value_3
FROM
(
(SELECT value_3, user_id FROM events_table where event_type IN (1, 2, 3, 4, 5))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (6, 7, 8, 9, 10))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (11, 12, 13, 14, 15))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (16, 17, 18, 19, 20))
UNION ALL
(SELECT value_3, user_id FROM events_table where event_type IN (21, 22, 23, 24, 25))
UNION ALL
(SELECT 1, 2)
) b
ORDER BY 1 DESC, 2 DESC
LIMIT 5;
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
FROM
( SELECT *, random()
FROM
( SELECT "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types"
FROM
( SELECT "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events
FROM (
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 0 AS event
FROM
events_table as "events"
WHERE
event_type IN (10, 11, 12, 13, 14, 15)) events_subquery_1)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 1 AS event
FROM
events_table as "events"
WHERE
event_type IN (15, 16, 17, 18, 19) ) events_subquery_2)
UNION
(SELECT *
FROM
(SELECT
"events"."user_id", "events"."time", 2 AS event
FROM
events_table as "events"
WHERE
event_type IN (20, 21, 22, 23, 24, 25) ) events_subquery_3)
UNION
(SELECT *
FROM
(SELECT 1, now(), 3 AS event) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q"
) as final_query
GROUP BY types
ORDER BY types;
SET citus.enable_router_execution TO true;

View File

@ -117,23 +117,303 @@ SELECT l_suppkey, count(*) FROM
GROUP BY l_suppkey, l_shipdate) supps
GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5;
-- repartition query on view with single table subquery
CREATE VIEW supp_count_view AS SELECT * FROM (SELECT l_suppkey, count(*) FROM lineitem_hash_part GROUP BY 1) s1;
SELECT * FROM supp_count_view ORDER BY 2 DESC, 1 LIMIT 10;
SET citus.task_executor_type to DEFAULT;
-- create a view with aggregate
CREATE VIEW lineitems_by_shipping_method AS
SELECT l_shipmode, count(*) as cnt FROM lineitem_hash_part GROUP BY 1;
-- following will fail due to non-flattening of subquery due to GROUP BY
-- following will fail due to non GROUP BY of partition key
SELECT * FROM lineitems_by_shipping_method;
-- create a view with group by on partition column
CREATE VIEW lineitems_by_orderkey AS
SELECT l_orderkey, count(*) FROM lineitem_hash_part GROUP BY 1;
SELECT
l_orderkey, count(*)
FROM
lineitem_hash_part
GROUP BY 1;
-- this will also fail due to same reason
SELECT * FROM lineitems_by_orderkey;
-- this should work since we're able to push down this query
SELECT * FROM lineitems_by_orderkey ORDER BY 2 DESC, 1 ASC LIMIT 10;
-- however it would work if it is made router plannable
-- it would also work since it is made router plannable
SELECT * FROM lineitems_by_orderkey WHERE l_orderkey = 100;
DROP TABLE temp_lineitem CASCADE;
DROP VIEW supp_count_view;
DROP VIEW lineitems_by_orderkey;
DROP VIEW lineitems_by_shipping_method;
DROP VIEW air_shipped_lineitems;
DROP VIEW priority_lineitem;
DROP VIEW priority_orders;
-- new tests for real time use case including views and subqueries
-- create view to display recent user who has an activity after a timestamp
CREATE VIEW recent_users AS
SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC;
SELECT * FROM recent_users;
-- create a view for recent_events
CREATE VIEW recent_events AS
SELECT user_id, time FROM events_table
WHERE time > '2014-01-20 01:45:49.978738'::timestamp;
SELECT count(*) FROM recent_events;
-- count number of events of recent_users
SELECT count(*) FROM recent_users ru JOIN events_table et ON (ru.user_id = et.user_id);
-- count number of events of per recent users order by count
SELECT ru.user_id, count(*)
FROM recent_users ru
JOIN events_table et
ON (ru.user_id = et.user_id)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1;
-- the same query with a left join however, it would still generate the same result
SELECT ru.user_id, count(*)
FROM recent_users ru
LEFT JOIN events_table et
ON (ru.user_id = et.user_id)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1;
-- query wrapped inside a subquery, it needs another top level order by
SELECT * FROM
(SELECT ru.user_id, count(*)
FROM recent_users ru
JOIN events_table et
ON (ru.user_id = et.user_id)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1) s1
ORDER BY 2 DESC, 1;
-- non-partition key joins are not supported inside subquery
SELECT * FROM
(SELECT ru.user_id, count(*)
FROM recent_users ru
JOIN events_table et
ON (ru.user_id = et.event_type)
GROUP BY ru.user_id
ORDER BY 2 DESC, 1) s1
ORDER BY 2 DESC, 1;
-- join between views
-- recent users who has an event in recent events
SELECT ru.user_id FROM recent_users ru JOIN recent_events re USING(user_id) GROUP BY ru.user_id ORDER BY ru.user_id;
-- outer join inside a subquery
-- recent_events who are not done by recent users
SELECT count(*) FROM (
SELECT re.*, ru.user_id AS recent_user
FROM recent_events re LEFT JOIN recent_users ru USING(user_id)) reu
WHERE recent_user IS NULL;
-- same query with anti-join
SELECT count(*)
FROM recent_events re LEFT JOIN recent_users ru ON(ru.user_id = re.user_id)
WHERE ru.user_id IS NULL;
-- join between view and table
-- users who has recent activity and they have an entry with value_1 is less than 15
SELECT ut.* FROM recent_users ru JOIN users_table ut USING (user_id) WHERE ut.value_1 < 15 ORDER BY 1,2;
-- determine if a recent user has done a given event type or not
SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event
FROM recent_users ru
LEFT JOIN events_table et
ON(ru.user_id = et.user_id AND et.event_type = 625)
ORDER BY 2 DESC, 1;
-- view vs table join wrapped inside a subquery
SELECT * FROM
(SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event
FROM recent_users ru
LEFT JOIN events_table et
ON(ru.user_id = et.user_id AND et.event_type = 625)
) s1
ORDER BY 2 DESC, 1;
-- event vs table non-partition-key join is not supported
SELECT * FROM
(SELECT ru.user_id, CASE WHEN et.user_id IS NULL THEN 'NO' ELSE 'YES' END as done_event
FROM recent_users ru
LEFT JOIN events_table et
ON(ru.user_id = et.event_type)
) s1
ORDER BY 2 DESC, 1;
-- create a select only view
CREATE VIEW selected_users AS SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150;
CREATE VIEW recent_selected_users AS SELECT su.* FROM selected_users su JOIN recent_users ru USING(user_id);
SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1;
-- this would be supported when we implement where partition_key in (subquery) support
SELECT et.* FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users);
-- it is supported when it is a router query
SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90);
-- expected this to work but it did not
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users);
-- wrapping it inside a SELECT * works
SELECT *
FROM (
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10
ORDER BY user_id;
-- union all also works for views
SELECT *
FROM (
(SELECT user_id FROM recent_users)
UNION ALL
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10
ORDER BY user_id;
SELECT count(*)
FROM (
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10;
-- expected this to work but it does not
SELECT count(*)
FROM (
(SELECT user_id FROM recent_users)
UNION ALL
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10;
-- expand view definitions and re-run last 2 queries
SELECT count(*)
FROM (
(SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa
)
UNION
(SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u
WHERE user_id < 15 AND user_id > 10;
SELECT count(*)
FROM (
(SELECT user_id FROM (SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
HAVING max(time) > '2014-01-21 05:45:49.978738'::timestamp order by 2 DESC) aa
)
UNION ALL
(SELECT user_id FROM (SELECT * FROM users_table WHERE value_1 >= 120 and value_1 <150) bb) ) u
WHERE user_id < 15 AND user_id > 10;
-- test distinct
-- distinct is supported if it is on a partition key
CREATE VIEW distinct_user_with_value_1_15 AS SELECT DISTINCT user_id FROM users_table WHERE value_1 = 15;
SELECT * FROM distinct_user_with_value_1_15 ORDER BY user_id;
-- distinct is not supported if it is on a non-partition key
CREATE VIEW distinct_value_1 AS SELECT DISTINCT value_1 FROM users_table WHERE value_2 = 15;
SELECT * FROM distinct_value_1;
-- CTEs are not supported even if they are on views
CREATE VIEW cte_view_1 AS
WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 15) SELECT * FROM c1 WHERE value_2 < 500;
SELECT * FROM cte_view_1;
-- this is single shard query but still not supported since it has view + cte
-- router planner can't detect it
SELECT * FROM cte_view_1 WHERE user_id = 8;
-- if CTE itself prunes down to a single shard than the view is supported (router plannable)
CREATE VIEW cte_view_2 AS
WITH c1 AS (SELECT * FROM users_table WHERE user_id = 8) SELECT * FROM c1 WHERE value_1 = 15;
SELECT * FROM cte_view_2;
CREATE VIEW router_view AS SELECT * FROM users_table WHERE user_id = 2;
-- router plannable
SELECT user_id FROM router_view GROUP BY 1;
-- There is a known issue with router plannable subqueries joined with non-router
-- plannable subqueries. Following tests should be uncommented when we fix it
-- join a router view (not implement error)
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id);
-- it still does not work when converted to 2 subquery join
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id);
-- views are completely removed and still it does not work
-- SELECT * FROM
-- (SELECT user_id FROM (SELECT * FROM users_table WHERE user_id = 2) rv1 GROUP BY 1) rv2
-- JOIN (SELECT user_id, time FROM events_table
-- WHERE time > '2014-01-20 01:45:49.978738'::timestamp) re
-- USING (user_id);
-- views with limits
CREATE VIEW recent_10_users AS
SELECT user_id, max(time) as lastseen FROM users_table
GROUP BY user_id
ORDER BY lastseen DESC
LIMIT 10;
-- this is not supported since it has limit in it and subquery_pushdown is not set
SELECT * FROM recent_10_users;
SET citus.subquery_pushdown to ON;
-- still not supported since outer query does not have limit
-- it shows a different (subquery with single relation) error message
SELECT * FROM recent_10_users;
-- now it displays more correct error message
SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id);
-- now both are supported when there is a limit on the outer most query
SELECT * FROM recent_10_users ORDER BY lastseen DESC LIMIT 10;
SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10;
RESET citus.subquery_pushdown;
-- explain tests
EXPLAIN (COSTS FALSE) SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1;
EXPLAIN (COSTS FALSE) SELECT *
FROM (
(SELECT user_id FROM recent_users)
UNION
(SELECT user_id FROM selected_users) ) u
WHERE user_id < 15 AND user_id > 10
ORDER BY user_id;
EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10;
SET citus.subquery_pushdown to ON;
EXPLAIN (COSTS FALSE) SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id) ORDER BY et.time DESC LIMIT 10;
RESET citus.subquery_pushdown;
DROP VIEW recent_10_users;
DROP VIEW router_view;
DROP VIEW cte_view_2;
DROP VIEW cte_view_1;
DROP VIEW distinct_value_1;
DROP VIEW distinct_user_with_value_1_15;
DROP VIEW recent_selected_users;
DROP VIEW selected_users;
DROP VIEW recent_events;
DROP VIEW recent_users;