mirror of https://github.com/citusdata/citus.git
Merge pull request #1876 from citusdata/subqueries
Recursively plan subqueries that are not safe to pushdown (i.e., requires merge step)pull/1883/head
commit
54ccfb24be
|
@ -593,7 +593,6 @@ CreateDistributedSelectPlan(uint64 planId, Query *originalQuery, Query *query,
|
|||
|
||||
DistributedPlan *distributedPlan = NULL;
|
||||
MultiTreeRoot *logicalPlan = NULL;
|
||||
DeferredErrorMessage *error = NULL;
|
||||
List *subPlanList = NIL;
|
||||
|
||||
/*
|
||||
|
@ -646,14 +645,10 @@ CreateDistributedSelectPlan(uint64 planId, Query *originalQuery, Query *query,
|
|||
|
||||
/*
|
||||
* Plan subqueries and CTEs that cannot be pushed down by recursively
|
||||
* calling the planner and add the resulting plans to subPlanList.
|
||||
* calling the planner and return the resulting plans to subPlanList.
|
||||
*/
|
||||
error = RecursivelyPlanSubqueriesAndCTEs(originalQuery, plannerRestrictionContext,
|
||||
planId, &subPlanList);
|
||||
if (error != NULL)
|
||||
{
|
||||
RaiseDeferredError(error, ERROR);
|
||||
}
|
||||
subPlanList = GenerateSubplansForSubqueriesAndCTEs(planId, originalQuery,
|
||||
plannerRestrictionContext);
|
||||
|
||||
/*
|
||||
* If subqueries were recursively planned then we need to replan the query
|
||||
|
|
|
@ -33,6 +33,7 @@
|
|||
#include "distributed/distributed_planner.h"
|
||||
#include "distributed/multi_server_executor.h"
|
||||
#include "distributed/remote_commands.h"
|
||||
#include "distributed/recursive_planning.h"
|
||||
#include "distributed/placement_connection.h"
|
||||
#include "distributed/worker_protocol.h"
|
||||
#include "lib/stringinfo.h"
|
||||
|
|
|
@ -80,7 +80,6 @@ typedef MultiNode *(*RuleApplyFunction) (MultiNode *leftNode, MultiNode *rightNo
|
|||
static RuleApplyFunction RuleApplyFunctionArray[JOIN_RULE_LAST] = { 0 }; /* join rules */
|
||||
|
||||
/* Local functions forward declarations */
|
||||
static bool SingleRelationRepartitionSubquery(Query *queryTree);
|
||||
static DeferredErrorMessage * DeferErrorIfUnsupportedSubqueryPushdown(Query *
|
||||
originalQuery,
|
||||
PlannerRestrictionContext
|
||||
|
@ -98,9 +97,6 @@ static bool RangeTableArrayContainsAnyRTEIdentities(RangeTblEntry **rangeTableEn
|
|||
queryRteIdentities);
|
||||
static Relids QueryRteIdentities(Query *queryTree);
|
||||
static DeferredErrorMessage * DeferErrorIfFromClauseRecurs(Query *queryTree);
|
||||
static DeferredErrorMessage * DeferErrorIfCannotPushdownSubquery(Query *subqueryTree,
|
||||
bool
|
||||
outerMostQueryHasLimit);
|
||||
static DeferredErrorMessage * DeferErrorIfUnsupportedUnionQuery(Query *queryTree,
|
||||
bool
|
||||
outerMostQueryHasLimit);
|
||||
|
@ -108,12 +104,10 @@ static bool ExtractSetOperationStatmentWalker(Node *node, List **setOperationLis
|
|||
static DeferredErrorMessage * DeferErrorIfUnsupportedTableCombination(Query *queryTree);
|
||||
static bool WindowPartitionOnDistributionColumn(Query *query);
|
||||
static bool AllTargetExpressionsAreColumnReferences(List *targetEntryList);
|
||||
static bool FindNodeCheckInRangeTableList(List *rtable, bool (*check)(Node *));
|
||||
static bool IsDistributedTableRTE(Node *node);
|
||||
static FieldSelect * CompositeFieldRecursive(Expr *expression, Query *query);
|
||||
static bool FullCompositeFieldList(List *compositeFieldList);
|
||||
static MultiNode * MultiNodeTree(Query *queryTree);
|
||||
static void ErrorIfQueryNotSupported(Query *queryTree);
|
||||
static DeferredErrorMessage * DeferredErrorIfUnsupportedRecurringTuplesJoin(
|
||||
PlannerRestrictionContext *plannerRestrictionContext);
|
||||
static bool ShouldRecurseForRecurringTuplesJoinChecks(RelOptInfo *relOptInfo);
|
||||
|
@ -396,12 +390,17 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree,
|
|||
{
|
||||
MultiNode *multiQueryNode = NULL;
|
||||
DeferredErrorMessage *subqueryPushdownError = NULL;
|
||||
DeferredErrorMessage *unsupportedQueryError = NULL;
|
||||
|
||||
/*
|
||||
* This is a generic error check that applies to both subquery pushdown
|
||||
* and single table repartition subquery.
|
||||
*/
|
||||
ErrorIfQueryNotSupported(originalQuery);
|
||||
unsupportedQueryError = DeferErrorIfQueryNotSupported(originalQuery);
|
||||
if (unsupportedQueryError != NULL)
|
||||
{
|
||||
RaiseDeferredError(unsupportedQueryError, ERROR);
|
||||
}
|
||||
|
||||
/*
|
||||
* In principle, we're first trying subquery pushdown planner. If it fails
|
||||
|
@ -468,7 +467,7 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree,
|
|||
* to ensure that Citus supports the subquery. Also, this function is designed to run
|
||||
* on the original query.
|
||||
*/
|
||||
static bool
|
||||
bool
|
||||
SingleRelationRepartitionSubquery(Query *queryTree)
|
||||
{
|
||||
List *rangeTableIndexList = NULL;
|
||||
|
@ -916,7 +915,7 @@ DeferErrorIfFromClauseRecurs(Query *queryTree)
|
|||
* the join condition must be partition columns.
|
||||
* c. If there is a distinct clause, it must be on the partition column.
|
||||
*
|
||||
* This function is very similar to ErrorIfQueryNotSupported() in logical
|
||||
* This function is very similar to DeferErrorIfQueryNotSupported() in logical
|
||||
* planner, but we don't reuse it, because differently for subqueries we support
|
||||
* a subset of distinct, union and left joins.
|
||||
*
|
||||
|
@ -926,7 +925,7 @@ DeferErrorIfFromClauseRecurs(Query *queryTree)
|
|||
* limit, we let this query to run, but results could be wrong depending on the
|
||||
* features of underlying tables.
|
||||
*/
|
||||
static DeferredErrorMessage *
|
||||
DeferredErrorMessage *
|
||||
DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerMostQueryHasLimit)
|
||||
{
|
||||
bool preconditionsSatisfied = true;
|
||||
|
@ -1513,7 +1512,7 @@ AllTargetExpressionsAreColumnReferences(List *targetEntryList)
|
|||
* FindNodeCheckInRangeTableList relies on FindNodeCheck() but only
|
||||
* considers the range table entries.
|
||||
*/
|
||||
static bool
|
||||
bool
|
||||
FindNodeCheckInRangeTableList(List *rtable, bool (*check)(Node *))
|
||||
{
|
||||
return range_table_walker(rtable, FindNodeCheck, check, QTW_EXAMINE_RTES);
|
||||
|
@ -1760,9 +1759,14 @@ MultiNodeTree(Query *queryTree)
|
|||
MultiProject *projectNode = NULL;
|
||||
MultiExtendedOp *extendedOpNode = NULL;
|
||||
MultiNode *currentTopNode = NULL;
|
||||
DeferredErrorMessage *unsupportedQueryError = NULL;
|
||||
|
||||
/* verify we can perform distributed planning on this query */
|
||||
ErrorIfQueryNotSupported(queryTree);
|
||||
unsupportedQueryError = DeferErrorIfQueryNotSupported(queryTree);
|
||||
if (unsupportedQueryError != NULL)
|
||||
{
|
||||
RaiseDeferredError(unsupportedQueryError, ERROR);
|
||||
}
|
||||
|
||||
/* extract where clause qualifiers and verify we can plan for them */
|
||||
whereClauseList = WhereClauseList(queryTree->jointree);
|
||||
|
@ -2220,8 +2224,8 @@ IsReadIntermediateResultFunction(Node *node)
|
|||
* the given query. The checks in this function will be removed as we support
|
||||
* more functionality in our distributed planning.
|
||||
*/
|
||||
static void
|
||||
ErrorIfQueryNotSupported(Query *queryTree)
|
||||
DeferredErrorMessage *
|
||||
DeferErrorIfQueryNotSupported(Query *queryTree)
|
||||
{
|
||||
char *errorMessage = NULL;
|
||||
bool hasTablesample = false;
|
||||
|
@ -2250,8 +2254,12 @@ ErrorIfQueryNotSupported(Query *queryTree)
|
|||
if (queryTree->hasWindowFuncs)
|
||||
{
|
||||
preconditionsSatisfied = false;
|
||||
errorMessage = "could not run distributed query with window functions";
|
||||
errorHint = filterHint;
|
||||
errorMessage = "could not run distributed query because the window "
|
||||
"function that is used cannot be pushed down";
|
||||
errorHint = "Window functions are supported in two ways. Either add "
|
||||
"an equality filter on the distributed tables' partition "
|
||||
"column or use the window functions inside a subquery with "
|
||||
"a PARTITION BY clause containing the distribution column";
|
||||
}
|
||||
|
||||
if (queryTree->setOperations)
|
||||
|
@ -2329,10 +2337,12 @@ ErrorIfQueryNotSupported(Query *queryTree)
|
|||
if (!preconditionsSatisfied)
|
||||
{
|
||||
bool showHint = ErrorHintRequired(errorHint, queryTree);
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("%s", errorMessage),
|
||||
showHint ? errhint("%s", errorHint) : 0));
|
||||
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
|
||||
errorMessage, NULL,
|
||||
showHint ? errorHint : NULL);
|
||||
}
|
||||
|
||||
return NULL;
|
||||
}
|
||||
|
||||
|
||||
|
@ -2631,10 +2641,12 @@ HasComplexRangeTableType(Query *queryTree)
|
|||
|
||||
/*
|
||||
* Check if the range table in the join tree is a simple relation or a
|
||||
* subquery.
|
||||
* subquery or a function. Note that RTE_FUNCTIONs are handled via (sub)query
|
||||
* pushdown.
|
||||
*/
|
||||
if (rangeTableEntry->rtekind != RTE_RELATION &&
|
||||
rangeTableEntry->rtekind != RTE_SUBQUERY)
|
||||
rangeTableEntry->rtekind != RTE_SUBQUERY &&
|
||||
rangeTableEntry->rtekind != RTE_FUNCTION)
|
||||
{
|
||||
hasComplexRangeTableType = true;
|
||||
}
|
||||
|
@ -3847,9 +3859,14 @@ SubqueryPushdownMultiNodeTree(Query *queryTree)
|
|||
Query *pushedDownQuery = NULL;
|
||||
List *subqueryTargetEntryList = NIL;
|
||||
List *havingClauseColumnList = NIL;
|
||||
DeferredErrorMessage *unsupportedQueryError = NULL;
|
||||
|
||||
/* verify we can perform distributed planning on this query */
|
||||
ErrorIfQueryNotSupported(queryTree);
|
||||
unsupportedQueryError = DeferErrorIfQueryNotSupported(queryTree);
|
||||
if (unsupportedQueryError != NULL)
|
||||
{
|
||||
RaiseDeferredError(unsupportedQueryError, ERROR);
|
||||
}
|
||||
|
||||
/*
|
||||
* We would be creating a new Query and pushing down top level query's
|
||||
|
|
|
@ -2093,7 +2093,19 @@ SubquerySqlTaskList(Job *job, PlannerRestrictionContext *plannerRestrictionConte
|
|||
*/
|
||||
if (targetCacheEntry == NULL)
|
||||
{
|
||||
RangeTblEntry *rangeTableEntry = (RangeTblEntry *) linitial(rangeTableList);
|
||||
RangeTblEntry *rangeTableEntry = NULL;
|
||||
|
||||
if (list_length(rangeTableList) == 0)
|
||||
{
|
||||
/*
|
||||
* User disabled the router planner and forced planner go through
|
||||
* subquery pushdown, but we cannot continue anymore.
|
||||
*/
|
||||
ereport(ERROR, (errmsg("cannot handle complex subqueries when the "
|
||||
"router executor is disabled")));
|
||||
}
|
||||
|
||||
rangeTableEntry = (RangeTblEntry *) linitial(rangeTableList);
|
||||
relationId = rangeTableEntry->relid;
|
||||
targetCacheEntry = DistributedTableCacheEntry(relationId);
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@
|
|||
#include "postgres.h"
|
||||
|
||||
#include "catalog/pg_type.h"
|
||||
#include "catalog/pg_class.h"
|
||||
#include "distributed/citus_nodes.h"
|
||||
#include "distributed/citus_ruleutils.h"
|
||||
#include "distributed/distributed_planner.h"
|
||||
|
@ -62,6 +63,7 @@
|
|||
#include "distributed/multi_router_planner.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/recursive_planning.h"
|
||||
#include "distributed/multi_server_executor.h"
|
||||
#include "distributed/relation_restriction_equivalence.h"
|
||||
#include "lib/stringinfo.h"
|
||||
#include "optimizer/planner.h"
|
||||
|
@ -86,6 +88,7 @@ typedef struct RecursivePlanningContext
|
|||
PlannerRestrictionContext *plannerRestrictionContext;
|
||||
} RecursivePlanningContext;
|
||||
|
||||
|
||||
/*
|
||||
* CteReferenceWalkerContext is used to collect CTE references in
|
||||
* CteReferenceListWalker.
|
||||
|
@ -107,8 +110,16 @@ typedef struct VarLevelsUpWalkerContext
|
|||
|
||||
|
||||
/* local function forward declarations */
|
||||
static DeferredErrorMessage * RecursivelyPlanSubqueriesAndCTEs(Query *query,
|
||||
RecursivePlanningContext *
|
||||
context);
|
||||
static DeferredErrorMessage * RecursivelyPlanCTEs(Query *query,
|
||||
RecursivePlanningContext *context);
|
||||
static bool RecursivelyPlanSubqueryWalker(Node *node, RecursivePlanningContext *context);
|
||||
static bool ShouldRecursivelyPlanSubquery(Query *subquery);
|
||||
static bool IsLocalTableRTE(Node *node);
|
||||
static void RecursivelyPlanSubquery(Query *subquery,
|
||||
RecursivePlanningContext *planningContext);
|
||||
static DistributedSubPlan * CreateDistributedSubPlan(uint32 subPlanId,
|
||||
Query *subPlanQuery);
|
||||
static bool CteReferenceListWalker(Node *node, CteReferenceWalkerContext *context);
|
||||
|
@ -118,6 +129,39 @@ static bool ContainsReferencesToOuterQueryWalker(Node *node,
|
|||
static Query * BuildSubPlanResultQuery(Query *subquery, uint64 planId, uint32 subPlanId);
|
||||
|
||||
|
||||
/*
|
||||
* GenerateSubplansForSubqueriesAndCTEs is a wrapper around RecursivelyPlanSubqueriesAndCTEs.
|
||||
* The function returns the subplans if necessary. For the details of when/how subplans are
|
||||
* generated, see RecursivelyPlanSubqueriesAndCTEs().
|
||||
*
|
||||
* Note that the input originalQuery query is modified if any subplans are generated.
|
||||
*/
|
||||
List *
|
||||
GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery,
|
||||
PlannerRestrictionContext *plannerRestrictionContext)
|
||||
{
|
||||
RecursivePlanningContext context;
|
||||
DeferredErrorMessage *error = NULL;
|
||||
|
||||
/*
|
||||
* Plan subqueries and CTEs that cannot be pushed down by recursively
|
||||
* calling the planner and add the resulting plans to subPlanList.
|
||||
*/
|
||||
context.level = 0;
|
||||
context.planId = planId;
|
||||
context.subPlanList = NIL;
|
||||
context.plannerRestrictionContext = plannerRestrictionContext;
|
||||
|
||||
error = RecursivelyPlanSubqueriesAndCTEs(originalQuery, &context);
|
||||
if (error != NULL)
|
||||
{
|
||||
RaiseDeferredError(error, ERROR);
|
||||
}
|
||||
|
||||
return context.subPlanList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* RecursivelyPlanSubqueriesAndCTEs finds subqueries and CTEs that cannot be pushed down to
|
||||
* workers directly and instead plans them by recursively calling the planner and
|
||||
|
@ -132,20 +176,12 @@ static Query * BuildSubPlanResultQuery(Query *subquery, uint64 planId, uint32 su
|
|||
* If recursive planning results in an error then the error is returned. Otherwise, the
|
||||
* subplans will be added to subPlanList.
|
||||
*/
|
||||
DeferredErrorMessage *
|
||||
RecursivelyPlanSubqueriesAndCTEs(Query *query,
|
||||
PlannerRestrictionContext *plannerRestrictionContext,
|
||||
uint64 planId, List **subPlanList)
|
||||
static DeferredErrorMessage *
|
||||
RecursivelyPlanSubqueriesAndCTEs(Query *query, RecursivePlanningContext *context)
|
||||
{
|
||||
DeferredErrorMessage *error = NULL;
|
||||
RecursivePlanningContext context;
|
||||
|
||||
context.level = 0;
|
||||
context.planId = planId;
|
||||
context.subPlanList = NIL;
|
||||
context.plannerRestrictionContext = plannerRestrictionContext;
|
||||
|
||||
error = RecursivelyPlanCTEs(query, &context);
|
||||
error = RecursivelyPlanCTEs(query, context);
|
||||
if (error != NULL)
|
||||
{
|
||||
return error;
|
||||
|
@ -166,9 +202,8 @@ RecursivelyPlanSubqueriesAndCTEs(Query *query,
|
|||
return NULL;
|
||||
}
|
||||
|
||||
/* XXX: plan subqueries */
|
||||
|
||||
*subPlanList = context.subPlanList;
|
||||
/* descend into subqueries */
|
||||
query_tree_walker(query, RecursivelyPlanSubqueryWalker, context, 0);
|
||||
|
||||
return NULL;
|
||||
}
|
||||
|
@ -246,7 +281,7 @@ RecursivelyPlanCTEs(Query *query, RecursivePlanningContext *planningContext)
|
|||
|
||||
subPlanId = list_length(planningContext->subPlanList) + 1;
|
||||
|
||||
if (log_min_messages >= DEBUG1)
|
||||
if (log_min_messages <= DEBUG1 || client_min_messages <= DEBUG1)
|
||||
{
|
||||
StringInfo subPlanString = makeStringInfo();
|
||||
pg_get_query_def(subquery, subPlanString);
|
||||
|
@ -316,6 +351,201 @@ RecursivelyPlanCTEs(Query *query, RecursivePlanningContext *planningContext)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* RecursivelyPlanSubqueryWalker recursively finds all the Query nodes and
|
||||
* recursively plans if necessary.
|
||||
*/
|
||||
static bool
|
||||
RecursivelyPlanSubqueryWalker(Node *node, RecursivePlanningContext *context)
|
||||
{
|
||||
if (node == NULL)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
if (IsA(node, Query))
|
||||
{
|
||||
Query *query = (Query *) node;
|
||||
DeferredErrorMessage *error = NULL;
|
||||
|
||||
context->level += 1;
|
||||
|
||||
/*
|
||||
* First, make sure any subqueries and CTEs within this subquery
|
||||
* are recursively planned if necessary.
|
||||
*/
|
||||
error = RecursivelyPlanSubqueriesAndCTEs(query, context);
|
||||
if (error != NULL)
|
||||
{
|
||||
RaiseDeferredError(error, ERROR);
|
||||
}
|
||||
context->level -= 1;
|
||||
|
||||
/*
|
||||
* Recursively plan this subquery if it cannot be pushed down and is
|
||||
* eligible for recursive planning.
|
||||
*/
|
||||
if (ShouldRecursivelyPlanSubquery(query))
|
||||
{
|
||||
RecursivelyPlanSubquery(query, context);
|
||||
}
|
||||
|
||||
/* we're done, no need to recurse anymore for this query */
|
||||
return false;
|
||||
}
|
||||
|
||||
return expression_tree_walker(node, RecursivelyPlanSubqueryWalker, context);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ShouldRecursivelyPlanSubquery decides whether the input subquery should be recursively
|
||||
* planned or not.
|
||||
*
|
||||
* For the details, see the cases in the function.
|
||||
*/
|
||||
static bool
|
||||
ShouldRecursivelyPlanSubquery(Query *subquery)
|
||||
{
|
||||
if (FindNodeCheckInRangeTableList(subquery->rtable, IsLocalTableRTE))
|
||||
{
|
||||
/*
|
||||
* Postgres can always plan queries that don't require distributed planning.
|
||||
* Note that we need to check this first, otherwise the calls to the many other
|
||||
* Citus planner functions would error our due to local relations.
|
||||
*
|
||||
* TODO: We could only successfully create distributed plans with local tables
|
||||
* when the local tables are on the leaf queries and the upper level queries
|
||||
* do not contain any other local tables.
|
||||
*/
|
||||
}
|
||||
else if (DeferErrorIfCannotPushdownSubquery(subquery, false) == NULL)
|
||||
{
|
||||
/*
|
||||
* Citus can pushdown this subquery, no need to recursively
|
||||
* plan which is much expensive than pushdown.
|
||||
*/
|
||||
return false;
|
||||
}
|
||||
else if (TaskExecutorType == MULTI_EXECUTOR_TASK_TRACKER &&
|
||||
SingleRelationRepartitionSubquery(subquery))
|
||||
{
|
||||
/*
|
||||
* Citus can plan this and execute via repartitioning. Thus,
|
||||
* no need to recursively plan.
|
||||
*/
|
||||
return false;
|
||||
}
|
||||
|
||||
/*
|
||||
* Even if we could recursively plan the subquery, we should ensure
|
||||
* that the subquery doesn't contain any references to the outer
|
||||
* queries.
|
||||
*/
|
||||
if (ContainsReferencesToOuterQuery(subquery))
|
||||
{
|
||||
elog(DEBUG2, "skipping recursive planning for the subquery since it "
|
||||
"contains references to outer queries");
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* IsLocalTableRTE gets a node and returns true if the node
|
||||
* is a range table relation entry that points to a local
|
||||
* relation (i.e., not a distributed relation).
|
||||
*/
|
||||
static bool
|
||||
IsLocalTableRTE(Node *node)
|
||||
{
|
||||
RangeTblEntry *rangeTableEntry = NULL;
|
||||
Oid relationId = InvalidOid;
|
||||
|
||||
if (node == NULL)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!IsA(node, RangeTblEntry))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
rangeTableEntry = (RangeTblEntry *) node;
|
||||
if (rangeTableEntry->rtekind != RTE_RELATION)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
if (rangeTableEntry->relkind == RELKIND_VIEW)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
relationId = rangeTableEntry->relid;
|
||||
if (IsDistributedTable(relationId))
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/* local table found */
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* RecursivelyPlanQuery recursively plans a query, replaces it with a
|
||||
* result query and returns the subplan.
|
||||
*/
|
||||
static void
|
||||
RecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *planningContext)
|
||||
{
|
||||
DistributedSubPlan *subPlan = NULL;
|
||||
uint64 planId = planningContext->planId;
|
||||
int subPlanId = 0;
|
||||
|
||||
Query *resultQuery = NULL;
|
||||
Query *debugQuery = NULL;
|
||||
|
||||
/*
|
||||
* Subquery will go through the standard planner, thus to properly deparse it
|
||||
* we keep its copy: debugQuery.
|
||||
*/
|
||||
if (log_min_messages <= DEBUG1 || client_min_messages <= DEBUG1)
|
||||
{
|
||||
debugQuery = copyObject(subquery);
|
||||
}
|
||||
|
||||
/*
|
||||
* Create the subplan and append it to the list in the planning context.
|
||||
*/
|
||||
subPlanId = list_length(planningContext->subPlanList) + 1;
|
||||
|
||||
subPlan = CreateDistributedSubPlan(subPlanId, subquery);
|
||||
planningContext->subPlanList = lappend(planningContext->subPlanList, subPlan);
|
||||
|
||||
resultQuery = BuildSubPlanResultQuery(subquery, planId, subPlanId);
|
||||
|
||||
if (log_min_messages <= DEBUG1 || client_min_messages <= DEBUG1)
|
||||
{
|
||||
StringInfo subqueryString = makeStringInfo();
|
||||
|
||||
pg_get_query_def(debugQuery, subqueryString);
|
||||
|
||||
ereport(DEBUG1, (errmsg("generating subplan " UINT64_FORMAT "_%u for "
|
||||
"subquery %s",
|
||||
planId, subPlanId, subqueryString->data)));
|
||||
}
|
||||
|
||||
/* finally update the input subquery to point the result query */
|
||||
memcpy(subquery, resultQuery, sizeof(Query));
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CreateDistributedSubPlan creates a distributed subplan by recursively calling
|
||||
* the planner from the top, which may either generate a local plan or another
|
||||
|
|
|
@ -92,9 +92,6 @@ extern bool IsModifyDistributedPlan(struct DistributedPlan *distributedPlan);
|
|||
extern bool IsMultiTaskPlan(struct DistributedPlan *distributedPlan);
|
||||
extern bool IsMultiShardModifyPlan(struct DistributedPlan *distributedPlan);
|
||||
extern RangeTblEntry * RemoteScanRangeTableEntry(List *columnNameList);
|
||||
extern char * GenerateResultId(uint64 planId, uint32 subPlanId);
|
||||
|
||||
|
||||
extern int GetRTEIdentity(RangeTblEntry *rte);
|
||||
|
||||
#endif /* DISTRIBUTED_PLANNER_H */
|
||||
|
|
|
@ -15,6 +15,7 @@
|
|||
#define MULTI_LOGICAL_PLANNER_H
|
||||
|
||||
#include "distributed/citus_nodes.h"
|
||||
#include "distributed/errormessage.h"
|
||||
#include "distributed/multi_join_order.h"
|
||||
#include "distributed/relation_restriction_equivalence.h"
|
||||
#include "nodes/nodes.h"
|
||||
|
@ -186,11 +187,16 @@ extern bool SubqueryPushdown;
|
|||
extern MultiTreeRoot * MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree,
|
||||
PlannerRestrictionContext *
|
||||
plannerRestrictionContext);
|
||||
extern bool SingleRelationRepartitionSubquery(Query *queryTree);
|
||||
extern DeferredErrorMessage * DeferErrorIfCannotPushdownSubquery(Query *subqueryTree,
|
||||
bool
|
||||
outerMostQueryHasLimit);
|
||||
extern PlannerRestrictionContext * FilterPlannerRestrictionForQuery(
|
||||
PlannerRestrictionContext *plannerRestrictionContext,
|
||||
Query *query);
|
||||
extern bool SafeToPushdownWindowFunction(Query *query, StringInfo *errorDetail);
|
||||
extern bool TargetListOnPartitionColumn(Query *query, List *targetEntryList);
|
||||
extern bool FindNodeCheckInRangeTableList(List *rtable, bool (*check)(Node *));
|
||||
extern bool ContainsReadIntermediateResultFunction(Node *node);
|
||||
extern MultiNode * ParentNode(MultiNode *multiNode);
|
||||
extern MultiNode * ChildNode(MultiUnaryNode *multiNode);
|
||||
|
@ -205,6 +211,7 @@ extern List * FindNodesOfType(MultiNode *node, int type);
|
|||
extern List * JoinClauseList(List *whereClauseList);
|
||||
extern bool IsJoinClause(Node *clause);
|
||||
extern List * SubqueryEntryList(Query *queryTree);
|
||||
extern DeferredErrorMessage * DeferErrorIfQueryNotSupported(Query *queryTree);
|
||||
extern bool ExtractRangeTableIndexWalker(Node *node, List **rangeTableIndexList);
|
||||
extern List * WhereClauseList(FromExpr *fromExpr);
|
||||
extern List * QualifierList(FromExpr *fromExpr);
|
||||
|
|
|
@ -18,11 +18,9 @@
|
|||
#include "nodes/relation.h"
|
||||
|
||||
|
||||
extern DeferredErrorMessage * RecursivelyPlanSubqueriesAndCTEs(Query *query,
|
||||
PlannerRestrictionContext *
|
||||
plannerRestrictionContext,
|
||||
uint64 planId,
|
||||
List **subPlanList);
|
||||
extern List * GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery,
|
||||
PlannerRestrictionContext *
|
||||
plannerRestrictionContext);
|
||||
extern char * GenerateResultId(uint64 planId, uint32 subPlanId);
|
||||
|
||||
|
||||
|
|
|
@ -346,15 +346,28 @@ SELECT count(*) FROM lineitem, orders WHERE l_orderkey + 1 = o_orderkey;
|
|||
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 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
|
||||
-- the subquery is recursively planned since it contains OFFSET, which is not pushdownable
|
||||
SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq ORDER BY 1 LIMIT 5;
|
||||
o_custkey
|
||||
-----------
|
||||
35
|
||||
37
|
||||
38
|
||||
40
|
||||
41
|
||||
(5 rows)
|
||||
|
||||
-- the subquery is recursively planned since it contains OFFSET, which is not pushdownable
|
||||
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq ORDER BY 1 LIMIT 5;
|
||||
o_orderkey
|
||||
------------
|
||||
69
|
||||
70
|
||||
71
|
||||
96
|
||||
97
|
||||
(5 rows)
|
||||
|
||||
-- Simple LIMIT/OFFSET with ORDER BY
|
||||
SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20;
|
||||
o_orderkey
|
||||
|
|
|
@ -0,0 +1,507 @@
|
|||
--
|
||||
-- MULTI_COMPLEX_EXPRESSIONS
|
||||
--
|
||||
-- Check that we can correctly handle complex expressions and aggregates.
|
||||
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem;
|
||||
?column?
|
||||
------------------------
|
||||
12000.0000000000000000
|
||||
(1 row)
|
||||
|
||||
SELECT sum(l_quantity) / (10 * avg(l_quantity)) FROM lineitem;
|
||||
?column?
|
||||
-----------------------
|
||||
1200.0000000000000000
|
||||
(1 row)
|
||||
|
||||
SELECT (sum(l_quantity) / (10 * avg(l_quantity))) + 11 FROM lineitem;
|
||||
?column?
|
||||
-----------------------
|
||||
1211.0000000000000000
|
||||
(1 row)
|
||||
|
||||
SELECT avg(l_quantity) as average FROM lineitem;
|
||||
average
|
||||
---------------------
|
||||
25.4462500000000000
|
||||
(1 row)
|
||||
|
||||
SELECT 100 * avg(l_quantity) as average_times_hundred FROM lineitem;
|
||||
average_times_hundred
|
||||
-----------------------
|
||||
2544.6250000000000000
|
||||
(1 row)
|
||||
|
||||
SELECT 100 * avg(l_quantity) / 10 as average_times_ten FROM lineitem;
|
||||
average_times_ten
|
||||
----------------------
|
||||
254.4625000000000000
|
||||
(1 row)
|
||||
|
||||
SELECT l_quantity, 10 * count(*) count_quantity FROM lineitem
|
||||
GROUP BY l_quantity ORDER BY count_quantity, l_quantity;
|
||||
l_quantity | count_quantity
|
||||
------------+----------------
|
||||
44.00 | 2150
|
||||
38.00 | 2160
|
||||
45.00 | 2180
|
||||
13.00 | 2190
|
||||
47.00 | 2200
|
||||
29.00 | 2220
|
||||
36.00 | 2230
|
||||
49.00 | 2230
|
||||
3.00 | 2270
|
||||
35.00 | 2280
|
||||
18.00 | 2290
|
||||
31.00 | 2290
|
||||
43.00 | 2290
|
||||
14.00 | 2300
|
||||
16.00 | 2300
|
||||
17.00 | 2300
|
||||
26.00 | 2300
|
||||
7.00 | 2320
|
||||
10.00 | 2340
|
||||
34.00 | 2340
|
||||
15.00 | 2350
|
||||
25.00 | 2360
|
||||
33.00 | 2360
|
||||
42.00 | 2360
|
||||
2.00 | 2370
|
||||
12.00 | 2410
|
||||
37.00 | 2410
|
||||
6.00 | 2420
|
||||
22.00 | 2420
|
||||
1.00 | 2430
|
||||
19.00 | 2430
|
||||
4.00 | 2440
|
||||
20.00 | 2460
|
||||
48.00 | 2460
|
||||
41.00 | 2470
|
||||
24.00 | 2490
|
||||
27.00 | 2490
|
||||
8.00 | 2500
|
||||
11.00 | 2500
|
||||
5.00 | 2540
|
||||
21.00 | 2550
|
||||
32.00 | 2550
|
||||
9.00 | 2580
|
||||
39.00 | 2600
|
||||
46.00 | 2600
|
||||
50.00 | 2600
|
||||
23.00 | 2610
|
||||
30.00 | 2640
|
||||
40.00 | 2690
|
||||
28.00 | 2730
|
||||
(50 rows)
|
||||
|
||||
-- Check that we can handle complex select clause expressions.
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE octet_length(l_comment || l_comment) > 40;
|
||||
count
|
||||
-------
|
||||
8148
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE octet_length(concat(l_comment, l_comment)) > 40;
|
||||
count
|
||||
-------
|
||||
8148
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE octet_length(l_comment) + octet_length('randomtext'::text) > 40;
|
||||
count
|
||||
-------
|
||||
4611
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE octet_length(l_comment) + 10 > 40;
|
||||
count
|
||||
-------
|
||||
4611
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE (l_receiptdate::timestamp - l_shipdate::timestamp) > interval '5 days';
|
||||
count
|
||||
-------
|
||||
10008
|
||||
(1 row)
|
||||
|
||||
-- can push down queries where no columns present on the WHERE clause
|
||||
SELECT count(*) FROM lineitem WHERE random() = -0.1;
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
-- boolean tests can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE (l_partkey > 10000) is true;
|
||||
count
|
||||
-------
|
||||
11423
|
||||
(1 row)
|
||||
|
||||
-- scalar array operator expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE l_partkey = ANY(ARRAY[19353, 19354, 19355]);
|
||||
count
|
||||
-------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- some more scalar array operator expressions
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE l_partkey = ALL(ARRAY[19353]);
|
||||
count
|
||||
-------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- operator expressions involving arrays
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE ARRAY[19353, 19354, 19355] @> ARRAY[l_partkey];
|
||||
count
|
||||
-------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- coerced via io expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE (l_quantity/100)::int::bool::text::bool;
|
||||
count
|
||||
-------
|
||||
260
|
||||
(1 row)
|
||||
|
||||
-- case expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE (CASE WHEN l_orderkey > 4000 THEN l_partkey / 100 > 1 ELSE false END);
|
||||
count
|
||||
-------
|
||||
7948
|
||||
(1 row)
|
||||
|
||||
-- coalesce expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE COALESCE((l_partkey/50000)::bool, false);
|
||||
count
|
||||
-------
|
||||
9122
|
||||
(1 row)
|
||||
|
||||
-- nullif expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE NULLIF((l_partkey/50000)::bool, false);
|
||||
count
|
||||
-------
|
||||
9122
|
||||
(1 row)
|
||||
|
||||
-- null test expressions can be pushed down
|
||||
SELECT count(*) FROM orders
|
||||
WHERE o_comment IS NOT null;
|
||||
count
|
||||
-------
|
||||
2984
|
||||
(1 row)
|
||||
|
||||
-- functions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE isfinite(l_shipdate);
|
||||
count
|
||||
-------
|
||||
12000
|
||||
(1 row)
|
||||
|
||||
-- constant expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE 0 != 0;
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
-- distinct expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE l_partkey IS DISTINCT FROM 50040;
|
||||
count
|
||||
-------
|
||||
11999
|
||||
(1 row)
|
||||
|
||||
-- row compare expression can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE row(l_partkey, 2, 3) > row(2000, 2, 3);
|
||||
count
|
||||
-------
|
||||
11882
|
||||
(1 row)
|
||||
|
||||
-- combination of different expressions can be pushed down
|
||||
SELECT count(*) FROM lineitem
|
||||
WHERE
|
||||
(l_quantity/100)::int::bool::text::bool AND
|
||||
CASE WHEN l_orderkey > 4000 THEN l_partkey / 100 > 1 ELSE false END AND
|
||||
COALESCE((l_partkey/50000)::bool, false) AND
|
||||
NULLIF((l_partkey/50000)::bool, false) AND
|
||||
isfinite(l_shipdate) AND
|
||||
l_partkey IS DISTINCT FROM 50040 AND
|
||||
row(l_partkey, 2, 3) > row(2000, 2, 3);
|
||||
count
|
||||
-------
|
||||
137
|
||||
(1 row)
|
||||
|
||||
-- constant expression in the WHERE clause with a column in the target list
|
||||
SELECT l_linenumber FROM lineitem
|
||||
WHERE
|
||||
1!=0
|
||||
ORDER BY
|
||||
l_linenumber
|
||||
LIMIT 1;
|
||||
l_linenumber
|
||||
--------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- constant expression in the WHERE clause with expressions and a column the target list
|
||||
SELECT count(*) * l_discount as total_discount, count(*), sum(l_tax), l_discount FROM lineitem
|
||||
WHERE
|
||||
1!=0
|
||||
GROUP BY
|
||||
l_discount
|
||||
ORDER BY
|
||||
total_discount DESC, sum(l_tax) DESC;
|
||||
total_discount | count | sum | l_discount
|
||||
----------------+-------+-------+------------
|
||||
104.80 | 1048 | 41.08 | 0.10
|
||||
98.55 | 1095 | 44.15 | 0.09
|
||||
90.64 | 1133 | 45.94 | 0.08
|
||||
71.05 | 1015 | 41.19 | 0.07
|
||||
69.42 | 1157 | 45.75 | 0.06
|
||||
53.60 | 1072 | 42.82 | 0.05
|
||||
43.64 | 1091 | 44.40 | 0.04
|
||||
32.55 | 1085 | 43.30 | 0.03
|
||||
22.22 | 1111 | 45.07 | 0.02
|
||||
11.22 | 1122 | 44.54 | 0.01
|
||||
0.00 | 1071 | 44.00 | 0.00
|
||||
(11 rows)
|
||||
|
||||
-- distinct expressions in the WHERE clause with a column in the target list
|
||||
SELECT l_linenumber FROM lineitem
|
||||
WHERE
|
||||
l_linenumber IS DISTINCT FROM 1 AND
|
||||
l_orderkey IS DISTINCT FROM 8997
|
||||
ORDER BY
|
||||
l_linenumber
|
||||
LIMIT 1;
|
||||
l_linenumber
|
||||
--------------
|
||||
2
|
||||
(1 row)
|
||||
|
||||
-- distinct expressions in the WHERE clause with expressions and a column the target list
|
||||
SELECT max(l_linenumber), min(l_discount), l_receiptdate FROM lineitem
|
||||
WHERE
|
||||
l_linenumber IS DISTINCT FROM 1 AND
|
||||
l_orderkey IS DISTINCT FROM 8997
|
||||
GROUP BY
|
||||
l_receiptdate
|
||||
ORDER BY
|
||||
l_receiptdate
|
||||
LIMIT 1;
|
||||
max | min | l_receiptdate
|
||||
-----+------+---------------
|
||||
3 | 0.07 | 01-09-1992
|
||||
(1 row)
|
||||
|
||||
-- Check that we can handle implicit and explicit join clause definitions.
|
||||
SELECT count(*) FROM lineitem, orders
|
||||
WHERE l_orderkey = o_orderkey AND l_quantity < 5;
|
||||
count
|
||||
-------
|
||||
951
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM lineitem
|
||||
JOIN orders ON l_orderkey = o_orderkey AND l_quantity < 5;
|
||||
count
|
||||
-------
|
||||
951
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM lineitem JOIN orders ON l_orderkey = o_orderkey
|
||||
WHERE l_quantity < 5;
|
||||
count
|
||||
-------
|
||||
951
|
||||
(1 row)
|
||||
|
||||
-- Check that we make sure local joins are between columns only.
|
||||
SELECT count(*) FROM lineitem, orders WHERE l_orderkey + 1 = o_orderkey;
|
||||
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
|
||||
-- the subquery is recursively planned since it contains OFFSET, which is not pushdownable
|
||||
SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq ORDER BY 1 LIMIT 5;
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Subqueries with offset are not supported yet
|
||||
-- the subquery is recursively planned since it contains OFFSET, which is not pushdownable
|
||||
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq ORDER BY 1 LIMIT 5;
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Subqueries with offset are not supported yet
|
||||
-- Simple LIMIT/OFFSET with ORDER BY
|
||||
SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20;
|
||||
o_orderkey
|
||||
------------
|
||||
69
|
||||
70
|
||||
71
|
||||
96
|
||||
97
|
||||
98
|
||||
99
|
||||
100
|
||||
101
|
||||
102
|
||||
(10 rows)
|
||||
|
||||
-- LIMIT/OFFSET with a subquery
|
||||
SET citus.task_executor_type TO 'task-tracker';
|
||||
SELECT
|
||||
customer_keys.o_custkey,
|
||||
SUM(order_count) AS total_order_count
|
||||
FROM
|
||||
(SELECT o_custkey, o_orderstatus, COUNT(*) AS order_count
|
||||
FROM orders GROUP BY o_custkey, o_orderstatus ) customer_keys
|
||||
GROUP BY
|
||||
customer_keys.o_custkey
|
||||
ORDER BY
|
||||
customer_keys.o_custkey DESC
|
||||
LIMIT 10 OFFSET 20;
|
||||
o_custkey | total_order_count
|
||||
-----------+-------------------
|
||||
1466 | 1
|
||||
1465 | 2
|
||||
1463 | 4
|
||||
1462 | 10
|
||||
1460 | 1
|
||||
1459 | 6
|
||||
1457 | 1
|
||||
1456 | 3
|
||||
1454 | 2
|
||||
1453 | 5
|
||||
(10 rows)
|
||||
|
||||
SET citus.task_executor_type TO 'real-time';
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- Ensure that we push down LIMIT and OFFSET properly
|
||||
-- No Group-By -> Push Down
|
||||
CREATE TEMP TABLE temp_limit_test_1 AS
|
||||
SELECT o_custkey FROM orders LIMIT 10 OFFSET 15;
|
||||
DEBUG: push down of limit count: 25
|
||||
-- GROUP BY without ORDER BY -> No push-down
|
||||
CREATE TEMP TABLE temp_limit_test_2 AS
|
||||
SELECT o_custkey FROM orders GROUP BY o_custkey LIMIT 10 OFFSET 15;
|
||||
-- GROUP BY and ORDER BY non-aggregate -> push-down
|
||||
CREATE TEMP TABLE temp_limit_test_3 AS
|
||||
SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey LIMIT 10 OFFSET 15;
|
||||
DEBUG: push down of limit count: 25
|
||||
-- GROUP BY and ORDER BY aggregate -> No push-down
|
||||
CREATE TEMP TABLE temp_limit_test_4 AS
|
||||
SELECT o_custkey, COUNT(*) AS ccnt FROM orders GROUP BY o_custkey ORDER BY ccnt DESC LIMIT 10 OFFSET 15;
|
||||
-- OFFSET without LIMIT
|
||||
SELECT o_custkey FROM orders ORDER BY o_custkey OFFSET 2980;
|
||||
o_custkey
|
||||
-----------
|
||||
1498
|
||||
1499
|
||||
1499
|
||||
1499
|
||||
(4 rows)
|
||||
|
||||
-- LIMIT/OFFSET with Joins
|
||||
SELECT
|
||||
li.l_partkey,
|
||||
o.o_custkey,
|
||||
li.l_quantity
|
||||
FROM
|
||||
lineitem li JOIN orders o ON li.l_orderkey = o.o_orderkey
|
||||
WHERE
|
||||
li.l_quantity > 25
|
||||
ORDER BY 1, 2, 3
|
||||
LIMIT 10 OFFSET 20;
|
||||
DEBUG: push down of limit count: 30
|
||||
l_partkey | o_custkey | l_quantity
|
||||
-----------+-----------+------------
|
||||
655 | 58 | 50.00
|
||||
669 | 319 | 34.00
|
||||
699 | 1255 | 50.00
|
||||
716 | 61 | 45.00
|
||||
723 | 14 | 36.00
|
||||
802 | 754 | 50.00
|
||||
831 | 589 | 32.00
|
||||
835 | 67 | 33.00
|
||||
864 | 439 | 32.00
|
||||
875 | 13 | 43.00
|
||||
(10 rows)
|
||||
|
||||
RESET client_min_messages;
|
||||
-- FILTERs
|
||||
SELECT
|
||||
l_orderkey,
|
||||
sum(l_extendedprice),
|
||||
sum(l_extendedprice) FILTER (WHERE l_shipmode = 'AIR'),
|
||||
count(*),
|
||||
count(*) FILTER (WHERE l_shipmode = 'AIR'),
|
||||
max(l_extendedprice),
|
||||
max(l_extendedprice) FILTER (WHERE l_quantity < 30)
|
||||
FROM lineitem
|
||||
GROUP BY l_orderkey
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 10;
|
||||
l_orderkey | sum | sum | count | count | max | max
|
||||
------------+-----------+-----------+-------+-------+-----------+----------
|
||||
12804 | 440012.71 | 45788.16 | 7 | 1 | 94398.00 | 45788.16
|
||||
9863 | 412560.63 | 175647.63 | 7 | 3 | 85723.77 | 50769.14
|
||||
2567 | 412076.77 | 59722.26 | 7 | 1 | 94894.00 | 9784.02
|
||||
11142 | 410502.38 | 44965.95 | 7 | 1 | 83989.44 | 44965.95
|
||||
12039 | 407048.94 | 76406.30 | 7 | 2 | 94471.02 | 19679.30
|
||||
2306 | 405629.96 | 28032.60 | 7 | 1 | 92838.00 | 44384.50
|
||||
5606 | 403595.91 | 36531.51 | 7 | 2 | 94890.18 | 30582.75
|
||||
11296 | 399079.89 | | 6 | 0 | 102449.00 | 33122.93
|
||||
11046 | 391163.26 | 31436.34 | 7 | 2 | 94506.24 | 47519.76
|
||||
4421 | 387313.12 | | 7 | 0 | 67301.52 | 23783.40
|
||||
(10 rows)
|
||||
|
||||
SELECT
|
||||
l_orderkey,
|
||||
sum(l_extendedprice),
|
||||
sum(l_extendedprice) FILTER (WHERE l_shipmode = 'AIR'),
|
||||
count(*),
|
||||
count(*) FILTER (WHERE l_shipmode = 'AIR'),
|
||||
max(l_extendedprice),
|
||||
max(l_extendedprice) FILTER (WHERE l_quantity < 30)
|
||||
FROM lineitem
|
||||
GROUP BY l_orderkey
|
||||
HAVING count(*) FILTER (WHERE l_shipmode = 'AIR') > 1
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 10;
|
||||
l_orderkey | sum | sum | count | count | max | max
|
||||
------------+-----------+-----------+-------+-------+----------+----------
|
||||
9863 | 412560.63 | 175647.63 | 7 | 3 | 85723.77 | 50769.14
|
||||
12039 | 407048.94 | 76406.30 | 7 | 2 | 94471.02 | 19679.30
|
||||
5606 | 403595.91 | 36531.51 | 7 | 2 | 94890.18 | 30582.75
|
||||
11046 | 391163.26 | 31436.34 | 7 | 2 | 94506.24 | 47519.76
|
||||
14499 | 384140.30 | 67867.08 | 7 | 2 | 84335.36 | 46169.75
|
||||
11623 | 380598.48 | 133709.82 | 7 | 2 | 93701.54 | 21487.65
|
||||
10787 | 375688.09 | 99424.78 | 7 | 2 | 76732.67 | 50946.91
|
||||
12902 | 358191.24 | 76891.00 | 7 | 2 | 82008.08 | 35602.08
|
||||
3747 | 353701.23 | 68592.23 | 7 | 2 | 67181.10 | 46252.77
|
||||
5158 | 349889.05 | 159753.19 | 7 | 3 | 78714.67 | 29729.20
|
||||
(10 rows)
|
||||
|
|
@ -726,8 +726,8 @@ INSERT INTO
|
|||
(SELECT user_id FROM raw_events_first);
|
||||
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
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.
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- If the query is router plannable then it is executed via the coordinator
|
||||
INSERT INTO
|
||||
raw_events_first(user_id)
|
||||
|
@ -1047,7 +1047,7 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
GROUP BY raw_events_second.user_id) AS foo;
|
||||
ERROR: cannot perform distributed planning for the given modification
|
||||
DETAIL: Select query cannot be pushed down to the worker.
|
||||
-- INSERT partition column does not match with SELECT partition column
|
||||
-- INSERT returns NULL partition key value via coordinator
|
||||
INSERT INTO agg_events
|
||||
(value_4_agg,
|
||||
value_1_agg,
|
||||
|
@ -1064,8 +1064,22 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
GROUP BY raw_events_second.value_3) AS foo;
|
||||
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Group by list without partition column is currently unsupported
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [0,1073741823]
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [0,1073741823]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
|
||||
DEBUG: generating subplan 86_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.value_3 AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id = raw_events_second.user_id) GROUP BY raw_events_second.value_3
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
ERROR: the partition column of table public.agg_events cannot be NULL
|
||||
-- error cases
|
||||
-- no part column at all
|
||||
INSERT INTO raw_events_second
|
||||
|
@ -1155,8 +1169,7 @@ DETAIL: The target table's partition column should correspond to a partition co
|
|||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
-- unsupported joins between subqueries
|
||||
-- we do not return bare partition column on the inner query
|
||||
-- foo2 is recursively planned and INSERT...SELECT is done via coordinator
|
||||
INSERT INTO agg_events
|
||||
(user_id)
|
||||
SELECT f2.id FROM
|
||||
|
@ -1181,11 +1194,23 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
ON (f.id = f2.id);
|
||||
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
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.
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [0,1073741823]
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [0,1073741823]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
|
||||
DEBUG: generating subplan 105_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id = raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) > (10)::numeric)
|
||||
-- the second part of the query is not routable since
|
||||
-- GROUP BY not on the partition column (i.e., value_1) and thus join
|
||||
-- on f.id = f2.id is not on the partition key (instead on the sum of partition key)
|
||||
-- but we still recursively plan foo2 and run the query
|
||||
INSERT INTO agg_events
|
||||
(user_id)
|
||||
SELECT f.id FROM
|
||||
|
@ -1210,8 +1235,19 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
ON (f.id = f2.id);
|
||||
DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
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.
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [0,1073741823]
|
||||
DEBUG: join prunable for intervals [-2147483648,-1073741825] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [0,1073741823]
|
||||
DEBUG: join prunable for intervals [-1073741824,-1] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [0,1073741823] and [1073741824,2147483647]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-1073741825]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1]
|
||||
DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823]
|
||||
DEBUG: generating subplan 108_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id = raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) > (10)::numeric)
|
||||
-- cannot pushdown the query since the JOIN is not equi JOIN
|
||||
INSERT INTO agg_events
|
||||
(user_id, value_4_agg)
|
||||
|
@ -1627,8 +1663,8 @@ HINT: Consider using an equality filter on the distributed table's partition co
|
|||
SET client_min_messages TO INFO;
|
||||
-- avoid constraint violations
|
||||
TRUNCATE raw_events_first;
|
||||
-- we don't support LIMIT even if it exists in the subqueries
|
||||
-- in where clause
|
||||
-- we don't support LIMIT for subquery pushdown, but
|
||||
-- we recursively plan the query and run it via coordinator
|
||||
INSERT INTO agg_events(user_id)
|
||||
SELECT user_id
|
||||
FROM users_table
|
||||
|
@ -1650,8 +1686,6 @@ WHERE user_id
|
|||
) as f_inner
|
||||
)
|
||||
) AS f2);
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
-- Altering a table and selecting from it using a multi-shard statement
|
||||
-- in the same transaction is allowed because we will use the same
|
||||
-- connections for all co-located placements.
|
||||
|
|
|
@ -676,7 +676,7 @@ 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
|
||||
-- supported via recursive planning
|
||||
INSERT INTO agg_results (user_id, agg_time, value_2_agg)
|
||||
SELECT
|
||||
user_id,
|
||||
|
@ -702,5 +702,3 @@ FROM (
|
|||
GROUP BY user_id
|
||||
) AS shard_union
|
||||
ORDER BY user_lastseen DESC;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause can only contain immutable functions
|
||||
|
|
|
@ -653,8 +653,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- user needs to supply partition by which should
|
||||
-- include the distribution key
|
||||
INSERT INTO agg_results_window (user_id, agg_time, value_2_agg)
|
||||
|
@ -672,8 +672,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- user needs to supply partition by which should
|
||||
-- include the distribution key
|
||||
INSERT INTO agg_results_window (user_id, agg_time, value_2_agg)
|
||||
|
@ -691,8 +691,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- w2 should not be pushed down
|
||||
INSERT INTO agg_results_window (user_id, value_1_agg, value_2_agg)
|
||||
SELECT * FROM
|
||||
|
@ -709,8 +709,8 @@ SELECT * FROM
|
|||
) as foo
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- GROUP BY includes the partition key, but not the WINDOW function
|
||||
INSERT INTO agg_results_window (user_id, agg_time, value_2_agg)
|
||||
SELECT
|
||||
|
@ -727,8 +727,8 @@ FROM
|
|||
) as foo
|
||||
WHERE
|
||||
my_rank > 125;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- GROUP BY includes the partition key, but not the WINDOW function
|
||||
INSERT INTO agg_results_window (user_id, agg_time, value_2_agg)
|
||||
SELECT
|
||||
|
@ -745,8 +745,8 @@ FROM
|
|||
) as foo
|
||||
WHERE
|
||||
my_rank > 125;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- w2 should not be allowed
|
||||
INSERT INTO agg_results_window (user_id, value_2_agg, value_3_agg)
|
||||
SELECT * FROM
|
||||
|
@ -761,8 +761,8 @@ SELECT * FROM
|
|||
WINDOW w1 AS (PARTITION BY users_table.user_id, events_table.event_type ORDER BY events_table.time),
|
||||
w2 AS (ORDER BY events_table.time)
|
||||
) as foo;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- unsupported window function with an override
|
||||
INSERT INTO agg_results_window(user_id, agg_time, value_2_agg)
|
||||
SELECT * FROM (
|
||||
|
@ -779,8 +779,8 @@ SELECT * FROM (
|
|||
WINDOW
|
||||
w2 as (PARTITION BY user_id, time)
|
||||
) a;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- Subquery in where with unsupported window function
|
||||
INSERT INTO agg_results_window(user_id)
|
||||
SELECT
|
||||
|
@ -811,8 +811,8 @@ SELECT * FROM (
|
|||
GROUP BY
|
||||
user_id
|
||||
) a;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- UNION with only one subquery which has a partition on non-distribution column should
|
||||
-- error out
|
||||
INSERT INTO agg_results_window(user_id, value_1_agg)
|
||||
|
@ -849,6 +849,6 @@ FROM (
|
|||
user_id
|
||||
)
|
||||
) AS ftop;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
DROP VIEW view_with_window_func;
|
||||
|
|
|
@ -425,10 +425,18 @@ DEBUG: Plan is router executable
|
|||
43 | 3 | affixal | 12723 | 3
|
||||
(10 rows)
|
||||
|
||||
-- they are not supported if multiple workers are involved
|
||||
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2;
|
||||
ERROR: could not run distributed query with complex table expressions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- they are supported via (sub)query pushdown if multiple workers are involved
|
||||
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2 ORDER BY 4 DESC, 1 DESC, 2 DESC LIMIT 5;
|
||||
DEBUG: push down of limit count: 5
|
||||
id | author_id | title | word_count | position
|
||||
----+-----------+------------+------------+----------
|
||||
12 | 2 | archiblast | 18185 | 3
|
||||
42 | 2 | ausable | 15885 | 3
|
||||
2 | 2 | abducing | 13642 | 3
|
||||
41 | 1 | aznavour | 11814 | 3
|
||||
32 | 2 | amazon | 11342 | 3
|
||||
(5 rows)
|
||||
|
||||
-- subqueries are supported in FROM clause but they are not router plannable
|
||||
SELECT articles_hash_mx.id,test.word_count
|
||||
FROM articles_hash_mx, (SELECT id, word_count FROM articles_hash_mx) AS test WHERE test.id = articles_hash_mx.id
|
||||
|
@ -500,6 +508,7 @@ HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
|||
-- subqueries are not supported in SELECT clause
|
||||
SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard_hash_mx a2 WHERE a.id = a2.id LIMIT 1)
|
||||
AS special_price FROM articles_hash_mx a;
|
||||
DEBUG: skipping recursive planning for the subquery since it contains references to outer queries
|
||||
ERROR: could not run distributed query with subquery outside the FROM and WHERE clauses
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- simple lookup query
|
||||
|
@ -1137,13 +1146,13 @@ DEBUG: Plan is router executable
|
|||
SELECT id, MIN(id) over (order by word_count)
|
||||
FROM articles_hash_mx
|
||||
WHERE author_id = 1 or author_id = 2;
|
||||
ERROR: could not run distributed query with window functions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count
|
||||
FROM articles_hash_mx
|
||||
WHERE author_id = 5 or author_id = 2;
|
||||
ERROR: could not run distributed query with window functions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- complex query hitting a single shard
|
||||
SELECT
|
||||
count(DISTINCT CASE
|
||||
|
|
|
@ -51,6 +51,29 @@ SELECT a FROM dest_table;
|
|||
2
|
||||
(2 rows)
|
||||
|
||||
-- subqueries are also allowed
|
||||
SET client_min_messages TO DEBUG1;
|
||||
SELECT
|
||||
foo.a
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT dest_table.a
|
||||
FROM
|
||||
dest_table, source_table
|
||||
WHERE
|
||||
source_table.a = dest_table.a AND
|
||||
dest_table.b IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC LIMIT 5
|
||||
) as foo;
|
||||
DEBUG: generating subplan 4_1 for CTE cte: SELECT DISTINCT dest_table.a FROM public.dest_table, public.source_table WHERE ((source_table.a = dest_table.a) AND (dest_table.b = ANY (ARRAY[1, 2, 3, 4])))
|
||||
DEBUG: generating subplan 4_2 for subquery SELECT a FROM (SELECT intermediate_result.a FROM read_intermediate_result('4_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) cte ORDER BY a DESC LIMIT 5
|
||||
a
|
||||
---
|
||||
(0 rows)
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
-- insert into is definitely not allowed
|
||||
INSERT INTO dest_table (a, b)
|
||||
SELECT a, b FROM source_table;
|
||||
|
|
|
@ -0,0 +1,246 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on partitioned tables
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_and_partitioning;
|
||||
SET search_path TO subquery_and_partitioning, public;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
CREATE TABLE partitioning_test(id int, value_1 int, time date) PARTITION BY RANGE (time);
|
||||
ERROR: syntax error at or near "PARTITION"
|
||||
LINE 1: ...partitioning_test(id int, value_1 int, time date) PARTITION ...
|
||||
^
|
||||
|
||||
-- create its partitions
|
||||
CREATE TABLE partitioning_test_2017 PARTITION OF partitioning_test FOR VALUES FROM ('2017-01-01') TO ('2018-01-01');
|
||||
ERROR: syntax error at or near "PARTITION"
|
||||
LINE 1: CREATE TABLE partitioning_test_2017 PARTITION OF partitionin...
|
||||
^
|
||||
CREATE TABLE partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01');
|
||||
ERROR: syntax error at or near "PARTITION"
|
||||
LINE 1: CREATE TABLE partitioning_test_2010 PARTITION OF partitionin...
|
||||
^
|
||||
-- load some data and distribute tables
|
||||
INSERT INTO partitioning_test VALUES (1, 1, '2017-11-23');
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test VALUES (1, 1, '2017-11-23');
|
||||
^
|
||||
INSERT INTO partitioning_test VALUES (2, 1, '2010-07-07');
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test VALUES (2, 1, '2010-07-07');
|
||||
^
|
||||
INSERT INTO partitioning_test_2017 VALUES (3, 3, '2017-11-22');
|
||||
ERROR: relation "partitioning_test_2017" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test_2017 VALUES (3, 3, '2017-11-22...
|
||||
^
|
||||
INSERT INTO partitioning_test_2010 VALUES (4, 4, '2010-03-03');
|
||||
ERROR: relation "partitioning_test_2010" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test_2010 VALUES (4, 4, '2010-03-03...
|
||||
^
|
||||
-- distribute partitioned table
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SELECT create_distributed_table('partitioning_test', 'id');
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 1: SELECT create_distributed_table('partitioning_test', 'id');
|
||||
^
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- subplan for partitioned tables
|
||||
SELECT
|
||||
id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- final query is router on partitioned tables
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as bar
|
||||
WHERE foo.id = date_part('day', bar.time)
|
||||
ORDER BY 2 DESC, 1;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- final query is real-time
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar
|
||||
WHERE date_part('day', foo.time) = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- final query is real-time that is joined with partitioned table
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar,
|
||||
partitioning_test
|
||||
WHERE date_part('day', foo.time) = bar.id AND partitioning_test.id = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- subquery in WHERE clause
|
||||
SELECT DISTINCT id
|
||||
FROM partitioning_test
|
||||
WHERE
|
||||
id IN (SELECT DISTINCT date_part('day', time) FROM partitioning_test);
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 2: FROM partitioning_test
|
||||
^
|
||||
-- repartition subquery
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT p1.value_1 FROM partitioning_test as p1, partitioning_test as p2 WHERE p1.id = p2.value_1
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_1 = bar.user_id;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 5: SELECT DISTINCT p1.value_1 FROM partitioning_test as p1, pa...
|
||||
^
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
-- subquery, cte, view and non-partitioned tables
|
||||
CREATE VIEW subquery_and_ctes AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_1 FROM partitioning_test OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_1 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT events_table.user_id
|
||||
FROM
|
||||
partitioning_test, events_table
|
||||
WHERE
|
||||
events_table.user_id = partitioning_test.id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 15: (SELECT DISTINCT value_1 FROM partitioning_test OFFSET 0)...
|
||||
^
|
||||
SELECT * FROM subquery_and_ctes
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
ERROR: relation "subquery_and_ctes" does not exist
|
||||
LINE 1: SELECT * FROM subquery_and_ctes
|
||||
^
|
||||
-- deep subquery, partitioned and non-partitioned tables together
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(partitioning_test.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_1 FROM partitioning_test GROUP BY value_1) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, partitioning_test
|
||||
WHERE
|
||||
level_5.avg_ev_type = partitioning_test.id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 20: (SELECT count(*) as cnt, value_1 FROM partitioning_...
|
||||
^
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_and_partitioning CASCADE;
|
||||
NOTICE: drop cascades to 2 other objects
|
||||
DETAIL: drop cascades to table users_table_local
|
||||
drop cascades to table events_table_local
|
||||
SET search_path TO public;
|
|
@ -525,10 +525,18 @@ DEBUG: Plan is router executable
|
|||
43 | 3 | affixal | 12723 | 3
|
||||
(10 rows)
|
||||
|
||||
-- they are not supported if multiple workers are involved
|
||||
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2;
|
||||
ERROR: could not run distributed query with complex table expressions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- they are supported via (sub)query pushdown if multiple workers are involved
|
||||
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2 ORDER BY 4 DESC, 1 DESC, 2 DESC LIMIT 5;
|
||||
DEBUG: push down of limit count: 5
|
||||
id | author_id | title | word_count | position
|
||||
----+-----------+------------+------------+----------
|
||||
12 | 2 | archiblast | 18185 | 3
|
||||
42 | 2 | ausable | 15885 | 3
|
||||
2 | 2 | abducing | 13642 | 3
|
||||
41 | 1 | aznavour | 11814 | 3
|
||||
32 | 2 | amazon | 11342 | 3
|
||||
(5 rows)
|
||||
|
||||
-- unless the query can be transformed into a join
|
||||
SELECT * FROM articles_hash
|
||||
WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 2)
|
||||
|
@ -615,6 +623,7 @@ HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
|||
-- subqueries are not supported in SELECT clause
|
||||
SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard_hash a2 WHERE a.id = a2.id LIMIT 1)
|
||||
AS special_price FROM articles_hash a;
|
||||
DEBUG: skipping recursive planning for the subquery since it contains references to outer queries
|
||||
ERROR: could not run distributed query with subquery outside the FROM and WHERE clauses
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- simple lookup query
|
||||
|
@ -1267,13 +1276,13 @@ DEBUG: Plan is router executable
|
|||
SELECT id, MIN(id) over (order by word_count)
|
||||
FROM articles_hash
|
||||
WHERE author_id = 1 or author_id = 2;
|
||||
ERROR: could not run distributed query with window functions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
SELECT LAG(title, 1) over (ORDER BY word_count) prev, title, word_count
|
||||
FROM articles_hash
|
||||
WHERE author_id = 5 or author_id = 2;
|
||||
ERROR: could not run distributed query with window functions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- where false queries are router plannable
|
||||
SELECT *
|
||||
FROM articles_hash
|
||||
|
|
|
@ -184,13 +184,23 @@ SELECT title FROM articles ORDER BY 1 LIMIT 5;
|
|||
ablation
|
||||
(5 rows)
|
||||
|
||||
-- queries which involve functions in FROM clause are unsupported.
|
||||
SELECT * FROM articles, position('om' in 'Thomas');
|
||||
ERROR: could not run distributed query with complex table expressions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- subqueries are not supported in WHERE clause in Citus
|
||||
-- queries which involve functions in FROM clause are recursively planned
|
||||
SELECT * FROM articles, position('om' in 'Thomas') ORDER BY 2 DESC, 1 DESC, 3 DESC LIMIT 5;
|
||||
id | author_id | title | word_count | position
|
||||
----+-----------+------------+------------+----------
|
||||
50 | 10 | anjanette | 19519 | 3
|
||||
40 | 10 | attemper | 14976 | 3
|
||||
30 | 10 | andelee | 6363 | 3
|
||||
20 | 10 | absentness | 1820 | 3
|
||||
10 | 10 | aggrandize | 17277 | 3
|
||||
(5 rows)
|
||||
|
||||
-- subqueries are supported in WHERE clause in Citus even if the relations are not distributed
|
||||
SELECT * FROM articles WHERE author_id IN (SELECT id FROM authors WHERE name LIKE '%a');
|
||||
ERROR: relation authors is not distributed
|
||||
id | author_id | title | word_count
|
||||
----+-----------+-------+------------
|
||||
(0 rows)
|
||||
|
||||
-- subqueries are supported in FROM clause
|
||||
SELECT articles.id,test.word_count
|
||||
FROM articles, (SELECT id, word_count FROM articles) AS test WHERE test.id = articles.id
|
||||
|
|
|
@ -0,0 +1,595 @@
|
|||
SET citus.next_shard_id TO 850000;
|
||||
-- ===================================================================
|
||||
-- test end-to-end query functionality
|
||||
-- ===================================================================
|
||||
CREATE TABLE articles (
|
||||
id bigint NOT NULL,
|
||||
author_id bigint NOT NULL,
|
||||
title varchar(20) NOT NULL,
|
||||
word_count integer NOT NULL CHECK (word_count > 0)
|
||||
);
|
||||
-- this table is used in a CTE test
|
||||
CREATE TABLE authors ( name text, id bigint );
|
||||
-- this table is used in router executor tests
|
||||
CREATE TABLE articles_single_shard (LIKE articles);
|
||||
SELECT master_create_distributed_table('articles', 'author_id', 'hash');
|
||||
master_create_distributed_table
|
||||
---------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT master_create_distributed_table('articles_single_shard', 'author_id', 'hash');
|
||||
master_create_distributed_table
|
||||
---------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT master_create_worker_shards('articles', 2, 1);
|
||||
master_create_worker_shards
|
||||
-----------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT master_create_worker_shards('articles_single_shard', 1, 1);
|
||||
master_create_worker_shards
|
||||
-----------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- create a bunch of test data
|
||||
INSERT INTO articles VALUES ( 1, 1, 'arsenous', 9572);
|
||||
INSERT INTO articles VALUES ( 2, 2, 'abducing', 13642);
|
||||
INSERT INTO articles VALUES ( 3, 3, 'asternal', 10480);
|
||||
INSERT INTO articles VALUES ( 4, 4, 'altdorfer', 14551);
|
||||
INSERT INTO articles VALUES ( 5, 5, 'aruru', 11389);
|
||||
INSERT INTO articles VALUES ( 6, 6, 'atlases', 15459);
|
||||
INSERT INTO articles VALUES ( 7, 7, 'aseptic', 12298);
|
||||
INSERT INTO articles VALUES ( 8, 8, 'agatized', 16368);
|
||||
INSERT INTO articles VALUES ( 9, 9, 'alligate', 438);
|
||||
INSERT INTO articles VALUES (10, 10, 'aggrandize', 17277);
|
||||
INSERT INTO articles VALUES (11, 1, 'alamo', 1347);
|
||||
INSERT INTO articles VALUES (12, 2, 'archiblast', 18185);
|
||||
INSERT INTO articles VALUES (13, 3, 'aseyev', 2255);
|
||||
INSERT INTO articles VALUES (14, 4, 'andesite', 19094);
|
||||
INSERT INTO articles VALUES (15, 5, 'adversa', 3164);
|
||||
INSERT INTO articles VALUES (16, 6, 'allonym', 2);
|
||||
INSERT INTO articles VALUES (17, 7, 'auriga', 4073);
|
||||
INSERT INTO articles VALUES (18, 8, 'assembly', 911);
|
||||
INSERT INTO articles VALUES (19, 9, 'aubergiste', 4981);
|
||||
INSERT INTO articles VALUES (20, 10, 'absentness', 1820);
|
||||
INSERT INTO articles VALUES (21, 1, 'arcading', 5890);
|
||||
INSERT INTO articles VALUES (22, 2, 'antipope', 2728);
|
||||
INSERT INTO articles VALUES (23, 3, 'abhorring', 6799);
|
||||
INSERT INTO articles VALUES (24, 4, 'audacious', 3637);
|
||||
INSERT INTO articles VALUES (25, 5, 'antehall', 7707);
|
||||
INSERT INTO articles VALUES (26, 6, 'abington', 4545);
|
||||
INSERT INTO articles VALUES (27, 7, 'arsenous', 8616);
|
||||
INSERT INTO articles VALUES (28, 8, 'aerophyte', 5454);
|
||||
INSERT INTO articles VALUES (29, 9, 'amateur', 9524);
|
||||
INSERT INTO articles VALUES (30, 10, 'andelee', 6363);
|
||||
INSERT INTO articles VALUES (31, 1, 'athwartships', 7271);
|
||||
INSERT INTO articles VALUES (32, 2, 'amazon', 11342);
|
||||
INSERT INTO articles VALUES (33, 3, 'autochrome', 8180);
|
||||
INSERT INTO articles VALUES (34, 4, 'amnestied', 12250);
|
||||
INSERT INTO articles VALUES (35, 5, 'aminate', 9089);
|
||||
INSERT INTO articles VALUES (36, 6, 'ablation', 13159);
|
||||
INSERT INTO articles VALUES (37, 7, 'archduchies', 9997);
|
||||
INSERT INTO articles VALUES (38, 8, 'anatine', 14067);
|
||||
INSERT INTO articles VALUES (39, 9, 'anchises', 10906);
|
||||
INSERT INTO articles VALUES (40, 10, 'attemper', 14976);
|
||||
INSERT INTO articles VALUES (41, 1, 'aznavour', 11814);
|
||||
INSERT INTO articles VALUES (42, 2, 'ausable', 15885);
|
||||
INSERT INTO articles VALUES (43, 3, 'affixal', 12723);
|
||||
INSERT INTO articles VALUES (44, 4, 'anteport', 16793);
|
||||
INSERT INTO articles VALUES (45, 5, 'afrasia', 864);
|
||||
INSERT INTO articles VALUES (46, 6, 'atlanta', 17702);
|
||||
INSERT INTO articles VALUES (47, 7, 'abeyance', 1772);
|
||||
INSERT INTO articles VALUES (48, 8, 'alkylic', 18610);
|
||||
INSERT INTO articles VALUES (49, 9, 'anyone', 2681);
|
||||
INSERT INTO articles VALUES (50, 10, 'anjanette', 19519);
|
||||
-- insert a single row for the test
|
||||
INSERT INTO articles_single_shard VALUES (50, 10, 'anjanette', 19519);
|
||||
-- zero-shard modifications should succeed
|
||||
UPDATE articles SET title = '' WHERE author_id = 1 AND author_id = 2;
|
||||
UPDATE articles SET title = '' WHERE 0 = 1;
|
||||
DELETE FROM articles WHERE author_id = 1 AND author_id = 2;
|
||||
-- single-shard tests
|
||||
-- test simple select for a single row
|
||||
SELECT * FROM articles WHERE author_id = 10 AND id = 50;
|
||||
id | author_id | title | word_count
|
||||
----+-----------+-----------+------------
|
||||
50 | 10 | anjanette | 19519
|
||||
(1 row)
|
||||
|
||||
-- get all titles by a single author
|
||||
SELECT title FROM articles WHERE author_id = 10;
|
||||
title
|
||||
------------
|
||||
aggrandize
|
||||
absentness
|
||||
andelee
|
||||
attemper
|
||||
anjanette
|
||||
(5 rows)
|
||||
|
||||
-- try ordering them by word count
|
||||
SELECT title, word_count FROM articles
|
||||
WHERE author_id = 10
|
||||
ORDER BY word_count DESC NULLS LAST;
|
||||
title | word_count
|
||||
------------+------------
|
||||
anjanette | 19519
|
||||
aggrandize | 17277
|
||||
attemper | 14976
|
||||
andelee | 6363
|
||||
absentness | 1820
|
||||
(5 rows)
|
||||
|
||||
-- look at last two articles by an author
|
||||
SELECT title, id FROM articles
|
||||
WHERE author_id = 5
|
||||
ORDER BY id
|
||||
LIMIT 2;
|
||||
title | id
|
||||
---------+----
|
||||
aruru | 5
|
||||
adversa | 15
|
||||
(2 rows)
|
||||
|
||||
-- find all articles by two authors in same shard
|
||||
SELECT title, author_id FROM articles
|
||||
WHERE author_id = 7 OR author_id = 8
|
||||
ORDER BY author_id ASC, id;
|
||||
title | author_id
|
||||
-------------+-----------
|
||||
aseptic | 7
|
||||
auriga | 7
|
||||
arsenous | 7
|
||||
archduchies | 7
|
||||
abeyance | 7
|
||||
agatized | 8
|
||||
assembly | 8
|
||||
aerophyte | 8
|
||||
anatine | 8
|
||||
alkylic | 8
|
||||
(10 rows)
|
||||
|
||||
-- add in some grouping expressions
|
||||
SELECT author_id, sum(word_count) AS corpus_size FROM articles
|
||||
WHERE author_id = 1 OR author_id = 2 OR author_id = 8 OR author_id = 10
|
||||
GROUP BY author_id
|
||||
HAVING sum(word_count) > 40000
|
||||
ORDER BY sum(word_count) DESC;
|
||||
author_id | corpus_size
|
||||
-----------+-------------
|
||||
2 | 61782
|
||||
10 | 59955
|
||||
8 | 55410
|
||||
(3 rows)
|
||||
|
||||
-- UNION/INTERSECT queries are unsupported if on multiple shards
|
||||
SELECT * FROM articles WHERE author_id = 10 UNION
|
||||
SELECT * FROM articles WHERE author_id = 2;
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- queries using CTEs are supported
|
||||
WITH long_names AS ( SELECT id FROM authors WHERE char_length(name) > 15 )
|
||||
SELECT title FROM articles ORDER BY 1 LIMIT 5;
|
||||
title
|
||||
-----------
|
||||
abducing
|
||||
abeyance
|
||||
abhorring
|
||||
abington
|
||||
ablation
|
||||
(5 rows)
|
||||
|
||||
-- queries which involve functions in FROM clause are recursively planned
|
||||
SELECT * FROM articles, position('om' in 'Thomas') ORDER BY 2 DESC, 1 DESC, 3 DESC LIMIT 5;
|
||||
id | author_id | title | word_count | position
|
||||
----+-----------+------------+------------+----------
|
||||
50 | 10 | anjanette | 19519 | 3
|
||||
40 | 10 | attemper | 14976 | 3
|
||||
30 | 10 | andelee | 6363 | 3
|
||||
20 | 10 | absentness | 1820 | 3
|
||||
10 | 10 | aggrandize | 17277 | 3
|
||||
(5 rows)
|
||||
|
||||
-- subqueries are supported in WHERE clause in Citus even if the relations are not distributed
|
||||
SELECT * FROM articles WHERE author_id IN (SELECT id FROM authors WHERE name LIKE '%a');
|
||||
ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker'
|
||||
-- subqueries are supported in FROM clause
|
||||
SELECT articles.id,test.word_count
|
||||
FROM articles, (SELECT id, word_count FROM articles) AS test WHERE test.id = articles.id
|
||||
ORDER BY articles.id;
|
||||
id | word_count
|
||||
----+------------
|
||||
1 | 9572
|
||||
2 | 13642
|
||||
3 | 10480
|
||||
4 | 14551
|
||||
5 | 11389
|
||||
6 | 15459
|
||||
7 | 12298
|
||||
8 | 16368
|
||||
9 | 438
|
||||
10 | 17277
|
||||
11 | 1347
|
||||
12 | 18185
|
||||
13 | 2255
|
||||
14 | 19094
|
||||
15 | 3164
|
||||
16 | 2
|
||||
17 | 4073
|
||||
18 | 911
|
||||
19 | 4981
|
||||
20 | 1820
|
||||
21 | 5890
|
||||
22 | 2728
|
||||
23 | 6799
|
||||
24 | 3637
|
||||
25 | 7707
|
||||
26 | 4545
|
||||
27 | 8616
|
||||
28 | 5454
|
||||
29 | 9524
|
||||
30 | 6363
|
||||
31 | 7271
|
||||
32 | 11342
|
||||
33 | 8180
|
||||
34 | 12250
|
||||
35 | 9089
|
||||
36 | 13159
|
||||
37 | 9997
|
||||
38 | 14067
|
||||
39 | 10906
|
||||
40 | 14976
|
||||
41 | 11814
|
||||
42 | 15885
|
||||
43 | 12723
|
||||
44 | 16793
|
||||
45 | 864
|
||||
46 | 17702
|
||||
47 | 1772
|
||||
48 | 18610
|
||||
49 | 2681
|
||||
50 | 19519
|
||||
(50 rows)
|
||||
|
||||
-- subqueries are not supported in SELECT clause
|
||||
SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard a2 WHERE a.id = a2.id LIMIT 1)
|
||||
AS special_price FROM articles a;
|
||||
ERROR: could not run distributed query with subquery outside the FROM and WHERE clauses
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- joins are not supported between local and distributed tables
|
||||
SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id;
|
||||
ERROR: relation authors is not distributed
|
||||
-- inner joins are not supported (I think)
|
||||
SELECT * FROM (articles INNER JOIN authors ON articles.id = authors.id);
|
||||
ERROR: relation authors is not distributed
|
||||
-- test use of EXECUTE statements within plpgsql
|
||||
DO $sharded_execute$
|
||||
BEGIN
|
||||
EXECUTE 'SELECT COUNT(*) FROM articles ' ||
|
||||
'WHERE author_id = $1 AND author_id = $2' USING 1, 2;
|
||||
END
|
||||
$sharded_execute$;
|
||||
-- test use of bare SQL within plpgsql
|
||||
DO $sharded_sql$
|
||||
BEGIN
|
||||
SELECT COUNT(*) FROM articles WHERE author_id = 1 AND author_id = 2;
|
||||
END
|
||||
$sharded_sql$;
|
||||
ERROR: query has no destination for result data
|
||||
HINT: If you want to discard the results of a SELECT, use PERFORM instead.
|
||||
CONTEXT: PL/pgSQL function inline_code_block line 3 at SQL statement
|
||||
-- test cross-shard queries
|
||||
SELECT COUNT(*) FROM articles;
|
||||
count
|
||||
-------
|
||||
50
|
||||
(1 row)
|
||||
|
||||
-- test with empty target list
|
||||
SELECT FROM articles;
|
||||
--
|
||||
(50 rows)
|
||||
|
||||
SELECT FROM articles WHERE author_id = 3737;
|
||||
--
|
||||
(0 rows)
|
||||
|
||||
SELECT FROM articles WHERE word_count = 65500;
|
||||
--
|
||||
(0 rows)
|
||||
|
||||
-- having queries supported in Citus
|
||||
SELECT author_id, sum(word_count) AS corpus_size FROM articles
|
||||
GROUP BY author_id
|
||||
HAVING sum(word_count) > 25000
|
||||
ORDER BY sum(word_count) DESC
|
||||
LIMIT 5;
|
||||
author_id | corpus_size
|
||||
-----------+-------------
|
||||
4 | 66325
|
||||
2 | 61782
|
||||
10 | 59955
|
||||
8 | 55410
|
||||
6 | 50867
|
||||
(5 rows)
|
||||
|
||||
SELECT author_id FROM articles
|
||||
GROUP BY author_id
|
||||
HAVING sum(word_count) > 50000
|
||||
ORDER BY author_id;
|
||||
author_id
|
||||
-----------
|
||||
2
|
||||
4
|
||||
6
|
||||
8
|
||||
10
|
||||
(5 rows)
|
||||
|
||||
SELECT author_id FROM articles
|
||||
GROUP BY author_id
|
||||
HAVING sum(word_count) > 50000 AND author_id < 5
|
||||
ORDER BY author_id;
|
||||
author_id
|
||||
-----------
|
||||
2
|
||||
4
|
||||
(2 rows)
|
||||
|
||||
SELECT author_id FROM articles
|
||||
GROUP BY author_id
|
||||
HAVING sum(word_count) > 50000 OR author_id < 5
|
||||
ORDER BY author_id;
|
||||
author_id
|
||||
-----------
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
6
|
||||
8
|
||||
10
|
||||
(7 rows)
|
||||
|
||||
SELECT author_id FROM articles
|
||||
GROUP BY author_id
|
||||
HAVING author_id <= 2 OR author_id = 8
|
||||
ORDER BY author_id;
|
||||
author_id
|
||||
-----------
|
||||
1
|
||||
2
|
||||
8
|
||||
(3 rows)
|
||||
|
||||
SELECT o_orderstatus, count(*), avg(o_totalprice) FROM orders
|
||||
GROUP BY o_orderstatus
|
||||
HAVING count(*) > 1450 OR avg(o_totalprice) > 150000
|
||||
ORDER BY o_orderstatus;
|
||||
o_orderstatus | count | avg
|
||||
---------------+-------+---------------------
|
||||
O | 1460 | 143355.847013698630
|
||||
P | 75 | 164847.914533333333
|
||||
(2 rows)
|
||||
|
||||
SELECT o_orderstatus, sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||
WHERE l_orderkey = o_orderkey AND l_orderkey > 9030
|
||||
GROUP BY o_orderstatus
|
||||
HAVING sum(l_linenumber) > 1000
|
||||
ORDER BY o_orderstatus;
|
||||
o_orderstatus | sum | avg
|
||||
---------------+------+--------------------
|
||||
F | 8559 | 3.0126715945089757
|
||||
O | 8901 | 3.0050641458474004
|
||||
(2 rows)
|
||||
|
||||
-- now, test the cases where Citus do or do not need to create
|
||||
-- the master queries
|
||||
SET citus.large_table_shard_count TO 2;
|
||||
SET client_min_messages TO 'DEBUG2';
|
||||
SET citus.task_executor_type TO 'real-time';
|
||||
-- start with the simple lookup query
|
||||
SELECT *
|
||||
FROM articles
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
----+-----------+--------------+------------
|
||||
1 | 1 | arsenous | 9572
|
||||
11 | 1 | alamo | 1347
|
||||
21 | 1 | arcading | 5890
|
||||
31 | 1 | athwartships | 7271
|
||||
41 | 1 | aznavour | 11814
|
||||
(5 rows)
|
||||
|
||||
-- below query hits a single shard, so no need to create the master query
|
||||
SELECT *
|
||||
FROM articles
|
||||
WHERE author_id = 1 OR author_id = 17;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
----+-----------+--------------+------------
|
||||
1 | 1 | arsenous | 9572
|
||||
11 | 1 | alamo | 1347
|
||||
21 | 1 | arcading | 5890
|
||||
31 | 1 | athwartships | 7271
|
||||
41 | 1 | aznavour | 11814
|
||||
(5 rows)
|
||||
|
||||
-- below query hits two shards, so needs to create the master query
|
||||
SELECT *
|
||||
FROM articles
|
||||
WHERE author_id = 1 OR author_id = 18;
|
||||
id | author_id | title | word_count
|
||||
----+-----------+--------------+------------
|
||||
1 | 1 | arsenous | 9572
|
||||
11 | 1 | alamo | 1347
|
||||
21 | 1 | arcading | 5890
|
||||
31 | 1 | athwartships | 7271
|
||||
41 | 1 | aznavour | 11814
|
||||
(5 rows)
|
||||
|
||||
-- rename the output columns on a no master query case
|
||||
SELECT id as article_id, word_count * id as random_value
|
||||
FROM articles
|
||||
WHERE author_id = 1;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
article_id | random_value
|
||||
------------+--------------
|
||||
1 | 9572
|
||||
11 | 14817
|
||||
21 | 123690
|
||||
31 | 225401
|
||||
41 | 484374
|
||||
(5 rows)
|
||||
|
||||
-- we can push down co-located joins to a single worker without the
|
||||
-- master query being required for only the same tables
|
||||
SELECT a.author_id as first_author, b.word_count as second_word_count
|
||||
FROM articles a, articles b
|
||||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
first_author | second_word_count
|
||||
--------------+-------------------
|
||||
10 | 17277
|
||||
10 | 1820
|
||||
10 | 6363
|
||||
(3 rows)
|
||||
|
||||
-- now show that JOINs with multiple tables are not router executable
|
||||
-- they are executed by real-time executor
|
||||
SELECT a.author_id as first_author, b.word_count as second_word_count
|
||||
FROM articles a, articles_single_shard b
|
||||
WHERE a.author_id = 10 and a.author_id = b.author_id
|
||||
LIMIT 3;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
first_author | second_word_count
|
||||
--------------+-------------------
|
||||
10 | 19519
|
||||
10 | 19519
|
||||
10 | 19519
|
||||
(3 rows)
|
||||
|
||||
-- do not create the master query for LIMIT on a single shard SELECT
|
||||
SELECT *
|
||||
FROM articles
|
||||
WHERE author_id = 1
|
||||
LIMIT 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id | author_id | title | word_count
|
||||
----+-----------+----------+------------
|
||||
1 | 1 | arsenous | 9572
|
||||
11 | 1 | alamo | 1347
|
||||
(2 rows)
|
||||
|
||||
-- This query hits a single shard. So GROUP BY can be
|
||||
-- pushed down to the workers directly. This query is
|
||||
-- equivalent to SELECT DISTINCT on a single shard.
|
||||
SELECT id
|
||||
FROM articles
|
||||
WHERE author_id = 1
|
||||
GROUP BY id
|
||||
ORDER BY id;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
id
|
||||
----
|
||||
1
|
||||
11
|
||||
21
|
||||
31
|
||||
41
|
||||
(5 rows)
|
||||
|
||||
-- copying from a single shard table does not require the master query
|
||||
COPY articles_single_shard TO stdout;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
50 10 anjanette 19519
|
||||
-- error out for queries with aggregates
|
||||
SELECT avg(word_count)
|
||||
FROM articles
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
avg
|
||||
--------------------
|
||||
12356.400000000000
|
||||
(1 row)
|
||||
|
||||
-- max, min, sum, count is somehow implemented
|
||||
-- differently in distributed planning
|
||||
SELECT max(word_count) as max, min(word_count) as min,
|
||||
sum(word_count) as sum, count(word_count) as cnt
|
||||
FROM articles
|
||||
WHERE author_id = 2;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Plan is router executable
|
||||
max | min | sum | cnt
|
||||
-------+------+-------+-----
|
||||
18185 | 2728 | 61782 | 5
|
||||
(1 row)
|
||||
|
||||
-- error out for queries with repartition jobs
|
||||
SELECT *
|
||||
FROM articles a, articles b
|
||||
WHERE a.id = b.id AND a.author_id = 1;
|
||||
DEBUG: join prunable for task partitionId 0 and 1
|
||||
DEBUG: join prunable for task partitionId 0 and 2
|
||||
DEBUG: join prunable for task partitionId 0 and 3
|
||||
DEBUG: join prunable for task partitionId 1 and 0
|
||||
DEBUG: join prunable for task partitionId 1 and 2
|
||||
DEBUG: join prunable for task partitionId 1 and 3
|
||||
DEBUG: join prunable for task partitionId 2 and 0
|
||||
DEBUG: join prunable for task partitionId 2 and 1
|
||||
DEBUG: join prunable for task partitionId 2 and 3
|
||||
DEBUG: join prunable for task partitionId 3 and 0
|
||||
DEBUG: join prunable for task partitionId 3 and 1
|
||||
DEBUG: join prunable for task partitionId 3 and 2
|
||||
DEBUG: pruning merge fetch taskId 1
|
||||
DETAIL: Creating dependency on merge taskId 3
|
||||
DEBUG: pruning merge fetch taskId 2
|
||||
DETAIL: Creating dependency on merge taskId 5
|
||||
DEBUG: pruning merge fetch taskId 4
|
||||
DETAIL: Creating dependency on merge taskId 5
|
||||
DEBUG: pruning merge fetch taskId 5
|
||||
DETAIL: Creating dependency on merge taskId 8
|
||||
DEBUG: pruning merge fetch taskId 7
|
||||
DETAIL: Creating dependency on merge taskId 7
|
||||
DEBUG: pruning merge fetch taskId 8
|
||||
DETAIL: Creating dependency on merge taskId 11
|
||||
DEBUG: pruning merge fetch taskId 10
|
||||
DETAIL: Creating dependency on merge taskId 9
|
||||
DEBUG: pruning merge fetch taskId 11
|
||||
DETAIL: Creating dependency on merge taskId 14
|
||||
ERROR: the query contains a join that requires repartitioning
|
||||
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
||||
-- system columns from shard tables can be queried and retrieved
|
||||
SELECT count(*) FROM (
|
||||
SELECT tableoid, ctid, cmin, cmax, xmin, xmax
|
||||
FROM articles
|
||||
WHERE tableoid IS NOT NULL OR
|
||||
ctid IS NOT NULL OR
|
||||
cmin IS NOT NULL OR
|
||||
cmax IS NOT NULL OR
|
||||
xmin IS NOT NULL OR
|
||||
xmax IS NOT NULL
|
||||
) x;
|
||||
count
|
||||
-------
|
||||
50
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages to 'NOTICE';
|
|
@ -95,7 +95,7 @@ from
|
|||
group by
|
||||
suppkey_bin
|
||||
order by
|
||||
avg_count desc
|
||||
avg_count desc, suppkey_bin DESC
|
||||
limit 20;
|
||||
suppkey_bin | avg_count
|
||||
-------------+--------------------
|
||||
|
@ -179,7 +179,7 @@ from
|
|||
1.00083402835696413678
|
||||
(1 row)
|
||||
|
||||
-- Check that we don't support subqueries with limit.
|
||||
-- we don't support subqueries with limit.
|
||||
select
|
||||
l_suppkey,
|
||||
sum(suppkey_count) as total_suppkey_count
|
||||
|
@ -195,19 +195,23 @@ from
|
|||
l_suppkey
|
||||
limit 100) as distributed_table
|
||||
group by
|
||||
l_suppkey;
|
||||
l_suppkey
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Subqueries with limit are not supported yet
|
||||
-- Check that we don't support subqueries without aggregates.
|
||||
select
|
||||
rounded_tax
|
||||
DISTINCT rounded_tax
|
||||
from
|
||||
(select
|
||||
round(l_tax) as rounded_tax
|
||||
from
|
||||
lineitem
|
||||
group by
|
||||
l_tax) as distributed_table;
|
||||
l_tax) as distributed_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5;
|
||||
ERROR: cannot perform distributed planning on this query
|
||||
DETAIL: Subqueries without aggregates are not supported yet
|
||||
-- Check that we support subqueries with count(distinct).
|
||||
|
|
|
@ -28,8 +28,7 @@ SET
|
|||
shardmaxvalue = '14947'
|
||||
WHERE
|
||||
shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'orders_subquery'::regclass ORDER BY shardid DESC LIMIT 1);
|
||||
-- If group by is not on partition column then we error out from single table
|
||||
-- repartition code path
|
||||
-- If group by is not on partition column then we recursively plan
|
||||
SELECT
|
||||
avg(order_count)
|
||||
FROM
|
||||
|
@ -40,8 +39,7 @@ FROM
|
|||
lineitem_subquery
|
||||
GROUP BY
|
||||
l_suppkey) AS order_counts;
|
||||
ERROR: the query contains a join that requires repartitioning
|
||||
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
-- Check that we error out if join is not on partition columns.
|
||||
SELECT
|
||||
avg(unit_price)
|
||||
|
@ -71,20 +69,22 @@ FROM
|
|||
l_orderkey) AS unit_prices;
|
||||
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.
|
||||
-- Subqueries without relation with a volatile functions (non-constant)
|
||||
-- Subqueries without relation with a volatile functions (non-constant) are planned recursively
|
||||
SELECT count(*) FROM (
|
||||
SELECT l_orderkey FROM lineitem_subquery JOIN (SELECT random()::int r) sub ON (l_orderkey = r)
|
||||
SELECT l_orderkey FROM lineitem_subquery JOIN (SELECT random()::int r) sub ON (l_orderkey = r) WHERE r > 10
|
||||
) b;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause can only contain immutable functions
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
-- 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: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
-- Check that we error out if queries in union do not include partition columns.
|
||||
SELECT count(*) FROM
|
||||
(
|
||||
|
@ -104,7 +104,8 @@ SELECT count(*) FROM
|
|||
2985
|
||||
(1 row)
|
||||
|
||||
-- Check that we error out if inner query has Limit but subquery_pushdown is not set
|
||||
-- we'd error out if inner query has Limit but subquery_pushdown is not set
|
||||
-- but we recursively plan the query
|
||||
SELECT
|
||||
avg(o_totalprice/l_quantity)
|
||||
FROM
|
||||
|
@ -124,8 +125,11 @@ FROM
|
|||
orders_subquery
|
||||
WHERE
|
||||
lineitem_quantities.l_orderkey = o_orderkey) orders_price ON true;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
avg
|
||||
-------------------------
|
||||
129027.1270000000000000
|
||||
(1 row)
|
||||
|
||||
-- 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;
|
||||
|
|
|
@ -1185,7 +1185,9 @@ limit 50;
|
|||
|
||||
-- reset subquery_pushdown
|
||||
SET citus.subquery_pushdown to OFF;
|
||||
-- not supported since JOIN is not on the partition key
|
||||
-- we recursively plan recent_events_1
|
||||
-- but not some_users_data since it has a reference
|
||||
-- from an outer query which is not recursively planned
|
||||
SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT user_id, max(time) AS lastseen
|
||||
|
@ -1218,10 +1220,11 @@ FROM
|
|||
ORDER BY
|
||||
user_id
|
||||
limit 50;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- not supported since JOIN is not on the partition key
|
||||
-- see (2 * user_id as user_id) target list element
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
-- we recursively plan some queries but fail in the end
|
||||
-- since some_users_data since it has a reference
|
||||
-- from an outer query which is not recursively planned
|
||||
SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT 2 * user_id as user_id, max(time) AS lastseen
|
||||
|
@ -1254,8 +1257,8 @@ FROM
|
|||
ORDER BY
|
||||
user_id
|
||||
limit 50;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
-- LATERAL JOINs used with INNER JOINs
|
||||
SET citus.subquery_pushdown to ON;
|
||||
SELECT user_id, lastseen
|
||||
|
@ -1550,7 +1553,9 @@ ORDER BY
|
|||
LIMIT 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
-- not supported since lower LATERAL JOIN is not on the partition key
|
||||
-- not pushdownable since lower LATERAL JOIN is not on the partition key
|
||||
-- not recursively plannable due to LATERAL join where there is a reference
|
||||
-- from an outer query
|
||||
SELECT user_id, lastseen
|
||||
FROM
|
||||
(SELECT
|
||||
|
@ -1603,8 +1608,8 @@ FROM
|
|||
ORDER BY
|
||||
user_id DESC
|
||||
LIMIT 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
-- NESTED INNER JOINs
|
||||
SELECT
|
||||
count(*) AS value, "generated_group_field"
|
||||
|
@ -2168,7 +2173,9 @@ LIMIT 10;
|
|||
(1 row)
|
||||
|
||||
SET citus.subquery_pushdown to OFF;
|
||||
-- not supported since join is not on the partition key
|
||||
-- not pushdownable since lower LATERAL JOIN is not on the partition key
|
||||
-- not recursively plannable due to LATERAL join where there is a reference
|
||||
-- from an outer query
|
||||
SELECT *
|
||||
FROM
|
||||
(SELECT
|
||||
|
@ -2206,8 +2213,8 @@ FROM
|
|||
ORDER BY
|
||||
value_2 DESC, user_id DESC
|
||||
LIMIT 10;
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
-- lets test some unsupported set operations
|
||||
-- not supported since we use INTERSECT
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
|
@ -2270,8 +2277,8 @@ GROUP BY
|
|||
types
|
||||
ORDER BY
|
||||
types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Intersect and Except are currently unsupported
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- not supported due to offset
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
FROM
|
||||
|
@ -2333,8 +2340,8 @@ GROUP BY
|
|||
types
|
||||
ORDER BY
|
||||
types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Offset clause is currently unsupported
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- not supported due to non relation rte
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
FROM
|
||||
|
@ -2393,8 +2400,8 @@ GROUP BY
|
|||
types
|
||||
ORDER BY
|
||||
types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- similar to the above, but constant rte is on the right side of the query
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
FROM
|
||||
|
@ -2448,6 +2455,6 @@ GROUP BY
|
|||
types
|
||||
ORDER BY
|
||||
types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
SET citus.enable_router_execution TO TRUE;
|
||||
|
|
|
@ -230,8 +230,11 @@ SELECT count(*) FROM
|
|||
-- table function cannot be used without subquery pushdown
|
||||
SELECT count(*) FROM user_buy_test_table JOIN generate_series(1,10) AS users_ref_test_table(id)
|
||||
ON user_buy_test_table.item_id = users_ref_test_table.id;
|
||||
ERROR: could not run distributed query with complex table expressions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
count
|
||||
-------
|
||||
4
|
||||
(1 row)
|
||||
|
||||
-- table function can be the inner relationship in an outer join
|
||||
SELECT count(*) FROM
|
||||
(SELECT random() FROM user_buy_test_table LEFT JOIN generate_series(1,10) AS users_ref_test_table(id)
|
||||
|
@ -262,21 +265,19 @@ DETAIL: There exist a table function in the outer part of the outer join
|
|||
SELECT count(*) FROM
|
||||
(SELECT random() FROM user_buy_test_table JOIN random() AS users_ref_test_table(id)
|
||||
ON user_buy_test_table.item_id > users_ref_test_table.id) subquery_1;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Only immutable functions can be used as a table expressions in a multi-shard query
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
-- cannot sneak in a volatile function as a parameter
|
||||
SELECT count(*) FROM
|
||||
(SELECT random() FROM user_buy_test_table JOIN generate_series(random()::int,10) AS users_ref_test_table(id)
|
||||
ON user_buy_test_table.item_id > users_ref_test_table.id) subquery_1;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Only immutable functions can be used as a table expressions in a multi-shard query
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
-- cannot perform a union with table function
|
||||
SELECT count(*) FROM
|
||||
(SELECT user_id FROM user_buy_test_table
|
||||
UNION ALL
|
||||
SELECT id FROM generate_series(1,10) AS users_ref_test_table(id)) subquery_1;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Table functions are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- subquery without FROM can be the inner relationship in a join
|
||||
SELECT count(*) FROM
|
||||
(SELECT random() FROM user_buy_test_table JOIN (SELECT 4 AS id) users_ref_test_table
|
||||
|
@ -312,8 +313,8 @@ SELECT count(*) FROM
|
|||
(SELECT user_id FROM user_buy_test_table
|
||||
UNION ALL
|
||||
SELECT id FROM (SELECT 5 AS id) users_ref_test_table) subquery_1;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- should be able to pushdown since reference table is in the
|
||||
-- inner part of the left join
|
||||
SELECT
|
||||
|
@ -983,8 +984,8 @@ INNER JOIN
|
|||
ON (t.user_id = q.user_id)) as final_query
|
||||
ORDER BY
|
||||
types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Reference tables are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- reference table exist in the subquery of union, should error out
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
FROM
|
||||
|
@ -1056,8 +1057,8 @@ GROUP BY
|
|||
types
|
||||
ORDER BY
|
||||
types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Reference tables are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
--
|
||||
-- Should error out with UNION ALL Queries on reference tables
|
||||
--
|
||||
|
@ -1111,8 +1112,8 @@ INNER JOIN
|
|||
WHERE value_1 > 2 and value_1 < 4) AS t ON (t.user_id = q.user_id)) as final_query
|
||||
GROUP BY types
|
||||
ORDER BY types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Reference tables are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- just a sanity check that we don't allow this if the reference table is on the
|
||||
-- left part of the left join
|
||||
SELECT count(*) FROM
|
||||
|
@ -1202,15 +1203,16 @@ SELECT foo.user_id FROM
|
|||
---------
|
||||
(0 rows)
|
||||
|
||||
-- not supported since group by is on the reference table column
|
||||
-- not pushdownable since group by is on the reference table column
|
||||
-- recursively planned, but hits unsupported clause type error on the top level query
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
SELECT r.user_id, random() FROM users_table m JOIN events_reference_table r ON int4eq(m.user_id, r.user_id)
|
||||
GROUP BY r.user_id
|
||||
) as foo;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Group by list without partition column is currently unsupported
|
||||
-- supported since the group by contains at least one distributed table
|
||||
ERROR: unsupported clause type
|
||||
-- not pushdownable since the group by contains at least one distributed table
|
||||
-- recursively planned, but hits unsupported clause type error on the top level query
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
SELECT r.user_id, random() FROM users_table m JOIN events_reference_table r ON int4eq(m.user_id, r.user_id)
|
||||
|
@ -1224,20 +1226,19 @@ ORDER BY 1 LIMIT 3;
|
|||
3
|
||||
(3 rows)
|
||||
|
||||
-- not supported since distinct is on the reference table column
|
||||
-- not pushdownable since distinct is on the reference table column
|
||||
-- recursively planned, but hits unsupported clause type error on the top level query
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
SELECT DISTINCT r.user_id, random() FROM users_table m JOIN events_reference_table r ON int4eq(m.user_id, r.user_id)
|
||||
) as foo;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Distinct on columns without partition column is currently unsupported
|
||||
ERROR: unsupported clause type
|
||||
-- not supported since distinct on is on the reference table column
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
SELECT DISTINCT ON(r.user_id) r.user_id, random() FROM users_table m JOIN events_reference_table r ON int4eq(m.user_id, r.user_id)
|
||||
) as foo;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Distinct on columns without partition column is currently unsupported
|
||||
ERROR: unsupported clause type
|
||||
-- supported since the distinct on contains at least one distributed table
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
|
@ -1309,28 +1310,26 @@ OFFSET 0;
|
|||
(5 rows)
|
||||
|
||||
-- should not push down this query since there is a distributed table (i.e., events_table)
|
||||
-- which is not in the DISTINCT clause
|
||||
-- which is not in the DISTINCT clause. Recursive planning also fails since router execution
|
||||
-- is disabled
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT DISTINCT users_reference_table.user_id FROM users_reference_table, events_table WHERE users_reference_table.user_id = events_table.value_4
|
||||
) as foo;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Distinct on columns without partition column is currently unsupported
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT users_reference_table.user_id FROM users_reference_table, events_table WHERE users_reference_table.user_id = events_table.value_4
|
||||
GROUP BY 1
|
||||
) as foo;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Group by list without partition column is currently unsupported
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
-- similiar to the above examples, this time there is a subquery
|
||||
-- whose output is not in the DISTINCT clause
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT DISTINCT users_reference_table.user_id FROM users_reference_table, (SELECT user_id, random() FROM events_table) as us_events WHERE users_reference_table.user_id = us_events.user_id
|
||||
) as foo;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Distinct on columns without partition column is currently unsupported
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
-- the following query is safe to push down since the DISTINCT clause include distribution column
|
||||
SELECT * FROM
|
||||
(
|
||||
|
@ -1347,6 +1346,8 @@ LIMIT 4;
|
|||
(4 rows)
|
||||
|
||||
-- should not pushdown since there is a non partition column on the DISTINCT clause
|
||||
-- Recursive planning also fails since router execution
|
||||
-- is disabled
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT
|
||||
|
@ -1359,8 +1360,7 @@ SELECT * FROM
|
|||
) as foo
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 4;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Distinct on columns without partition column is currently unsupported
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
-- test the read_intermediate_result() for GROUP BYs
|
||||
BEGIN;
|
||||
|
||||
|
@ -1483,7 +1483,7 @@ LIMIT 5;
|
|||
5
|
||||
(5 rows)
|
||||
|
||||
-- should error out since there is a distributed table and
|
||||
-- should recursively plan since
|
||||
-- there are no columns on the GROUP BY from the distributed table
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
|
@ -1499,8 +1499,12 @@ JOIN
|
|||
ON (mx = user_id)
|
||||
ORDER BY 1
|
||||
LIMIT 5;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Group by list without partition column is currently unsupported
|
||||
user_id
|
||||
---------
|
||||
1
|
||||
4
|
||||
(2 rows)
|
||||
|
||||
ROLLBACK;
|
||||
-- should work since we're using an immutable function as recurring tuple
|
||||
SELECT
|
||||
|
@ -1521,7 +1525,7 @@ LIMIT 5;
|
|||
6
|
||||
(2 rows)
|
||||
|
||||
-- should not work since we're
|
||||
-- should recursively plan since we're
|
||||
-- using an immutable function as recurring tuple
|
||||
-- along with a distributed table, where GROUP BY is
|
||||
-- on the recurring tuple
|
||||
|
@ -1539,8 +1543,11 @@ JOIN
|
|||
ON (mx = user_id)
|
||||
ORDER BY 1
|
||||
LIMIT 5;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Group by list without partition column is currently unsupported
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
(1 row)
|
||||
|
||||
DROP TABLE user_buy_test_table;
|
||||
DROP TABLE users_ref_test_table;
|
||||
DROP TABLE users_return_test_table;
|
||||
|
|
|
@ -582,17 +582,20 @@ LIMIT 2;
|
|||
(2 rows)
|
||||
|
||||
-- subquery in where clause has a volatile function and no relation
|
||||
-- thus we recursively plan it
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
value_2 >
|
||||
(SELECT random())
|
||||
(SELECT random()) AND user_id < 0
|
||||
ORDER BY 1 ASC
|
||||
LIMIT 2;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause can only contain immutable functions
|
||||
user_id
|
||||
---------
|
||||
(0 rows)
|
||||
|
||||
-- OFFSET is not supported in the subquey
|
||||
SELECT
|
||||
user_id
|
||||
|
@ -614,7 +617,8 @@ ERROR: cannot push down this subquery
|
|||
DETAIL: Offset clause is currently unsupported
|
||||
-- we can detect unsupported subquerues even if they appear
|
||||
-- in WHERE subquery -> FROM subquery -> WHERE subquery
|
||||
SELECT user_id
|
||||
-- but we can recursively plan that anyway
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE user_id
|
||||
IN (SELECT
|
||||
|
@ -638,9 +642,13 @@ WHERE user_id
|
|||
AND e1.user_id IN (SELECT user_id FROM users_table LIMIT 3 )
|
||||
) as f_outer
|
||||
WHERE f_inner.user_id = f_outer.user_id
|
||||
);
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
) ORDER BY 1 LIMIT 3;
|
||||
user_id
|
||||
---------
|
||||
1
|
||||
5
|
||||
(2 rows)
|
||||
|
||||
-- semi join is not on the partition key for the third subquery
|
||||
SELECT user_id
|
||||
FROM users_table
|
||||
|
|
|
@ -47,8 +47,8 @@ FROM (
|
|||
) user_id
|
||||
ORDER BY 2 DESC,1
|
||||
LIMIT 5;
|
||||
ERROR: cannot pushdown this query
|
||||
DETAIL: Reference tables are not allowed with set operations
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- the same query with union all
|
||||
SELECT user_id, counter
|
||||
FROM (
|
||||
|
@ -76,8 +76,8 @@ FROM (
|
|||
) user_id
|
||||
ORDER BY 2 DESC,1
|
||||
LIMIT 5;
|
||||
ERROR: cannot pushdown this query
|
||||
DETAIL: Reference tables are not allowed with set operations
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- the same query with group by
|
||||
SELECT user_id, sum(counter)
|
||||
FROM (
|
||||
|
@ -212,8 +212,8 @@ FROM (
|
|||
SELECT user_id, sum(value_2) AS counter FROM users_table where value_1 < 5 and value_1 < 6 GROUP BY user_id HAVING sum(value_2) > 25
|
||||
) user_id
|
||||
GROUP BY user_id ORDER BY 1 DESC LIMIT 5;
|
||||
ERROR: cannot pushdown this query
|
||||
DETAIL: Reference tables are not allowed with set operations
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- similar query as above, with UNION ALL
|
||||
SELECT sum(counter)
|
||||
FROM (
|
||||
|
@ -329,8 +329,8 @@ FROM (
|
|||
user_id)) AS ftop
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
ERROR: cannot pushdown this query
|
||||
DETAIL: Reference tables are not allowed with set operations
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- top level unions are wrapped into top level aggregations
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
FROM
|
||||
|
@ -597,8 +597,8 @@ FROM
|
|||
UNION ALL
|
||||
(SELECT user_id FROM events_reference_table)
|
||||
) b;
|
||||
ERROR: cannot pushdown this query
|
||||
DETAIL: Reference tables are not allowed with set operations
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- similar query without top level agg
|
||||
SELECT
|
||||
user_id
|
||||
|
@ -739,6 +739,8 @@ LIMIT 5;
|
|||
|
||||
-- now lets also have some unsupported queries
|
||||
-- group by is not on the partition key
|
||||
-- but we can still recursively plan it, though that is not suffient for pushdown
|
||||
-- of the whole query
|
||||
SELECT user_id, sum(counter)
|
||||
FROM (
|
||||
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
|
||||
|
@ -746,8 +748,8 @@ FROM (
|
|||
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 not all subqueries in the UNION have the partition column in the same position
|
||||
DETAIL: Each leaf query of the UNION should return the partition column in the same position and all joins must be on the partition column
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- partition key is not selected
|
||||
SELECT sum(counter)
|
||||
FROM (
|
||||
|
@ -786,8 +788,8 @@ UNION
|
|||
) user_id_2
|
||||
GROUP BY user_id)
|
||||
) as ftop;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Intersect and Except are currently unsupported
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- non-equi join are not supported since there is no equivalence between the partition column
|
||||
SELECT user_id, sum(counter)
|
||||
FROM (
|
||||
|
@ -943,8 +945,8 @@ FROM (
|
|||
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
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- lower level union does not return partition key with the other relations
|
||||
SELECT *
|
||||
FROM (
|
||||
|
@ -1030,7 +1032,9 @@ FROM
|
|||
) b;
|
||||
ERROR: cannot pushdown the subquery since not all subqueries in the UNION have the partition column in the same position
|
||||
DETAIL: Each leaf query of the UNION should return the partition column in the same position and all joins must be on the partition column
|
||||
-- we don't support subqueries without relations
|
||||
-- we don't support pushing down subqueries without relations
|
||||
-- recursive planning can replace that query, though the whole
|
||||
-- query is not safe to pushdown
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
|
@ -1039,9 +1043,11 @@ FROM
|
|||
UNION ALL
|
||||
(SELECT 1)
|
||||
) b;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
-- we don't support subqueries without relations
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- we don't support pushing down subqueries without relations
|
||||
-- recursive planning can replace that query, though the whole
|
||||
-- query is not safe to pushdown
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
|
@ -1050,8 +1056,8 @@ FROM
|
|||
UNION ALL
|
||||
(SELECT (random() * 100)::int)
|
||||
) b;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- we don't support subqueries without relations
|
||||
SELECT
|
||||
user_id, value_3
|
||||
|
@ -1071,8 +1077,11 @@ FROM
|
|||
) b
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 5;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- we don't support pushing down subqueries without relations
|
||||
-- recursive planning can replace that query, though the whole
|
||||
-- query is not safe to pushdown
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
FROM
|
||||
( SELECT *, random()
|
||||
|
@ -1115,8 +1124,8 @@ FROM
|
|||
) as final_query
|
||||
GROUP BY types
|
||||
ORDER BY types;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Subqueries without a FROM clause are not supported with union operator
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
SET citus.enable_router_execution TO true;
|
||||
DROP TABLE events_reference_table;
|
||||
DROP TABLE users_reference_table;
|
||||
|
|
|
@ -790,8 +790,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- user needs to supply partition by which should
|
||||
-- include the distribution key
|
||||
SELECT
|
||||
|
@ -808,8 +808,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- user needs to supply partition by which should
|
||||
-- include the distribution key
|
||||
SELECT
|
||||
|
@ -826,8 +826,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- w2 should not be pushed down
|
||||
SELECT * FROM
|
||||
(
|
||||
|
@ -843,8 +843,8 @@ SELECT * FROM
|
|||
) as foo
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC NULLS LAST
|
||||
LIMIT 10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- w2 should not be pushed down
|
||||
SELECT * FROM
|
||||
(
|
||||
|
@ -862,8 +862,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC, 2 DESC NULLS LAST
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- GROUP BY includes the partition key, but not the WINDOW function
|
||||
SELECT
|
||||
user_id, time, my_rank
|
||||
|
@ -883,8 +883,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC,2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions without PARTITION BY on distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- GROUP BY includes the partition key, but not the WINDOW function
|
||||
SELECT
|
||||
user_id, time, my_rank
|
||||
|
@ -904,8 +904,8 @@ ORDER BY
|
|||
3 DESC, 1 DESC,2 DESC
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- Overriding window function but not supported
|
||||
SELECT * FROM (
|
||||
SELECT
|
||||
|
@ -923,8 +923,8 @@ SELECT * FROM (
|
|||
) a
|
||||
ORDER BY
|
||||
1,2,3;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- Aggregate function on distribution column should error out
|
||||
SELECT * FROM (
|
||||
SELECT
|
||||
|
@ -936,8 +936,8 @@ SELECT * FROM (
|
|||
) a
|
||||
ORDER BY
|
||||
1 DESC, 2 DESC;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- test with reference table partitioned on only a column from reference table
|
||||
SELECT *
|
||||
FROM
|
||||
|
@ -951,8 +951,8 @@ ORDER BY
|
|||
1, 2, 3
|
||||
LIMIT
|
||||
20;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- UNION ALL with only one of them is not partitioned over distribution column which
|
||||
-- should not be allowed.
|
||||
SELECT
|
||||
|
@ -974,8 +974,8 @@ FROM
|
|||
GROUP BY user_id
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- UNION with only one subquery which has a partition on non-distribution column should
|
||||
-- error out
|
||||
SELECT *
|
||||
|
@ -1011,6 +1011,6 @@ FROM (
|
|||
user_id)) AS ftop
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Window functions with PARTITION BY list missing distribution column is currently unsupported
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
DROP VIEW subq;
|
||||
|
|
|
@ -287,9 +287,17 @@ 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 GROUP BY of partition key
|
||||
SELECT * FROM lineitems_by_shipping_method;
|
||||
ERROR: Unrecognized range table id 1
|
||||
-- following will be supported via recursive planning
|
||||
SELECT * FROM lineitems_by_shipping_method ORDER BY 1,2 LIMIT 5;
|
||||
l_shipmode | cnt
|
||||
------------+------
|
||||
AIR | 1706
|
||||
FOB | 1709
|
||||
MAIL | 1739
|
||||
RAIL | 1706
|
||||
REG AIR | 1679
|
||||
(5 rows)
|
||||
|
||||
-- create a view with group by on partition column
|
||||
CREATE VIEW lineitems_by_orderkey AS
|
||||
SELECT
|
||||
|
@ -631,21 +639,39 @@ SELECT * FROM distinct_user_with_value_1_3 ORDER BY user_id;
|
|||
(6 rows)
|
||||
|
||||
-- distinct is not supported if it is on a non-partition key
|
||||
-- but will be supported via recursive planning
|
||||
CREATE VIEW distinct_value_1 AS SELECT DISTINCT value_1 FROM users_table WHERE value_2 = 3;
|
||||
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
|
||||
SELECT * FROM distinct_value_1 ORDER BY 1 DESC LIMIT 5;
|
||||
value_1
|
||||
---------
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
(5 rows)
|
||||
|
||||
-- CTEs are supported even if they are on views
|
||||
CREATE VIEW cte_view_1 AS
|
||||
WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 3) SELECT * FROM c1 WHERE value_2 < 4;
|
||||
SELECT * FROM cte_view_1;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: CTEs in subqueries are currently unsupported
|
||||
-- this is single shard query but still not supported since it has view + cte
|
||||
SELECT * FROM cte_view_1 ORDER BY 1,2,3,4,5 LIMIT 5;
|
||||
user_id | time | value_1 | value_2 | value_3 | value_4
|
||||
---------+---------------------------------+---------+---------+---------+---------
|
||||
1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 |
|
||||
2 | Thu Nov 23 13:52:54.83829 2017 | 3 | 1 | 4 |
|
||||
3 | Wed Nov 22 23:24:32.080584 2017 | 3 | 2 | 5 |
|
||||
4 | Wed Nov 22 23:59:46.493416 2017 | 3 | 1 | 3 |
|
||||
4 | Thu Nov 23 01:55:21.824618 2017 | 3 | 1 | 4 |
|
||||
(5 rows)
|
||||
|
||||
-- this is single shard query and still not supported since it has view + cte
|
||||
-- router planner can't detect it
|
||||
SELECT * FROM cte_view_1 WHERE user_id = 2;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: CTEs in subqueries are currently unsupported
|
||||
SELECT * FROM cte_view_1 WHERE user_id = 2 ORDER BY 1,2,3,4,5;
|
||||
user_id | time | value_1 | value_2 | value_3 | value_4
|
||||
---------+--------------------------------+---------+---------+---------+---------
|
||||
2 | Thu Nov 23 13:52:54.83829 2017 | 3 | 1 | 4 |
|
||||
(1 row)
|
||||
|
||||
-- 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 = 2) SELECT * FROM c1 WHERE value_1 = 3;
|
||||
|
@ -687,8 +713,16 @@ CREATE VIEW recent_10_users AS
|
|||
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
|
||||
user_id | lastseen
|
||||
---------+---------------------------------
|
||||
1 | Thu Nov 23 17:30:34.635085 2017
|
||||
3 | Thu Nov 23 17:18:51.048758 2017
|
||||
5 | Thu Nov 23 16:48:32.08896 2017
|
||||
4 | Thu Nov 23 15:32:02.360969 2017
|
||||
6 | Thu Nov 23 14:43:18.024104 2017
|
||||
2 | Thu Nov 23 13:52:54.83829 2017
|
||||
(6 rows)
|
||||
|
||||
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
|
||||
|
@ -792,8 +826,43 @@ EXPLAIN (COSTS FALSE) SELECT *
|
|||
(23 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
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------------------------------------------------------
|
||||
Limit
|
||||
-> Sort
|
||||
Sort Key: remote_scan."time" DESC
|
||||
-> Custom Scan (Citus Real-Time)
|
||||
-> Distributed Subplan 83_1
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: max((max(remote_scan.lastseen))) DESC
|
||||
-> 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
|
||||
-> Limit
|
||||
-> Sort
|
||||
Sort Key: (max("time")) DESC
|
||||
-> HashAggregate
|
||||
Group Key: user_id
|
||||
-> Seq Scan on users_table_1400000 users_table
|
||||
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: (intermediate_result.user_id = et.user_id)
|
||||
-> Function Scan on read_intermediate_result intermediate_result
|
||||
-> Hash
|
||||
-> Seq Scan on events_table_1400004 et
|
||||
(33 rows)
|
||||
|
||||
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
|
||||
|
|
|
@ -0,0 +1,194 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with subqueries and CTEs
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_deep;
|
||||
SET search_path TO subquery_and_ctes, public;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced due to OFFSET
|
||||
-- one level up subquery should be replaced due to GROUP BY on non partition key
|
||||
-- one level up subquery should be replaced due to LIMUT
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
avg(event_type) as avg_val
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar, users_table WHERE bar.user_id = users_table.user_id GROUP BY users_table.value_1
|
||||
) as baz
|
||||
WHERE baz.avg_val < users_table.user_id
|
||||
LIMIT 3
|
||||
) as sub1
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: generating subplan 1_1 for subquery SELECT user_id, event_type FROM public.events_table WHERE (value_2 < 3) OFFSET 3
|
||||
DEBUG: generating subplan 1_2 for subquery SELECT avg(bar.event_type) AS avg_val FROM (SELECT foo.event_type, users_table_1.user_id FROM public.users_table users_table_1, (SELECT intermediate_result.user_id, intermediate_result.event_type FROM read_intermediate_result('1_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, event_type integer)) foo WHERE (foo.user_id = users_table_1.user_id)) bar, public.users_table WHERE (bar.user_id = users_table.user_id) GROUP BY users_table.value_1
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 1_3 for subquery SELECT users_table.user_id FROM public.users_table, (SELECT intermediate_result.avg_val FROM read_intermediate_result('1_2'::text, 'binary'::citus_copy_format) intermediate_result(avg_val numeric)) baz WHERE (baz.avg_val < (users_table.user_id)::numeric) LIMIT 3
|
||||
user_id
|
||||
---------
|
||||
5
|
||||
(1 row)
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE -> WHERE should be replaced due to CTE
|
||||
-- subquery in FROM -> FROM -> WHERE should be replaced due to LIMIT
|
||||
-- one level above should be replaced due to DISTINCT on non-partition key
|
||||
-- one level above should be replaced due to GROUP BY on non-partition key
|
||||
SELECT event, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT event, array_agg(t.user_id) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
DISTINCT ON(e.event_type::text) e.event_type::text as event, e.time, e.user_id
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND EXISTS (WITH cte AS (SELECT count(*) FROM users_table) SELECT * FROM cte)
|
||||
LIMIT 5
|
||||
)
|
||||
) t, users_table WHERE users_table.value_1 = t.event::int
|
||||
GROUP BY event
|
||||
) q
|
||||
ORDER BY 2 DESC, 1;
|
||||
DEBUG: generating subplan 5_1 for CTE cte: SELECT count(*) AS count FROM public.users_table
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 5_2 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 >= 5) AND (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 1) AND (events_table.event_type <= 3) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))) AND (NOT (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 3) AND (events_table.event_type <= 4) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id))))) AND (EXISTS (SELECT cte.count FROM (SELECT intermediate_result.count FROM read_intermediate_result('5_1'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)) cte))) LIMIT 5
|
||||
DEBUG: generating subplan 5_3 for subquery SELECT DISTINCT ON ((e.event_type)::text) (e.event_type)::text AS event, e."time", e.user_id FROM public.users_table u, public.events_table e WHERE ((u.user_id = e.user_id) AND (u.user_id IN (SELECT intermediate_result.user_id FROM read_intermediate_result('5_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer))))
|
||||
DEBUG: generating subplan 5_4 for subquery SELECT t.event, array_agg(t.user_id) AS events_table FROM (SELECT intermediate_result.event, intermediate_result."time", intermediate_result.user_id FROM read_intermediate_result('5_3'::text, 'binary'::citus_copy_format) intermediate_result(event text, "time" timestamp without time zone, user_id integer)) t, public.users_table WHERE (users_table.value_1 = (t.event)::integer) GROUP BY t.event
|
||||
event | array_length
|
||||
-------+--------------
|
||||
3 | 26
|
||||
4 | 21
|
||||
2 | 18
|
||||
1 | 15
|
||||
0 | 12
|
||||
5 | 9
|
||||
(6 rows)
|
||||
|
||||
-- this test probably doesn't add too much value,
|
||||
-- but recurse 6 times for fun
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(users_table.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_2 FROM users_table GROUP BY value_2) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, users_table
|
||||
WHERE
|
||||
level_5.avg_ev_type = users_table.user_id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
DEBUG: generating subplan 10_1 for subquery SELECT count(*) AS cnt, value_2 FROM public.users_table GROUP BY value_2
|
||||
DEBUG: generating subplan 10_2 for subquery SELECT avg(events_table.event_type) AS avg FROM (SELECT level_1.cnt FROM (SELECT intermediate_result.cnt, intermediate_result.value_2 FROM read_intermediate_result('10_1'::text, 'binary'::citus_copy_format) intermediate_result(cnt bigint, value_2 integer)) level_1, public.users_table WHERE (users_table.user_id = level_1.cnt)) level_2, public.events_table WHERE (events_table.user_id = level_2.cnt) GROUP BY level_2.cnt
|
||||
DEBUG: generating subplan 10_3 for subquery SELECT max(users_table.value_1) AS mx_val_1 FROM (SELECT intermediate_result.avg FROM read_intermediate_result('10_2'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)) level_3, public.users_table WHERE ((users_table.user_id)::numeric = level_3.avg) GROUP BY level_3.avg
|
||||
DEBUG: generating subplan 10_4 for subquery SELECT avg(events_table.event_type) AS avg_ev_type FROM (SELECT intermediate_result.mx_val_1 FROM read_intermediate_result('10_3'::text, 'binary'::citus_copy_format) intermediate_result(mx_val_1 integer)) level_4, public.events_table WHERE (level_4.mx_val_1 = events_table.user_id) GROUP BY level_4.mx_val_1
|
||||
DEBUG: generating subplan 10_5 for subquery SELECT min(users_table.value_1) AS min FROM (SELECT intermediate_result.avg_ev_type FROM read_intermediate_result('10_4'::text, 'binary'::citus_copy_format) intermediate_result(avg_ev_type numeric)) level_5, public.users_table WHERE (level_5.avg_ev_type = (users_table.user_id)::numeric) GROUP BY level_5.avg_ev_type
|
||||
DEBUG: generating subplan 10_6 for subquery SELECT avg(level_6.min) AS avg FROM (SELECT intermediate_result.min FROM read_intermediate_result('10_5'::text, 'binary'::citus_copy_format) intermediate_result(min integer)) level_6, public.users_table WHERE (users_table.user_id = level_6.min) GROUP BY users_table.value_1
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
-- same query happening in the subqueries in WHERE
|
||||
-- this test probably doesn't add too much value,
|
||||
-- but recurse 6 times for fun
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
users_table
|
||||
WHERE user_id IN (
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(users_table.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_2 FROM users_table GROUP BY value_2) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, users_table
|
||||
WHERE
|
||||
level_5.avg_ev_type = users_table.user_id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar);
|
||||
DEBUG: generating subplan 17_1 for subquery SELECT count(*) AS cnt, value_2 FROM public.users_table GROUP BY value_2
|
||||
DEBUG: generating subplan 17_2 for subquery SELECT avg(events_table.event_type) AS avg FROM (SELECT level_1.cnt FROM (SELECT intermediate_result.cnt, intermediate_result.value_2 FROM read_intermediate_result('17_1'::text, 'binary'::citus_copy_format) intermediate_result(cnt bigint, value_2 integer)) level_1, public.users_table WHERE (users_table.user_id = level_1.cnt)) level_2, public.events_table WHERE (events_table.user_id = level_2.cnt) GROUP BY level_2.cnt
|
||||
DEBUG: generating subplan 17_3 for subquery SELECT max(users_table.value_1) AS mx_val_1 FROM (SELECT intermediate_result.avg FROM read_intermediate_result('17_2'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)) level_3, public.users_table WHERE ((users_table.user_id)::numeric = level_3.avg) GROUP BY level_3.avg
|
||||
DEBUG: generating subplan 17_4 for subquery SELECT avg(events_table.event_type) AS avg_ev_type FROM (SELECT intermediate_result.mx_val_1 FROM read_intermediate_result('17_3'::text, 'binary'::citus_copy_format) intermediate_result(mx_val_1 integer)) level_4, public.events_table WHERE (level_4.mx_val_1 = events_table.user_id) GROUP BY level_4.mx_val_1
|
||||
DEBUG: generating subplan 17_5 for subquery SELECT min(users_table.value_1) AS min FROM (SELECT intermediate_result.avg_ev_type FROM read_intermediate_result('17_4'::text, 'binary'::citus_copy_format) intermediate_result(avg_ev_type numeric)) level_5, public.users_table WHERE (level_5.avg_ev_type = (users_table.user_id)::numeric) GROUP BY level_5.avg_ev_type
|
||||
DEBUG: generating subplan 17_6 for subquery SELECT avg(level_6.min) AS avg FROM (SELECT intermediate_result.min FROM read_intermediate_result('17_5'::text, 'binary'::citus_copy_format) intermediate_result(min integer)) level_6, public.users_table WHERE (users_table.user_id = level_6.min) GROUP BY users_table.value_1
|
||||
DEBUG: generating subplan 17_7 for subquery SELECT count(*) AS count FROM (SELECT intermediate_result.avg FROM read_intermediate_result('17_6'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)) bar
|
||||
user_id | time | value_1 | value_2 | value_3 | value_4
|
||||
---------+------+---------+---------+---------+---------
|
||||
(0 rows)
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_deep CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,147 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on failure cases
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA not_supported;
|
||||
SET search_path TO not_supported, public;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
-- we don't support subqueries with local tables when they are not leaf queries
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
users_table_local.user_id
|
||||
FROM
|
||||
users_table_local, (SELECT user_id FROM events_table) as evs
|
||||
WHERE users_table_local.user_id = evs.user_id
|
||||
) as foo;
|
||||
ERROR: relation users_table_local is not distributed
|
||||
-- we don't support subqueries with local tables when they are not leaf queries
|
||||
SELECT user_id FROM users_table WHERE user_id IN
|
||||
(SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table_local JOIN (SELECT user_id FROM events_table_local) as foo
|
||||
USING (user_id)
|
||||
);
|
||||
ERROR: relation "events_table_local" does not exist
|
||||
LINE 5: users_table_local JOIN (SELECT user_id FROM events_table_...
|
||||
^
|
||||
-- we don't support aggregate distinct if the group by is not on partition key, expect for count distinct
|
||||
-- thus baz and bar are recursively planned but not foo
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT avg(DISTINCT value_1), random() FROM users_table GROUP BY user_id OFFSET 3
|
||||
) as baz,
|
||||
(
|
||||
SELECT count(DISTINCT value_1), random() FROM users_table GROUP BY value_2 OFFSET 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT avg(DISTINCT value_1), random() FROM users_table GROUP BY value_2 OFFSET 3
|
||||
) as foo;
|
||||
DEBUG: generating subplan 4_1 for subquery SELECT avg(DISTINCT value_1) AS avg, random() AS random FROM public.users_table GROUP BY user_id OFFSET 3
|
||||
DEBUG: generating subplan 4_2 for subquery SELECT count(DISTINCT value_1) AS count, random() AS random FROM public.users_table GROUP BY value_2 OFFSET 3
|
||||
ERROR: cannot compute aggregate (distinct)
|
||||
DETAIL: table partitioning is unsuitable for aggregate (distinct)
|
||||
-- we don't support array_aggs with ORDER BYs
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
array_agg(users_table.user_id ORDER BY users_table.time)
|
||||
FROM
|
||||
users_table, (SELECT user_id FROM events_table) as evs
|
||||
WHERE users_table.user_id = evs.user_id
|
||||
GROUP BY users_table.user_id
|
||||
LIMIT 5
|
||||
) as foo;
|
||||
ERROR: array_agg with order by is unsupported
|
||||
-- we don't support queries with recurring tuples in the FROM
|
||||
-- clause and subquery in WHERE clause
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, (SELECT user_id FROM events_table) as evs
|
||||
WHERE users_table.user_id = evs.user_id
|
||||
LIMIT 5
|
||||
) as foo WHERE user_id IN (SELECT count(*) FROM users_table GROUP BY user_id);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 10_1 for subquery SELECT users_table.user_id FROM public.users_table, (SELECT events_table.user_id FROM public.events_table) evs WHERE (users_table.user_id = evs.user_id) LIMIT 5
|
||||
ERROR: cannot pushdown the subquery
|
||||
DETAIL: Complex subqueries and CTEs are not allowed in the FROM clause when the query has subqueries in the WHERE clause
|
||||
-- we don't support recursive subqueries when router executor is disabled
|
||||
SET citus.enable_router_execution TO false;
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 12_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
ERROR: cannot handle complex subqueries when the router executor is disabled
|
||||
SET citus.enable_router_execution TO true;
|
||||
-- window functions are not allowed if they're not partitioned on the distribution column
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
user_id, time, rnk
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
*, rank() OVER my_win as rnk
|
||||
FROM
|
||||
events_table
|
||||
WINDOW my_win AS (PARTITION BY event_type ORDER BY time DESC)
|
||||
) as foo
|
||||
ORDER BY
|
||||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10) as foo;
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- top level join is not on the distribution key thus not supported
|
||||
-- (use random to prevent Postgres to pull subqueries)
|
||||
SELECT
|
||||
foo.value_2
|
||||
FROM
|
||||
(SELECT users_table.value_2, random() FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (1,2,3,4)) as foo,
|
||||
(SELECT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar
|
||||
WHERE
|
||||
foo.value_2 = bar.value_2;
|
||||
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.
|
||||
-- OUTER JOINs where the outer part is recursively planned and not the other way
|
||||
-- around is not supported
|
||||
SELECT
|
||||
foo.value_2
|
||||
FROM
|
||||
(SELECT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (1,2,3,4) LIMIT 5) as foo
|
||||
LEFT JOIN
|
||||
(SELECT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar
|
||||
ON(foo.value_2 = bar.value_2);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 17_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) LIMIT 5
|
||||
ERROR: cannot pushdown the subquery
|
||||
DETAIL: Complex subqueries and CTEs cannot be in the outer part of the outer join
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA not_supported CASCADE;
|
||||
NOTICE: drop cascades to table users_table_local
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,422 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with subqueries and CTEs
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_and_ctes;
|
||||
SET search_path TO subquery_and_ctes, public;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- CTEs are recursively planned, and subquery foo is also recursively planned
|
||||
-- final plan becomes a router plan
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id;
|
||||
DEBUG: generating subplan 2_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_ctes.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 3_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_ctes.users_table_local
|
||||
DEBUG: generating subplan 3_2 for CTE dist_cte: SELECT user_id FROM public.events_table
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 2_2 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
count
|
||||
-------
|
||||
1644
|
||||
(1 row)
|
||||
|
||||
-- CTEs are recursively planned, and subquery foo is also recursively planned
|
||||
-- final plan becomes a real-time plan since we also have events_table in the
|
||||
-- range table entries
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo, events_table
|
||||
WHERE foo.user_id = cte.user_id AND events_table.user_id = cte.user_id;
|
||||
DEBUG: generating subplan 6_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_ctes.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 7_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_ctes.users_table_local
|
||||
DEBUG: generating subplan 7_2 for CTE dist_cte: SELECT user_id FROM public.events_table
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 6_2 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
count
|
||||
-------
|
||||
30608
|
||||
(1 row)
|
||||
|
||||
-- CTEs are replaced and subquery in WHERE is also replaced
|
||||
-- but the query is still real-time query since users_table is in the
|
||||
-- range table list
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM users_table, cte
|
||||
WHERE
|
||||
users_table.user_id = cte.user_id AND
|
||||
users_table.user_id IN (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5)
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: generating subplan 10_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_ctes.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 11_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_ctes.users_table_local
|
||||
DEBUG: generating subplan 11_2 for CTE dist_cte: SELECT user_id FROM public.events_table
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 10_2 for subquery SELECT DISTINCT value_2 FROM public.users_table WHERE ((value_1 >= 1) AND (value_1 <= 20)) ORDER BY value_2 LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
(4 rows)
|
||||
|
||||
-- a very similar query as the above, but this time errors
|
||||
-- out since we don't support subqueries in WHERE clause
|
||||
-- when there is only intermediate results on the range table
|
||||
-- note that this time subquery in WHERE clause is not replaced
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM cte
|
||||
WHERE
|
||||
cte.user_id IN (SELECT DISTINCT user_id FROM users_table WHERE value_1 >= 1 AND value_1 <= 20)
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: generating subplan 14_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_ctes.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 15_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_ctes.users_table_local
|
||||
DEBUG: generating subplan 15_2 for CTE dist_cte: SELECT user_id FROM public.events_table
|
||||
ERROR: cannot pushdown the subquery
|
||||
DETAIL: Complex subqueries and CTEs are not allowed in the FROM clause when the query has subqueries in the WHERE clause
|
||||
-- CTEs inside a subquery and the final query becomes a router
|
||||
-- query
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo;
|
||||
DEBUG: generating subplan 17_1 for CTE cte: SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4])))
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
(6 rows)
|
||||
|
||||
-- CTEs inside a subquery and the final query becomes a
|
||||
-- real-time query since the other subquery is safe to pushdown
|
||||
SELECT
|
||||
bar.user_id
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
|
||||
) as bar
|
||||
WHERE foo.user_id = bar.user_id;
|
||||
DEBUG: generating subplan 19_1 for CTE cte: SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4])))
|
||||
user_id
|
||||
---------
|
||||
5
|
||||
1
|
||||
4
|
||||
3
|
||||
6
|
||||
2
|
||||
(6 rows)
|
||||
|
||||
-- CTEs inside a deeper subquery
|
||||
-- and also the subquery that contains the CTE is replaced
|
||||
SELECT
|
||||
DISTINCT bar.user_id
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
users_table.user_id, some_events.event_type
|
||||
FROM
|
||||
users_table,
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
event_type, users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
value_1 IN (1,2)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as some_events
|
||||
WHERE
|
||||
users_table.user_id = some_events.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 2,1
|
||||
LIMIT 2
|
||||
|
||||
) as bar
|
||||
WHERE foo.user_id = bar.user_id
|
||||
ORDER BY 1 DESC LIMIT 5;
|
||||
DEBUG: generating subplan 21_1 for CTE cte: SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4])))
|
||||
DEBUG: generating subplan 21_2 for CTE cte: SELECT events_table.event_type, users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (users_table.value_1 = ANY (ARRAY[1, 2])))
|
||||
DEBUG: push down of limit count: 2
|
||||
DEBUG: generating subplan 21_3 for subquery SELECT users_table.user_id, some_events.event_type FROM public.users_table, (SELECT cte.event_type, cte.user_id FROM (SELECT intermediate_result.event_type, intermediate_result.user_id FROM read_intermediate_result('21_2'::text, 'binary'::citus_copy_format) intermediate_result(event_type integer, user_id integer)) cte ORDER BY cte.event_type DESC) some_events WHERE ((users_table.user_id = some_events.user_id) AND (some_events.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY some_events.event_type, users_table.user_id LIMIT 2
|
||||
user_id
|
||||
---------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- CTEs on the different parts of the query is replaced
|
||||
-- and subquery foo is also replaced since it contains
|
||||
-- DISTINCT on a non-partition key
|
||||
SELECT * FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM users_table, cte
|
||||
WHERE
|
||||
users_table.user_id = cte.user_id AND
|
||||
users_table.user_id IN
|
||||
(WITH cte_in_where AS (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5) SELECT * FROM cte_in_where)
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
events_table
|
||||
WHERE
|
||||
foo.user_id = events_table.value_2
|
||||
ORDER BY 3 DESC, 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: generating subplan 25_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_ctes.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 26_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_ctes.users_table_local
|
||||
DEBUG: generating subplan 26_2 for CTE dist_cte: SELECT user_id FROM public.events_table
|
||||
DEBUG: generating subplan 25_2 for CTE cte_in_where: SELECT DISTINCT value_2 FROM public.users_table WHERE ((value_1 >= 1) AND (value_1 <= 20)) ORDER BY value_2 LIMIT 5
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 25_3 for subquery SELECT DISTINCT cte.user_id FROM public.users_table, (SELECT intermediate_result.user_id FROM read_intermediate_result('25_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) cte WHERE ((users_table.user_id = cte.user_id) AND (users_table.user_id IN (SELECT cte_in_where.value_2 FROM (SELECT intermediate_result.value_2 FROM read_intermediate_result('25_2'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) cte_in_where))) ORDER BY cte.user_id DESC
|
||||
DEBUG: push down of limit count: 5
|
||||
user_id | user_id | time | event_type | value_2 | value_3 | value_4
|
||||
---------+---------+---------------------------------+------------+---------+---------+---------
|
||||
4 | 1 | Thu Nov 23 21:54:46.924477 2017 | 6 | 4 | 5 |
|
||||
2 | 4 | Thu Nov 23 18:10:21.338399 2017 | 1 | 2 | 4 |
|
||||
4 | 3 | Thu Nov 23 18:08:26.550729 2017 | 2 | 4 | 3 |
|
||||
2 | 3 | Thu Nov 23 16:44:41.903713 2017 | 4 | 2 | 2 |
|
||||
1 | 3 | Thu Nov 23 16:31:56.219594 2017 | 5 | 1 | 2 |
|
||||
(5 rows)
|
||||
|
||||
-- now recursively plan subqueries inside the CTEs that contains LIMIT and OFFSET
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_2 FROM users_table OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_2 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id;
|
||||
DEBUG: generating subplan 30_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_ctes.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT users_table.value_2 FROM public.users_table OFFSET 0) foo WHERE ((events_table.user_id = foo.value_2) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 31_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_ctes.users_table_local
|
||||
DEBUG: generating subplan 31_2 for CTE dist_cte: SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT users_table.value_2 FROM public.users_table OFFSET 0) foo WHERE ((events_table.user_id = foo.value_2) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 32_1 for subquery SELECT DISTINCT value_1 FROM public.users_table ORDER BY value_1 LIMIT 3
|
||||
DEBUG: generating subplan 32_2 for subquery SELECT DISTINCT value_2 FROM public.users_table OFFSET 0
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 30_2 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
count
|
||||
-------
|
||||
432
|
||||
(1 row)
|
||||
|
||||
-- the same query, but this time the CTEs also live inside a subquery
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_2 FROM users_table OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_2 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: generating subplan 36_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_ctes.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT users_table.value_2 FROM public.users_table OFFSET 0) foo WHERE ((events_table.user_id = foo.value_2) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 37_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_ctes.users_table_local
|
||||
DEBUG: generating subplan 37_2 for CTE dist_cte: SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT users_table.value_2 FROM public.users_table OFFSET 0) foo WHERE ((events_table.user_id = foo.value_2) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 38_1 for subquery SELECT DISTINCT value_1 FROM public.users_table ORDER BY value_1 LIMIT 3
|
||||
DEBUG: generating subplan 38_2 for subquery SELECT DISTINCT value_2 FROM public.users_table OFFSET 0
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 36_2 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
DEBUG: generating subplan 36_3 for subquery SELECT count(*) AS cnt FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('36_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) cte, (SELECT intermediate_result.user_id FROM read_intermediate_result('36_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo WHERE (foo.user_id = cte.user_id)
|
||||
DEBUG: push down of limit count: 5
|
||||
cnt | user_id | time | value_1 | value_2 | value_3 | value_4
|
||||
-----+---------+---------------------------------+---------+---------+---------+---------
|
||||
432 | 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 |
|
||||
432 | 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 |
|
||||
432 | 3 | Thu Nov 23 17:18:51.048758 2017 | 1 | 5 | 5 |
|
||||
432 | 3 | Thu Nov 23 17:10:35.959913 2017 | 4 | 3 | 1 |
|
||||
432 | 5 | Thu Nov 23 16:48:32.08896 2017 | 5 | 2 | 1 |
|
||||
(5 rows)
|
||||
|
||||
-- recursive CTES are not supported inside subqueries as well
|
||||
SELECT
|
||||
bar.user_id
|
||||
FROM
|
||||
(
|
||||
WITH RECURSIVE cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
|
||||
) as bar
|
||||
WHERE foo.user_id = bar.user_id;
|
||||
ERROR: recursive CTEs are not supported in distributed queries
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_and_ctes CASCADE;
|
||||
NOTICE: drop cascades to table users_table_local
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,347 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality
|
||||
-- ===================================================================
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- subqueries in FROM clause with LIMIT should be recursively planned
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 1_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
(5 rows)
|
||||
|
||||
-- subqueries in FROM clause with DISTINCT on non-partition key
|
||||
-- should be recursively planned
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.value_1
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: generating subplan 3_1 for subquery SELECT DISTINCT users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.value_1
|
||||
value_1
|
||||
---------
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
0
|
||||
(6 rows)
|
||||
|
||||
-- subqueries in FROM clause with GROUP BY on non-partition key
|
||||
-- should be recursively planned
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2, avg(value_1)
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo
|
||||
ORDER BY 2 DESC, 1;
|
||||
DEBUG: generating subplan 5_1 for subquery SELECT users_table.value_2, avg(users_table.value_1) AS avg FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) GROUP BY users_table.value_2 ORDER BY users_table.value_2 DESC
|
||||
value_2 | avg
|
||||
---------+--------------------
|
||||
4 | 2.8453608247422680
|
||||
2 | 2.6833855799373041
|
||||
5 | 2.6238938053097345
|
||||
1 | 2.3569131832797428
|
||||
3 | 2.3424124513618677
|
||||
0 | 2.0940170940170940
|
||||
(6 rows)
|
||||
|
||||
-- multiple subqueries in FROM clause should be replaced
|
||||
-- and the final query is router query
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.value_3
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
GROUP BY users_table.value_3
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.value_3
|
||||
ORDER BY 2 DESC, 1;
|
||||
DEBUG: generating subplan 7_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) GROUP BY users_table.value_2 ORDER BY users_table.value_2 DESC
|
||||
DEBUG: generating subplan 7_2 for subquery SELECT users_table.value_3 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[5, 6, 7, 8]))) GROUP BY users_table.value_3 ORDER BY users_table.value_3 DESC
|
||||
value_2 | value_3
|
||||
---------+---------
|
||||
5 | 5
|
||||
4 | 4
|
||||
3 | 3
|
||||
2 | 2
|
||||
1 | 1
|
||||
0 | 0
|
||||
(6 rows)
|
||||
|
||||
-- same query with alias in the subquery
|
||||
SELECT
|
||||
DISTINCT ON (citus) citus, postgres, citus + 1 as c1, postgres-1 as p1
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo(postgres),
|
||||
(SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC
|
||||
) as bar (citus)
|
||||
WHERE foo.postgres = bar.citus
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: generating subplan 10_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) GROUP BY users_table.value_2 ORDER BY users_table.value_2 DESC
|
||||
DEBUG: push down of limit count: 3
|
||||
citus | postgres | c1 | p1
|
||||
-------+----------+----+----
|
||||
5 | 5 | 6 | 4
|
||||
4 | 4 | 5 | 3
|
||||
3 | 3 | 4 | 2
|
||||
(3 rows)
|
||||
|
||||
-- foo is replaced
|
||||
-- and the final query is real-time
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: generating subplan 12_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) GROUP BY users_table.value_2 ORDER BY users_table.value_2 DESC
|
||||
DEBUG: push down of limit count: 3
|
||||
value_2 | user_id
|
||||
---------+---------
|
||||
5 | 5
|
||||
5 | 5
|
||||
5 | 5
|
||||
(3 rows)
|
||||
|
||||
-- subqueries in WHERE should be replaced
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE
|
||||
user_id IN (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5)
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 14_1 for subquery SELECT DISTINCT value_2 FROM public.users_table WHERE ((value_1 >= 1) AND (value_1 <= 20)) ORDER BY value_2 LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
(4 rows)
|
||||
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced due to OFFSET
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
event_type, user_id
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar
|
||||
) as baz
|
||||
WHERE baz.user_id = users_table.user_id
|
||||
) as sub1
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: generating subplan 16_1 for subquery SELECT user_id, event_type FROM public.events_table WHERE (value_2 < 3) OFFSET 3
|
||||
DEBUG: push down of limit count: 3
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
5
|
||||
4
|
||||
(3 rows)
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE should be replaced due to LIMIT
|
||||
SELECT user_id, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT user_id, array_agg(event ORDER BY time) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
u.user_id, e.event_type::text AS event, e.time
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
LIMIT 5
|
||||
)
|
||||
) t
|
||||
GROUP BY user_id
|
||||
) q
|
||||
ORDER BY 2 DESC, 1;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 18_1 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 >= 5) AND (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 1) AND (events_table.event_type <= 3) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))) AND (NOT (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 3) AND (events_table.event_type <= 4) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))))) LIMIT 5
|
||||
user_id | array_length
|
||||
---------+--------------
|
||||
5 | 364
|
||||
(1 row)
|
||||
|
||||
-- subquery (i.e., subquery_2) in WHERE->FROM should be replaced due to LIMIT
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM (
|
||||
SELECT
|
||||
subquery_1.user_id, count_pay
|
||||
FROM
|
||||
(
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>1' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 1 AND events_table.event_type < 3
|
||||
)
|
||||
UNION
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>2' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 2 AND events_table.event_type < 4
|
||||
)
|
||||
) AS subquery_1
|
||||
LEFT JOIN
|
||||
(SELECT
|
||||
user_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
user_id >= 1 AND
|
||||
user_id <= 3 AND
|
||||
users_table.value_1 > 3 AND users_table.value_1 < 5
|
||||
GROUP BY
|
||||
user_id
|
||||
HAVING
|
||||
COUNT(*) > 1
|
||||
LIMIT 10
|
||||
) AS subquery_2
|
||||
ON
|
||||
subquery_1.user_id = subquery_2.user_id
|
||||
GROUP BY
|
||||
subquery_1.user_id,
|
||||
count_pay) AS subquery_top
|
||||
GROUP BY
|
||||
count_pay, user_id
|
||||
)
|
||||
GROUP BY user_id
|
||||
HAVING count(*) > 1 AND sum(value_2) > 29
|
||||
ORDER BY 1;
|
||||
DEBUG: push down of limit count: 10
|
||||
DEBUG: generating subplan 20_1 for subquery SELECT user_id, count(*) AS count_pay FROM public.users_table WHERE ((user_id >= 1) AND (user_id <= 3) AND (value_1 > 3) AND (value_1 < 5)) GROUP BY user_id HAVING (count(*) > 1) LIMIT 10
|
||||
user_id
|
||||
---------
|
||||
2
|
||||
3
|
||||
(2 rows)
|
||||
|
|
@ -0,0 +1,389 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with complex target entries
|
||||
-- and some utilities
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_complex;
|
||||
SET search_path TO subquery_complex, public;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- COUNT DISTINCT at the top level query
|
||||
SELECT
|
||||
event_type, count(distinct value_2)
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id IN (SELECT user_id FROM users_table GROUP BY user_id ORDER BY count(*) DESC LIMIT 20)
|
||||
GROUP BY
|
||||
event_type
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: push down of limit count: 20
|
||||
DEBUG: generating subplan 1_1 for subquery SELECT user_id FROM public.users_table GROUP BY user_id ORDER BY (count(*)) DESC LIMIT 20
|
||||
event_type | count
|
||||
------------+-------
|
||||
6 | 1
|
||||
5 | 3
|
||||
4 | 6
|
||||
(3 rows)
|
||||
|
||||
-- aggregate distinct in the subqueries
|
||||
-- avg distinct on partition key
|
||||
-- count distinct on partition key
|
||||
-- count distinct on non-partition key
|
||||
-- sum distinct on non-partition key when group by is partition key
|
||||
-- and the final query is real-time query
|
||||
SELECT
|
||||
DISTINCT ON (avg) avg, cnt_1, cnt_2, sum
|
||||
FROM
|
||||
(
|
||||
SELECT avg(distinct user_id) as avg FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as foo,
|
||||
(
|
||||
SELECT count(distinct user_id) as cnt_1 FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT count(distinct value_2) as cnt_2 FROM users_table ORDER BY 1 DESC LIMIT 4
|
||||
) as baz,
|
||||
(
|
||||
SELECT user_id, sum(distinct value_2) as sum FROM users_table GROUP BY user_id ORDER BY 1 DESC LIMIT 4
|
||||
) as bat, events_table
|
||||
WHERE foo.avg != bar.cnt_1 AND baz.cnt_2 = events_table.event_type
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 3_1 for subquery SELECT avg(DISTINCT user_id) AS avg FROM public.users_table ORDER BY (avg(DISTINCT user_id)) DESC LIMIT 3
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 3_2 for subquery SELECT count(DISTINCT user_id) AS cnt_1 FROM public.users_table ORDER BY (count(DISTINCT user_id)) DESC LIMIT 3
|
||||
DEBUG: generating subplan 3_3 for subquery SELECT count(DISTINCT value_2) AS cnt_2 FROM public.users_table ORDER BY (count(DISTINCT value_2)) DESC LIMIT 4
|
||||
DEBUG: push down of limit count: 4
|
||||
DEBUG: generating subplan 3_4 for subquery SELECT user_id, sum(DISTINCT value_2) AS sum FROM public.users_table GROUP BY user_id ORDER BY user_id DESC LIMIT 4
|
||||
avg | cnt_1 | cnt_2 | sum
|
||||
--------------------+-------+-------+-----
|
||||
3.5000000000000000 | 6 | 6 | 10
|
||||
(1 row)
|
||||
|
||||
-- Aggregate type conversions inside the subqueries
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
min(user_id) * 2, max(user_id) / 2, sum(user_id), count(user_id)::float, avg(user_id)::bigint
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
min(value_3) * 2, max(value_3) / 2, sum(value_3), count(value_3), avg(value_3)
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT
|
||||
min(time), max(time), count(time),
|
||||
count(*) FILTER (WHERE user_id = 3) as cnt_with_filter,
|
||||
count(*) FILTER (WHERE user_id::text LIKE '%3%') as cnt_with_filter_2
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as baz
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 8_1 for subquery SELECT (min(user_id) * 2), (max(user_id) / 2), sum(user_id) AS sum, (count(user_id))::double precision AS count, (avg(user_id))::bigint AS avg FROM public.users_table ORDER BY (min(user_id) * 2) DESC LIMIT 3
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 8_2 for subquery SELECT (min(value_3) * (2)::double precision), (max(value_3) / (2)::double precision), sum(value_3) AS sum, count(value_3) AS count, avg(value_3) AS avg FROM public.users_table ORDER BY (min(value_3) * (2)::double precision) DESC LIMIT 3
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 8_3 for subquery SELECT min("time") AS min, max("time") AS max, count("time") AS count, count(*) FILTER (WHERE (user_id = 3)) AS cnt_with_filter, count(*) FILTER (WHERE ((user_id)::text ~~ '%3%'::text)) AS cnt_with_filter_2 FROM public.users_table ORDER BY (min("time")) DESC LIMIT 3
|
||||
?column? | ?column? | sum | count | avg | ?column? | ?column? | sum | count | avg | min | max | count | cnt_with_filter | cnt_with_filter_2
|
||||
----------+----------+-----+-------+-----+----------+----------+-----+-------+-----------------+---------------------------------+---------------------------------+-------+-----------------+-------------------
|
||||
2 | 3 | 376 | 101 | 4 | 0 | 2.5 | 273 | 101 | 2.7029702970297 | Wed Nov 22 18:19:49.944985 2017 | Thu Nov 23 17:30:34.635085 2017 | 101 | 17 | 17
|
||||
(1 row)
|
||||
|
||||
-- Expressions inside the aggregates
|
||||
-- parts of the query is inspired by TPCH queries
|
||||
SELECT
|
||||
DISTINCT ON (avg) avg, cnt_1, cnt_2, cnt_3, sum_1,l_year, pos, count_pay
|
||||
FROM
|
||||
(
|
||||
SELECT avg(user_id * (5.0 / (value_1 + 0.1))) as avg FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as foo,
|
||||
(
|
||||
SELECT sum(user_id * (5.0 / (value_1 + value_2 + 0.1)) * value_3) as cnt_1 FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT
|
||||
avg(case
|
||||
when user_id > 4
|
||||
then value_1
|
||||
end) as cnt_2,
|
||||
avg(case
|
||||
when user_id > 500
|
||||
then value_1
|
||||
end) as cnt_3,
|
||||
sum(case
|
||||
when value_1 = 1
|
||||
OR value_2 = 1
|
||||
then 1
|
||||
else 0
|
||||
end) as sum_1,
|
||||
extract(year FROM max(time)) as l_year,
|
||||
strpos(max(user_id)::text, '1') as pos
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY
|
||||
1 DESC
|
||||
LIMIT 4
|
||||
) as baz,
|
||||
(
|
||||
SELECT COALESCE(value_3, 20) AS count_pay FROM users_table ORDER BY 1 OFFSET 20 LIMIT 5
|
||||
) as tar,
|
||||
events_table
|
||||
WHERE foo.avg != bar.cnt_1 AND baz.cnt_2 != events_table.event_type
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 12_1 for subquery SELECT avg(((user_id)::numeric * (5.0 / ((value_1)::numeric + 0.1)))) AS avg FROM public.users_table ORDER BY (avg(((user_id)::numeric * (5.0 / ((value_1)::numeric + 0.1))))) DESC LIMIT 3
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 12_2 for subquery SELECT sum(((((user_id)::numeric * (5.0 / (((value_1 + value_2))::numeric + 0.1))))::double precision * value_3)) AS cnt_1 FROM public.users_table ORDER BY (sum(((((user_id)::numeric * (5.0 / (((value_1 + value_2))::numeric + 0.1))))::double precision * value_3))) DESC LIMIT 3
|
||||
DEBUG: push down of limit count: 4
|
||||
DEBUG: generating subplan 12_3 for subquery SELECT avg(CASE WHEN (user_id > 4) THEN value_1 ELSE NULL::integer END) AS cnt_2, avg(CASE WHEN (user_id > 500) THEN value_1 ELSE NULL::integer END) AS cnt_3, sum(CASE WHEN ((value_1 = 1) OR (value_2 = 1)) THEN 1 ELSE 0 END) AS sum_1, date_part('year'::text, max("time")) AS l_year, strpos((max(user_id))::text, '1'::text) AS pos FROM public.users_table ORDER BY (avg(CASE WHEN (user_id > 4) THEN value_1 ELSE NULL::integer END)) DESC LIMIT 4
|
||||
DEBUG: push down of limit count: 25
|
||||
DEBUG: generating subplan 12_4 for subquery SELECT COALESCE(value_3, (20)::double precision) AS count_pay FROM public.users_table ORDER BY COALESCE(value_3, (20)::double precision) OFFSET 20 LIMIT 5
|
||||
avg | cnt_1 | cnt_2 | cnt_3 | sum_1 | l_year | pos | count_pay
|
||||
-------------------------+------------------+--------------------+-------+-------+--------+-----+-----------
|
||||
30.14666771571734992301 | 3308.14619815793 | 2.5000000000000000 | | 31 | 2017 | 0 | 1
|
||||
(1 row)
|
||||
|
||||
-- Multiple columns in GROUP BYs
|
||||
-- foo needs to be recursively planned, bar can be pushded down
|
||||
SELECT
|
||||
DISTINCT ON (avg) avg, avg2
|
||||
FROM
|
||||
(
|
||||
SELECT avg(value_3) as avg FROM users_table GROUP BY value_1, value_2
|
||||
) as foo,
|
||||
(
|
||||
SELECT avg(value_3) as avg2 FROM users_table GROUP BY value_1, value_2, user_id
|
||||
) as bar
|
||||
WHERE foo.avg = bar.avg2
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: generating subplan 17_1 for subquery SELECT avg(value_3) AS avg FROM public.users_table GROUP BY value_1, value_2
|
||||
DEBUG: push down of limit count: 3
|
||||
avg | avg2
|
||||
-----+------
|
||||
5 | 5
|
||||
4 | 4
|
||||
3.5 | 3.5
|
||||
(3 rows)
|
||||
|
||||
-- HAVING and ORDER BY tests
|
||||
SELECT a.user_id, b.value_2, c.avg
|
||||
FROM (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
(value_1 > 2)
|
||||
GROUP BY
|
||||
user_id
|
||||
HAVING
|
||||
count(distinct value_1) > 2
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as a,
|
||||
(
|
||||
SELECT
|
||||
value_2
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
(value_1 > 2)
|
||||
GROUP BY
|
||||
value_2
|
||||
HAVING
|
||||
count(distinct value_1) > 2
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as b,
|
||||
(
|
||||
SELECT
|
||||
avg(user_id) as avg
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
(value_1 > 2)
|
||||
GROUP BY
|
||||
value_2
|
||||
HAVING
|
||||
sum(value_1) > 10
|
||||
ORDER BY (sum(value_3) - avg(value_1) - COALESCE(array_upper(ARRAY[max(user_id)],1) * 5,0)) DESC
|
||||
LIMIT 3
|
||||
) as c
|
||||
WHERE b.value_2 != a.user_id
|
||||
ORDER BY 3 DESC, 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 19_1 for subquery SELECT user_id FROM public.users_table WHERE (value_1 > 2) GROUP BY user_id HAVING (count(DISTINCT value_1) > 2) ORDER BY user_id DESC LIMIT 3
|
||||
DEBUG: generating subplan 19_2 for subquery SELECT value_2 FROM public.users_table WHERE (value_1 > 2) GROUP BY value_2 HAVING (count(DISTINCT value_1) > 2) ORDER BY value_2 DESC LIMIT 3
|
||||
DEBUG: generating subplan 19_3 for subquery SELECT avg(user_id) AS avg FROM public.users_table WHERE (value_1 > 2) GROUP BY value_2 HAVING (sum(value_1) > 10) ORDER BY ((sum(value_3) - (avg(value_1))::double precision) - (COALESCE((array_upper(ARRAY[max(user_id)], 1) * 5), 0))::double precision) DESC LIMIT 3
|
||||
user_id | value_2 | avg
|
||||
---------+---------+--------------------
|
||||
4 | 5 | 4.1666666666666667
|
||||
3 | 5 | 4.1666666666666667
|
||||
5 | 4 | 4.1666666666666667
|
||||
3 | 4 | 4.1666666666666667
|
||||
5 | 3 | 4.1666666666666667
|
||||
(5 rows)
|
||||
|
||||
-- zero shard subquery joined with a regular one
|
||||
SELECT
|
||||
bar.user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND false AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as bar
|
||||
WHERE foo.user_id > bar.user_id
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 23_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
DEBUG: generating subplan 23_2 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND false AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
(0 rows)
|
||||
|
||||
-- window functions tests, both is recursively planned
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT
|
||||
user_id, time, rnk
|
||||
FROM
|
||||
(
|
||||
SELECT * FROM (
|
||||
SELECT
|
||||
*, rank() OVER my_win as rnk
|
||||
FROM
|
||||
events_table
|
||||
WINDOW my_win AS (PARTITION BY user_id ORDER BY time DESC)
|
||||
ORDER BY rnk DESC
|
||||
) as foo_inner
|
||||
LIMIT 4
|
||||
) as foo
|
||||
ORDER BY
|
||||
3 DESC, 1 DESC, 2 DESC
|
||||
) foo,
|
||||
(
|
||||
SELECT
|
||||
user_id, time, rnk
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
*, rank() OVER my_win as rnk
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id = 3
|
||||
WINDOW my_win AS (PARTITION BY event_type ORDER BY time DESC)
|
||||
) as foo
|
||||
ORDER BY
|
||||
3 DESC, 1 DESC, 2 DESC
|
||||
) bar WHERE foo.user_id = bar.user_id
|
||||
ORDER BY foo.rnk DESC, foo.time DESC, bar.time LIMIT 5;
|
||||
DEBUG: push down of limit count: 4
|
||||
DEBUG: generating subplan 26_1 for subquery SELECT user_id, "time", event_type, value_2, value_3, value_4, rnk FROM (SELECT events_table.user_id, events_table."time", events_table.event_type, events_table.value_2, events_table.value_3, events_table.value_4, rank() OVER my_win AS rnk FROM public.events_table WINDOW my_win AS (PARTITION BY events_table.user_id ORDER BY events_table."time" DESC) ORDER BY (rank() OVER my_win) DESC) foo_inner LIMIT 4
|
||||
DEBUG: generating subplan 26_2 for subquery SELECT user_id, "time", event_type, value_2, value_3, value_4, rank() OVER my_win AS rnk FROM public.events_table WHERE (user_id = 3) WINDOW my_win AS (PARTITION BY event_type ORDER BY "time" DESC)
|
||||
user_id | time | rnk | user_id | time | rnk
|
||||
---------+------+-----+---------+------+-----
|
||||
(0 rows)
|
||||
|
||||
-- cursor test
|
||||
BEGIN;
|
||||
|
||||
DECLARE recursive_subquery CURSOR FOR
|
||||
SELECT
|
||||
event_type, count(distinct value_2)
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id IN (SELECT user_id FROM users_table GROUP BY user_id ORDER BY count(*) DESC LIMIT 20)
|
||||
GROUP BY
|
||||
event_type
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: push down of limit count: 20
|
||||
DEBUG: generating subplan 29_1 for subquery SELECT user_id FROM public.users_table GROUP BY user_id ORDER BY (count(*)) DESC LIMIT 20
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
event_type | count
|
||||
------------+-------
|
||||
6 | 1
|
||||
(1 row)
|
||||
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
event_type | count
|
||||
------------+-------
|
||||
5 | 3
|
||||
(1 row)
|
||||
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
event_type | count
|
||||
------------+-------
|
||||
4 | 6
|
||||
(1 row)
|
||||
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
event_type | count
|
||||
------------+-------
|
||||
(0 rows)
|
||||
|
||||
COMMIT;
|
||||
-- cursor test with FETCH ALL
|
||||
BEGIN;
|
||||
|
||||
DECLARE recursive_subquery CURSOR FOR
|
||||
SELECT
|
||||
event_type, count(distinct value_2)
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id IN (SELECT user_id FROM users_table GROUP BY user_id ORDER BY count(*) DESC LIMIT 20)
|
||||
GROUP BY
|
||||
event_type
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: push down of limit count: 20
|
||||
DEBUG: generating subplan 31_1 for subquery SELECT user_id FROM public.users_table GROUP BY user_id ORDER BY (count(*)) DESC LIMIT 20
|
||||
FETCH ALL FROM recursive_subquery;
|
||||
event_type | count
|
||||
------------+-------
|
||||
6 | 1
|
||||
5 | 3
|
||||
4 | 6
|
||||
(3 rows)
|
||||
|
||||
FETCH ALL FROM recursive_subquery;
|
||||
event_type | count
|
||||
------------+-------
|
||||
(0 rows)
|
||||
|
||||
COMMIT;
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_complex CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,143 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with different executors
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_executor;
|
||||
SET search_path TO subquery_executor, public;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- subquery with router planner
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
DEBUG: generating subplan 2_1 for subquery SELECT value_2 FROM public.users_table WHERE (user_id = 15) OFFSET 0
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
-- subquery with router but not logical plannable
|
||||
-- should fail
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT user_id, sum(value_2) over (partition by user_id) AS counter FROM users_table WHERE user_id = 15
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.counter = bar.user_id;
|
||||
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.
|
||||
-- subquery with real-time query
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id != 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
DEBUG: generating subplan 5_1 for subquery SELECT value_2 FROM public.users_table WHERE (user_id <> 15) OFFSET 0
|
||||
count
|
||||
-------
|
||||
1612
|
||||
(1 row)
|
||||
|
||||
-- subquery with repartition query
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
DEBUG: cannot use real time executor with repartition jobs
|
||||
HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker.
|
||||
DEBUG: generating subplan 7_1 for subquery SELECT DISTINCT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.value_2) AND (users_table.user_id < 2))
|
||||
count
|
||||
-------
|
||||
58
|
||||
(1 row)
|
||||
|
||||
-- mixed of all executors (including local execution)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table OFFSET 0
|
||||
) as bar,
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) baz,
|
||||
(
|
||||
SELECT user_id FROM users_table_local WHERE user_id = 2
|
||||
) baw
|
||||
WHERE foo.value_2 = bar.user_id AND baz.value_2 = bar.user_id AND bar.user_id = baw.user_id;
|
||||
DEBUG: generating subplan 9_1 for subquery SELECT value_2 FROM public.users_table WHERE (user_id = 15) OFFSET 0
|
||||
DEBUG: generating subplan 9_2 for subquery SELECT user_id FROM public.users_table OFFSET 0
|
||||
DEBUG: cannot use real time executor with repartition jobs
|
||||
HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker.
|
||||
DEBUG: generating subplan 9_3 for subquery SELECT DISTINCT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.value_2) AND (users_table.user_id < 2))
|
||||
DEBUG: generating subplan 9_4 for subquery SELECT user_id FROM subquery_executor.users_table_local WHERE (user_id = 2)
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
-- final query is router
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 1 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table WHERE user_id = 2 OFFSET 0
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
DEBUG: generating subplan 13_1 for subquery SELECT value_2 FROM public.users_table WHERE (user_id = 1) OFFSET 0
|
||||
DEBUG: generating subplan 13_2 for subquery SELECT user_id FROM public.users_table WHERE (user_id = 2) OFFSET 0
|
||||
count
|
||||
-------
|
||||
18
|
||||
(1 row)
|
||||
|
||||
-- final query is real-time
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 1 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table WHERE user_id != 2
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
DEBUG: generating subplan 16_1 for subquery SELECT value_2 FROM public.users_table WHERE (user_id = 1) OFFSET 0
|
||||
count
|
||||
-------
|
||||
103
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_executor CASCADE;
|
||||
NOTICE: drop cascades to table users_table_local
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,243 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on local tables
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_local_tables;
|
||||
SET search_path TO subquery_local_tables, public;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- foo is only on the local tables, thus can be replaced
|
||||
-- bar is on the distributed tables with LIMIT, should be replaced
|
||||
SELECT
|
||||
foo.user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table_local.user_id
|
||||
FROM
|
||||
users_table_local, events_table_local
|
||||
WHERE
|
||||
users_table_local.user_id = events_table_local.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as bar
|
||||
WHERE bar.user_id = foo.user_id
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: generating subplan 3_1 for subquery SELECT DISTINCT users_table_local.user_id FROM subquery_local_tables.users_table_local, subquery_local_tables.events_table_local WHERE ((users_table_local.user_id = events_table_local.user_id) AND (events_table_local.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table_local.user_id DESC LIMIT 5
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 3_2 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[5, 6, 7, 8]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
(5 rows)
|
||||
|
||||
-- foo is only on the local tables, thus can be replaced
|
||||
SELECT
|
||||
foo.user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table_local.user_id
|
||||
FROM
|
||||
users_table_local, events_table_local
|
||||
WHERE
|
||||
users_table_local.user_id = events_table_local.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
) as bar
|
||||
WHERE bar.user_id = foo.user_id
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: generating subplan 5_1 for subquery SELECT DISTINCT users_table_local.user_id FROM subquery_local_tables.users_table_local, subquery_local_tables.events_table_local WHERE ((users_table_local.user_id = events_table_local.user_id) AND (events_table_local.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table_local.user_id DESC LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
(5 rows)
|
||||
|
||||
-- subqueries in WHERE could be replaced even if they are on the local tables
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE
|
||||
user_id IN (SELECT DISTINCT value_2 FROM users_table_local WHERE value_1 = 1)
|
||||
ORDER BY 1 LIMIT 5;
|
||||
DEBUG: generating subplan 6_1 for subquery SELECT DISTINCT value_2 FROM subquery_local_tables.users_table_local WHERE (value_1 = 1)
|
||||
DEBUG: push down of limit count: 5
|
||||
user_id
|
||||
---------
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
(5 rows)
|
||||
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced if
|
||||
-- it contains onle local tables
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
event_type, user_id
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table_local WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar
|
||||
) as baz
|
||||
WHERE baz.user_id = users_table.user_id
|
||||
) as sub1
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: generating subplan 7_1 for subquery SELECT user_id, event_type FROM subquery_local_tables.events_table_local WHERE (value_2 < 3) OFFSET 3
|
||||
DEBUG: push down of limit count: 3
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
5
|
||||
4
|
||||
(3 rows)
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE -> WHERE should be replaced if
|
||||
-- it contains onle local tables
|
||||
-- Later the upper level query is also recursively planned due to LIMIT
|
||||
SELECT user_id, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT user_id, array_agg(event ORDER BY time) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
u.user_id, e.event_type::text AS event, e.time
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table_local WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
LIMIT 5
|
||||
)
|
||||
) t
|
||||
GROUP BY user_id
|
||||
) q
|
||||
ORDER BY 2 DESC, 1;
|
||||
DEBUG: generating subplan 8_1 for subquery SELECT user_id FROM subquery_local_tables.events_table_local WHERE ((event_type > 1) AND (event_type <= 3) AND (value_3 > (1)::double precision))
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 8_2 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 >= 5) AND (EXISTS (SELECT intermediate_result.user_id FROM read_intermediate_result('8_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer))) AND (NOT (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 3) AND (events_table.event_type <= 4) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))))) LIMIT 5
|
||||
user_id | array_length
|
||||
---------+--------------
|
||||
5 | 364
|
||||
(1 row)
|
||||
|
||||
-- subquery (i.e., subquery_2) in WHERE->FROM should be replaced due to local tables
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM (
|
||||
SELECT
|
||||
subquery_1.user_id, count_pay
|
||||
FROM
|
||||
(
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>1' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 1 AND events_table.event_type < 3
|
||||
)
|
||||
UNION
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>2' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 2 AND events_table.event_type < 4
|
||||
)
|
||||
) AS subquery_1
|
||||
LEFT JOIN
|
||||
(SELECT
|
||||
user_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
users_table_local
|
||||
WHERE
|
||||
user_id >= 1 AND
|
||||
user_id <= 3 AND
|
||||
users_table_local.value_1 > 3 AND users_table_local.value_1 < 5
|
||||
GROUP BY
|
||||
user_id
|
||||
HAVING
|
||||
COUNT(*) > 1
|
||||
LIMIT 10
|
||||
) AS subquery_2
|
||||
ON
|
||||
subquery_1.user_id = subquery_2.user_id
|
||||
GROUP BY
|
||||
subquery_1.user_id,
|
||||
count_pay) AS subquery_top
|
||||
GROUP BY
|
||||
count_pay, user_id
|
||||
)
|
||||
GROUP BY user_id
|
||||
HAVING count(*) > 1 AND sum(value_2) > 29
|
||||
ORDER BY 1;
|
||||
DEBUG: generating subplan 10_1 for subquery SELECT user_id, count(*) AS count_pay FROM subquery_local_tables.users_table_local WHERE ((user_id >= 1) AND (user_id <= 3) AND (value_1 > 3) AND (value_1 < 5)) GROUP BY user_id HAVING (count(*) > 1) LIMIT 10
|
||||
user_id
|
||||
---------
|
||||
2
|
||||
3
|
||||
(2 rows)
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_local_tables CASCADE;
|
||||
NOTICE: drop cascades to 2 other objects
|
||||
DETAIL: drop cascades to table users_table_local
|
||||
drop cascades to table events_table_local
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,283 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on partitioned tables
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_and_partitioning;
|
||||
SET search_path TO subquery_and_partitioning, public;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
CREATE TABLE partitioning_test(id int, value_1 int, time date) PARTITION BY RANGE (time);
|
||||
|
||||
-- create its partitions
|
||||
CREATE TABLE partitioning_test_2017 PARTITION OF partitioning_test FOR VALUES FROM ('2017-01-01') TO ('2018-01-01');
|
||||
CREATE TABLE partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01');
|
||||
-- load some data and distribute tables
|
||||
INSERT INTO partitioning_test VALUES (1, 1, '2017-11-23');
|
||||
INSERT INTO partitioning_test VALUES (2, 1, '2010-07-07');
|
||||
INSERT INTO partitioning_test_2017 VALUES (3, 3, '2017-11-22');
|
||||
INSERT INTO partitioning_test_2010 VALUES (4, 4, '2010-03-03');
|
||||
-- distribute partitioned table
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SELECT create_distributed_table('partitioning_test', 'id');
|
||||
NOTICE: Copying data from local table...
|
||||
NOTICE: Copying data from local table...
|
||||
create_distributed_table
|
||||
--------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- subplan for partitioned tables
|
||||
SELECT
|
||||
id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 3_1 for subquery SELECT DISTINCT id FROM subquery_and_partitioning.partitioning_test LIMIT 5
|
||||
id
|
||||
----
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
(4 rows)
|
||||
|
||||
-- final query is router on partitioned tables
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as bar
|
||||
WHERE foo.id = date_part('day', bar.time)
|
||||
ORDER BY 2 DESC, 1;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 5_1 for subquery SELECT DISTINCT id FROM subquery_and_partitioning.partitioning_test LIMIT 5
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 5_2 for subquery SELECT DISTINCT "time" FROM subquery_and_partitioning.partitioning_test LIMIT 5
|
||||
id | time
|
||||
----+------------
|
||||
3 | 03-03-2010
|
||||
(1 row)
|
||||
|
||||
-- final query is real-time
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar
|
||||
WHERE date_part('day', foo.time) = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 8_1 for subquery SELECT DISTINCT "time" FROM subquery_and_partitioning.partitioning_test ORDER BY "time" DESC LIMIT 5
|
||||
DEBUG: push down of limit count: 3
|
||||
time | id
|
||||
------------+----
|
||||
03-03-2010 | 3
|
||||
(1 row)
|
||||
|
||||
-- final query is real-time that is joined with partitioned table
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar,
|
||||
partitioning_test
|
||||
WHERE date_part('day', foo.time) = bar.id AND partitioning_test.id = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 10_1 for subquery SELECT DISTINCT "time" FROM subquery_and_partitioning.partitioning_test ORDER BY "time" DESC LIMIT 5
|
||||
DEBUG: push down of limit count: 3
|
||||
time | id | id | value_1 | time
|
||||
------------+----+----+---------+------------
|
||||
03-03-2010 | 3 | 3 | 3 | 11-22-2017
|
||||
(1 row)
|
||||
|
||||
-- subquery in WHERE clause
|
||||
SELECT DISTINCT id
|
||||
FROM partitioning_test
|
||||
WHERE
|
||||
id IN (SELECT DISTINCT date_part('day', time) FROM partitioning_test);
|
||||
DEBUG: generating subplan 12_1 for subquery SELECT DISTINCT date_part('day'::text, "time") AS date_part FROM subquery_and_partitioning.partitioning_test
|
||||
id
|
||||
----
|
||||
3
|
||||
(1 row)
|
||||
|
||||
-- repartition subquery
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT p1.value_1 FROM partitioning_test as p1, partitioning_test as p2 WHERE p1.id = p2.value_1
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_1 = bar.user_id;
|
||||
DEBUG: cannot use real time executor with repartition jobs
|
||||
HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker.
|
||||
DEBUG: generating subplan 14_1 for subquery SELECT DISTINCT p1.value_1 FROM subquery_and_partitioning.partitioning_test p1, subquery_and_partitioning.partitioning_test p2 WHERE (p1.id = p2.value_1)
|
||||
count
|
||||
-------
|
||||
47
|
||||
(1 row)
|
||||
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
-- subquery, cte, view and non-partitioned tables
|
||||
CREATE VIEW subquery_and_ctes AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_1 FROM partitioning_test OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_1 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT events_table.user_id
|
||||
FROM
|
||||
partitioning_test, events_table
|
||||
WHERE
|
||||
events_table.user_id = partitioning_test.id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2;
|
||||
SELECT * FROM subquery_and_ctes
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: generating subplan 16_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_and_partitioning.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT partitioning_test.value_1 FROM subquery_and_partitioning.partitioning_test OFFSET 0) foo WHERE ((events_table.user_id = foo.value_1) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 17_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_and_partitioning.users_table_local
|
||||
DEBUG: generating subplan 17_2 for CTE dist_cte: SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT partitioning_test.value_1 FROM subquery_and_partitioning.partitioning_test OFFSET 0) foo WHERE ((events_table.user_id = foo.value_1) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 18_1 for subquery SELECT DISTINCT value_1 FROM public.users_table ORDER BY value_1 LIMIT 3
|
||||
DEBUG: generating subplan 18_2 for subquery SELECT DISTINCT value_1 FROM subquery_and_partitioning.partitioning_test OFFSET 0
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 16_2 for subquery SELECT DISTINCT events_table.user_id FROM subquery_and_partitioning.partitioning_test, public.events_table WHERE ((events_table.user_id = partitioning_test.id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY events_table.user_id DESC LIMIT 5
|
||||
DEBUG: generating subplan 16_3 for subquery SELECT count(*) AS cnt FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('16_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) cte, (SELECT intermediate_result.user_id FROM read_intermediate_result('16_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo WHERE (foo.user_id = cte.user_id)
|
||||
DEBUG: push down of limit count: 5
|
||||
cnt | user_id | time | value_1 | value_2 | value_3 | value_4
|
||||
-----+---------+---------------------------------+---------+---------+---------+---------
|
||||
105 | 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 |
|
||||
105 | 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 |
|
||||
105 | 3 | Thu Nov 23 17:18:51.048758 2017 | 1 | 5 | 5 |
|
||||
105 | 3 | Thu Nov 23 17:10:35.959913 2017 | 4 | 3 | 1 |
|
||||
105 | 5 | Thu Nov 23 16:48:32.08896 2017 | 5 | 2 | 1 |
|
||||
(5 rows)
|
||||
|
||||
-- deep subquery, partitioned and non-partitioned tables together
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(partitioning_test.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_1 FROM partitioning_test GROUP BY value_1) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, partitioning_test
|
||||
WHERE
|
||||
level_5.avg_ev_type = partitioning_test.id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
DEBUG: generating subplan 23_1 for subquery SELECT count(*) AS cnt, value_1 FROM subquery_and_partitioning.partitioning_test GROUP BY value_1
|
||||
DEBUG: generating subplan 23_2 for subquery SELECT avg(events_table.event_type) AS avg FROM (SELECT level_1.cnt FROM (SELECT intermediate_result.cnt, intermediate_result.value_1 FROM read_intermediate_result('23_1'::text, 'binary'::citus_copy_format) intermediate_result(cnt bigint, value_1 integer)) level_1, public.users_table WHERE (users_table.user_id = level_1.cnt)) level_2, public.events_table WHERE (events_table.user_id = level_2.cnt) GROUP BY level_2.cnt
|
||||
DEBUG: generating subplan 23_3 for subquery SELECT max(users_table.value_1) AS mx_val_1 FROM (SELECT intermediate_result.avg FROM read_intermediate_result('23_2'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)) level_3, public.users_table WHERE ((users_table.user_id)::numeric = level_3.avg) GROUP BY level_3.avg
|
||||
DEBUG: generating subplan 23_4 for subquery SELECT avg(events_table.event_type) AS avg_ev_type FROM (SELECT intermediate_result.mx_val_1 FROM read_intermediate_result('23_3'::text, 'binary'::citus_copy_format) intermediate_result(mx_val_1 integer)) level_4, public.events_table WHERE (level_4.mx_val_1 = events_table.user_id) GROUP BY level_4.mx_val_1
|
||||
DEBUG: generating subplan 23_5 for subquery SELECT min(partitioning_test.value_1) AS min FROM (SELECT intermediate_result.avg_ev_type FROM read_intermediate_result('23_4'::text, 'binary'::citus_copy_format) intermediate_result(avg_ev_type numeric)) level_5, subquery_and_partitioning.partitioning_test WHERE (level_5.avg_ev_type = (partitioning_test.id)::numeric) GROUP BY level_5.avg_ev_type
|
||||
DEBUG: generating subplan 23_6 for subquery SELECT avg(level_6.min) AS avg FROM (SELECT intermediate_result.min FROM read_intermediate_result('23_5'::text, 'binary'::citus_copy_format) intermediate_result(min integer)) level_6, public.users_table WHERE (users_table.user_id = level_6.min) GROUP BY users_table.value_1
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_and_partitioning CASCADE;
|
||||
NOTICE: drop cascades to 4 other objects
|
||||
DETAIL: drop cascades to table users_table_local
|
||||
drop cascades to table events_table_local
|
||||
drop cascades to table partitioning_test
|
||||
drop cascades to view subquery_and_ctes
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,246 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on partitioned tables
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_and_partitioning;
|
||||
SET search_path TO subquery_and_partitioning, public;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
CREATE TABLE partitioning_test(id int, value_1 int, time date) PARTITION BY RANGE (time);
|
||||
ERROR: syntax error at or near "PARTITION"
|
||||
LINE 1: ...partitioning_test(id int, value_1 int, time date) PARTITION ...
|
||||
^
|
||||
|
||||
-- create its partitions
|
||||
CREATE TABLE partitioning_test_2017 PARTITION OF partitioning_test FOR VALUES FROM ('2017-01-01') TO ('2018-01-01');
|
||||
ERROR: syntax error at or near "PARTITION"
|
||||
LINE 1: CREATE TABLE partitioning_test_2017 PARTITION OF partitionin...
|
||||
^
|
||||
CREATE TABLE partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01');
|
||||
ERROR: syntax error at or near "PARTITION"
|
||||
LINE 1: CREATE TABLE partitioning_test_2010 PARTITION OF partitionin...
|
||||
^
|
||||
-- load some data and distribute tables
|
||||
INSERT INTO partitioning_test VALUES (1, 1, '2017-11-23');
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test VALUES (1, 1, '2017-11-23');
|
||||
^
|
||||
INSERT INTO partitioning_test VALUES (2, 1, '2010-07-07');
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test VALUES (2, 1, '2010-07-07');
|
||||
^
|
||||
INSERT INTO partitioning_test_2017 VALUES (3, 3, '2017-11-22');
|
||||
ERROR: relation "partitioning_test_2017" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test_2017 VALUES (3, 3, '2017-11-22...
|
||||
^
|
||||
INSERT INTO partitioning_test_2010 VALUES (4, 4, '2010-03-03');
|
||||
ERROR: relation "partitioning_test_2010" does not exist
|
||||
LINE 1: INSERT INTO partitioning_test_2010 VALUES (4, 4, '2010-03-03...
|
||||
^
|
||||
-- distribute partitioned table
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SELECT create_distributed_table('partitioning_test', 'id');
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 1: SELECT create_distributed_table('partitioning_test', 'id');
|
||||
^
|
||||
SET client_min_messages TO DEBUG1;
|
||||
-- subplan for partitioned tables
|
||||
SELECT
|
||||
id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- final query is router on partitioned tables
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as bar
|
||||
WHERE foo.id = date_part('day', bar.time)
|
||||
ORDER BY 2 DESC, 1;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- final query is real-time
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar
|
||||
WHERE date_part('day', foo.time) = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- final query is real-time that is joined with partitioned table
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar,
|
||||
partitioning_test
|
||||
WHERE date_part('day', foo.time) = bar.id AND partitioning_test.id = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 7: partitioning_test
|
||||
^
|
||||
-- subquery in WHERE clause
|
||||
SELECT DISTINCT id
|
||||
FROM partitioning_test
|
||||
WHERE
|
||||
id IN (SELECT DISTINCT date_part('day', time) FROM partitioning_test);
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 2: FROM partitioning_test
|
||||
^
|
||||
-- repartition subquery
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT p1.value_1 FROM partitioning_test as p1, partitioning_test as p2 WHERE p1.id = p2.value_1
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_1 = bar.user_id;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 5: SELECT DISTINCT p1.value_1 FROM partitioning_test as p1, pa...
|
||||
^
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
-- subquery, cte, view and non-partitioned tables
|
||||
CREATE VIEW subquery_and_ctes AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_1 FROM partitioning_test OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_1 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT events_table.user_id
|
||||
FROM
|
||||
partitioning_test, events_table
|
||||
WHERE
|
||||
events_table.user_id = partitioning_test.id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 15: (SELECT DISTINCT value_1 FROM partitioning_test OFFSET 0)...
|
||||
^
|
||||
SELECT * FROM subquery_and_ctes
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
ERROR: relation "subquery_and_ctes" does not exist
|
||||
LINE 1: SELECT * FROM subquery_and_ctes
|
||||
^
|
||||
-- deep subquery, partitioned and non-partitioned tables together
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(partitioning_test.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_1 FROM partitioning_test GROUP BY value_1) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, partitioning_test
|
||||
WHERE
|
||||
level_5.avg_ev_type = partitioning_test.id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
ERROR: relation "partitioning_test" does not exist
|
||||
LINE 20: (SELECT count(*) as cnt, value_1 FROM partitioning_...
|
||||
^
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_and_partitioning CASCADE;
|
||||
NOTICE: drop cascades to 2 other objects
|
||||
DETAIL: drop cascades to table users_table_local
|
||||
drop cascades to table events_table_local
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,285 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on prepared statements
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_prepared_statements;
|
||||
SET search_path TO subquery_prepared_statements, public;
|
||||
CREATE TYPE xy AS (x int, y int);
|
||||
SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements');
|
||||
run_command_on_workers
|
||||
-------------------------------------
|
||||
(localhost,57637,t,"CREATE SCHEMA")
|
||||
(localhost,57638,t,"CREATE SCHEMA")
|
||||
(2 rows)
|
||||
|
||||
SELECT run_command_on_workers('CREATE TYPE subquery_prepared_statements.xy AS (x int, y int)');
|
||||
run_command_on_workers
|
||||
-----------------------------------
|
||||
(localhost,57637,t,"CREATE TYPE")
|
||||
(localhost,57638,t,"CREATE TYPE")
|
||||
(2 rows)
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
PREPARE subquery_prepare_without_param AS
|
||||
SELECT
|
||||
DISTINCT values_of_subquery
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT (users_table.user_id, events_table.event_type)::xy as values_of_subquery
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
PREPARE subquery_prepare_param_on_partkey(int) AS
|
||||
SELECT
|
||||
DISTINCT values_of_subquery
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT (users_table.user_id, events_table.event_type)::xy as values_of_subquery
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
(users_table.user_id = $1 OR users_table.user_id = 2) AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
PREPARE subquery_prepare_param_non_partkey(int) AS
|
||||
SELECT
|
||||
DISTINCT values_of_subquery
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT (users_table.user_id, events_table.event_type)::xy as values_of_subquery
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type = $1
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
-- execute each test with 6 times
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 1_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,4)
|
||||
(6,3)
|
||||
(6,2)
|
||||
(6,1)
|
||||
(5,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,4)
|
||||
(6,3)
|
||||
(6,2)
|
||||
(6,1)
|
||||
(5,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,4)
|
||||
(6,3)
|
||||
(6,2)
|
||||
(6,1)
|
||||
(5,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,4)
|
||||
(6,3)
|
||||
(6,2)
|
||||
(6,1)
|
||||
(5,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,4)
|
||||
(6,3)
|
||||
(6,2)
|
||||
(6,1)
|
||||
(5,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,4)
|
||||
(6,3)
|
||||
(6,2)
|
||||
(6,1)
|
||||
(5,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,4)
|
||||
(6,3)
|
||||
(6,2)
|
||||
(6,1)
|
||||
(5,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 3_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND ((users_table.user_id = 1) OR (users_table.user_id = 2)) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(2,4)
|
||||
(2,3)
|
||||
(2,2)
|
||||
(2,1)
|
||||
(1,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 5_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND ((users_table.user_id = 1) OR (users_table.user_id = 2)) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(2,4)
|
||||
(2,3)
|
||||
(2,2)
|
||||
(2,1)
|
||||
(1,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 7_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND ((users_table.user_id = 1) OR (users_table.user_id = 2)) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(2,4)
|
||||
(2,3)
|
||||
(2,2)
|
||||
(2,1)
|
||||
(1,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 9_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND ((users_table.user_id = 1) OR (users_table.user_id = 2)) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(2,4)
|
||||
(2,3)
|
||||
(2,2)
|
||||
(2,1)
|
||||
(1,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 11_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND ((users_table.user_id = 1) OR (users_table.user_id = 2)) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(2,4)
|
||||
(2,3)
|
||||
(2,2)
|
||||
(2,1)
|
||||
(1,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 14_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND ((users_table.user_id = 1) OR (users_table.user_id = 2)) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(2,4)
|
||||
(2,3)
|
||||
(2,2)
|
||||
(2,1)
|
||||
(1,4)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 16_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = 1)) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,1)
|
||||
(5,1)
|
||||
(4,1)
|
||||
(3,1)
|
||||
(2,1)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 18_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = 1)) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,1)
|
||||
(5,1)
|
||||
(4,1)
|
||||
(3,1)
|
||||
(2,1)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 20_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = 1)) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,1)
|
||||
(5,1)
|
||||
(4,1)
|
||||
(3,1)
|
||||
(2,1)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 22_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = 1)) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,1)
|
||||
(5,1)
|
||||
(4,1)
|
||||
(3,1)
|
||||
(2,1)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 24_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = 1)) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,1)
|
||||
(5,1)
|
||||
(4,1)
|
||||
(3,1)
|
||||
(2,1)
|
||||
(5 rows)
|
||||
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 27_1 for subquery SELECT DISTINCT ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy AS values_of_subquery FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = 1)) ORDER BY ROW(users_table.user_id, events_table.event_type)::subquery_prepared_statements.xy DESC LIMIT 5
|
||||
values_of_subquery
|
||||
--------------------
|
||||
(6,1)
|
||||
(5,1)
|
||||
(4,1)
|
||||
(3,1)
|
||||
(2,1)
|
||||
(5 rows)
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_prepared_statements CASCADE;
|
||||
NOTICE: drop cascades to type xy
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,566 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on views
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_view;
|
||||
SET search_path TO subquery_view, public;
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
SET client_min_messages TO DEBUG1;
|
||||
CREATE VIEW view_without_subquery AS
|
||||
SELECT
|
||||
DISTINCT users_table.value_1
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
view_without_subquery
|
||||
ORDER BY 1 DESC LIMIT 5;
|
||||
DEBUG: generating subplan 3_1 for subquery SELECT DISTINCT users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.value_1 DESC
|
||||
value_1
|
||||
---------
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
(5 rows)
|
||||
|
||||
CREATE VIEW view_without_subquery_second AS
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
view_without_subquery_second
|
||||
ORDER BY 1;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 5_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
(5 rows)
|
||||
|
||||
-- subqueries in FROM clause with LIMIT should be recursively planned
|
||||
CREATE VIEW subquery_limit AS
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
SELECT * FROM subquery_limit ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 7_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
(5 rows)
|
||||
|
||||
-- subqueries in FROM clause with GROUP BY non-distribution column should be recursively planned
|
||||
CREATE VIEW subquery_non_p_key_group_by AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.value_1
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
SELECT * FROM subquery_non_p_key_group_by ORDER BY 1 DESC;
|
||||
DEBUG: generating subplan 9_1 for subquery SELECT DISTINCT users_table.value_1 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.value_1
|
||||
value_1
|
||||
---------
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
0
|
||||
(6 rows)
|
||||
|
||||
CREATE VIEW final_query_router AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.value_3
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
GROUP BY users_table.value_3
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.value_3
|
||||
ORDER BY 2 DESC, 1;
|
||||
SELECT * FROM final_query_router ORDER BY 1;
|
||||
DEBUG: generating subplan 11_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) GROUP BY users_table.value_2 ORDER BY users_table.value_2 DESC
|
||||
DEBUG: generating subplan 11_2 for subquery SELECT users_table.value_3 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[5, 6, 7, 8]))) GROUP BY users_table.value_3 ORDER BY users_table.value_3 DESC
|
||||
value_2 | value_3
|
||||
---------+---------
|
||||
0 | 0
|
||||
1 | 1
|
||||
2 | 2
|
||||
3 | 3
|
||||
4 | 4
|
||||
5 | 5
|
||||
(6 rows)
|
||||
|
||||
CREATE VIEW final_query_realtime AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
SELECT
|
||||
DISTINCT ON (users_table.value_2) users_table.value_2, time, value_3
|
||||
FROM
|
||||
final_query_realtime, users_table
|
||||
WHERE
|
||||
users_table.user_id = final_query_realtime.user_id
|
||||
ORDER BY 1 DESC, 2 DESC, 3 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: generating subplan 14_1 for subquery SELECT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) GROUP BY users_table.value_2 ORDER BY users_table.value_2 DESC
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 14_2 for subquery SELECT foo.value_2, bar.user_id FROM (SELECT intermediate_result.value_2 FROM read_intermediate_result('14_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) foo, (SELECT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[5, 6, 7, 8]))) ORDER BY users_table.user_id DESC) bar WHERE (foo.value_2 = bar.user_id) ORDER BY bar.user_id DESC, foo.value_2 DESC LIMIT 3
|
||||
DEBUG: push down of limit count: 3
|
||||
value_2 | time | value_3
|
||||
---------+---------------------------------+---------
|
||||
5 | Thu Nov 23 16:28:38.455322 2017 | 4
|
||||
4 | Thu Nov 23 10:22:39.468816 2017 | 3
|
||||
3 | Thu Nov 23 15:55:08.493462 2017 | 3
|
||||
(3 rows)
|
||||
|
||||
CREATE VIEW subquery_in_where AS
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE
|
||||
user_id IN (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5);
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
subquery_in_where
|
||||
ORDER BY 1 DESC;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 17_1 for subquery SELECT DISTINCT value_2 FROM public.users_table WHERE ((value_1 >= 1) AND (value_1 <= 20)) ORDER BY value_2 LIMIT 5
|
||||
user_id
|
||||
---------
|
||||
4
|
||||
3
|
||||
2
|
||||
1
|
||||
(4 rows)
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE should be replaced due to LIMIT
|
||||
CREATE VIEW subquery_from_from_where AS
|
||||
SELECT user_id, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT user_id, array_agg(event ORDER BY time) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
u.user_id, e.event_type::text AS event, e.time
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
LIMIT 5
|
||||
)
|
||||
) t
|
||||
GROUP BY user_id
|
||||
) q;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
subquery_from_from_where
|
||||
ORDER BY
|
||||
2 DESC, 1;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 19_1 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 >= 5) AND (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 1) AND (events_table.event_type <= 3) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))) AND (NOT (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 3) AND (events_table.event_type <= 4) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))))) LIMIT 5
|
||||
user_id | array_length
|
||||
---------+--------------
|
||||
5 | 364
|
||||
(1 row)
|
||||
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced if
|
||||
-- it contains onle local tables
|
||||
CREATE VIEW subquery_from_from_where_local_table AS
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
event_type, user_id
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table_local WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar
|
||||
) as baz
|
||||
WHERE baz.user_id = users_table.user_id
|
||||
) as sub1;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
subquery_from_from_where
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 21_1 for subquery SELECT user_id FROM public.users_table WHERE ((value_2 >= 5) AND (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 1) AND (events_table.event_type <= 3) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))) AND (NOT (EXISTS (SELECT events_table.user_id FROM public.events_table WHERE ((events_table.event_type > 3) AND (events_table.event_type <= 4) AND (events_table.value_3 > (1)::double precision) AND (events_table.user_id = users_table.user_id)))))) LIMIT 5
|
||||
DEBUG: push down of limit count: 3
|
||||
user_id | array_length
|
||||
---------+--------------
|
||||
5 | 364
|
||||
(1 row)
|
||||
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
CREATE VIEW repartition_view AS
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
repartition_view;
|
||||
DEBUG: cannot use real time executor with repartition jobs
|
||||
HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker.
|
||||
DEBUG: generating subplan 23_1 for subquery SELECT DISTINCT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.value_2) AND (users_table.user_id < 2))
|
||||
DEBUG: generating subplan 23_2 for subquery SELECT count(*) AS count FROM (SELECT intermediate_result.value_2 FROM read_intermediate_result('23_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) foo, (SELECT users_table.user_id FROM public.users_table) bar WHERE (foo.value_2 = bar.user_id)
|
||||
count
|
||||
-------
|
||||
58
|
||||
(1 row)
|
||||
|
||||
CREATE VIEW all_executors_view AS
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table OFFSET 0
|
||||
) as bar,
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) baz,
|
||||
(
|
||||
SELECT user_id FROM users_table_local WHERE user_id = 2
|
||||
) baw
|
||||
WHERE foo.value_2 = bar.user_id AND baz.value_2 = bar.user_id AND bar.user_id = baw.user_id;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
all_executors_view;
|
||||
DEBUG: generating subplan 26_1 for subquery SELECT value_2 FROM public.users_table WHERE (user_id = 15) OFFSET 0
|
||||
DEBUG: generating subplan 26_2 for subquery SELECT user_id FROM public.users_table OFFSET 0
|
||||
DEBUG: cannot use real time executor with repartition jobs
|
||||
HINT: Since you enabled citus.enable_repartition_joins Citus chose to use task-tracker.
|
||||
DEBUG: generating subplan 26_3 for subquery SELECT DISTINCT users_table.value_2 FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.value_2) AND (users_table.user_id < 2))
|
||||
DEBUG: generating subplan 26_4 for subquery SELECT user_id FROM subquery_view.users_table_local WHERE (user_id = 2)
|
||||
DEBUG: generating subplan 26_5 for subquery SELECT count(*) AS count FROM (SELECT intermediate_result.value_2 FROM read_intermediate_result('26_1'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) foo, (SELECT intermediate_result.user_id FROM read_intermediate_result('26_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) bar, (SELECT intermediate_result.value_2 FROM read_intermediate_result('26_3'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) baz, (SELECT intermediate_result.user_id FROM read_intermediate_result('26_4'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) baw WHERE ((foo.value_2 = bar.user_id) AND (baz.value_2 = bar.user_id) AND (bar.user_id = baw.user_id))
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
-- the same query, but this time the CTEs also live inside a subquery
|
||||
CREATE VIEW subquery_and_ctes AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_2 FROM users_table OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_2 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2;
|
||||
SELECT * FROM subquery_and_ctes
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: generating subplan 31_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_view.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT users_table.value_2 FROM public.users_table OFFSET 0) foo WHERE ((events_table.user_id = foo.value_2) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 32_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_view.users_table_local
|
||||
DEBUG: generating subplan 32_2 for CTE dist_cte: SELECT events_table.user_id FROM public.events_table, (SELECT DISTINCT users_table.value_2 FROM public.users_table OFFSET 0) foo WHERE ((events_table.user_id = foo.value_2) AND (events_table.user_id IN (SELECT DISTINCT users_table.value_1 FROM public.users_table ORDER BY users_table.value_1 LIMIT 3)))
|
||||
DEBUG: push down of limit count: 3
|
||||
DEBUG: generating subplan 33_1 for subquery SELECT DISTINCT value_1 FROM public.users_table ORDER BY value_1 LIMIT 3
|
||||
DEBUG: generating subplan 33_2 for subquery SELECT DISTINCT value_2 FROM public.users_table OFFSET 0
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 31_2 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
DEBUG: generating subplan 31_3 for subquery SELECT count(*) AS cnt FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('31_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) cte, (SELECT intermediate_result.user_id FROM read_intermediate_result('31_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo WHERE (foo.user_id = cte.user_id)
|
||||
DEBUG: push down of limit count: 5
|
||||
cnt | user_id | time | value_1 | value_2 | value_3 | value_4
|
||||
-----+---------+---------------------------------+---------+---------+---------+---------
|
||||
432 | 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 |
|
||||
432 | 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 |
|
||||
432 | 3 | Thu Nov 23 17:18:51.048758 2017 | 1 | 5 | 5 |
|
||||
432 | 3 | Thu Nov 23 17:10:35.959913 2017 | 4 | 3 | 1 |
|
||||
432 | 5 | Thu Nov 23 16:48:32.08896 2017 | 5 | 2 | 1 |
|
||||
(5 rows)
|
||||
|
||||
CREATE VIEW subquery_and_ctes_second AS
|
||||
SELECT time, event_type, value_2, value_3 FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM users_table, cte
|
||||
WHERE
|
||||
users_table.user_id = cte.user_id AND
|
||||
users_table.user_id IN
|
||||
(WITH cte_in_where AS (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5) SELECT * FROM cte_in_where)
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
events_table
|
||||
WHERE
|
||||
foo.user_id = events_table.value_2;
|
||||
SELECT * FROM subquery_and_ctes_second
|
||||
ORDER BY 3 DESC, 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
DEBUG: generating subplan 38_1 for CTE cte: WITH local_cte AS (SELECT users_table_local.user_id, users_table_local."time", users_table_local.value_1, users_table_local.value_2, users_table_local.value_3, users_table_local.value_4 FROM subquery_view.users_table_local), dist_cte AS (SELECT events_table.user_id FROM public.events_table) SELECT dist_cte.user_id FROM (local_cte JOIN dist_cte ON ((dist_cte.user_id = local_cte.user_id)))
|
||||
DEBUG: generating subplan 39_1 for CTE local_cte: SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM subquery_view.users_table_local
|
||||
DEBUG: generating subplan 39_2 for CTE dist_cte: SELECT user_id FROM public.events_table
|
||||
DEBUG: generating subplan 38_2 for CTE cte_in_where: SELECT DISTINCT value_2 FROM public.users_table WHERE ((value_1 >= 1) AND (value_1 <= 20)) ORDER BY value_2 LIMIT 5
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 38_3 for subquery SELECT DISTINCT cte.user_id FROM public.users_table, (SELECT intermediate_result.user_id FROM read_intermediate_result('38_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) cte WHERE ((users_table.user_id = cte.user_id) AND (users_table.user_id IN (SELECT cte_in_where.value_2 FROM (SELECT intermediate_result.value_2 FROM read_intermediate_result('38_2'::text, 'binary'::citus_copy_format) intermediate_result(value_2 integer)) cte_in_where))) ORDER BY cte.user_id DESC
|
||||
DEBUG: push down of limit count: 5
|
||||
time | event_type | value_2 | value_3
|
||||
---------------------------------+------------+---------+---------
|
||||
Thu Nov 23 21:54:46.924477 2017 | 6 | 4 | 5
|
||||
Wed Nov 22 21:24:22.849224 2017 | 5 | 4 | 1
|
||||
Wed Nov 22 21:05:25.194441 2017 | 5 | 4 | 1
|
||||
Thu Nov 23 04:01:12.29256 2017 | 4 | 4 | 3
|
||||
Thu Nov 23 09:33:16.992454 2017 | 3 | 4 | 1
|
||||
(5 rows)
|
||||
|
||||
CREATE VIEW deep_subquery AS
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(users_table.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_2 FROM users_table GROUP BY value_2) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, users_table
|
||||
WHERE
|
||||
level_5.avg_ev_type = users_table.user_id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
deep_subquery;
|
||||
DEBUG: generating subplan 43_1 for subquery SELECT count(*) AS cnt, value_2 FROM public.users_table GROUP BY value_2
|
||||
DEBUG: generating subplan 43_2 for subquery SELECT avg(events_table.event_type) AS avg FROM (SELECT level_1.cnt FROM (SELECT intermediate_result.cnt, intermediate_result.value_2 FROM read_intermediate_result('43_1'::text, 'binary'::citus_copy_format) intermediate_result(cnt bigint, value_2 integer)) level_1, public.users_table WHERE (users_table.user_id = level_1.cnt)) level_2, public.events_table WHERE (events_table.user_id = level_2.cnt) GROUP BY level_2.cnt
|
||||
DEBUG: generating subplan 43_3 for subquery SELECT max(users_table.value_1) AS mx_val_1 FROM (SELECT intermediate_result.avg FROM read_intermediate_result('43_2'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)) level_3, public.users_table WHERE ((users_table.user_id)::numeric = level_3.avg) GROUP BY level_3.avg
|
||||
DEBUG: generating subplan 43_4 for subquery SELECT avg(events_table.event_type) AS avg_ev_type FROM (SELECT intermediate_result.mx_val_1 FROM read_intermediate_result('43_3'::text, 'binary'::citus_copy_format) intermediate_result(mx_val_1 integer)) level_4, public.events_table WHERE (level_4.mx_val_1 = events_table.user_id) GROUP BY level_4.mx_val_1
|
||||
DEBUG: generating subplan 43_5 for subquery SELECT min(users_table.value_1) AS min FROM (SELECT intermediate_result.avg_ev_type FROM read_intermediate_result('43_4'::text, 'binary'::citus_copy_format) intermediate_result(avg_ev_type numeric)) level_5, public.users_table WHERE (level_5.avg_ev_type = (users_table.user_id)::numeric) GROUP BY level_5.avg_ev_type
|
||||
DEBUG: generating subplan 43_6 for subquery SELECT avg(level_6.min) AS avg FROM (SELECT intermediate_result.min FROM read_intermediate_result('43_5'::text, 'binary'::citus_copy_format) intermediate_result(min integer)) level_6, public.users_table WHERE (users_table.user_id = level_6.min) GROUP BY users_table.value_1
|
||||
DEBUG: generating subplan 43_7 for subquery SELECT count(*) AS count FROM (SELECT intermediate_result.avg FROM read_intermediate_result('43_6'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric)) bar
|
||||
count
|
||||
-------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
CREATE VIEW result_of_view_is_also_recursively_planned AS
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
*
|
||||
FROM
|
||||
result_of_view_is_also_recursively_planned, events_table
|
||||
WHERE
|
||||
events_table.value_2 = result_of_view_is_also_recursively_planned.user_id
|
||||
ORDER BY time DESC
|
||||
LIMIT 5
|
||||
OFFSET 4
|
||||
) as foo
|
||||
ORDER BY time DESC LIMIT 5;
|
||||
DEBUG: push down of limit count: 5
|
||||
DEBUG: generating subplan 51_1 for subquery SELECT DISTINCT users_table.user_id FROM public.users_table, public.events_table WHERE ((users_table.user_id = events_table.user_id) AND (events_table.event_type = ANY (ARRAY[1, 2, 3, 4]))) ORDER BY users_table.user_id DESC LIMIT 5
|
||||
DEBUG: push down of limit count: 9
|
||||
DEBUG: generating subplan 51_2 for subquery SELECT result_of_view_is_also_recursively_planned.user_id, events_table.user_id, events_table."time", events_table.event_type, events_table.value_2, events_table.value_3, events_table.value_4 FROM (SELECT foo.user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('51_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) foo ORDER BY foo.user_id DESC) result_of_view_is_also_recursively_planned, public.events_table WHERE (events_table.value_2 = result_of_view_is_also_recursively_planned.user_id) ORDER BY events_table."time" DESC OFFSET 4 LIMIT 5
|
||||
user_id | user_id | time | event_type | value_2 | value_3 | value_4
|
||||
---------+---------+---------------------------------+------------+---------+---------+---------
|
||||
2 | 3 | Thu Nov 23 16:44:41.903713 2017 | 4 | 2 | 2 |
|
||||
2 | 5 | Thu Nov 23 16:11:02.929469 2017 | 4 | 2 | 0 |
|
||||
4 | 5 | Thu Nov 23 14:40:40.467511 2017 | 1 | 4 | 1 |
|
||||
3 | 2 | Thu Nov 23 14:02:47.738901 2017 | 1 | 3 | 2 |
|
||||
3 | 6 | Thu Nov 23 14:00:13.20013 2017 | 3 | 3 | 3 |
|
||||
(5 rows)
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
DROP SCHEMA subquery_view CASCADE;
|
||||
NOTICE: drop cascades to 17 other objects
|
||||
DETAIL: drop cascades to table users_table_local
|
||||
drop cascades to table events_table_local
|
||||
drop cascades to view view_without_subquery
|
||||
drop cascades to view view_without_subquery_second
|
||||
drop cascades to view subquery_limit
|
||||
drop cascades to view subquery_non_p_key_group_by
|
||||
drop cascades to view final_query_router
|
||||
drop cascades to view final_query_realtime
|
||||
drop cascades to view subquery_in_where
|
||||
drop cascades to view subquery_from_from_where
|
||||
drop cascades to view subquery_from_from_where_local_table
|
||||
drop cascades to view repartition_view
|
||||
drop cascades to view all_executors_view
|
||||
drop cascades to view subquery_and_ctes
|
||||
drop cascades to view subquery_and_ctes_second
|
||||
drop cascades to view deep_subquery
|
||||
drop cascades to view result_of_view_is_also_recursively_planned
|
||||
SET search_path TO public;
|
|
@ -86,22 +86,27 @@ LIMIT
|
|||
6 | 5
|
||||
(5 rows)
|
||||
|
||||
-- CTE in subquery errors out
|
||||
-- CTE in subquery recursively planned
|
||||
SELECT user_id FROM (
|
||||
WITH cte AS (
|
||||
SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 2 LIMIT 5
|
||||
)
|
||||
SELECT user_id FROM cte WHERE value_2 > 0
|
||||
) a;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: CTEs in subqueries are currently unsupported
|
||||
) a ORDER BY 1 LIMIT 3;
|
||||
user_id
|
||||
---------
|
||||
2
|
||||
2
|
||||
2
|
||||
(3 rows)
|
||||
|
||||
-- CTE outside of FROM/WHERE errors out
|
||||
WITH cte AS (
|
||||
SELECT user_id FROM users_table WHERE value_2 IN (1, 2)
|
||||
)
|
||||
SELECT (SELECT * FROM cte);
|
||||
WARNING: more than one row returned by a subquery used as an expression
|
||||
CONTEXT: while executing command on localhost:57638
|
||||
CONTEXT: while executing command on localhost:57637
|
||||
ERROR: could not receive query results
|
||||
WITH cte_basic AS (
|
||||
SELECT user_id FROM users_table WHERE user_id = 1
|
||||
|
@ -218,8 +223,20 @@ FROM
|
|||
(SELECT min(user_id) AS user_id FROM top_users) top_users
|
||||
JOIN
|
||||
users_table USING (user_id);
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Aggregates without group by are currently unsupported
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
(10 rows)
|
||||
|
||||
-- FOR UPDATE in subquery on CTE
|
||||
WITH top_users AS (
|
||||
SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10
|
||||
|
@ -257,8 +274,15 @@ ORDER BY
|
|||
user_id
|
||||
LIMIT
|
||||
5;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Limit in subquery is currently unsupported
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
(5 rows)
|
||||
|
||||
-- OFFSET in subquery on CTE
|
||||
WITH top_users AS (
|
||||
SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10
|
||||
|
@ -273,8 +297,15 @@ ORDER BY
|
|||
user_id
|
||||
LIMIT
|
||||
5;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Offset clause is currently unsupported
|
||||
user_id
|
||||
---------
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
6
|
||||
(5 rows)
|
||||
|
||||
-- Unsupported join in CTE
|
||||
WITH top_users AS (
|
||||
SELECT DISTINCT e.user_id FROM users_table u JOIN events_table e ON (u.user_id = e.user_id AND u.value_1 > e.value_2)
|
||||
|
@ -330,8 +361,8 @@ ORDER BY
|
|||
user_id
|
||||
LIMIT
|
||||
5;
|
||||
ERROR: could not run distributed query with window functions
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
ERROR: could not run distributed query because the window function that is used cannot be pushed down
|
||||
HINT: Window functions are supported in two ways. Either add an equality filter on the distributed tables' partition column or use the window functions inside a subquery with a PARTITION BY clause containing the distribution column
|
||||
-- Window functions that partition by the distribution column in subqueries in CTEs are ok
|
||||
WITH top_users AS
|
||||
(SELECT *
|
||||
|
@ -434,8 +465,8 @@ ORDER BY
|
|||
1,2,3,4,5,6
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: Complex subqueries and CTEs are not supported within a UNION
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
SELECT * FROM (
|
||||
SELECT * FROM (WITH cte AS (
|
||||
SELECT * FROM users_table
|
||||
|
@ -446,8 +477,8 @@ ORDER BY
|
|||
1,2,3,4,5,6
|
||||
LIMIT
|
||||
10;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: CTEs in subqueries are currently unsupported
|
||||
ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT
|
||||
HINT: Consider using an equality filter on the distributed table's partition column.
|
||||
-- SELECT * FROM (SELECT * FROM cte UNION SELECT * FROM cte) a; should work
|
||||
WITH cte AS (
|
||||
SELECT * FROM users_table WHERE user_id IN (1, 2)
|
||||
|
@ -515,8 +546,7 @@ FROM
|
|||
SELECT * FROM users_table WHERE user_id>1
|
||||
)
|
||||
SELECT * FROM basic_recursive ORDER BY user_id LIMIT 1) cte_rec;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: CTEs in subqueries are currently unsupported
|
||||
ERROR: recursive CTEs are not supported in distributed queries
|
||||
-- basic_recursive in WHERE with UNION ALL
|
||||
SELECT
|
||||
*
|
||||
|
@ -530,8 +560,7 @@ WHERE
|
|||
SELECT user_id FROM users_table WHERE user_id>1
|
||||
)
|
||||
SELECT * FROM basic_recursive ORDER BY user_id LIMIT 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.
|
||||
ERROR: recursive CTEs are not supported in distributed queries
|
||||
-- one recursive one regular CTE should error out
|
||||
WITH RECURSIVE basic_recursive(x) AS(
|
||||
VALUES (1)
|
||||
|
@ -590,15 +619,24 @@ SELECT user_id, sum(value_2) FROM cte_user GROUP BY 1 ORDER BY 1, 2;
|
|||
6 | 220
|
||||
(6 rows)
|
||||
|
||||
SELECT * FROM cte_view;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: CTEs in subqueries are currently unsupported
|
||||
SELECT * FROM cte_view ORDER BY 1, 2 LIMIT 5;
|
||||
user_id | value_1
|
||||
---------+---------
|
||||
1 | 5
|
||||
2 | 4
|
||||
3 | 5
|
||||
4 | 5
|
||||
5 | 5
|
||||
(5 rows)
|
||||
|
||||
WITH cte_user_with_view AS
|
||||
(
|
||||
SELECT * FROM cte_view WHERE user_id < 3
|
||||
)
|
||||
SELECT user_id, value_1 FROM cte_user_with_view ORDER BY 1, 2 LIMIT 10 OFFSET 3;
|
||||
ERROR: cannot push down this subquery
|
||||
DETAIL: CTEs in subqueries are currently unsupported
|
||||
SELECT user_id, value_1 FROM cte_user_with_view ORDER BY 1, 2 LIMIT 10 OFFSET 2;
|
||||
user_id | value_1
|
||||
---------+---------
|
||||
(0 rows)
|
||||
|
||||
DROP VIEW basic_view;
|
||||
DROP VIEW cte_view;
|
||||
|
|
|
@ -84,6 +84,7 @@ ORDER BY
|
|||
|
||||
-- Subqueries in WHERE and FROM are mixed
|
||||
-- In this query, only subquery in WHERE is not a colocated join
|
||||
-- but we're able to recursively plan that as well
|
||||
WITH users_events AS (
|
||||
WITH colocated_join AS (
|
||||
SELECT
|
||||
|
@ -135,15 +136,22 @@ WITH users_events AS (
|
|||
)
|
||||
)
|
||||
SELECT
|
||||
*
|
||||
DISTINCT uid
|
||||
FROM
|
||||
users_events
|
||||
ORDER BY
|
||||
1, 2
|
||||
1 DESC
|
||||
LIMIT
|
||||
20;
|
||||
ERROR: cannot pushdown the subquery
|
||||
DETAIL: Complex subqueries and CTEs are not allowed in the FROM clause when the query has subqueries in the WHERE clause
|
||||
5;
|
||||
uid
|
||||
-----
|
||||
6
|
||||
5
|
||||
4
|
||||
3
|
||||
2
|
||||
(5 rows)
|
||||
|
||||
-- cte LEFT JOIN distributed_table should error out
|
||||
WITH cte AS (
|
||||
SELECT * FROM users_table WHERE user_id = 1 ORDER BY value_1
|
||||
|
|
|
@ -122,6 +122,10 @@ WHERE
|
|||
(1 row)
|
||||
|
||||
-- CTE in WHERE basic
|
||||
-- this is a tricky query that hits an aggresive
|
||||
-- check in subquery puwhdown after the recursive planning
|
||||
-- where LIMIT should be allowed
|
||||
-- if the query contains only intermediate results
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
|
@ -141,9 +145,16 @@ IN
|
|||
1)
|
||||
SELECT * FROM events LIMIT 10
|
||||
);
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
count
|
||||
-------
|
||||
101
|
||||
(1 row)
|
||||
|
||||
-- CTE with non-colocated join in WHERE
|
||||
-- this is a tricky query that hits an aggresive
|
||||
-- check in subquery puwhdown after the recursive planning
|
||||
-- where LIMIT should be allowed
|
||||
-- if the query contains only intermediate results
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
|
@ -164,5 +175,8 @@ WHERE
|
|||
)
|
||||
SELECT * FROM users LIMIT 10
|
||||
);
|
||||
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
|
||||
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
|
||||
count
|
||||
-------
|
||||
101
|
||||
(1 row)
|
||||
|
||||
|
|
|
@ -38,6 +38,14 @@ test: multi_behavioral_analytics_basics multi_behavioral_analytics_single_shard_
|
|||
# ----------
|
||||
test: multi_partitioning_utils multi_partitioning
|
||||
|
||||
|
||||
# ----------
|
||||
# Tests for recursive subquery planning
|
||||
# ----------
|
||||
test: subquery_basics subquery_local_tables subquery_executors subquery_and_cte
|
||||
test: subqueries_deep subquery_view subquery_partitioning subquery_complex_target_list subqueries_not_supported
|
||||
test: subquery_prepared_statements
|
||||
|
||||
# ----------
|
||||
# Miscellaneous tests to check our query planning behavior
|
||||
# ----------
|
||||
|
|
|
@ -157,12 +157,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 single repartition subquery
|
||||
SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq;
|
||||
-- the subquery is recursively planned since it contains OFFSET, which is not pushdownable
|
||||
SELECT * FROM (SELECT o_custkey FROM orders GROUP BY o_custkey ORDER BY o_custkey OFFSET 20) sq ORDER BY 1 LIMIT 5;
|
||||
|
||||
-- Error in the optimizer when subquery pushdown is on
|
||||
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq;
|
||||
-- the subquery is recursively planned since it contains OFFSET, which is not pushdownable
|
||||
SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq ORDER BY 1 LIMIT 5;
|
||||
|
||||
-- Simple LIMIT/OFFSET with ORDER BY
|
||||
SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20;
|
||||
|
|
|
@ -795,7 +795,7 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4,
|
|||
GROUP BY raw_events_second.user_id) AS foo;
|
||||
|
||||
|
||||
-- INSERT partition column does not match with SELECT partition column
|
||||
-- INSERT returns NULL partition key value via coordinator
|
||||
INSERT INTO agg_events
|
||||
(value_4_agg,
|
||||
value_1_agg,
|
||||
|
@ -874,8 +874,7 @@ SELECT
|
|||
FROM
|
||||
reference_table;
|
||||
|
||||
-- unsupported joins between subqueries
|
||||
-- we do not return bare partition column on the inner query
|
||||
-- foo2 is recursively planned and INSERT...SELECT is done via coordinator
|
||||
INSERT INTO agg_events
|
||||
(user_id)
|
||||
SELECT f2.id FROM
|
||||
|
@ -903,6 +902,7 @@ ON (f.id = f2.id);
|
|||
-- the second part of the query is not routable since
|
||||
-- GROUP BY not on the partition column (i.e., value_1) and thus join
|
||||
-- on f.id = f2.id is not on the partition key (instead on the sum of partition key)
|
||||
-- but we still recursively plan foo2 and run the query
|
||||
INSERT INTO agg_events
|
||||
(user_id)
|
||||
SELECT f.id FROM
|
||||
|
@ -1315,8 +1315,8 @@ SET client_min_messages TO INFO;
|
|||
-- avoid constraint violations
|
||||
TRUNCATE raw_events_first;
|
||||
|
||||
-- we don't support LIMIT even if it exists in the subqueries
|
||||
-- in where clause
|
||||
-- we don't support LIMIT for subquery pushdown, but
|
||||
-- we recursively plan the query and run it via coordinator
|
||||
INSERT INTO agg_events(user_id)
|
||||
SELECT user_id
|
||||
FROM users_table
|
||||
|
|
|
@ -662,7 +662,7 @@ FROM
|
|||
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
|
||||
-- supported via recursive planning
|
||||
INSERT INTO agg_results (user_id, agg_time, value_2_agg)
|
||||
SELECT
|
||||
user_id,
|
||||
|
|
|
@ -216,8 +216,8 @@ SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1;
|
|||
|
||||
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 3;
|
||||
|
||||
-- they are not supported if multiple workers are involved
|
||||
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2;
|
||||
-- they are supported via (sub)query pushdown if multiple workers are involved
|
||||
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2 ORDER BY 4 DESC, 1 DESC, 2 DESC LIMIT 5;
|
||||
|
||||
-- subqueries are supported in FROM clause but they are not router plannable
|
||||
SELECT articles_hash_mx.id,test.word_count
|
||||
|
|
|
@ -33,6 +33,24 @@ SELECT a FROM dest_table WHERE a = 1;
|
|||
-- real-time selects are also allowed
|
||||
SELECT a FROM dest_table;
|
||||
|
||||
-- subqueries are also allowed
|
||||
SET client_min_messages TO DEBUG1;
|
||||
SELECT
|
||||
foo.a
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT dest_table.a
|
||||
FROM
|
||||
dest_table, source_table
|
||||
WHERE
|
||||
source_table.a = dest_table.a AND
|
||||
dest_table.b IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC LIMIT 5
|
||||
) as foo;
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
-- insert into is definitely not allowed
|
||||
INSERT INTO dest_table (a, b)
|
||||
SELECT a, b FROM source_table;
|
||||
|
|
|
@ -280,8 +280,8 @@ SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1;
|
|||
|
||||
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 3;
|
||||
|
||||
-- they are not supported if multiple workers are involved
|
||||
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2;
|
||||
-- they are supported via (sub)query pushdown if multiple workers are involved
|
||||
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2 ORDER BY 4 DESC, 1 DESC, 2 DESC LIMIT 5;
|
||||
|
||||
-- unless the query can be transformed into a join
|
||||
SELECT * FROM articles_hash
|
||||
|
|
|
@ -124,10 +124,10 @@ SELECT * FROM articles WHERE author_id = 2;
|
|||
WITH long_names AS ( SELECT id FROM authors WHERE char_length(name) > 15 )
|
||||
SELECT title FROM articles ORDER BY 1 LIMIT 5;
|
||||
|
||||
-- queries which involve functions in FROM clause are unsupported.
|
||||
SELECT * FROM articles, position('om' in 'Thomas');
|
||||
-- queries which involve functions in FROM clause are recursively planned
|
||||
SELECT * FROM articles, position('om' in 'Thomas') ORDER BY 2 DESC, 1 DESC, 3 DESC LIMIT 5;
|
||||
|
||||
-- subqueries are not supported in WHERE clause in Citus
|
||||
-- subqueries are supported in WHERE clause in Citus even if the relations are not distributed
|
||||
SELECT * FROM articles WHERE author_id IN (SELECT id FROM authors WHERE name LIKE '%a');
|
||||
|
||||
-- subqueries are supported in FROM clause
|
||||
|
|
|
@ -74,7 +74,7 @@ from
|
|||
group by
|
||||
suppkey_bin
|
||||
order by
|
||||
avg_count desc
|
||||
avg_count desc, suppkey_bin DESC
|
||||
limit 20;
|
||||
|
||||
select
|
||||
|
@ -122,8 +122,7 @@ from
|
|||
(l_orderkey/4)::int,
|
||||
l_suppkey ) as distributed_table;
|
||||
|
||||
-- Check that we don't support subqueries with limit.
|
||||
|
||||
-- we don't support subqueries with limit.
|
||||
select
|
||||
l_suppkey,
|
||||
sum(suppkey_count) as total_suppkey_count
|
||||
|
@ -139,19 +138,23 @@ from
|
|||
l_suppkey
|
||||
limit 100) as distributed_table
|
||||
group by
|
||||
l_suppkey;
|
||||
l_suppkey
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
|
||||
-- Check that we don't support subqueries without aggregates.
|
||||
|
||||
select
|
||||
rounded_tax
|
||||
DISTINCT rounded_tax
|
||||
from
|
||||
(select
|
||||
round(l_tax) as rounded_tax
|
||||
from
|
||||
lineitem
|
||||
group by
|
||||
l_tax) as distributed_table;
|
||||
l_tax) as distributed_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5;
|
||||
|
||||
-- Check that we support subqueries with count(distinct).
|
||||
|
||||
|
|
|
@ -30,8 +30,7 @@ SET
|
|||
WHERE
|
||||
shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'orders_subquery'::regclass ORDER BY shardid DESC LIMIT 1);
|
||||
|
||||
-- If group by is not on partition column then we error out from single table
|
||||
-- repartition code path
|
||||
-- If group by is not on partition column then we recursively plan
|
||||
SELECT
|
||||
avg(order_count)
|
||||
FROM
|
||||
|
@ -70,9 +69,9 @@ FROM
|
|||
GROUP BY
|
||||
l_orderkey) AS unit_prices;
|
||||
|
||||
-- Subqueries without relation with a volatile functions (non-constant)
|
||||
-- Subqueries without relation with a volatile functions (non-constant) are planned recursively
|
||||
SELECT count(*) FROM (
|
||||
SELECT l_orderkey FROM lineitem_subquery JOIN (SELECT random()::int r) sub ON (l_orderkey = r)
|
||||
SELECT l_orderkey FROM lineitem_subquery JOIN (SELECT random()::int r) sub ON (l_orderkey = r) WHERE r > 10
|
||||
) b;
|
||||
|
||||
-- Check that we error out if there is non relation subqueries
|
||||
|
@ -98,7 +97,8 @@ SELECT count(*) FROM
|
|||
(SELECT l_orderkey FROM lineitem_subquery) UNION
|
||||
(SELECT l_orderkey FROM lineitem_subquery)
|
||||
) b;
|
||||
-- Check that we error out if inner query has Limit but subquery_pushdown is not set
|
||||
-- we'd error out if inner query has Limit but subquery_pushdown is not set
|
||||
-- but we recursively plan the query
|
||||
SELECT
|
||||
avg(o_totalprice/l_quantity)
|
||||
FROM
|
||||
|
|
|
@ -1120,7 +1120,9 @@ limit 50;
|
|||
-- reset subquery_pushdown
|
||||
SET citus.subquery_pushdown to OFF;
|
||||
|
||||
-- not supported since JOIN is not on the partition key
|
||||
-- we recursively plan recent_events_1
|
||||
-- but not some_users_data since it has a reference
|
||||
-- from an outer query which is not recursively planned
|
||||
SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT user_id, max(time) AS lastseen
|
||||
|
@ -1154,8 +1156,9 @@ ORDER BY
|
|||
user_id
|
||||
limit 50;
|
||||
|
||||
-- not supported since JOIN is not on the partition key
|
||||
-- see (2 * user_id as user_id) target list element
|
||||
-- we recursively plan some queries but fail in the end
|
||||
-- since some_users_data since it has a reference
|
||||
-- from an outer query which is not recursively planned
|
||||
SELECT "some_users_data".user_id, lastseen
|
||||
FROM
|
||||
(SELECT 2 * user_id as user_id, max(time) AS lastseen
|
||||
|
@ -1465,7 +1468,9 @@ ORDER BY
|
|||
user_id DESC
|
||||
LIMIT 10;
|
||||
|
||||
-- not supported since lower LATERAL JOIN is not on the partition key
|
||||
-- not pushdownable since lower LATERAL JOIN is not on the partition key
|
||||
-- not recursively plannable due to LATERAL join where there is a reference
|
||||
-- from an outer query
|
||||
SELECT user_id, lastseen
|
||||
FROM
|
||||
(SELECT
|
||||
|
@ -1995,7 +2000,9 @@ ORDER BY
|
|||
LIMIT 10;
|
||||
SET citus.subquery_pushdown to OFF;
|
||||
|
||||
-- not supported since join is not on the partition key
|
||||
-- not pushdownable since lower LATERAL JOIN is not on the partition key
|
||||
-- not recursively plannable due to LATERAL join where there is a reference
|
||||
-- from an outer query
|
||||
SELECT *
|
||||
FROM
|
||||
(SELECT
|
||||
|
|
|
@ -980,14 +980,16 @@ SELECT foo.user_id FROM
|
|||
WHERE event_type > 100
|
||||
) as foo;
|
||||
|
||||
-- not supported since group by is on the reference table column
|
||||
-- not pushdownable since group by is on the reference table column
|
||||
-- recursively planned, but hits unsupported clause type error on the top level query
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
SELECT r.user_id, random() FROM users_table m JOIN events_reference_table r ON int4eq(m.user_id, r.user_id)
|
||||
GROUP BY r.user_id
|
||||
) as foo;
|
||||
|
||||
-- supported since the group by contains at least one distributed table
|
||||
-- not pushdownable since the group by contains at least one distributed table
|
||||
-- recursively planned, but hits unsupported clause type error on the top level query
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
SELECT r.user_id, random() FROM users_table m JOIN events_reference_table r ON int4eq(m.user_id, r.user_id)
|
||||
|
@ -995,7 +997,8 @@ SELECT foo.user_id FROM
|
|||
) as foo
|
||||
ORDER BY 1 LIMIT 3;
|
||||
|
||||
-- not supported since distinct is on the reference table column
|
||||
-- not pushdownable since distinct is on the reference table column
|
||||
-- recursively planned, but hits unsupported clause type error on the top level query
|
||||
SELECT foo.user_id FROM
|
||||
(
|
||||
SELECT DISTINCT r.user_id, random() FROM users_table m JOIN events_reference_table r ON int4eq(m.user_id, r.user_id)
|
||||
|
@ -1048,7 +1051,8 @@ LIMIT 5
|
|||
OFFSET 0;
|
||||
|
||||
-- should not push down this query since there is a distributed table (i.e., events_table)
|
||||
-- which is not in the DISTINCT clause
|
||||
-- which is not in the DISTINCT clause. Recursive planning also fails since router execution
|
||||
-- is disabled
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT DISTINCT users_reference_table.user_id FROM users_reference_table, events_table WHERE users_reference_table.user_id = events_table.value_4
|
||||
|
@ -1076,6 +1080,8 @@ ORDER BY 1 DESC
|
|||
LIMIT 4;
|
||||
|
||||
-- should not pushdown since there is a non partition column on the DISTINCT clause
|
||||
-- Recursive planning also fails since router execution
|
||||
-- is disabled
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT
|
||||
|
@ -1172,7 +1178,7 @@ FROM
|
|||
ORDER BY 1
|
||||
LIMIT 5;
|
||||
|
||||
-- should error out since there is a distributed table and
|
||||
-- should recursively plan since
|
||||
-- there are no columns on the GROUP BY from the distributed table
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
|
@ -1206,7 +1212,7 @@ ORDER BY 1
|
|||
LIMIT 5;
|
||||
|
||||
|
||||
-- should not work since we're
|
||||
-- should recursively plan since we're
|
||||
-- using an immutable function as recurring tuple
|
||||
-- along with a distributed table, where GROUP BY is
|
||||
-- on the recurring tuple
|
||||
|
|
|
@ -489,13 +489,14 @@ ORDER BY 1 ASC
|
|||
LIMIT 2;
|
||||
|
||||
-- subquery in where clause has a volatile function and no relation
|
||||
-- thus we recursively plan it
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
value_2 >
|
||||
(SELECT random())
|
||||
(SELECT random()) AND user_id < 0
|
||||
ORDER BY 1 ASC
|
||||
LIMIT 2;
|
||||
|
||||
|
@ -519,7 +520,8 @@ WHERE
|
|||
|
||||
-- we can detect unsupported subquerues even if they appear
|
||||
-- in WHERE subquery -> FROM subquery -> WHERE subquery
|
||||
SELECT user_id
|
||||
-- but we can recursively plan that anyway
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE user_id
|
||||
IN (SELECT
|
||||
|
@ -543,7 +545,7 @@ WHERE user_id
|
|||
AND e1.user_id IN (SELECT user_id FROM users_table LIMIT 3 )
|
||||
) as f_outer
|
||||
WHERE f_inner.user_id = f_outer.user_id
|
||||
);
|
||||
) ORDER BY 1 LIMIT 3;
|
||||
|
||||
-- semi join is not on the partition key for the third subquery
|
||||
SELECT user_id
|
||||
|
|
|
@ -569,6 +569,8 @@ LIMIT 5;
|
|||
-- now lets also have some unsupported queries
|
||||
|
||||
-- group by is not on the partition key
|
||||
-- but we can still recursively plan it, though that is not suffient for pushdown
|
||||
-- of the whole query
|
||||
SELECT user_id, sum(counter)
|
||||
FROM (
|
||||
SELECT user_id, sum(value_2) AS counter FROM events_table GROUP BY user_id
|
||||
|
@ -803,7 +805,9 @@ FROM
|
|||
(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
|
||||
-- we don't support pushing down subqueries without relations
|
||||
-- recursive planning can replace that query, though the whole
|
||||
-- query is not safe to pushdown
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
|
@ -813,7 +817,9 @@ FROM
|
|||
(SELECT 1)
|
||||
) b;
|
||||
|
||||
-- we don't support subqueries without relations
|
||||
-- we don't support pushing down subqueries without relations
|
||||
-- recursive planning can replace that query, though the whole
|
||||
-- query is not safe to pushdown
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
|
@ -843,6 +849,9 @@ FROM
|
|||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 5;
|
||||
|
||||
-- we don't support pushing down subqueries without relations
|
||||
-- recursive planning can replace that query, though the whole
|
||||
-- query is not safe to pushdown
|
||||
SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType
|
||||
FROM
|
||||
( SELECT *, random()
|
||||
|
|
|
@ -146,8 +146,8 @@ SET citus.task_executor_type to DEFAULT;
|
|||
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 GROUP BY of partition key
|
||||
SELECT * FROM lineitems_by_shipping_method;
|
||||
-- following will be supported via recursive planning
|
||||
SELECT * FROM lineitems_by_shipping_method ORDER BY 1,2 LIMIT 5;
|
||||
|
||||
-- create a view with group by on partition column
|
||||
CREATE VIEW lineitems_by_orderkey AS
|
||||
|
@ -348,18 +348,19 @@ CREATE VIEW distinct_user_with_value_1_3 AS SELECT DISTINCT user_id FROM users_t
|
|||
SELECT * FROM distinct_user_with_value_1_3 ORDER BY user_id;
|
||||
|
||||
-- distinct is not supported if it is on a non-partition key
|
||||
-- but will be supported via recursive planning
|
||||
CREATE VIEW distinct_value_1 AS SELECT DISTINCT value_1 FROM users_table WHERE value_2 = 3;
|
||||
SELECT * FROM distinct_value_1;
|
||||
SELECT * FROM distinct_value_1 ORDER BY 1 DESC LIMIT 5;
|
||||
|
||||
-- CTEs are not supported even if they are on views
|
||||
-- CTEs are supported even if they are on views
|
||||
CREATE VIEW cte_view_1 AS
|
||||
WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 3) SELECT * FROM c1 WHERE value_2 < 4;
|
||||
|
||||
SELECT * FROM cte_view_1;
|
||||
SELECT * FROM cte_view_1 ORDER BY 1,2,3,4,5 LIMIT 5;
|
||||
|
||||
-- this is single shard query but still not supported since it has view + cte
|
||||
-- this is single shard query and still not supported since it has view + cte
|
||||
-- router planner can't detect it
|
||||
SELECT * FROM cte_view_1 WHERE user_id = 2;
|
||||
SELECT * FROM cte_view_1 WHERE user_id = 2 ORDER BY 1,2,3,4,5;
|
||||
|
||||
-- if CTE itself prunes down to a single shard than the view is supported (router plannable)
|
||||
CREATE VIEW cte_view_2 AS
|
||||
|
|
|
@ -0,0 +1,160 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with subqueries and CTEs
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_deep;
|
||||
SET search_path TO subquery_and_ctes, public;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced due to OFFSET
|
||||
-- one level up subquery should be replaced due to GROUP BY on non partition key
|
||||
-- one level up subquery should be replaced due to LIMUT
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
avg(event_type) as avg_val
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar, users_table WHERE bar.user_id = users_table.user_id GROUP BY users_table.value_1
|
||||
|
||||
) as baz
|
||||
WHERE baz.avg_val < users_table.user_id
|
||||
LIMIT 3
|
||||
|
||||
) as sub1
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE -> WHERE should be replaced due to CTE
|
||||
-- subquery in FROM -> FROM -> WHERE should be replaced due to LIMIT
|
||||
-- one level above should be replaced due to DISTINCT on non-partition key
|
||||
-- one level above should be replaced due to GROUP BY on non-partition key
|
||||
SELECT event, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT event, array_agg(t.user_id) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
DISTINCT ON(e.event_type::text) e.event_type::text as event, e.time, e.user_id
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND EXISTS (WITH cte AS (SELECT count(*) FROM users_table) SELECT * FROM cte)
|
||||
LIMIT 5
|
||||
)
|
||||
) t, users_table WHERE users_table.value_1 = t.event::int
|
||||
GROUP BY event
|
||||
) q
|
||||
ORDER BY 2 DESC, 1;
|
||||
|
||||
-- this test probably doesn't add too much value,
|
||||
-- but recurse 6 times for fun
|
||||
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(users_table.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_2 FROM users_table GROUP BY value_2) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, users_table
|
||||
WHERE
|
||||
level_5.avg_ev_type = users_table.user_id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
|
||||
-- same query happening in the subqueries in WHERE
|
||||
|
||||
-- this test probably doesn't add too much value,
|
||||
-- but recurse 6 times for fun
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
users_table
|
||||
WHERE user_id IN (
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(users_table.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_2 FROM users_table GROUP BY value_2) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, users_table
|
||||
WHERE
|
||||
level_5.avg_ev_type = users_table.user_id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar);
|
||||
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_deep CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,137 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on failure cases
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA not_supported;
|
||||
SET search_path TO not_supported, public;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
|
||||
-- we don't support subqueries with local tables when they are not leaf queries
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
users_table_local.user_id
|
||||
FROM
|
||||
users_table_local, (SELECT user_id FROM events_table) as evs
|
||||
WHERE users_table_local.user_id = evs.user_id
|
||||
) as foo;
|
||||
|
||||
-- we don't support subqueries with local tables when they are not leaf queries
|
||||
SELECT user_id FROM users_table WHERE user_id IN
|
||||
(SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table_local JOIN (SELECT user_id FROM events_table_local) as foo
|
||||
USING (user_id)
|
||||
);
|
||||
|
||||
-- we don't support aggregate distinct if the group by is not on partition key, expect for count distinct
|
||||
-- thus baz and bar are recursively planned but not foo
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT avg(DISTINCT value_1), random() FROM users_table GROUP BY user_id OFFSET 3
|
||||
) as baz,
|
||||
(
|
||||
SELECT count(DISTINCT value_1), random() FROM users_table GROUP BY value_2 OFFSET 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT avg(DISTINCT value_1), random() FROM users_table GROUP BY value_2 OFFSET 3
|
||||
) as foo;
|
||||
|
||||
-- we don't support array_aggs with ORDER BYs
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
array_agg(users_table.user_id ORDER BY users_table.time)
|
||||
FROM
|
||||
users_table, (SELECT user_id FROM events_table) as evs
|
||||
WHERE users_table.user_id = evs.user_id
|
||||
GROUP BY users_table.user_id
|
||||
LIMIT 5
|
||||
) as foo;
|
||||
|
||||
-- we don't support queries with recurring tuples in the FROM
|
||||
-- clause and subquery in WHERE clause
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, (SELECT user_id FROM events_table) as evs
|
||||
WHERE users_table.user_id = evs.user_id
|
||||
LIMIT 5
|
||||
) as foo WHERE user_id IN (SELECT count(*) FROM users_table GROUP BY user_id);
|
||||
|
||||
-- we don't support recursive subqueries when router executor is disabled
|
||||
SET citus.enable_router_execution TO false;
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
SET citus.enable_router_execution TO true;
|
||||
|
||||
|
||||
-- window functions are not allowed if they're not partitioned on the distribution column
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
user_id, time, rnk
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
*, rank() OVER my_win as rnk
|
||||
FROM
|
||||
events_table
|
||||
WINDOW my_win AS (PARTITION BY event_type ORDER BY time DESC)
|
||||
) as foo
|
||||
ORDER BY
|
||||
3 DESC, 1 DESC, 2 DESC
|
||||
LIMIT
|
||||
10) as foo;
|
||||
|
||||
-- top level join is not on the distribution key thus not supported
|
||||
-- (use random to prevent Postgres to pull subqueries)
|
||||
SELECT
|
||||
foo.value_2
|
||||
FROM
|
||||
(SELECT users_table.value_2, random() FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (1,2,3,4)) as foo,
|
||||
(SELECT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar
|
||||
WHERE
|
||||
foo.value_2 = bar.value_2;
|
||||
|
||||
|
||||
-- OUTER JOINs where the outer part is recursively planned and not the other way
|
||||
-- around is not supported
|
||||
SELECT
|
||||
foo.value_2
|
||||
FROM
|
||||
(SELECT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (1,2,3,4) LIMIT 5) as foo
|
||||
LEFT JOIN
|
||||
(SELECT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.user_id AND event_type IN (5,6,7,8)) as bar
|
||||
ON(foo.value_2 = bar.value_2);
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA not_supported CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,326 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with subqueries and CTEs
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_and_ctes;
|
||||
SET search_path TO subquery_and_ctes, public;
|
||||
|
||||
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
-- CTEs are recursively planned, and subquery foo is also recursively planned
|
||||
-- final plan becomes a router plan
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id;
|
||||
|
||||
-- CTEs are recursively planned, and subquery foo is also recursively planned
|
||||
-- final plan becomes a real-time plan since we also have events_table in the
|
||||
-- range table entries
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo, events_table
|
||||
WHERE foo.user_id = cte.user_id AND events_table.user_id = cte.user_id;
|
||||
|
||||
-- CTEs are replaced and subquery in WHERE is also replaced
|
||||
-- but the query is still real-time query since users_table is in the
|
||||
-- range table list
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM users_table, cte
|
||||
WHERE
|
||||
users_table.user_id = cte.user_id AND
|
||||
users_table.user_id IN (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5)
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- a very similar query as the above, but this time errors
|
||||
-- out since we don't support subqueries in WHERE clause
|
||||
-- when there is only intermediate results on the range table
|
||||
-- note that this time subquery in WHERE clause is not replaced
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM cte
|
||||
WHERE
|
||||
cte.user_id IN (SELECT DISTINCT user_id FROM users_table WHERE value_1 >= 1 AND value_1 <= 20)
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- CTEs inside a subquery and the final query becomes a router
|
||||
-- query
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo;
|
||||
|
||||
|
||||
-- CTEs inside a subquery and the final query becomes a
|
||||
-- real-time query since the other subquery is safe to pushdown
|
||||
SELECT
|
||||
bar.user_id
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
|
||||
) as bar
|
||||
WHERE foo.user_id = bar.user_id;
|
||||
|
||||
-- CTEs inside a deeper subquery
|
||||
-- and also the subquery that contains the CTE is replaced
|
||||
SELECT
|
||||
DISTINCT bar.user_id
|
||||
FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
users_table.user_id, some_events.event_type
|
||||
FROM
|
||||
users_table,
|
||||
(
|
||||
WITH cte AS (
|
||||
SELECT
|
||||
event_type, users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
value_1 IN (1,2)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as some_events
|
||||
WHERE
|
||||
users_table.user_id = some_events.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 2,1
|
||||
LIMIT 2
|
||||
|
||||
) as bar
|
||||
WHERE foo.user_id = bar.user_id
|
||||
ORDER BY 1 DESC LIMIT 5;
|
||||
|
||||
|
||||
|
||||
-- CTEs on the different parts of the query is replaced
|
||||
-- and subquery foo is also replaced since it contains
|
||||
-- DISTINCT on a non-partition key
|
||||
SELECT * FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM users_table, cte
|
||||
WHERE
|
||||
users_table.user_id = cte.user_id AND
|
||||
users_table.user_id IN
|
||||
(WITH cte_in_where AS (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5) SELECT * FROM cte_in_where)
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
events_table
|
||||
WHERE
|
||||
foo.user_id = events_table.value_2
|
||||
ORDER BY 3 DESC, 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
|
||||
|
||||
-- now recursively plan subqueries inside the CTEs that contains LIMIT and OFFSET
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_2 FROM users_table OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_2 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id;
|
||||
|
||||
-- the same query, but this time the CTEs also live inside a subquery
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_2 FROM users_table OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_2 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
|
||||
-- recursive CTES are not supported inside subqueries as well
|
||||
SELECT
|
||||
bar.user_id
|
||||
FROM
|
||||
(
|
||||
WITH RECURSIVE cte AS (
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
) SELECT * FROM cte ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
|
||||
) as bar
|
||||
WHERE foo.user_id = bar.user_id;
|
||||
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_and_ctes CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,268 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality
|
||||
-- ===================================================================
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
-- subqueries in FROM clause with LIMIT should be recursively planned
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
|
||||
-- subqueries in FROM clause with DISTINCT on non-partition key
|
||||
-- should be recursively planned
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.value_1
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- subqueries in FROM clause with GROUP BY on non-partition key
|
||||
-- should be recursively planned
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2, avg(value_1)
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo
|
||||
ORDER BY 2 DESC, 1;
|
||||
|
||||
-- multiple subqueries in FROM clause should be replaced
|
||||
-- and the final query is router query
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.value_3
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
GROUP BY users_table.value_3
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.value_3
|
||||
ORDER BY 2 DESC, 1;
|
||||
|
||||
-- same query with alias in the subquery
|
||||
SELECT
|
||||
DISTINCT ON (citus) citus, postgres, citus + 1 as c1, postgres-1 as p1
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo(postgres),
|
||||
(SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC
|
||||
) as bar (citus)
|
||||
WHERE foo.postgres = bar.citus
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
|
||||
-- foo is replaced
|
||||
-- and the final query is real-time
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
|
||||
-- subqueries in WHERE should be replaced
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE
|
||||
user_id IN (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5)
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced due to OFFSET
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
event_type, user_id
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar
|
||||
|
||||
) as baz
|
||||
WHERE baz.user_id = users_table.user_id
|
||||
|
||||
) as sub1
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3;
|
||||
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE should be replaced due to LIMIT
|
||||
SELECT user_id, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT user_id, array_agg(event ORDER BY time) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
u.user_id, e.event_type::text AS event, e.time
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
LIMIT 5
|
||||
)
|
||||
) t
|
||||
GROUP BY user_id
|
||||
) q
|
||||
ORDER BY 2 DESC, 1;
|
||||
|
||||
|
||||
-- subquery (i.e., subquery_2) in WHERE->FROM should be replaced due to LIMIT
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM (
|
||||
SELECT
|
||||
subquery_1.user_id, count_pay
|
||||
FROM
|
||||
(
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>1' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 1 AND events_table.event_type < 3
|
||||
)
|
||||
UNION
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>2' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 2 AND events_table.event_type < 4
|
||||
)
|
||||
) AS subquery_1
|
||||
LEFT JOIN
|
||||
(SELECT
|
||||
user_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
user_id >= 1 AND
|
||||
user_id <= 3 AND
|
||||
users_table.value_1 > 3 AND users_table.value_1 < 5
|
||||
GROUP BY
|
||||
user_id
|
||||
HAVING
|
||||
COUNT(*) > 1
|
||||
LIMIT 10
|
||||
|
||||
) AS subquery_2
|
||||
ON
|
||||
subquery_1.user_id = subquery_2.user_id
|
||||
GROUP BY
|
||||
subquery_1.user_id,
|
||||
count_pay) AS subquery_top
|
||||
GROUP BY
|
||||
count_pay, user_id
|
||||
)
|
||||
GROUP BY user_id
|
||||
HAVING count(*) > 1 AND sum(value_2) > 29
|
||||
ORDER BY 1;
|
||||
|
|
@ -0,0 +1,293 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with complex target entries
|
||||
-- and some utilities
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_complex;
|
||||
SET search_path TO subquery_complex, public;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
-- COUNT DISTINCT at the top level query
|
||||
SELECT
|
||||
event_type, count(distinct value_2)
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id IN (SELECT user_id FROM users_table GROUP BY user_id ORDER BY count(*) DESC LIMIT 20)
|
||||
GROUP BY
|
||||
event_type
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
|
||||
|
||||
-- aggregate distinct in the subqueries
|
||||
-- avg distinct on partition key
|
||||
-- count distinct on partition key
|
||||
-- count distinct on non-partition key
|
||||
-- sum distinct on non-partition key when group by is partition key
|
||||
-- and the final query is real-time query
|
||||
SELECT
|
||||
DISTINCT ON (avg) avg, cnt_1, cnt_2, sum
|
||||
FROM
|
||||
(
|
||||
SELECT avg(distinct user_id) as avg FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as foo,
|
||||
(
|
||||
SELECT count(distinct user_id) as cnt_1 FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT count(distinct value_2) as cnt_2 FROM users_table ORDER BY 1 DESC LIMIT 4
|
||||
) as baz,
|
||||
(
|
||||
SELECT user_id, sum(distinct value_2) as sum FROM users_table GROUP BY user_id ORDER BY 1 DESC LIMIT 4
|
||||
) as bat, events_table
|
||||
WHERE foo.avg != bar.cnt_1 AND baz.cnt_2 = events_table.event_type
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- Aggregate type conversions inside the subqueries
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
min(user_id) * 2, max(user_id) / 2, sum(user_id), count(user_id)::float, avg(user_id)::bigint
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
min(value_3) * 2, max(value_3) / 2, sum(value_3), count(value_3), avg(value_3)
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT
|
||||
min(time), max(time), count(time),
|
||||
count(*) FILTER (WHERE user_id = 3) as cnt_with_filter,
|
||||
count(*) FILTER (WHERE user_id::text LIKE '%3%') as cnt_with_filter_2
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as baz
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- Expressions inside the aggregates
|
||||
-- parts of the query is inspired by TPCH queries
|
||||
SELECT
|
||||
DISTINCT ON (avg) avg, cnt_1, cnt_2, cnt_3, sum_1,l_year, pos, count_pay
|
||||
FROM
|
||||
(
|
||||
SELECT avg(user_id * (5.0 / (value_1 + 0.1))) as avg FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as foo,
|
||||
(
|
||||
SELECT sum(user_id * (5.0 / (value_1 + value_2 + 0.1)) * value_3) as cnt_1 FROM users_table ORDER BY 1 DESC LIMIT 3
|
||||
) as bar,
|
||||
(
|
||||
SELECT
|
||||
avg(case
|
||||
when user_id > 4
|
||||
then value_1
|
||||
end) as cnt_2,
|
||||
avg(case
|
||||
when user_id > 500
|
||||
then value_1
|
||||
end) as cnt_3,
|
||||
sum(case
|
||||
when value_1 = 1
|
||||
OR value_2 = 1
|
||||
then 1
|
||||
else 0
|
||||
end) as sum_1,
|
||||
extract(year FROM max(time)) as l_year,
|
||||
strpos(max(user_id)::text, '1') as pos
|
||||
FROM
|
||||
users_table
|
||||
ORDER BY
|
||||
1 DESC
|
||||
LIMIT 4
|
||||
) as baz,
|
||||
(
|
||||
SELECT COALESCE(value_3, 20) AS count_pay FROM users_table ORDER BY 1 OFFSET 20 LIMIT 5
|
||||
) as tar,
|
||||
events_table
|
||||
WHERE foo.avg != bar.cnt_1 AND baz.cnt_2 != events_table.event_type
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- Multiple columns in GROUP BYs
|
||||
-- foo needs to be recursively planned, bar can be pushded down
|
||||
SELECT
|
||||
DISTINCT ON (avg) avg, avg2
|
||||
FROM
|
||||
(
|
||||
SELECT avg(value_3) as avg FROM users_table GROUP BY value_1, value_2
|
||||
) as foo,
|
||||
(
|
||||
SELECT avg(value_3) as avg2 FROM users_table GROUP BY value_1, value_2, user_id
|
||||
) as bar
|
||||
WHERE foo.avg = bar.avg2
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
|
||||
-- HAVING and ORDER BY tests
|
||||
SELECT a.user_id, b.value_2, c.avg
|
||||
FROM (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
(value_1 > 2)
|
||||
GROUP BY
|
||||
user_id
|
||||
HAVING
|
||||
count(distinct value_1) > 2
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as a,
|
||||
(
|
||||
SELECT
|
||||
value_2
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
(value_1 > 2)
|
||||
GROUP BY
|
||||
value_2
|
||||
HAVING
|
||||
count(distinct value_1) > 2
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3
|
||||
) as b,
|
||||
(
|
||||
SELECT
|
||||
avg(user_id) as avg
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
(value_1 > 2)
|
||||
GROUP BY
|
||||
value_2
|
||||
HAVING
|
||||
sum(value_1) > 10
|
||||
ORDER BY (sum(value_3) - avg(value_1) - COALESCE(array_upper(ARRAY[max(user_id)],1) * 5,0)) DESC
|
||||
LIMIT 3
|
||||
) as c
|
||||
|
||||
WHERE b.value_2 != a.user_id
|
||||
ORDER BY 3 DESC, 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
|
||||
-- zero shard subquery joined with a regular one
|
||||
SELECT
|
||||
bar.user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND false AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as bar
|
||||
WHERE foo.user_id > bar.user_id
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- window functions tests, both is recursively planned
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT
|
||||
user_id, time, rnk
|
||||
FROM
|
||||
(
|
||||
SELECT * FROM (
|
||||
SELECT
|
||||
*, rank() OVER my_win as rnk
|
||||
FROM
|
||||
events_table
|
||||
WINDOW my_win AS (PARTITION BY user_id ORDER BY time DESC)
|
||||
ORDER BY rnk DESC
|
||||
) as foo_inner
|
||||
|
||||
LIMIT 4
|
||||
) as foo
|
||||
ORDER BY
|
||||
3 DESC, 1 DESC, 2 DESC
|
||||
) foo,
|
||||
(
|
||||
SELECT
|
||||
user_id, time, rnk
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
*, rank() OVER my_win as rnk
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id = 3
|
||||
WINDOW my_win AS (PARTITION BY event_type ORDER BY time DESC)
|
||||
|
||||
) as foo
|
||||
ORDER BY
|
||||
3 DESC, 1 DESC, 2 DESC
|
||||
) bar WHERE foo.user_id = bar.user_id
|
||||
ORDER BY foo.rnk DESC, foo.time DESC, bar.time LIMIT 5;
|
||||
|
||||
-- cursor test
|
||||
BEGIN;
|
||||
|
||||
DECLARE recursive_subquery CURSOR FOR
|
||||
SELECT
|
||||
event_type, count(distinct value_2)
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id IN (SELECT user_id FROM users_table GROUP BY user_id ORDER BY count(*) DESC LIMIT 20)
|
||||
GROUP BY
|
||||
event_type
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
FETCH 1 FROM recursive_subquery;
|
||||
COMMIT;
|
||||
|
||||
-- cursor test with FETCH ALL
|
||||
BEGIN;
|
||||
|
||||
DECLARE recursive_subquery CURSOR FOR
|
||||
SELECT
|
||||
event_type, count(distinct value_2)
|
||||
FROM
|
||||
events_table
|
||||
WHERE
|
||||
user_id IN (SELECT user_id FROM users_table GROUP BY user_id ORDER BY count(*) DESC LIMIT 20)
|
||||
GROUP BY
|
||||
event_type
|
||||
ORDER BY 1 DESC, 2 DESC
|
||||
LIMIT 3;
|
||||
|
||||
FETCH ALL FROM recursive_subquery;
|
||||
FETCH ALL FROM recursive_subquery;
|
||||
COMMIT;
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_complex CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,113 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality with different executors
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_executor;
|
||||
SET search_path TO subquery_executor, public;
|
||||
|
||||
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
-- subquery with router planner
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
|
||||
-- subquery with router but not logical plannable
|
||||
-- should fail
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT user_id, sum(value_2) over (partition by user_id) AS counter FROM users_table WHERE user_id = 15
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.counter = bar.user_id;
|
||||
|
||||
-- subquery with real-time query
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id != 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
|
||||
|
||||
-- subquery with repartition query
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
|
||||
-- mixed of all executors (including local execution)
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table OFFSET 0
|
||||
) as bar,
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) baz,
|
||||
(
|
||||
SELECT user_id FROM users_table_local WHERE user_id = 2
|
||||
) baw
|
||||
WHERE foo.value_2 = bar.user_id AND baz.value_2 = bar.user_id AND bar.user_id = baw.user_id;
|
||||
|
||||
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
|
||||
|
||||
-- final query is router
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 1 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table WHERE user_id = 2 OFFSET 0
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
|
||||
-- final query is real-time
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 1 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table WHERE user_id != 2
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_executor CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,204 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on local tables
|
||||
-- ===================================================================
|
||||
|
||||
CREATE SCHEMA subquery_local_tables;
|
||||
SET search_path TO subquery_local_tables, public;
|
||||
|
||||
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
-- foo is only on the local tables, thus can be replaced
|
||||
-- bar is on the distributed tables with LIMIT, should be replaced
|
||||
SELECT
|
||||
foo.user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table_local.user_id
|
||||
FROM
|
||||
users_table_local, events_table_local
|
||||
WHERE
|
||||
users_table_local.user_id = events_table_local.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as bar
|
||||
|
||||
WHERE bar.user_id = foo.user_id
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- foo is only on the local tables, thus can be replaced
|
||||
SELECT
|
||||
foo.user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table_local.user_id
|
||||
FROM
|
||||
users_table_local, events_table_local
|
||||
WHERE
|
||||
users_table_local.user_id = events_table_local.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
) as bar
|
||||
WHERE bar.user_id = foo.user_id
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
|
||||
-- subqueries in WHERE could be replaced even if they are on the local tables
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE
|
||||
user_id IN (SELECT DISTINCT value_2 FROM users_table_local WHERE value_1 = 1)
|
||||
ORDER BY 1 LIMIT 5;
|
||||
|
||||
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced if
|
||||
-- it contains onle local tables
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
event_type, user_id
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table_local WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar
|
||||
|
||||
) as baz
|
||||
WHERE baz.user_id = users_table.user_id
|
||||
|
||||
) as sub1
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3;
|
||||
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE -> WHERE should be replaced if
|
||||
-- it contains onle local tables
|
||||
-- Later the upper level query is also recursively planned due to LIMIT
|
||||
SELECT user_id, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT user_id, array_agg(event ORDER BY time) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
u.user_id, e.event_type::text AS event, e.time
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table_local WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
LIMIT 5
|
||||
)
|
||||
) t
|
||||
GROUP BY user_id
|
||||
) q
|
||||
ORDER BY 2 DESC, 1;
|
||||
|
||||
|
||||
|
||||
-- subquery (i.e., subquery_2) in WHERE->FROM should be replaced due to local tables
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE
|
||||
user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM (
|
||||
SELECT
|
||||
subquery_1.user_id, count_pay
|
||||
FROM
|
||||
(
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>1' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 1 AND events_table.event_type < 3
|
||||
)
|
||||
UNION
|
||||
(SELECT
|
||||
users_table.user_id,
|
||||
'action=>2' AS event,
|
||||
events_table.time
|
||||
FROM
|
||||
users_table,
|
||||
events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
users_table.user_id >= 1 AND
|
||||
users_table.user_id <= 3 AND
|
||||
events_table.event_type > 2 AND events_table.event_type < 4
|
||||
)
|
||||
) AS subquery_1
|
||||
LEFT JOIN
|
||||
(SELECT
|
||||
user_id,
|
||||
COUNT(*) AS count_pay
|
||||
FROM
|
||||
users_table_local
|
||||
WHERE
|
||||
user_id >= 1 AND
|
||||
user_id <= 3 AND
|
||||
users_table_local.value_1 > 3 AND users_table_local.value_1 < 5
|
||||
GROUP BY
|
||||
user_id
|
||||
HAVING
|
||||
COUNT(*) > 1
|
||||
LIMIT 10
|
||||
|
||||
) AS subquery_2
|
||||
ON
|
||||
subquery_1.user_id = subquery_2.user_id
|
||||
GROUP BY
|
||||
subquery_1.user_id,
|
||||
count_pay) AS subquery_top
|
||||
GROUP BY
|
||||
count_pay, user_id
|
||||
)
|
||||
GROUP BY user_id
|
||||
HAVING count(*) > 1 AND sum(value_2) > 29
|
||||
ORDER BY 1;
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_local_tables CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,213 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on partitioned tables
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_and_partitioning;
|
||||
SET search_path TO subquery_and_partitioning, public;
|
||||
|
||||
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
|
||||
CREATE TABLE partitioning_test(id int, value_1 int, time date) PARTITION BY RANGE (time);
|
||||
|
||||
-- create its partitions
|
||||
CREATE TABLE partitioning_test_2017 PARTITION OF partitioning_test FOR VALUES FROM ('2017-01-01') TO ('2018-01-01');
|
||||
CREATE TABLE partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01');
|
||||
|
||||
-- load some data and distribute tables
|
||||
INSERT INTO partitioning_test VALUES (1, 1, '2017-11-23');
|
||||
INSERT INTO partitioning_test VALUES (2, 1, '2010-07-07');
|
||||
|
||||
INSERT INTO partitioning_test_2017 VALUES (3, 3, '2017-11-22');
|
||||
INSERT INTO partitioning_test_2010 VALUES (4, 4, '2010-03-03');
|
||||
|
||||
-- distribute partitioned table
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SELECT create_distributed_table('partitioning_test', 'id');
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
-- subplan for partitioned tables
|
||||
SELECT
|
||||
id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- final query is router on partitioned tables
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
LIMIT 5
|
||||
) as bar
|
||||
WHERE foo.id = date_part('day', bar.time)
|
||||
ORDER BY 2 DESC, 1;
|
||||
|
||||
-- final query is real-time
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar
|
||||
WHERE date_part('day', foo.time) = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
|
||||
-- final query is real-time that is joined with partitioned table
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT partitioning_test.time
|
||||
FROM
|
||||
partitioning_test
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5
|
||||
) as foo,
|
||||
(
|
||||
SELECT
|
||||
DISTINCT partitioning_test.id
|
||||
FROM
|
||||
partitioning_test
|
||||
) as bar,
|
||||
partitioning_test
|
||||
WHERE date_part('day', foo.time) = bar.id AND partitioning_test.id = bar.id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
|
||||
-- subquery in WHERE clause
|
||||
SELECT DISTINCT id
|
||||
FROM partitioning_test
|
||||
WHERE
|
||||
id IN (SELECT DISTINCT date_part('day', time) FROM partitioning_test);
|
||||
|
||||
-- repartition subquery
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT p1.value_1 FROM partitioning_test as p1, partitioning_test as p2 WHERE p1.id = p2.value_1
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_1 = bar.user_id;
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
|
||||
|
||||
-- subquery, cte, view and non-partitioned tables
|
||||
CREATE VIEW subquery_and_ctes AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_1 FROM partitioning_test OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_1 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT events_table.user_id
|
||||
FROM
|
||||
partitioning_test, events_table
|
||||
WHERE
|
||||
events_table.user_id = partitioning_test.id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2;
|
||||
|
||||
SELECT * FROM subquery_and_ctes
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
|
||||
-- deep subquery, partitioned and non-partitioned tables together
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(partitioning_test.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_1 FROM partitioning_test GROUP BY value_1) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, partitioning_test
|
||||
WHERE
|
||||
level_5.avg_ev_type = partitioning_test.id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_and_partitioning CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,89 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on prepared statements
|
||||
-- ===================================================================
|
||||
CREATE SCHEMA subquery_prepared_statements;
|
||||
SET search_path TO subquery_prepared_statements, public;
|
||||
|
||||
CREATE TYPE xy AS (x int, y int);
|
||||
|
||||
SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements');
|
||||
SELECT run_command_on_workers('CREATE TYPE subquery_prepared_statements.xy AS (x int, y int)');
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
PREPARE subquery_prepare_without_param AS
|
||||
SELECT
|
||||
DISTINCT values_of_subquery
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT (users_table.user_id, events_table.event_type)::xy as values_of_subquery
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
PREPARE subquery_prepare_param_on_partkey(int) AS
|
||||
SELECT
|
||||
DISTINCT values_of_subquery
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT (users_table.user_id, events_table.event_type)::xy as values_of_subquery
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
(users_table.user_id = $1 OR users_table.user_id = 2) AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
PREPARE subquery_prepare_param_non_partkey(int) AS
|
||||
SELECT
|
||||
DISTINCT values_of_subquery
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT (users_table.user_id, events_table.event_type)::xy as values_of_subquery
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type = $1
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
-- execute each test with 6 times
|
||||
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
EXECUTE subquery_prepare_without_param;
|
||||
|
||||
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
EXECUTE subquery_prepare_param_on_partkey(1);
|
||||
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
EXECUTE subquery_prepare_param_non_partkey(1);
|
||||
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_prepared_statements CASCADE;
|
||||
SET search_path TO public;
|
|
@ -0,0 +1,427 @@
|
|||
-- ===================================================================
|
||||
-- test recursive planning functionality on views
|
||||
-- ===================================================================
|
||||
|
||||
CREATE SCHEMA subquery_view;
|
||||
SET search_path TO subquery_view, public;
|
||||
|
||||
|
||||
CREATE TABLE users_table_local AS SELECT * FROM users_table;
|
||||
CREATE TABLE events_table_local AS SELECT * FROM events_table;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
|
||||
CREATE VIEW view_without_subquery AS
|
||||
SELECT
|
||||
DISTINCT users_table.value_1
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
view_without_subquery
|
||||
ORDER BY 1 DESC LIMIT 5;
|
||||
|
||||
CREATE VIEW view_without_subquery_second AS
|
||||
SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 5;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
view_without_subquery_second
|
||||
ORDER BY 1;
|
||||
|
||||
-- subqueries in FROM clause with LIMIT should be recursively planned
|
||||
CREATE VIEW subquery_limit AS
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
SELECT * FROM subquery_limit ORDER BY 1 DESC;
|
||||
|
||||
-- subqueries in FROM clause with GROUP BY non-distribution column should be recursively planned
|
||||
CREATE VIEW subquery_non_p_key_group_by AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.value_1
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
SELECT * FROM subquery_non_p_key_group_by ORDER BY 1 DESC;
|
||||
|
||||
|
||||
|
||||
CREATE VIEW final_query_router AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.value_3
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
GROUP BY users_table.value_3
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.value_3
|
||||
ORDER BY 2 DESC, 1;
|
||||
|
||||
SELECT * FROM final_query_router ORDER BY 1;
|
||||
|
||||
CREATE VIEW final_query_realtime AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
users_table.value_2
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
GROUP BY users_table.value_2
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
(SELECT
|
||||
users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (5,6,7,8)
|
||||
ORDER BY 1 DESC
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id
|
||||
ORDER BY 2 DESC, 1 DESC
|
||||
LIMIT 3;
|
||||
|
||||
SELECT
|
||||
DISTINCT ON (users_table.value_2) users_table.value_2, time, value_3
|
||||
FROM
|
||||
final_query_realtime, users_table
|
||||
WHERE
|
||||
users_table.user_id = final_query_realtime.user_id
|
||||
ORDER BY 1 DESC, 2 DESC, 3 DESC
|
||||
LIMIT 3;
|
||||
|
||||
|
||||
CREATE VIEW subquery_in_where AS
|
||||
SELECT DISTINCT user_id
|
||||
FROM users_table
|
||||
WHERE
|
||||
user_id IN (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5);
|
||||
|
||||
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
subquery_in_where
|
||||
ORDER BY 1 DESC;
|
||||
|
||||
|
||||
-- subquery in FROM -> FROM -> WHERE should be replaced due to LIMIT
|
||||
CREATE VIEW subquery_from_from_where AS
|
||||
SELECT user_id, array_length(events_table, 1)
|
||||
FROM (
|
||||
SELECT user_id, array_agg(event ORDER BY time) AS events_table
|
||||
FROM (
|
||||
SELECT
|
||||
u.user_id, e.event_type::text AS event, e.time
|
||||
FROM
|
||||
users_table AS u,
|
||||
events_table AS e
|
||||
WHERE u.user_id = e.user_id AND
|
||||
u.user_id IN
|
||||
(
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
users_table
|
||||
WHERE value_2 >= 5
|
||||
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 1 AND event_type <= 3 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 3 AND event_type <= 4 AND value_3 > 1 AND user_id = users_table.user_id)
|
||||
LIMIT 5
|
||||
)
|
||||
) t
|
||||
GROUP BY user_id
|
||||
) q;
|
||||
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
subquery_from_from_where
|
||||
ORDER BY
|
||||
2 DESC, 1;
|
||||
|
||||
|
||||
-- subquery in FROM -> FROM -> FROM should be replaced if
|
||||
-- it contains onle local tables
|
||||
CREATE VIEW subquery_from_from_where_local_table AS
|
||||
SELECT
|
||||
DISTINCT user_id
|
||||
FROM
|
||||
(
|
||||
SELECT users_table.user_id FROM users_table,
|
||||
(
|
||||
SELECT
|
||||
event_type, user_id
|
||||
FROM
|
||||
(SELECT event_type, users_table.user_id FROM users_table,
|
||||
(SELECT user_id, event_type FROM events_table_local WHERE value_2 < 3 OFFSET 3) as foo
|
||||
WHERE foo.user_id = users_table.user_id
|
||||
) bar
|
||||
|
||||
) as baz
|
||||
WHERE baz.user_id = users_table.user_id
|
||||
|
||||
) as sub1;
|
||||
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
subquery_from_from_where
|
||||
ORDER BY 1 DESC
|
||||
LIMIT 3;
|
||||
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
|
||||
CREATE VIEW repartition_view AS
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table
|
||||
) as bar
|
||||
WHERE foo.value_2 = bar.user_id;
|
||||
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
repartition_view;
|
||||
|
||||
CREATE VIEW all_executors_view AS
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT value_2 FROM users_table WHERE user_id = 15 OFFSET 0
|
||||
) as foo,
|
||||
(
|
||||
SELECT user_id FROM users_table OFFSET 0
|
||||
) as bar,
|
||||
(
|
||||
SELECT DISTINCT users_table.value_2 FROM users_table, events_table WHERE users_table.user_id = events_table.value_2 AND users_table.user_id < 2
|
||||
) baz,
|
||||
(
|
||||
SELECT user_id FROM users_table_local WHERE user_id = 2
|
||||
) baw
|
||||
WHERE foo.value_2 = bar.user_id AND baz.value_2 = bar.user_id AND bar.user_id = baw.user_id;
|
||||
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
all_executors_view;
|
||||
|
||||
SET citus.enable_repartition_joins to OFF;
|
||||
|
||||
|
||||
-- the same query, but this time the CTEs also live inside a subquery
|
||||
CREATE VIEW subquery_and_ctes AS
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(
|
||||
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
events_table,
|
||||
(SELECT DISTINCT value_2 FROM users_table OFFSET 0) as foo
|
||||
WHERE
|
||||
events_table.user_id = foo.value_2 AND
|
||||
events_table.user_id IN (SELECT DISTINCT value_1 FROM users_table ORDER BY 1 LIMIT 3)
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT
|
||||
count(*) as cnt
|
||||
FROM
|
||||
cte,
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
WHERE foo.user_id = cte.user_id
|
||||
|
||||
) as foo, users_table WHERE foo.cnt > users_table.value_2;
|
||||
|
||||
SELECT * FROM subquery_and_ctes
|
||||
ORDER BY 3 DESC, 1 DESC, 2 DESC, 4 DESC
|
||||
LIMIT 5;
|
||||
|
||||
|
||||
CREATE VIEW subquery_and_ctes_second AS
|
||||
SELECT time, event_type, value_2, value_3 FROM
|
||||
(
|
||||
WITH cte AS (
|
||||
WITH local_cte AS (
|
||||
SELECT * FROM users_table_local
|
||||
),
|
||||
dist_cte AS (
|
||||
SELECT user_id FROM events_table
|
||||
)
|
||||
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
|
||||
)
|
||||
SELECT DISTINCT cte.user_id
|
||||
FROM users_table, cte
|
||||
WHERE
|
||||
users_table.user_id = cte.user_id AND
|
||||
users_table.user_id IN
|
||||
(WITH cte_in_where AS (SELECT DISTINCT value_2 FROM users_table WHERE value_1 >= 1 AND value_1 <= 20 ORDER BY 1 LIMIT 5) SELECT * FROM cte_in_where)
|
||||
ORDER BY 1 DESC
|
||||
) as foo,
|
||||
events_table
|
||||
WHERE
|
||||
foo.user_id = events_table.value_2;
|
||||
|
||||
|
||||
SELECT * FROM subquery_and_ctes_second
|
||||
ORDER BY 3 DESC, 2 DESC, 1 DESC
|
||||
LIMIT 5;
|
||||
|
||||
CREATE VIEW deep_subquery AS
|
||||
SELECT count(*)
|
||||
FROM
|
||||
(
|
||||
SELECT avg(min) FROM
|
||||
(
|
||||
SELECT min(users_table.value_1) FROM
|
||||
(
|
||||
SELECT avg(event_type) as avg_ev_type FROM
|
||||
(
|
||||
SELECT
|
||||
max(value_1) as mx_val_1
|
||||
FROM (
|
||||
SELECT
|
||||
avg(event_type) as avg
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
cnt
|
||||
FROM
|
||||
(SELECT count(*) as cnt, value_2 FROM users_table GROUP BY value_2) as level_1, users_table
|
||||
WHERE
|
||||
users_table.user_id = level_1.cnt
|
||||
) as level_2, events_table
|
||||
WHERE events_table.user_id = level_2.cnt
|
||||
GROUP BY level_2.cnt
|
||||
) as level_3, users_table
|
||||
WHERE user_id = level_3.avg
|
||||
GROUP BY level_3.avg
|
||||
) as level_4, events_table
|
||||
WHERE level_4.mx_val_1 = events_table.user_id
|
||||
GROUP BY level_4.mx_val_1
|
||||
) as level_5, users_table
|
||||
WHERE
|
||||
level_5.avg_ev_type = users_table.user_id
|
||||
GROUP BY
|
||||
level_5.avg_ev_type
|
||||
) as level_6, users_table WHERE users_table.user_id = level_6.min
|
||||
GROUP BY users_table.value_1
|
||||
) as bar;
|
||||
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
deep_subquery;
|
||||
|
||||
|
||||
CREATE VIEW result_of_view_is_also_recursively_planned AS
|
||||
SELECT
|
||||
user_id
|
||||
FROM
|
||||
(SELECT
|
||||
DISTINCT users_table.user_id
|
||||
FROM
|
||||
users_table, events_table
|
||||
WHERE
|
||||
users_table.user_id = events_table.user_id AND
|
||||
event_type IN (1,2,3,4)
|
||||
ORDER BY 1 DESC LIMIT 5
|
||||
) as foo
|
||||
ORDER BY 1 DESC;
|
||||
SELECT
|
||||
*
|
||||
FROM
|
||||
(SELECT
|
||||
*
|
||||
FROM
|
||||
result_of_view_is_also_recursively_planned, events_table
|
||||
WHERE
|
||||
events_table.value_2 = result_of_view_is_also_recursively_planned.user_id
|
||||
ORDER BY time DESC
|
||||
LIMIT 5
|
||||
OFFSET 4
|
||||
|
||||
) as foo
|
||||
ORDER BY time DESC LIMIT 5;
|
||||
|
||||
SET client_min_messages TO DEFAULT;
|
||||
|
||||
DROP SCHEMA subquery_view CASCADE;
|
||||
SET search_path TO public;
|
|
@ -54,13 +54,13 @@ ORDER BY
|
|||
LIMIT
|
||||
5;
|
||||
|
||||
-- CTE in subquery errors out
|
||||
-- CTE in subquery recursively planned
|
||||
SELECT user_id FROM (
|
||||
WITH cte AS (
|
||||
SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 2 LIMIT 5
|
||||
)
|
||||
SELECT user_id FROM cte WHERE value_2 > 0
|
||||
) a;
|
||||
) a ORDER BY 1 LIMIT 3;
|
||||
|
||||
-- CTE outside of FROM/WHERE errors out
|
||||
WITH cte AS (
|
||||
|
@ -458,14 +458,14 @@ WITH cte_user AS (
|
|||
)
|
||||
SELECT user_id, sum(value_2) FROM cte_user GROUP BY 1 ORDER BY 1, 2;
|
||||
|
||||
SELECT * FROM cte_view;
|
||||
SELECT * FROM cte_view ORDER BY 1, 2 LIMIT 5;
|
||||
|
||||
|
||||
WITH cte_user_with_view AS
|
||||
(
|
||||
SELECT * FROM cte_view WHERE user_id < 3
|
||||
)
|
||||
SELECT user_id, value_1 FROM cte_user_with_view ORDER BY 1, 2 LIMIT 10 OFFSET 3;
|
||||
SELECT user_id, value_1 FROM cte_user_with_view ORDER BY 1, 2 LIMIT 10 OFFSET 2;
|
||||
|
||||
DROP VIEW basic_view;
|
||||
DROP VIEW cte_view;
|
||||
|
|
|
@ -66,6 +66,7 @@ ORDER BY
|
|||
|
||||
-- Subqueries in WHERE and FROM are mixed
|
||||
-- In this query, only subquery in WHERE is not a colocated join
|
||||
-- but we're able to recursively plan that as well
|
||||
WITH users_events AS (
|
||||
WITH colocated_join AS (
|
||||
SELECT
|
||||
|
@ -117,13 +118,13 @@ WITH users_events AS (
|
|||
)
|
||||
)
|
||||
SELECT
|
||||
*
|
||||
DISTINCT uid
|
||||
FROM
|
||||
users_events
|
||||
ORDER BY
|
||||
1, 2
|
||||
1 DESC
|
||||
LIMIT
|
||||
20;
|
||||
5;
|
||||
|
||||
-- cte LEFT JOIN distributed_table should error out
|
||||
WITH cte AS (
|
||||
|
|
|
@ -110,6 +110,10 @@ WHERE
|
|||
);
|
||||
|
||||
-- CTE in WHERE basic
|
||||
-- this is a tricky query that hits an aggresive
|
||||
-- check in subquery puwhdown after the recursive planning
|
||||
-- where LIMIT should be allowed
|
||||
-- if the query contains only intermediate results
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
|
@ -131,6 +135,10 @@ IN
|
|||
);
|
||||
|
||||
-- CTE with non-colocated join in WHERE
|
||||
-- this is a tricky query that hits an aggresive
|
||||
-- check in subquery puwhdown after the recursive planning
|
||||
-- where LIMIT should be allowed
|
||||
-- if the query contains only intermediate results
|
||||
SELECT
|
||||
count(*)
|
||||
FROM
|
||||
|
|
Loading…
Reference in New Issue