mirror of https://github.com/citusdata/citus.git
Merge branch 'master' into fix-cache-invalidation-crash
commit
6fc69f1175
|
@ -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
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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 *)
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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 };
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
@ -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;
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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';
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
@ -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';
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
@ -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;
|
|
@ -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;
|
||||
|
|
Loading…
Reference in New Issue