Merge branch 'master' into fix-cache-invalidation-crash

pull/1363/head
Brian Cloutier 2017-04-29 04:15:47 +03:00 committed by GitHub
commit 6fc69f1175
40 changed files with 15061 additions and 2052 deletions

View File

@ -303,7 +303,7 @@ uint32
WorkerGetLiveNodeCount(void)
{
List *workerNodeList = ActiveWorkerNodeList();
uint32 liveWorkerCount = workerNodeList->length;
uint32 liveWorkerCount = list_length(workerNodeList);
return liveWorkerCount;
}

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

@ -72,8 +72,7 @@ static PlannedStmt * FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *cust
static void CheckNodeIsDumpable(Node *node);
static List * CopyPlanParamList(List *originalPlanParamList);
static PlannerRestrictionContext * CreateAndPushPlannerRestrictionContext(void);
static RelationRestrictionContext * CurrentRelationRestrictionContext(void);
static JoinRestrictionContext * CurrentJoinRestrictionContext(void);
static PlannerRestrictionContext * CurrentPlannerRestrictionContext(void);
static void PopPlannerRestrictionContext(void);
static bool HasUnresolvedExternParamsWalker(Node *expression, ParamListInfo boundParams);
@ -310,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);
/*
@ -324,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);
@ -658,20 +658,40 @@ multi_join_restriction_hook(PlannerInfo *root,
JoinType jointype,
JoinPathExtraData *extra)
{
JoinRestrictionContext *joinContext = NULL;
JoinRestriction *joinRestriction = palloc0(sizeof(JoinRestriction));
PlannerRestrictionContext *plannerRestrictionContext = NULL;
JoinRestrictionContext *joinRestrictionContext = NULL;
JoinRestriction *joinRestriction = NULL;
MemoryContext restrictionsMemoryContext = NULL;
MemoryContext oldMemoryContext = NULL;
List *restrictInfoList = NIL;
restrictInfoList = extra->restrictlist;
joinContext = CurrentJoinRestrictionContext();
Assert(joinContext != NULL);
/*
* Use a memory context that's guaranteed to live long enough, could be
* called in a more shorted lived one (e.g. with GEQO).
*/
plannerRestrictionContext = CurrentPlannerRestrictionContext();
restrictionsMemoryContext = plannerRestrictionContext->memoryContext;
oldMemoryContext = MemoryContextSwitchTo(restrictionsMemoryContext);
/*
* We create a copy of restrictInfoList because it may be created in a memory
* context which will be deleted when we still need it, thus we create a copy
* of it in our memory context.
*/
restrictInfoList = copyObject(extra->restrictlist);
joinRestrictionContext = plannerRestrictionContext->joinRestrictionContext;
Assert(joinRestrictionContext != NULL);
joinRestriction = palloc0(sizeof(JoinRestriction));
joinRestriction->joinType = jointype;
joinRestriction->joinRestrictInfoList = restrictInfoList;
joinRestriction->plannerInfo = root;
joinContext->joinRestrictionList =
lappend(joinContext->joinRestrictionList, joinRestriction);
joinRestrictionContext->joinRestrictionList =
lappend(joinRestrictionContext->joinRestrictionList, joinRestriction);
MemoryContextSwitchTo(oldMemoryContext);
}
@ -684,7 +704,10 @@ void
multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo, Index index,
RangeTblEntry *rte)
{
RelationRestrictionContext *restrictionContext = NULL;
PlannerRestrictionContext *plannerRestrictionContext = NULL;
RelationRestrictionContext *relationRestrictionContext = NULL;
MemoryContext restrictionsMemoryContext = NULL;
MemoryContext oldMemoryContext = NULL;
RelationRestriction *relationRestriction = NULL;
DistTableCacheEntry *cacheEntry = NULL;
bool distributedTable = false;
@ -695,12 +718,17 @@ multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo, Index
return;
}
/*
* Use a memory context that's guaranteed to live long enough, could be
* called in a more shorted lived one (e.g. with GEQO).
*/
plannerRestrictionContext = CurrentPlannerRestrictionContext();
restrictionsMemoryContext = plannerRestrictionContext->memoryContext;
oldMemoryContext = MemoryContextSwitchTo(restrictionsMemoryContext);
distributedTable = IsDistributedTable(rte->relid);
localTable = !distributedTable;
restrictionContext = CurrentRelationRestrictionContext();
Assert(restrictionContext != NULL);
relationRestriction = palloc0(sizeof(RelationRestriction));
relationRestriction->index = index;
relationRestriction->relationId = rte->relid;
@ -718,8 +746,9 @@ multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo, Index
CopyPlanParamList(root->parent_root->plan_params);
}
restrictionContext->hasDistributedRelation |= distributedTable;
restrictionContext->hasLocalRelation |= localTable;
relationRestrictionContext = plannerRestrictionContext->relationRestrictionContext;
relationRestrictionContext->hasDistributedRelation |= distributedTable;
relationRestrictionContext->hasLocalRelation |= localTable;
/*
* We're also keeping track of whether all participant
@ -729,12 +758,14 @@ multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo, Index
{
cacheEntry = DistributedTableCacheEntry(rte->relid);
restrictionContext->allReferenceTables &=
relationRestrictionContext->allReferenceTables &=
(cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE);
}
restrictionContext->relationRestrictionList =
lappend(restrictionContext->relationRestrictionList, relationRestriction);
relationRestrictionContext->relationRestrictionList =
lappend(relationRestrictionContext->relationRestrictionList, relationRestriction);
MemoryContextSwitchTo(oldMemoryContext);
}
@ -766,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)
@ -784,6 +814,8 @@ CreateAndPushPlannerRestrictionContext(void)
plannerRestrictionContext->joinRestrictionContext =
palloc0(sizeof(JoinRestrictionContext));
plannerRestrictionContext->memoryContext = CurrentMemoryContext;
/* we'll apply logical AND as we add tables */
plannerRestrictionContext->relationRestrictionContext->allReferenceTables = true;
@ -795,44 +827,20 @@ CreateAndPushPlannerRestrictionContext(void)
/*
* CurrentRelationRestrictionContext returns the the last restriction context from the
* relationRestrictionContext list.
* CurrentRestrictionContext returns the the most recently added
* PlannerRestrictionContext from the plannerRestrictionContextList list.
*/
static RelationRestrictionContext *
CurrentRelationRestrictionContext(void)
static PlannerRestrictionContext *
CurrentPlannerRestrictionContext(void)
{
PlannerRestrictionContext *plannerRestrictionContext = NULL;
RelationRestrictionContext *relationRestrictionContext = NULL;
Assert(plannerRestrictionContextList != NIL);
plannerRestrictionContext =
(PlannerRestrictionContext *) linitial(plannerRestrictionContextList);
relationRestrictionContext = plannerRestrictionContext->relationRestrictionContext;
return relationRestrictionContext;
}
/*
* CurrentJoinRestrictionContext returns the the last restriction context from the
* list.
*/
static JoinRestrictionContext *
CurrentJoinRestrictionContext(void)
{
PlannerRestrictionContext *plannerRestrictionContext = NULL;
JoinRestrictionContext *joinRestrictionContext = NULL;
Assert(plannerRestrictionContextList != NIL);
plannerRestrictionContext =
(PlannerRestrictionContext *) linitial(plannerRestrictionContextList);
joinRestrictionContext = plannerRestrictionContext->joinRestrictionContext;
return joinRestrictionContext;
return plannerRestrictionContext;
}

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

@ -186,6 +186,9 @@ static List * PruneWithBoundaries(DistTableCacheEntry *cacheEntry,
static List * ExhaustivePrune(DistTableCacheEntry *cacheEntry,
ClauseWalkerContext *context,
PruningInstance *prune);
static bool ExhaustivePruneOne(ShardInterval *curInterval,
ClauseWalkerContext *context,
PruningInstance *prune);
static int UpperShardBoundary(Datum partitionColumnValue,
ShardInterval **shardIntervalCache,
int shardCount, FunctionCallInfoData *compareFunction,
@ -894,14 +897,26 @@ PruneOne(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
}
/*
* If previous pruning method yielded a single shard, we could also
* attempt range based pruning to exclude it further. But that seems
* rarely useful in practice, and thus likely a waste of runtime and code
* complexity.
* If previous pruning method yielded a single shard, and the table is not
* hash partitioned, attempt range based pruning to exclude it further.
*
* That's particularly important in particular for subquery pushdown,
* where it's very common to have a user specified equality restriction,
* and a range based restriction for shard boundaries, added by the
* subquery machinery.
*/
if (shardInterval)
{
return list_make1(shardInterval);
if (context->partitionMethod != DISTRIBUTE_BY_HASH &&
ExhaustivePruneOne(shardInterval, context, prune))
{
return NIL;
}
else
{
/* no chance to prune further, return */
return list_make1(shardInterval);
}
}
/*
@ -1158,9 +1173,19 @@ PruneWithBoundaries(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *contex
}
if (prune->greaterConsts)
{
lowerBound = prune->greaterConsts->constvalue;
lowerBoundInclusive = false;
hasLowerBound = true;
/*
* Use the more restrictive one, if both greater and greaterEqual
* constraints are specified.
*/
if (!hasLowerBound ||
PerformValueCompare(compareFunctionCall,
prune->greaterConsts->constvalue,
lowerBound) >= 0)
{
lowerBound = prune->greaterConsts->constvalue;
lowerBoundInclusive = false;
hasLowerBound = true;
}
}
if (prune->lessEqualConsts)
{
@ -1170,9 +1195,19 @@ PruneWithBoundaries(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *contex
}
if (prune->lessConsts)
{
upperBound = prune->lessConsts->constvalue;
upperBoundInclusive = false;
hasUpperBound = true;
/*
* Use the more restrictive one, if both less and lessEqual
* constraints are specified.
*/
if (!hasUpperBound ||
PerformValueCompare(compareFunctionCall,
prune->lessConsts->constvalue,
upperBound) <= 0)
{
upperBound = prune->lessConsts->constvalue;
upperBoundInclusive = false;
hasUpperBound = true;
}
}
Assert(hasLowerBound || hasUpperBound);
@ -1232,88 +1267,104 @@ ExhaustivePrune(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
PruningInstance *prune)
{
List *remainingShardList = NIL;
FunctionCallInfo compareFunctionCall = &context->compareIntervalFunctionCall;
int shardCount = cacheEntry->shardIntervalArrayLength;
ShardInterval **sortedShardIntervalArray = cacheEntry->sortedShardIntervalArray;
int curIdx = 0;
for (curIdx = 0; curIdx < shardCount; curIdx++)
{
Datum compareWith = 0;
ShardInterval *curInterval = sortedShardIntervalArray[curIdx];
/* NULL boundaries can't be compared to */
if (!curInterval->minValueExists || !curInterval->maxValueExists)
if (!ExhaustivePruneOne(curInterval, context, prune))
{
remainingShardList = lappend(remainingShardList, curInterval);
continue;
}
if (prune->equalConsts)
{
compareWith = prune->equalConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
compareWith,
curInterval->minValue) < 0)
{
continue;
}
if (PerformValueCompare(compareFunctionCall,
compareWith,
curInterval->maxValue) > 0)
{
continue;
}
}
if (prune->greaterEqualConsts)
{
compareWith = prune->greaterEqualConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->maxValue,
compareWith) < 0)
{
continue;
}
}
if (prune->greaterConsts)
{
compareWith = prune->greaterConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->maxValue,
compareWith) <= 0)
{
continue;
}
}
if (prune->lessEqualConsts)
{
compareWith = prune->lessEqualConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->minValue,
compareWith) > 0)
{
continue;
}
}
if (prune->lessConsts)
{
compareWith = prune->lessConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->minValue,
compareWith) >= 0)
{
continue;
}
}
remainingShardList = lappend(remainingShardList, curInterval);
}
return remainingShardList;
}
/*
* ExhaustivePruneOne returns true if curInterval is pruned away, false
* otherwise.
*/
static bool
ExhaustivePruneOne(ShardInterval *curInterval,
ClauseWalkerContext *context,
PruningInstance *prune)
{
FunctionCallInfo compareFunctionCall = &context->compareIntervalFunctionCall;
Datum compareWith = 0;
/* NULL boundaries can't be compared to */
if (!curInterval->minValueExists || !curInterval->maxValueExists)
{
return false;
}
if (prune->equalConsts)
{
compareWith = prune->equalConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
compareWith,
curInterval->minValue) < 0)
{
return true;
}
if (PerformValueCompare(compareFunctionCall,
compareWith,
curInterval->maxValue) > 0)
{
return true;
}
}
if (prune->greaterEqualConsts)
{
compareWith = prune->greaterEqualConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->maxValue,
compareWith) < 0)
{
return true;
}
}
if (prune->greaterConsts)
{
compareWith = prune->greaterConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->maxValue,
compareWith) <= 0)
{
return true;
}
}
if (prune->lessEqualConsts)
{
compareWith = prune->lessEqualConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->minValue,
compareWith) > 0)
{
return true;
}
}
if (prune->lessConsts)
{
compareWith = prune->lessConsts->constvalue;
if (PerformValueCompare(compareFunctionCall,
curInterval->minValue,
compareWith) >= 0)
{
return true;
}
}
return false;
}

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

@ -58,6 +58,7 @@ typedef struct PlannerRestrictionContext
{
RelationRestrictionContext *relationRestrictionContext;
JoinRestrictionContext *joinRestrictionContext;
MemoryContext memoryContext;
} PlannerRestrictionContext;
typedef struct RelationShard
@ -70,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,
@ -646,8 +1132,10 @@ GROUP BY
count_pay
ORDER BY
count_pay;
-- Lateral join subquery pushdown
-- set subquery_pushdown since there is limit in the query
SET citus.subquery_pushdown to ON;
SELECT
tenant_id,
user_id,
@ -663,6 +1151,7 @@ FROM
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
lastseen
FROM
users
@ -681,8 +1170,7 @@ FROM
FROM
events
WHERE
(composite_id).tenant_id = subquery_top.tenant_id AND
(composite_id).user_id = subquery_top.user_id
(composite_id) = subquery_top.composite_id
ORDER BY
event_time DESC
LIMIT
@ -697,7 +1185,8 @@ ORDER BY
user_lastseen DESC
LIMIT
10;
-- reset the flag for next query
SET citus.subquery_pushdown to OFF;
-- Same queries above with explain
-- Simple join subquery pushdown
@ -720,8 +1209,7 @@ FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type IN ('click', 'submit', 'pay')) AS subquery
@ -745,14 +1233,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -760,14 +1248,14 @@ FROM
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id) as composite_id,
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -785,8 +1273,7 @@ FROM
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay') AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -810,14 +1297,14 @@ EXPLAIN (COSTS OFF)
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>1'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'click')
@ -825,14 +1312,14 @@ EXPLAIN (COSTS OFF)
(SELECT
(users.composite_id).tenant_id,
(users.composite_id).user_id,
(users.composite_id),
'action=>2'AS event,
events.event_time
FROM
users,
events
WHERE
(users.composite_id).tenant_id = (events.composite_id).tenant_id AND
(users.composite_id).user_id = (events.composite_id).user_id AND
(users.composite_id) = (events.composite_id) AND
users.composite_id >= '(1, -9223372036854775808)'::user_composite_type AND
users.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'submit')
@ -841,6 +1328,7 @@ EXPLAIN (COSTS OFF)
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
COUNT(*) AS count_pay
FROM
events
@ -849,13 +1337,11 @@ EXPLAIN (COSTS OFF)
events.composite_id <= '(1, 9223372036854775807)'::user_composite_type AND
event_type = 'pay'
GROUP BY
tenant_id,
user_id
composite_id
HAVING
COUNT(*) > 2) AS subquery_2
ON
subquery_1.tenant_id = subquery_2.tenant_id AND
subquery_1.user_id = subquery_2.user_id
subquery_1.composite_id = subquery_2.composite_id
GROUP BY
subquery_1.tenant_id,
subquery_1.user_id,
@ -868,6 +1354,8 @@ ORDER BY
count_pay;
-- Lateral join subquery pushdown
-- set subquery_pushdown due to limit in the query
SET citus.subquery_pushdown to ON;
EXPLAIN (COSTS OFF)
SELECT
tenant_id,
@ -884,6 +1372,7 @@ FROM
(SELECT
(composite_id).tenant_id,
(composite_id).user_id,
composite_id,
lastseen
FROM
users
@ -902,8 +1391,7 @@ FROM
FROM
events
WHERE
(composite_id).tenant_id = subquery_top.tenant_id AND
(composite_id).user_id = subquery_top.user_id
(composite_id) = subquery_top.composite_id
ORDER BY
event_time DESC
LIMIT
@ -918,5 +1406,5 @@ ORDER BY
user_lastseen DESC
LIMIT
10;
SET citus.subquery_pushdown to OFF;
SET citus.enable_router_execution TO 'true';

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;