Subqueries in where -- updated (#1372)

* Support for subqueries in WHERE clause

This commit enables subqueries in WHERE clause to be pushed down
by the subquery pushdown logic.

The support covers:
  - Correlated subqueries with IN, NOT IN, EXISTS, NOT EXISTS,
    operator expressions such as (>, <, =, ALL, ANY etc.)
  - Non-correlated subqueries with (partition_key) IN (SELECT partition_key ..)
    (partition_key) =ANY (SELECT partition_key ...)

Note that this commit heavily utilizes the attribute equivalence logic introduced
in the 1cb6a34ba8. In general, this commit mostly
adjusts the logical planner not to error out on the subqueries in WHERE clause.

* Improve error checks for subquery pushdown and INSERT ... SELECT

Since we allow subqueries in WHERE clause with the previous commit,
we should apply the same limitations to those subqueries.

With this commit, we do not iterate on each subquery one by one.
Instead, we extract all the subqueries and apply the checks directly
on those subqueries. The aim of this change is to (i) Simplify the
code (ii) Make it close to the checks on INSERT .. SELECT code base.

* Extend checks for unresolved paramaters to include SubLinks

With the presence of subqueries in where clause (i.e., SubPlans on the
query) the existing way for checking unresolved parameters fail. The
reason is that the parameters for SubPlans are kept on the parent plan not
on the query itself (see primnodes.h for the details).

With this commit, instead of checking SubPlans on the modified plans
we start to use originalQuery, where SubLinks represent the subqueries
in where clause. The unresolved parameters can be found on the SubLinks.

* Apply code-review feedback

* Remove unnecessary copying of shard interval list

This commit removes unnecessary copying of shard interval list. Note
that there are no copyObject function implemented for shard intervals.
pull/1376/head
Önder Kalacı 2017-05-01 17:20:21 +03:00 committed by GitHub
parent 8dab40da69
commit b74ed3c8e1
26 changed files with 1617 additions and 302 deletions

View File

@ -2978,12 +2978,12 @@ FindReferencedTableColumn(Expr *columnExpression, List *parentQueryList, Query *
/* /*
* ExtractQueryWalker walks over a query, and finds all queries in the query * ExtractQueryWalker walks over a query, and finds all queries in the query
* tree and returns these queries. * tree and returns these queries. Note that the function also recurses into
* the subqueries in WHERE clause.
*/ */
bool bool
ExtractQueryWalker(Node *node, List **queryList) ExtractQueryWalker(Node *node, List **queryList)
{ {
bool walkerResult = false;
if (node == NULL) if (node == NULL)
{ {
return false; return false;
@ -2994,11 +2994,10 @@ ExtractQueryWalker(Node *node, List **queryList)
Query *query = (Query *) node; Query *query = (Query *) node;
(*queryList) = lappend(*queryList, query); (*queryList) = lappend(*queryList, query);
walkerResult = query_tree_walker(query, ExtractQueryWalker, queryList, return query_tree_walker(query, ExtractQueryWalker, queryList, 0);
QTW_EXAMINE_RTES);
} }
return walkerResult; return expression_tree_walker(node, ExtractQueryWalker, queryList);
} }

View File

@ -93,6 +93,7 @@ static bool HasOuterJoinWalker(Node *node, void *maxJoinLevel);
static bool HasComplexJoinOrder(Query *queryTree); static bool HasComplexJoinOrder(Query *queryTree);
static bool HasComplexRangeTableType(Query *queryTree); static bool HasComplexRangeTableType(Query *queryTree);
static void ValidateClauseList(List *clauseList); static void ValidateClauseList(List *clauseList);
static void ValidateSubqueryPushdownClauseList(List *clauseList);
static bool ExtractFromExpressionWalker(Node *node, static bool ExtractFromExpressionWalker(Node *node,
QualifierWalkerContext *walkerContext); QualifierWalkerContext *walkerContext);
static List * MultiTableNodeList(List *tableEntryList, List *rangeTableList); static List * MultiTableNodeList(List *tableEntryList, List *rangeTableList);
@ -102,6 +103,7 @@ static MultiNode * MultiJoinTree(List *joinOrderList, List *collectTableList,
static MultiCollect * CollectNodeForTable(List *collectTableList, uint32 rangeTableId); static MultiCollect * CollectNodeForTable(List *collectTableList, uint32 rangeTableId);
static MultiSelect * MultiSelectNode(List *whereClauseList); static MultiSelect * MultiSelectNode(List *whereClauseList);
static bool IsSelectClause(Node *clause); static bool IsSelectClause(Node *clause);
static bool IsSublinkClause(Node *clause);
static MultiProject * MultiProjectNode(List *targetEntryList); static MultiProject * MultiProjectNode(List *targetEntryList);
static MultiExtendedOp * MultiExtendedOpNode(Query *queryTree); static MultiExtendedOp * MultiExtendedOpNode(Query *queryTree);
@ -134,6 +136,8 @@ static MultiNode * MultiSubqueryPlanTree(Query *originalQuery,
Query *queryTree, Query *queryTree,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext); plannerRestrictionContext);
static List * SublinkList(Query *originalQuery);
static bool ExtractSublinkWalker(Node *node, List **sublinkList);
static MultiNode * SubqueryPushdownMultiPlanTree(Query *queryTree); static MultiNode * SubqueryPushdownMultiPlanTree(Query *queryTree);
static List * CreateSubqueryTargetEntryList(List *columnList); static List * CreateSubqueryTargetEntryList(List *columnList);
@ -159,15 +163,18 @@ MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree,
{ {
MultiNode *multiQueryNode = NULL; MultiNode *multiQueryNode = NULL;
MultiTreeRoot *rootNode = NULL; MultiTreeRoot *rootNode = NULL;
List *subqueryEntryList = NULL;
/* /*
* We check the existence of subqueries in the modified query given that * We check the existence of subqueries in FROM clause on the modified query
* if postgres already flattened the subqueries, MultiPlanTree() can plan * given that if postgres already flattened the subqueries, MultiPlanTree()
* corresponding distributed plan. * can plan corresponding distributed plan.
*
* We also check the existence of subqueries in WHERE clause. Note that
* this check needs to be done on the original query given that
* standard_planner() may replace the sublinks with anti/semi joins and
* MultiPlanTree() cannot plan such queries.
*/ */
subqueryEntryList = SubqueryEntryList(queryTree); if (SubqueryEntryList(queryTree) != NIL || SublinkList(originalQuery) != NIL)
if (subqueryEntryList != NIL)
{ {
multiQueryNode = MultiSubqueryPlanTree(originalQuery, queryTree, multiQueryNode = MultiSubqueryPlanTree(originalQuery, queryTree,
plannerRestrictionContext); plannerRestrictionContext);
@ -185,6 +192,57 @@ MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree,
} }
/*
* SublinkList finds the subquery nodes in the where clause of the given query. Note
* that the function should be called on the original query given that postgres
* standard_planner() may convert the subqueries in WHERE clause to joins.
*/
static List *
SublinkList(Query *originalQuery)
{
FromExpr *joinTree = originalQuery->jointree;
Node *queryQuals = NULL;
List *sublinkList = NIL;
if (!joinTree)
{
return NIL;
}
queryQuals = joinTree->quals;
ExtractSublinkWalker(queryQuals, &sublinkList);
return sublinkList;
}
/*
* ExtractSublinkWalker walks over a quals node, and finds all sublinks
* in that node.
*/
static bool
ExtractSublinkWalker(Node *node, List **sublinkList)
{
bool walkerResult = false;
if (node == NULL)
{
return false;
}
if (IsA(node, SubLink))
{
(*sublinkList) = lappend(*sublinkList, node);
}
else
{
walkerResult = expression_tree_walker(node, ExtractSublinkWalker,
sublinkList);
}
return walkerResult;
}
/* /*
* MultiSubqueryPlanTree gets the query objects and returns logical plan * MultiSubqueryPlanTree gets the query objects and returns logical plan
* for subqueries. * for subqueries.
@ -335,9 +393,9 @@ DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext) plannerRestrictionContext)
{ {
ListCell *rangeTableEntryCell = NULL;
List *subqueryEntryList = NIL;
bool outerMostQueryHasLimit = false; bool outerMostQueryHasLimit = false;
ListCell *subqueryCell = NULL;
List *subqueryList = NIL;
DeferredErrorMessage *error = NULL; DeferredErrorMessage *error = NULL;
RelationRestrictionContext *relationRestrictionContext = RelationRestrictionContext *relationRestrictionContext =
plannerRestrictionContext->relationRestrictionContext; plannerRestrictionContext->relationRestrictionContext;
@ -376,13 +434,21 @@ DeferErrorIfUnsupportedSubqueryPushdown(Query *originalQuery,
"equality operator.", NULL); "equality operator.", NULL);
} }
subqueryEntryList = SubqueryEntryList(originalQuery); /*
foreach(rangeTableEntryCell, subqueryEntryList) * We first extract all the queries that appear in the original query. Later,
{ * we delete the original query given that error rules does not apply to the
RangeTblEntry *rangeTableEntry = lfirst(rangeTableEntryCell); * top level query. For instance, we could support any LIMIT/ORDER BY on the
Query *subquery = rangeTableEntry->subquery; * top level query.
*/
ExtractQueryWalker((Node *) originalQuery, &subqueryList);
subqueryList = list_delete(subqueryList, originalQuery);
error = DeferErrorIfCannotPushdownSubquery(subquery, outerMostQueryHasLimit); /* iterate on the subquery list and error out accordingly */
foreach(subqueryCell, subqueryList)
{
Query *subquery = lfirst(subqueryCell);
error = DeferErrorIfCannotPushdownSubquery(subquery,
outerMostQueryHasLimit);
if (error) if (error)
{ {
return error; return error;
@ -411,6 +477,18 @@ DeferErrorIfUnsupportedFilters(Query *subquery)
ListCell *queryCell = NULL; ListCell *queryCell = NULL;
List *subqueryOpExpressionList = NIL; List *subqueryOpExpressionList = NIL;
List *relationIdList = RelationIdList(subquery); List *relationIdList = RelationIdList(subquery);
Var *partitionColumn = NULL;
Oid relationId = InvalidOid;
/*
* If there are no appropriate relations, we're going to error out on
* DeferErrorIfCannotPushdownSubquery(). It may happen once the subquery
* does not include a relation.
*/
if (relationIdList == NIL)
{
return NULL;
}
/* /*
* Get relation id of any relation in the subquery and create partiton column * Get relation id of any relation in the subquery and create partiton column
@ -418,8 +496,8 @@ DeferErrorIfUnsupportedFilters(Query *subquery)
* expressions on different tables. Then we compare these operator expressions * expressions on different tables. Then we compare these operator expressions
* to see if they consist of same operator and constant value. * to see if they consist of same operator and constant value.
*/ */
Oid relationId = linitial_oid(relationIdList); relationId = linitial_oid(relationIdList);
Var *partitionColumn = PartitionColumn(relationId, 0); partitionColumn = PartitionColumn(relationId, 0);
ExtractQueryWalker((Node *) subquery, &queryList); ExtractQueryWalker((Node *) subquery, &queryList);
foreach(queryCell, queryList) foreach(queryCell, queryList)
@ -511,13 +589,11 @@ EqualOpExpressionLists(List *firstOpExpressionList, List *secondOpExpressionList
/* /*
* DeferErrorIfCannotPushdownSubquery recursively checks if we can push down the given * DeferErrorIfCannotPushdownSubquery checks if we can push down the given
* subquery to worker nodes. If we cannot push down the subquery, this function * subquery to worker nodes. If we cannot push down the subquery, this function
* returns a deferred error. * returns a deferred error.
* *
* We can push down a subquery if it follows rules below. We support nested queries * We can push down a subquery if it follows rules below:
* as long as they follow the same rules, and we recurse to validate each subquery
* for this given query.
* a. If there is an aggregate, it must be grouped on partition column. * a. If there is an aggregate, it must be grouped on partition column.
* b. If there is a join, it must be between two regular tables or two subqueries. * b. If there is a join, it must be between two regular tables or two subqueries.
* We don't support join between a regular table and a subquery. And columns on * We don't support join between a regular table and a subquery. And columns on
@ -539,8 +615,6 @@ DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerMostQueryHasLi
{ {
bool preconditionsSatisfied = true; bool preconditionsSatisfied = true;
char *errorDetail = NULL; char *errorDetail = NULL;
List *subqueryEntryList = NIL;
ListCell *rangeTableEntryCell = NULL;
DeferredErrorMessage *deferredError = NULL; DeferredErrorMessage *deferredError = NULL;
deferredError = DeferErrorIfUnsupportedTableCombination(subqueryTree); deferredError = DeferErrorIfUnsupportedTableCombination(subqueryTree);
@ -549,12 +623,6 @@ DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerMostQueryHasLi
return deferredError; return deferredError;
} }
if (subqueryTree->hasSubLinks)
{
preconditionsSatisfied = false;
errorDetail = "Subqueries other than from-clause subqueries are unsupported";
}
if (subqueryTree->rtable == NIL) if (subqueryTree->rtable == NIL)
{ {
preconditionsSatisfied = false; preconditionsSatisfied = false;
@ -676,41 +744,20 @@ DeferErrorIfCannotPushdownSubquery(Query *subqueryTree, bool outerMostQueryHasLi
errorDetail, NULL); errorDetail, NULL);
} }
/* recursively do same check for subqueries of this query */
subqueryEntryList = SubqueryEntryList(subqueryTree);
foreach(rangeTableEntryCell, subqueryEntryList)
{
RangeTblEntry *rangeTableEntry =
(RangeTblEntry *) lfirst(rangeTableEntryCell);
Query *innerSubquery = rangeTableEntry->subquery;
deferredError = DeferErrorIfCannotPushdownSubquery(innerSubquery,
outerMostQueryHasLimit);
if (deferredError)
{
return deferredError;
}
}
return NULL; return NULL;
} }
/* /*
* DeferErrorIfUnsupportedUnionQuery is a helper function for ErrorIfCannotPushdownSubquery(). * DeferErrorIfUnsupportedUnionQuery is a helper function for ErrorIfCannotPushdownSubquery().
* It basically iterates over the subqueries that reside under the given set operations.
*
* The function also errors out for set operations INTERSECT and EXCEPT. * The function also errors out for set operations INTERSECT and EXCEPT.
*/ */
static DeferredErrorMessage * static DeferredErrorMessage *
DeferErrorIfUnsupportedUnionQuery(Query *subqueryTree, DeferErrorIfUnsupportedUnionQuery(Query *subqueryTree,
bool outerMostQueryHasLimit) bool outerMostQueryHasLimit)
{ {
List *rangeTableIndexList = NIL;
ListCell *rangeTableIndexCell = NULL;
List *setOperationStatementList = NIL; List *setOperationStatementList = NIL;
ListCell *setOperationStatmentCell = NULL; ListCell *setOperationStatmentCell = NULL;
List *rangeTableList = subqueryTree->rtable;
ExtractSetOperationStatmentWalker((Node *) subqueryTree->setOperations, ExtractSetOperationStatmentWalker((Node *) subqueryTree->setOperations,
&setOperationStatementList); &setOperationStatementList);
@ -727,24 +774,6 @@ DeferErrorIfUnsupportedUnionQuery(Query *subqueryTree,
} }
} }
ExtractRangeTableIndexWalker((Node *) subqueryTree->setOperations,
&rangeTableIndexList);
foreach(rangeTableIndexCell, rangeTableIndexList)
{
int rangeTableIndex = lfirst_int(rangeTableIndexCell);
RangeTblEntry *rangeTableEntry = rt_fetch(rangeTableIndex, rangeTableList);
DeferredErrorMessage *deferredError = NULL;
Assert(rangeTableEntry->rtekind == RTE_SUBQUERY);
deferredError = DeferErrorIfCannotPushdownSubquery(rangeTableEntry->subquery,
outerMostQueryHasLimit);
if (deferredError)
{
return deferredError;
}
}
return NULL; return NULL;
} }
@ -1248,11 +1277,15 @@ ErrorIfQueryNotSupported(Query *queryTree)
const char *filterHint = "Consider using an equality filter on the distributed " const char *filterHint = "Consider using an equality filter on the distributed "
"table's partition column."; "table's partition column.";
if (queryTree->hasSubLinks) /*
* There could be Sublinks in the target list as well. To produce better
* error messages we're checking sublinks in the where clause.
*/
if (queryTree->hasSubLinks && SublinkList(queryTree) == NIL)
{ {
preconditionsSatisfied = false; preconditionsSatisfied = false;
errorMessage = "could not run distributed query with subquery outside the " errorMessage = "could not run distributed query with subquery outside the "
"FROM clause"; "FROM and WHERE clauses";
errorHint = filterHint; errorHint = filterHint;
} }
@ -1506,6 +1539,12 @@ DeferErrorIfUnsupportedSubqueryRepartition(Query *subqueryTree)
errorDetail = "Subqueries with offset are not supported yet"; errorDetail = "Subqueries with offset are not supported yet";
} }
if (subqueryTree->hasSubLinks)
{
preconditionsSatisfied = false;
errorDetail = "Subqueries other than from-clause subqueries are unsupported";
}
/* finally check and return error if conditions are not satisfied */ /* finally check and return error if conditions are not satisfied */
if (!preconditionsSatisfied) if (!preconditionsSatisfied)
{ {
@ -1752,11 +1791,40 @@ ValidateClauseList(List *clauseList)
{ {
Node *clause = (Node *) lfirst(clauseCell); Node *clause = (Node *) lfirst(clauseCell);
bool selectClause = IsSelectClause(clause); /*
bool joinClause = IsJoinClause(clause); * There could never be sublinks here given that it is handled
bool orClause = or_clause(clause); * in subquery pushdown code-path.
*/
Assert(!IsSublinkClause(clause));
if (!(selectClause || joinClause || orClause)) if (!(IsSelectClause(clause) || IsJoinClause(clause) || or_clause(clause)))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("unsupported clause type")));
}
}
}
/*
* ValidateSubqueryPushdownClauseList walks over the given list of clauses,
* and checks that we can recognize all the clauses. This function ensures
* that we do not drop an unsupported clause type on the floor, and thus
* prevents erroneous results.
*
* Note that this function is slightly different than ValidateClauseList(),
* additionally allowing sublinks.
*/
static void
ValidateSubqueryPushdownClauseList(List *clauseList)
{
ListCell *clauseCell = NULL;
foreach(clauseCell, clauseList)
{
Node *clause = (Node *) lfirst(clauseCell);
if (!(IsSublinkClause(clause) || IsSelectClause(clause) ||
IsJoinClause(clause) || or_clause(clause)))
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("unsupported clause type"))); errmsg("unsupported clause type")));
@ -1810,6 +1878,10 @@ JoinClauseList(List *whereClauseList)
* subqueries but differently from the outermost query, they are run on a copy * subqueries but differently from the outermost query, they are run on a copy
* of parse tree and changes do not get persisted as modifications to the original * of parse tree and changes do not get persisted as modifications to the original
* query tree. * query tree.
*
* Also this function adds SubLinks to the baseQualifierList when they appear on
* the query's WHERE clause. The callers of the function should consider processing
* Sublinks as well.
*/ */
static bool static bool
ExtractFromExpressionWalker(Node *node, QualifierWalkerContext *walkerContext) ExtractFromExpressionWalker(Node *node, QualifierWalkerContext *walkerContext)
@ -2203,7 +2275,8 @@ MultiSelectNode(List *whereClauseList)
/* /*
* IsSelectClause determines if the given node is a select clause according to * IsSelectClause determines if the given node is a select clause according to
* our criteria. Our criteria defines a select clause as an expression that has * our criteria. Our criteria defines a select clause as an expression that has
* zero or more columns belonging to only one table. * zero or more columns belonging to only one table. The function assumes that
* no sublinks exists in the clause.
*/ */
static bool static bool
IsSelectClause(Node *clause) IsSelectClause(Node *clause)
@ -2213,16 +2286,6 @@ IsSelectClause(Node *clause)
Var *firstColumn = NULL; Var *firstColumn = NULL;
Index firstColumnTableId = 0; Index firstColumnTableId = 0;
bool isSelectClause = true; bool isSelectClause = true;
NodeTag nodeTag = nodeTag(clause);
/* error out for subqueries in WHERE clause */
if (nodeTag == T_SubLink || nodeTag == T_SubPlan)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot perform distributed planning on this query"),
errdetail("Subqueries other than in from-clause are currently "
"unsupported")));
}
/* extract columns from the clause */ /* extract columns from the clause */
columnList = pull_var_clause_default(clause); columnList = pull_var_clause_default(clause);
@ -2249,6 +2312,23 @@ IsSelectClause(Node *clause)
} }
/*
* IsSublinkClause determines if the given node is a sublink or subplan.
*/
static bool
IsSublinkClause(Node *clause)
{
NodeTag nodeTag = nodeTag(clause);
if (nodeTag == T_SubLink || nodeTag == T_SubPlan)
{
return true;
}
return false;
}
/* /*
* MultiProjectNode builds the project node using the target entry information * MultiProjectNode builds the project node using the target entry information
* from the query tree. The project node only encapsulates projected columns, * from the query tree. The project node only encapsulates projected columns,
@ -2904,9 +2984,13 @@ SubqueryPushdownMultiPlanTree(Query *queryTree)
/* verify we can perform distributed planning on this query */ /* verify we can perform distributed planning on this query */
ErrorIfQueryNotSupported(queryTree); ErrorIfQueryNotSupported(queryTree);
/* extract qualifiers and verify we can plan for them */ /*
* Extract qualifiers and verify we can plan for them. Note that since
* subquery pushdown join planning is based on restriction equivalence,
* checking for these qualifiers may not be necessary.
*/
qualifierList = QualifierList(queryTree->jointree); qualifierList = QualifierList(queryTree->jointree);
ValidateClauseList(qualifierList); ValidateSubqueryPushdownClauseList(qualifierList);
/* /*
* We would be creating a new Query and pushing down top level query's * We would be creating a new Query and pushing down top level query's

View File

@ -265,7 +265,7 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query
PlannedStmt *resultPlan = NULL; PlannedStmt *resultPlan = NULL;
bool hasUnresolvedParams = false; bool hasUnresolvedParams = false;
if (HasUnresolvedExternParamsWalker((Node *) query, boundParams)) if (HasUnresolvedExternParamsWalker((Node *) originalQuery, boundParams))
{ {
hasUnresolvedParams = true; hasUnresolvedParams = true;
} }

View File

@ -2939,10 +2939,10 @@ InsertSelectQuery(Query *query)
* shallowly, for lack of copyObject support. * shallowly, for lack of copyObject support.
* *
* Note that CopyRelationRestrictionContext copies the following fields per relation * Note that CopyRelationRestrictionContext copies the following fields per relation
* context: index, relationId, distributedRelation, rte, relOptInfo->baserestrictinfo, * context: index, relationId, distributedRelation, rte, relOptInfo->baserestrictinfo
* relOptInfo->joininfo and prunedShardIntervalList. Also, the function shallowly copies * and relOptInfo->joininfo. Also, the function shallowly copies plannerInfo and
* plannerInfo which is read-only. All other parts of the relOptInfo is also shallowly * prunedShardIntervalList which are read-only. All other parts of the relOptInfo
* copied. * is also shallowly copied.
*/ */
RelationRestrictionContext * RelationRestrictionContext *
CopyRelationRestrictionContext(RelationRestrictionContext *oldContext) CopyRelationRestrictionContext(RelationRestrictionContext *oldContext)
@ -2981,8 +2981,7 @@ CopyRelationRestrictionContext(RelationRestrictionContext *oldContext)
/* not copyable, but readonly */ /* not copyable, but readonly */
newRestriction->plannerInfo = oldRestriction->plannerInfo; newRestriction->plannerInfo = oldRestriction->plannerInfo;
newRestriction->prunedShardIntervalList = newRestriction->prunedShardIntervalList = oldRestriction->prunedShardIntervalList;
copyObject(oldRestriction->prunedShardIntervalList);
newContext->relationRestrictionList = newContext->relationRestrictionList =
lappend(newContext->relationRestrictionList, newRestriction); lappend(newContext->relationRestrictionList, newRestriction);

View File

@ -121,16 +121,6 @@ FROM
WHERE WHERE
user_id < 0; user_id < 0;
NOTICE: evaluating on master NOTICE: evaluating on master
-- make sure stable functions in CTEs are evaluated
INSERT INTO raw_events_second (user_id, value_1)
WITH sub_cte AS (SELECT evaluate_on_master())
SELECT
user_id, (SELECT * FROM sub_cte)
FROM
raw_events_first
WHERE
user_id < 0;
NOTICE: evaluating on master
-- make sure we don't evaluate stable functions with column arguments -- make sure we don't evaluate stable functions with column arguments
CREATE OR REPLACE FUNCTION evaluate_on_master(x int) CREATE OR REPLACE FUNCTION evaluate_on_master(x int)
RETURNS int LANGUAGE plpgsql STABLE RETURNS int LANGUAGE plpgsql STABLE
@ -665,18 +655,14 @@ INSERT INTO agg_events
fist_table_agg; fist_table_agg;
ERROR: INSERT INTO ... SELECT partition columns in the source table and subquery do not match ERROR: INSERT INTO ... SELECT partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery. DETAIL: The target table's partition column should correspond to a partition column in the subquery.
-- We do support some CTEs -- We don't support CTEs that consist of const values as well
INSERT INTO agg_events INSERT INTO agg_events
WITH sub_cte AS (SELECT 1) WITH sub_cte AS (SELECT 1)
SELECT SELECT
raw_events_first.user_id, (SELECT * FROM sub_cte) raw_events_first.user_id, (SELECT * FROM sub_cte)
FROM FROM
raw_events_first; raw_events_first;
DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) WITH sub_cte AS (SELECT 1) SELECT user_id, (SELECT sub_cte."?column?" FROM sub_cte) FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) >= '-2147483648'::integer) AND (worker_hash(user_id) <= '-1073741825'::integer)) ERROR: Subqueries without relations are not allowed in INSERT ... SELECT queries
DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) WITH sub_cte AS (SELECT 1) SELECT user_id, (SELECT sub_cte."?column?" FROM sub_cte) FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) >= '-1073741824'::integer) AND (worker_hash(user_id) <= '-1'::integer))
DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) WITH sub_cte AS (SELECT 1) SELECT user_id, (SELECT sub_cte."?column?" FROM sub_cte) FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) >= 0) AND (worker_hash(user_id) <= 1073741823))
DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) WITH sub_cte AS (SELECT 1) SELECT user_id, (SELECT sub_cte."?column?" FROM sub_cte) FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) >= 1073741824) AND (worker_hash(user_id) <= 2147483647))
DEBUG: Plan is router executable
-- We do not support any set operations -- We do not support any set operations
INSERT INTO INSERT INTO
raw_events_first(user_id) raw_events_first(user_id)
@ -1581,6 +1567,30 @@ ERROR: grouping sets are not allowed in INSERT ... SELECT queries
SET client_min_messages TO INFO; SET client_min_messages TO INFO;
-- avoid constraint violations -- avoid constraint violations
TRUNCATE raw_events_first; TRUNCATE raw_events_first;
-- we don't support LIMIT even if it exists in the subqueries
-- in where clause
INSERT INTO agg_events(user_id)
SELECT user_id
FROM users_table
WHERE user_id
IN (SELECT
user_id
FROM (
(
SELECT
user_id
FROM
(
SELECT
e1.user_id
FROM
users_table u1, events_table e1
WHERE
e1.user_id = u1.user_id LIMIT 3
) as f_inner
)
) AS f2);
ERROR: LIMIT clauses are not allowed in INSERT ... SELECT queries
-- Altering a table and selecting from it using a multi-shard statement -- Altering a table and selecting from it using a multi-shard statement
-- in the same transaction is allowed because we will use the same -- in the same transaction is allowed because we will use the same
-- connections for all co-located placements. -- connections for all co-located placements.

View File

@ -422,15 +422,6 @@ DEBUG: Plan is router executable
SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2; 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 ERROR: could not run distributed query with complex table expressions
HINT: Consider using an equality filter on the distributed table's partition column. HINT: Consider using an equality filter on the distributed table's partition column.
-- subqueries are not supported in WHERE clause in Citus
SELECT * FROM articles_hash_mx WHERE author_id IN (SELECT id FROM authors_hash_mx WHERE name LIKE '%a');
ERROR: cannot plan queries that include both regular and partitioned relations
SELECT * FROM articles_hash_mx WHERE author_id IN (SELECT author_id FROM articles_hash_mx WHERE author_id = 1 or author_id = 3);
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS
HINT: Consider joining tables on partition column and have equal filter on joining columns.
SELECT * FROM articles_hash_mx WHERE author_id = (SELECT 1);
ERROR: could not run distributed query with subquery outside the FROM clause
HINT: Consider using an equality filter on the distributed table's partition column.
-- subqueries are supported in FROM clause but they are not router plannable -- subqueries are supported in FROM clause but they are not router plannable
SELECT articles_hash_mx.id,test.word_count 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 FROM articles_hash_mx, (SELECT id, word_count FROM articles_hash_mx) AS test WHERE test.id = articles_hash_mx.id
@ -502,7 +493,7 @@ HINT: Set citus.task_executor_type to "task-tracker".
-- subqueries are not supported in SELECT clause -- 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) 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; AS special_price FROM articles_hash_mx a;
ERROR: could not run distributed query with subquery outside the FROM clause 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. HINT: Consider using an equality filter on the distributed table's partition column.
-- simple lookup query -- simple lookup query
SELECT * SELECT *

View File

@ -520,15 +520,6 @@ DEBUG: Plan is router executable
SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2; 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 ERROR: could not run distributed query with complex table expressions
HINT: Consider using an equality filter on the distributed table's partition column. HINT: Consider using an equality filter on the distributed table's partition column.
-- subqueries are not supported in WHERE clause in Citus
SELECT * FROM articles_hash WHERE author_id IN (SELECT id FROM authors_hash WHERE name LIKE '%a');
ERROR: cannot plan queries that include both regular and partitioned relations
SELECT * FROM articles_hash WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 1 or author_id = 3);
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS
HINT: Consider joining tables on partition column and have equal filter on joining columns.
SELECT * FROM articles_hash WHERE author_id = (SELECT 1);
ERROR: could not run distributed query with subquery outside the FROM clause
HINT: Consider using an equality filter on the distributed table's partition column.
-- unless the query can be transformed into a join -- unless the query can be transformed into a join
SELECT * FROM articles_hash SELECT * FROM articles_hash
WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 2) WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 2)
@ -615,7 +606,7 @@ HINT: Set citus.task_executor_type to "task-tracker".
-- subqueries are not supported in SELECT clause -- 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) 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; AS special_price FROM articles_hash a;
ERROR: could not run distributed query with subquery outside the FROM clause 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. HINT: Consider using an equality filter on the distributed table's partition column.
-- simple lookup query -- simple lookup query
SELECT * SELECT *

View File

@ -247,7 +247,7 @@ ORDER BY articles.id;
-- subqueries are not supported in SELECT clause -- 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) 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; AS special_price FROM articles a;
ERROR: could not run distributed query with subquery outside the FROM clause 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. HINT: Consider using an equality filter on the distributed table's partition column.
-- joins are not supported between local and distributed tables -- joins are not supported between local and distributed tables
SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id; SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id;

View File

@ -316,8 +316,14 @@ RETURNS TABLE (a bigint)
AS $$ AS $$
SELECT count(*) AS count_val from test_parameterized_sql where org_id = org_id_val; SELECT count(*) AS count_val from test_parameterized_sql where org_id = org_id_val;
$$ LANGUAGE SQL STABLE; $$ LANGUAGE SQL STABLE;
CREATE OR REPLACE FUNCTION test_parameterized_sql_function_in_subquery_where(org_id_val integer)
RETURNS TABLE (a bigint)
AS $$
SELECT count(*) AS count_val from test_parameterized_sql as t1 where
org_id IN (SELECT org_id FROM test_parameterized_sql as t2 WHERE t2.org_id = t1.org_id AND org_id = org_id_val);
$$ LANGUAGE SQL STABLE;
INSERT INTO test_parameterized_sql VALUES(1, 1); INSERT INTO test_parameterized_sql VALUES(1, 1);
-- both of them should fail -- all of them should fail
SELECT * FROM test_parameterized_sql_function(1); SELECT * FROM test_parameterized_sql_function(1);
ERROR: cannot perform distributed planning on this query because parameterized queries for SQL functions referencing distributed tables are not supported ERROR: cannot perform distributed planning on this query because parameterized queries for SQL functions referencing distributed tables are not supported
HINT: Consider using PL/pgSQL functions instead. HINT: Consider using PL/pgSQL functions instead.
@ -326,6 +332,11 @@ ERROR: could not create distributed plan
DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus. DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus.
HINT: Consider using PL/pgSQL functions instead. HINT: Consider using PL/pgSQL functions instead.
CONTEXT: SQL function "test_parameterized_sql_function" statement 1 CONTEXT: SQL function "test_parameterized_sql_function" statement 1
SELECT test_parameterized_sql_function_in_subquery_where(1);
ERROR: could not create distributed plan
DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus.
HINT: Consider using PL/pgSQL functions instead.
CONTEXT: SQL function "test_parameterized_sql_function_in_subquery_where" statement 1
DROP TABLE temp_table; DROP TABLE temp_table;
DROP TABLE test_parameterized_sql; DROP TABLE test_parameterized_sql;
-- clean-up functions -- clean-up functions
@ -338,3 +349,4 @@ DROP FUNCTION non_partition_parameter_insert_sql(int);
DROP FUNCTION non_partition_parameter_update_sql(int, int); DROP FUNCTION non_partition_parameter_update_sql(int, int);
DROP FUNCTION non_partition_parameter_delete_sql(int); DROP FUNCTION non_partition_parameter_delete_sql(int);
DROP FUNCTION test_parameterized_sql_function(int); DROP FUNCTION test_parameterized_sql_function(int);
DROP FUNCTION test_parameterized_sql_function_in_subquery_where(int);

View File

@ -546,62 +546,103 @@ ORDER BY user_lastseen DESC, user_id;
------------------------------------ ------------------------------------
-- Count the number of distinct users_table who are in segment X and Y and Z -- Count the number of distinct users_table who are in segment X and Y and Z
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT DISTINCT user_id SELECT user_id
FROM users_table FROM users_table
WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20) WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20)
AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40) AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40)
AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60); AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60)
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS GROUP BY
HINT: Consider joining tables on partition column and have equal filter on joining columns. user_id
ORDER BY
user_id DESC
LIMIT 5;
user_id
---------
93
90
88
87
84
(5 rows)
------------------------------------ ------------------------------------
-- Find customers who have done X, and satisfy other customer specific criteria -- Find customers who have done X, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, value_2 FROM users_table WHERE
value_1 > 101 AND value_1 < 110 value_1 > 101 AND value_1 < 110
AND value_2 >= 5 AND value_2 >= 5
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 101 AND event_type < 110 AND value_3 > 100 AND user_id = users_table.user_id); AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 101 AND event_type < 110 AND value_3 > 100 AND user_id = users_table.user_id)
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS ORDER BY 2 DESC, 1 DESC
HINT: Consider joining tables on partition column and have equal filter on joining columns. LIMIT 5;
user_id | value_2
---------+---------
95 | 951
4 | 934
2 | 908
90 | 900
49 | 847
(5 rows)
------------------------------------ ------------------------------------
-- Customers who havent done X, and satisfy other customer specific criteria -- Customers who havent done X, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, value_2 FROM users_table WHERE
value_1 = 101 value_1 = 101
AND value_2 >= 5 AND value_2 >= 5
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND user_id = users_table.user_id); AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND user_id = users_table.user_id)
ERROR: could not run distributed query with subquery outside the FROM clause ORDER BY 1 DESC, 2 DESC
HINT: Consider using an equality filter on the distributed table's partition column. LIMIT 3;
user_id | value_2
---------+---------
58 | 585
51 | 1000
48 | 861
(3 rows)
------------------------------------ ------------------------------------
-- Customers who have done X and Y, and satisfy other customer specific criteria -- Customers who have done X and Y, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, sum(value_2) as cnt FROM users_table WHERE
value_1 > 100 value_1 > 100
AND value_2 >= 5 AND value_2 >= 5
AND EXISTS (SELECT user_id FROM events_table WHERE event_type != 100 AND value_3 > 100 AND user_id = users_table.user_id) AND EXISTS (SELECT user_id FROM events_table WHERE event_type != 100 AND value_3 > 100 AND user_id = users_table.user_id)
AND EXISTS (SELECT user_id FROM events_table WHERE event_type = 101 AND value_3 > 100 AND user_id = users_table.user_id); AND EXISTS (SELECT user_id FROM events_table WHERE event_type = 101 AND value_3 > 100 AND user_id = users_table.user_id)
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS GROUP BY
HINT: Consider joining tables on partition column and have equal filter on joining columns. user_id
ORDER BY cnt DESC, user_id DESC
LIMIT 5;
user_id | cnt
---------+-------
49 | 48606
69 | 46524
86 | 46163
80 | 45995
35 | 45437
(5 rows)
------------------------------------ ------------------------------------
-- Customers who have done X and havent done Y, and satisfy other customer specific criteria -- Customers who have done X and havent done Y, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, value_2 FROM users_table WHERE
value_2 >= 5 value_2 >= 5
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id) AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id)
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id); AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id)
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS ORDER BY 2 DESC, 1 DESC
HINT: Consider joining tables on partition column and have equal filter on joining columns. LIMIT 4;
user_id | value_2
---------+---------
8 | 996
96 | 995
8 | 995
96 | 989
(4 rows)
------------------------------------ ------------------------------------
-- Customers who have done X more than 2 times, and satisfy other customer specific criteria -- Customers who have done X more than 2 times, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, SELECT user_id,
value_2 avg(value_2)
FROM users_table FROM users_table
WHERE value_1 > 100 WHERE value_1 > 100
AND value_1 < 124 AND value_1 < 124
@ -613,9 +654,21 @@ SELECT user_id,
AND value_3 > 100 AND value_3 > 100
AND user_id = users_table.user_id AND user_id = users_table.user_id
GROUP BY user_id GROUP BY user_id
HAVING Count(*) > 2); HAVING Count(*) > 2)
ERROR: could not run distributed query with subquery outside the FROM clause GROUP BY
HINT: Consider using an equality filter on the distributed table's partition column. user_id
ORDER BY
1 DESC, 2 DESC
LIMIT 5;
user_id | avg
---------+----------------------
99 | 571.6666666666666667
98 | 758.0000000000000000
96 | 459.6666666666666667
90 | 453.3333333333333333
89 | 215.0000000000000000
(5 rows)
------------------------------------ ------------------------------------
-- Find me all users_table who logged in more than once -- Find me all users_table who logged in more than once
------------------------------------ ------------------------------------
@ -671,7 +724,6 @@ ORDER BY
------------------------------------ ------------------------------------
-- Find me all users_table who has done some event and has filters -- Find me all users_table who has done some event and has filters
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id SELECT user_id
FROM events_table FROM events_table
@ -684,18 +736,32 @@ WHERE
users_table users_table
WHERE WHERE
value_1 = 15 AND value_2 > 25 value_1 = 15 AND value_2 > 25
); )
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS ORDER BY 1;
HINT: Consider joining tables on partition column and have equal filter on joining columns. user_id
---------
7
53
(2 rows)
------------------------------------ ------------------------------------
-- Which events_table did people who has done some specific events_table -- Which events_table did people who has done some specific events_table
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, event_type FROM events_table SELECT
WHERE user_id in (SELECT user_id from events_table WHERE event_type > 500 and event_type < 505) user_id, event_type FROM events_table
GROUP BY user_id, event_type; WHERE
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS user_id in (SELECT user_id from events_table WHERE event_type > 500 and event_type < 505)
HINT: Consider joining tables on partition column and have equal filter on joining columns. GROUP BY
user_id, event_type
ORDER BY 2 DESC, 1
LIMIT 3;
user_id | event_type
---------+------------
18 | 999
23 | 999
26 | 999
(3 rows)
------------------------------------ ------------------------------------
-- Find me all the users_table who has done some event more than three times -- Find me all the users_table who has done some event more than three times
------------------------------------ ------------------------------------
@ -1950,5 +2016,45 @@ FROM (
ERROR: no value found for parameter 1 ERROR: no value found for parameter 1
SET client_min_messages TO DEFAULT; SET client_min_messages TO DEFAULT;
DROP FUNCTION volatile_func_test(); DROP FUNCTION volatile_func_test();
SET citus.subquery_pushdown to OFF; CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool
AS 'select $1 > $2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- we don't support joins via functions
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 test_join_function_2(u.user_id, e.user_id)
) t
GROUP BY user_id
) q
ORDER BY 2 DESC, 1;
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
-- note that the following query has joins on the partition keys
-- however we fail to push down it due to the function call on the
-- where clause. We probably need to relax that check
SELECT
users_table.user_id, users_table.value_1, prob
FROM
users_table
JOIN
(SELECT
ma.user_id, (GREATEST(coalesce(ma.value_4 / 250, 0.0) + GREATEST(1.0))) / 2 AS prob
FROM
users_table AS ma, events_table as short_list
WHERE
short_list.user_id = ma.user_id and ma.value_1 < 50 and short_list.event_type < 50
) temp
ON users_table.user_id = temp.user_id
WHERE
users_table.value_1 < 50 AND test_join_function_2(users_table.user_id, temp.user_id);
ERROR: unsupported clause type
DROP FUNCTION test_join_function_2(integer, integer);
SET citus.enable_router_execution TO TRUE; SET citus.enable_router_execution TO TRUE;
SET citus.subquery_pushdown to OFF;

View File

@ -2438,7 +2438,7 @@ FROM
FROM FROM
(SELECT (SELECT
1 as user_id, now(), 3 AS event 1 as user_id, now(), 3 AS event
) events_subquery_4) OFFSET 3) t1 ) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q" GROUP BY "t1"."user_id") AS t) "q"
INNER JOIN INNER JOIN
(SELECT (SELECT
@ -2498,7 +2498,7 @@ FROM
FROM FROM
(SELECT (SELECT
1 as user_id, now(), 3 AS event 1 as user_id, now(), 3 AS event
) events_subquery_4) OFFSET 3) t1 ) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q" GROUP BY "t1"."user_id") AS t) "q"
INNER JOIN INNER JOIN
(SELECT random()::int as user_id) AS t (SELECT random()::int as user_id) AS t

View File

@ -0,0 +1,582 @@
--
-- multi subquery in where queries aims to expand existing subquery pushdown
-- regression tests to cover more cases specifically subqueries in WHERE clause
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
--
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
-- subqueries in WHERE with greater operator
SELECT
user_id
FROM
users_table
WHERE
value_2 >
(SELECT
max(value_2)
FROM
events_table
WHERE
users_table.user_id = events_table.user_id AND event_type = 50
GROUP BY
user_id
)
GROUP BY user_id
HAVING count(*) > 66
ORDER BY user_id
LIMIT 5;
user_id
---------
49
55
56
63
(4 rows)
-- subqueries in where with ALL operator
SELECT
user_id
FROM
users_table
WHERE
value_2 > 545 AND
value_2 < ALL (SELECT avg(value_3) FROM events_table WHERE users_table.user_id = events_table.user_id GROUP BY user_id)
GROUP BY
1
ORDER BY
1 DESC
LIMIT 3;
user_id
---------
69
52
12
(3 rows)
-- IN operator on non-partition key
SELECT
user_id
FROM
events_table as e1
WHERE
event_type IN
(SELECT
event_type
FROM
events_table as e2
WHERE
value_2 = 15 AND value_3 > 25 AND
e1.user_id = e2.user_id
)
ORDER BY 1;
user_id
---------
8
17
33
47
54
54
56
71
79
86
(10 rows)
-- NOT IN on non-partition key
SELECT
user_id
FROM
events_table as e1
WHERE
event_type NOT IN
(SELECT
event_type
FROM
events_table as e2
WHERE
value_2 = 15 AND value_3 > 25 AND
e1.user_id = e2.user_id
)
GROUP BY 1
HAVING count(*) > 122
ORDER BY 1;
user_id
---------
23
25
(2 rows)
-- non-correlated query with =ANY on partition keys
SELECT
user_id, count(*)
FROM
users_table
WHERE
user_id =ANY(SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20) GROUP BY 1 ORDER BY 2 DESC LIMIT 5;
user_id | count
---------+-------
12 | 121
87 | 117
37 | 115
23 | 115
46 | 115
(5 rows)
-- users that appeared more than 118 times
SELECT
user_id
FROM
users_table
WHERE 118 <=
(SELECT
count(*)
FROM
events_table
WHERE
users_table.user_id = events_table.user_id
GROUP BY
user_id)
GROUP BY
user_id
ORDER BY
user_id;
user_id
---------
13
17
23
25
(4 rows)
-- the following query doesn't have a meaningful result
-- but it is a valid query with an arbitrary subquery in
-- WHERE clause
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 101 AND value_1 < 110
AND value_2 >= 5
AND user_id IN
(
SELECT
e1.user_id
FROM (
-- Get the first time each user viewed the homepage.
SELECT
user_id,
1 AS view_homepage,
min(time) AS view_homepage_time
FROM events_table
WHERE
event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90)
GROUP BY user_id
) e1 LEFT JOIN LATERAL (
SELECT
user_id,
1 AS use_demo,
time AS use_demo_time
FROM events_table
WHERE
user_id = e1.user_id AND
event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91)
ORDER BY time
) e2 ON true LEFT JOIN LATERAL (
SELECT
user_id,
1 AS enter_credit_card,
time AS enter_credit_card_time
FROM events_table
WHERE
user_id = e2.user_id AND
event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92)
ORDER BY time
) e3 ON true LEFT JOIN LATERAL (
SELECT
1 AS submit_card_info,
user_id,
time AS enter_credit_card_time
FROM events_table
WHERE
user_id = e3.user_id AND
event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93)
ORDER BY time
) e4 ON true LEFT JOIN LATERAL (
SELECT
1 AS see_bought_screen
FROM events_table
WHERE
user_id = e4.user_id AND
event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94)
ORDER BY time
) e5 ON true
group by e1.user_id
HAVING sum(submit_card_info) > 0
)
ORDER BY 1, 2;
user_id | value_2
---------+---------
5 | 884
42 | 55
42 | 471
51 | 758
72 | 897
82 | 691
95 | 951
(7 rows)
-- similar to the above query
-- the following query doesn't have a meaningful result
-- but it is a valid query with an arbitrary subquery in
-- WHERE clause
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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 10 AND events_table.event_type < 12
)
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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 12 AND events_table.event_type < 14
)
) AS subquery_1
LEFT JOIN
(SELECT
user_id,
COUNT(*) AS count_pay
FROM
users_table
WHERE
user_id >= 10 AND
user_id <= 70 AND
users_table.value_1 > 15 AND users_table.value_1 < 17
GROUP BY
user_id
HAVING
COUNT(*) > 1) 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(*) > 3 AND sum(value_2) > 49000
ORDER BY 1;
user_id
---------
18
29
40
49
58
69
(6 rows)
-- the following query doesn't have a meaningful result
-- but it is a valid query with an arbitrary subquery in
-- FROM clause involving a complex query in WHERE clause
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 > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id)
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id)
)
) t
GROUP BY user_id
) q
ORDER BY 2 DESC, 1;
user_id | array_length
---------+--------------
96 | 12204
8 | 8170
(2 rows)
--
-- below tests only aims for cases where all relations
-- are not joined on partition key
--
-- e4 is not joined on the partition key
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 101 AND value_1 < 110
AND value_2 >= 5
AND user_id IN
(
SELECT
e1.user_id
FROM (
-- Get the first time each user viewed the homepage.
SELECT
user_id,
1 AS view_homepage,
min(time) AS view_homepage_time
FROM events_table
WHERE
event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90)
GROUP BY user_id
) e1 LEFT JOIN LATERAL (
SELECT
user_id,
1 AS use_demo,
time AS use_demo_time
FROM events_table
WHERE
user_id = e1.user_id AND
event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91)
ORDER BY time
) e2 ON true LEFT JOIN LATERAL (
SELECT
user_id,
1 AS enter_credit_card,
time AS enter_credit_card_time
FROM events_table
WHERE
user_id = e2.user_id AND
event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92)
ORDER BY time
) e3 ON true LEFT JOIN LATERAL (
SELECT
1 AS submit_card_info,
user_id,
time AS enter_credit_card_time
FROM events_table
WHERE
value_2 = e3.user_id AND
event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93)
ORDER BY time
) e4 ON true LEFT JOIN LATERAL (
SELECT
1 AS see_bought_screen
FROM events_table
WHERE
user_id = e4.user_id AND
event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94)
ORDER BY time
) e5 ON true
group by e1.user_id
HAVING sum(submit_card_info) > 0
);
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.
-- left leaf query does not return partition key
SELECT
user_id
FROM
users_table
WHERE
user_id IN
(
SELECT
user_id
FROM (
SELECT
subquery_1.user_id, count_pay
FROM
(
(SELECT
2 * users_table.user_id as 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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 10 AND events_table.event_type < 12
)
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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 12 AND events_table.event_type < 14
)
) AS subquery_1
LEFT JOIN
(SELECT
user_id,
COUNT(*) AS count_pay
FROM
users_table
WHERE
user_id >= 10 AND
user_id <= 70 AND
users_table.value_1 > 15 AND users_table.value_1 < 17
GROUP BY
user_id
HAVING
COUNT(*) > 1) 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(*) > 3 AND sum(value_2) > 49000
ORDER BY 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.
-- NOT EXISTS query has non-equi join
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 > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id)
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id != users_table.user_id)
)
) t
GROUP BY user_id
) q
ORDER BY 2 DESC, 1;
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
-- subquery in where clause doesn't have a relation
SELECT
user_id
FROM
users_table
WHERE
value_2 >
(SELECT 1);
ERROR: cannot push down this subquery
DETAIL: Subqueries without relations are unsupported
-- OFFSET is not supported in the subquey
SELECT
user_id
FROM
users_table
WHERE
value_2 >
(SELECT
max(value_2)
FROM
events_table
WHERE
users_table.user_id = events_table.user_id AND event_type = 50
GROUP BY
user_id
OFFSET 3
);
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
FROM users_table
WHERE user_id
IN (SELECT
f_inner.user_id
FROM
(
SELECT
e1.user_id
FROM
users_table u1, events_table e1
WHERE
e1.user_id = u1.user_id
) as f_inner,
(
SELECT
e1.user_id
FROM
users_table u1, events_table e1
WHERE
e1.user_id = u1.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
-- semi join is not on the partition key for the third subquery
SELECT user_id
FROM users_table
WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20)
AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40)
AND value_2 IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60);
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
CREATE FUNCTION test_join_function(integer, integer) RETURNS bool
AS 'select $1 > $2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- we disallow JOINs via functions
SELECT user_id, value_2 FROM users_table WHERE
value_1 = 101
AND value_2 >= 5
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND test_join_function(events_table.user_id, users_table.user_id))
ORDER BY 1 DESC, 2 DESC
LIMIT 3;
ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys
DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator.
DROP FUNCTION test_join_function(int,int);

View File

@ -1,18 +0,0 @@
--
-- MULTI_VERIFY_NO_SUBQUERY
--
-- This test checks that we simply emit an error message instead of trying to
-- process a distributed unsupported SQL subquery.
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1030000;
SELECT * FROM lineitem WHERE l_orderkey IN
(SELECT l_orderkey FROM lineitem WHERE l_quantity > 0);
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS
SELECT l_quantity FROM lineitem WHERE EXISTS
(SELECT 1 FROM orders WHERE o_orderkey = l_orderkey);
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS
SELECT l_quantity FROM lineitem WHERE l_orderkey IN (SELECT o_orderkey FROM orders);
ERROR: could not run distributed query with join types other than INNER or OUTER JOINS
SELECT l_orderkey FROM lineitem WHERE l_quantity > ALL(SELECT o_orderkey FROM orders);
ERROR: could not run distributed query with subquery outside the FROM clause
SELECT l_quantity FROM lineitem WHERE l_orderkey = (SELECT min(o_orderkey) FROM orders);
ERROR: could not run distributed query with subquery outside the FROM clause

View File

@ -568,9 +568,16 @@ SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1;
(12 rows) (12 rows)
-- this would be supported when we implement where partition_key in (subquery) support -- this would be supported when we implement where partition_key in (subquery) support
SELECT et.* FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users); SELECT et.user_id, et.time FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users) GROUP BY 1,2 ORDER BY 1 DESC,2 DESC LIMIT 5;
ERROR: could not run distributed query with subquery outside the FROM clause user_id | time
HINT: Consider using an equality filter on the distributed table's partition column. ---------+---------------------------------
90 | Tue Jan 21 02:50:05.379732 2014
90 | Tue Jan 21 00:08:33.911898 2014
90 | Mon Jan 20 22:25:39.21906 2014
90 | Mon Jan 20 21:11:10.814326 2014
90 | Mon Jan 20 19:16:33.359257 2014
(5 rows)
-- it is supported when it is a router query -- it is supported when it is a router query
SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90); SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90);
count count
@ -725,18 +732,23 @@ SELECT user_id FROM router_view GROUP BY 1;
2 2
(1 row) (1 row)
-- There is a known issue with router plannable subqueries joined with non-router -- join a router view
-- plannable subqueries. Following tests should be uncommented when we fix it SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id) ORDER BY 2 LIMIT 3;
-- join a router view (not implement error) user_id | time
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id); ---------+---------------------------------
-- it still does not work when converted to 2 subquery join 2 | Mon Jan 20 02:02:03.208351 2014
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id); 2 | Mon Jan 20 02:34:14.54301 2014
-- views are completely removed and still it does not work 2 | Mon Jan 20 03:16:38.418772 2014
-- SELECT * FROM (3 rows)
-- (SELECT user_id FROM (SELECT * FROM users_table WHERE user_id = 2) rv1 GROUP BY 1) rv2
-- JOIN (SELECT user_id, time FROM events_table SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id) ORDER BY 2 LIMIT 3;
-- WHERE time > '2014-01-20 01:45:49.978738'::timestamp) re user_id | time
-- USING (user_id); ---------+---------------------------------
2 | Mon Jan 20 02:02:03.208351 2014
2 | Mon Jan 20 02:34:14.54301 2014
2 | Mon Jan 20 03:16:38.418772 2014
(3 rows)
-- views with limits -- views with limits
CREATE VIEW recent_10_users AS CREATE VIEW recent_10_users AS
SELECT user_id, max(time) as lastseen FROM users_table SELECT user_id, max(time) as lastseen FROM users_table

View File

@ -21,7 +21,7 @@ test: multi_table_ddl
test: multi_create_table test: multi_create_table
test: multi_load_data test: multi_load_data
test: multi_basic_queries multi_complex_expressions multi_verify_no_subquery test: multi_basic_queries multi_complex_expressions
test: multi_single_relation_subquery test: multi_single_relation_subquery
test: multi_binary_master_copy_format test: multi_binary_master_copy_format

View File

@ -38,10 +38,10 @@ test: multi_insert_select
# Miscellaneous tests to check our query planning behavior # Miscellaneous tests to check our query planning behavior
# ---------- # ----------
test: multi_deparse_shard_query test: multi_deparse_shard_query
test: multi_basic_queries multi_complex_expressions multi_verify_no_subquery test: multi_basic_queries multi_complex_expressions
test: multi_explain test: multi_explain
test: multi_subquery multi_subquery_complex_queries multi_subquery_behavioral_analytics test: multi_subquery multi_subquery_complex_queries multi_subquery_behavioral_analytics
test: multi_subquery_union test: multi_subquery_union multi_subquery_in_where_clause
test: multi_reference_table test: multi_reference_table
test: multi_outer_join_reference test: multi_outer_join_reference
test: multi_single_relation_subquery test: multi_single_relation_subquery

View File

@ -28,7 +28,7 @@ test: multi_load_data
# ---------- # ----------
# Miscellaneous tests to check our query planning behavior # Miscellaneous tests to check our query planning behavior
# ---------- # ----------
test: multi_basic_queries multi_complex_expressions multi_verify_no_subquery test: multi_basic_queries multi_complex_expressions
test: multi_agg_distinct multi_limit_clause multi_limit_clause_approximate test: multi_agg_distinct multi_limit_clause multi_limit_clause_approximate
test: multi_average_expression multi_working_columns test: multi_average_expression multi_working_columns
test: multi_array_agg test: multi_array_agg

View File

@ -96,16 +96,6 @@ FROM
WHERE WHERE
user_id < 0; user_id < 0;
-- make sure stable functions in CTEs are evaluated
INSERT INTO raw_events_second (user_id, value_1)
WITH sub_cte AS (SELECT evaluate_on_master())
SELECT
user_id, (SELECT * FROM sub_cte)
FROM
raw_events_first
WHERE
user_id < 0;
-- make sure we don't evaluate stable functions with column arguments -- make sure we don't evaluate stable functions with column arguments
CREATE OR REPLACE FUNCTION evaluate_on_master(x int) CREATE OR REPLACE FUNCTION evaluate_on_master(x int)
RETURNS int LANGUAGE plpgsql STABLE RETURNS int LANGUAGE plpgsql STABLE
@ -517,7 +507,7 @@ INSERT INTO agg_events
FROM FROM
fist_table_agg; fist_table_agg;
-- We do support some CTEs -- We don't support CTEs that consist of const values as well
INSERT INTO agg_events INSERT INTO agg_events
WITH sub_cte AS (SELECT 1) WITH sub_cte AS (SELECT 1)
SELECT SELECT
@ -1311,6 +1301,30 @@ SET client_min_messages TO INFO;
-- avoid constraint violations -- avoid constraint violations
TRUNCATE raw_events_first; TRUNCATE raw_events_first;
-- we don't support LIMIT even if it exists in the subqueries
-- in where clause
INSERT INTO agg_events(user_id)
SELECT user_id
FROM users_table
WHERE user_id
IN (SELECT
user_id
FROM (
(
SELECT
user_id
FROM
(
SELECT
e1.user_id
FROM
users_table u1, events_table e1
WHERE
e1.user_id = u1.user_id LIMIT 3
) as f_inner
)
) AS f2);
-- Altering a table and selecting from it using a multi-shard statement -- Altering a table and selecting from it using a multi-shard statement
-- in the same transaction is allowed because we will use the same -- in the same transaction is allowed because we will use the same
-- connections for all co-located placements. -- connections for all co-located placements.

View File

@ -219,14 +219,6 @@ SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 o
-- they are not supported if multiple workers are involved -- 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; SELECT * FROM articles_hash_mx, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2;
-- subqueries are not supported in WHERE clause in Citus
SELECT * FROM articles_hash_mx WHERE author_id IN (SELECT id FROM authors_hash_mx WHERE name LIKE '%a');
SELECT * FROM articles_hash_mx WHERE author_id IN (SELECT author_id FROM articles_hash_mx WHERE author_id = 1 or author_id = 3);
SELECT * FROM articles_hash_mx WHERE author_id = (SELECT 1);
-- subqueries are supported in FROM clause but they are not router plannable -- subqueries are supported in FROM clause but they are not router plannable
SELECT articles_hash_mx.id,test.word_count 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 FROM articles_hash_mx, (SELECT id, word_count FROM articles_hash_mx) AS test WHERE test.id = articles_hash_mx.id

View File

@ -281,13 +281,6 @@ SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or a
-- they are not supported if multiple workers are involved -- 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; SELECT * FROM articles_hash, position('om' in 'Thomas') WHERE author_id = 1 or author_id = 2;
-- subqueries are not supported in WHERE clause in Citus
SELECT * FROM articles_hash WHERE author_id IN (SELECT id FROM authors_hash WHERE name LIKE '%a');
SELECT * FROM articles_hash WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 1 or author_id = 3);
SELECT * FROM articles_hash WHERE author_id = (SELECT 1);
-- unless the query can be transformed into a join -- unless the query can be transformed into a join
SELECT * FROM articles_hash SELECT * FROM articles_hash
WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 2) WHERE author_id IN (SELECT author_id FROM articles_hash WHERE author_id = 2)

View File

@ -134,11 +134,20 @@ AS $$
SELECT count(*) AS count_val from test_parameterized_sql where org_id = org_id_val; SELECT count(*) AS count_val from test_parameterized_sql where org_id = org_id_val;
$$ LANGUAGE SQL STABLE; $$ LANGUAGE SQL STABLE;
CREATE OR REPLACE FUNCTION test_parameterized_sql_function_in_subquery_where(org_id_val integer)
RETURNS TABLE (a bigint)
AS $$
SELECT count(*) AS count_val from test_parameterized_sql as t1 where
org_id IN (SELECT org_id FROM test_parameterized_sql as t2 WHERE t2.org_id = t1.org_id AND org_id = org_id_val);
$$ LANGUAGE SQL STABLE;
INSERT INTO test_parameterized_sql VALUES(1, 1); INSERT INTO test_parameterized_sql VALUES(1, 1);
-- both of them should fail -- all of them should fail
SELECT * FROM test_parameterized_sql_function(1); SELECT * FROM test_parameterized_sql_function(1);
SELECT test_parameterized_sql_function(1); SELECT test_parameterized_sql_function(1);
SELECT test_parameterized_sql_function_in_subquery_where(1);
DROP TABLE temp_table; DROP TABLE temp_table;
DROP TABLE test_parameterized_sql; DROP TABLE test_parameterized_sql;
@ -153,3 +162,4 @@ DROP FUNCTION non_partition_parameter_insert_sql(int);
DROP FUNCTION non_partition_parameter_update_sql(int, int); DROP FUNCTION non_partition_parameter_update_sql(int, int);
DROP FUNCTION non_partition_parameter_delete_sql(int); DROP FUNCTION non_partition_parameter_delete_sql(int);
DROP FUNCTION test_parameterized_sql_function(int); DROP FUNCTION test_parameterized_sql_function(int);
DROP FUNCTION test_parameterized_sql_function_in_subquery_where(int);

View File

@ -458,57 +458,66 @@ ORDER BY user_lastseen DESC, user_id;
------------------------------------ ------------------------------------
-- Count the number of distinct users_table who are in segment X and Y and Z -- Count the number of distinct users_table who are in segment X and Y and Z
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT DISTINCT user_id SELECT user_id
FROM users_table FROM users_table
WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20) WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20)
AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40) AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40)
AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60); AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60)
GROUP BY
user_id
ORDER BY
user_id DESC
LIMIT 5;
------------------------------------ ------------------------------------
-- Find customers who have done X, and satisfy other customer specific criteria -- Find customers who have done X, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, value_2 FROM users_table WHERE
value_1 > 101 AND value_1 < 110 value_1 > 101 AND value_1 < 110
AND value_2 >= 5 AND value_2 >= 5
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 101 AND event_type < 110 AND value_3 > 100 AND user_id = users_table.user_id); AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 101 AND event_type < 110 AND value_3 > 100 AND user_id = users_table.user_id)
ORDER BY 2 DESC, 1 DESC
LIMIT 5;
------------------------------------ ------------------------------------
-- Customers who havent done X, and satisfy other customer specific criteria -- Customers who havent done X, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, value_2 FROM users_table WHERE
value_1 = 101 value_1 = 101
AND value_2 >= 5 AND value_2 >= 5
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND user_id = users_table.user_id); AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND user_id = users_table.user_id)
ORDER BY 1 DESC, 2 DESC
LIMIT 3;
------------------------------------ ------------------------------------
-- Customers who have done X and Y, and satisfy other customer specific criteria -- Customers who have done X and Y, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, sum(value_2) as cnt FROM users_table WHERE
value_1 > 100 value_1 > 100
AND value_2 >= 5 AND value_2 >= 5
AND EXISTS (SELECT user_id FROM events_table WHERE event_type != 100 AND value_3 > 100 AND user_id = users_table.user_id) AND EXISTS (SELECT user_id FROM events_table WHERE event_type != 100 AND value_3 > 100 AND user_id = users_table.user_id)
AND EXISTS (SELECT user_id FROM events_table WHERE event_type = 101 AND value_3 > 100 AND user_id = users_table.user_id); AND EXISTS (SELECT user_id FROM events_table WHERE event_type = 101 AND value_3 > 100 AND user_id = users_table.user_id)
GROUP BY
user_id
ORDER BY cnt DESC, user_id DESC
LIMIT 5;
------------------------------------ ------------------------------------
-- Customers who have done X and havent done Y, and satisfy other customer specific criteria -- Customers who have done X and havent done Y, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, value_2 FROM users_table WHERE SELECT user_id, value_2 FROM users_table WHERE
value_2 >= 5 value_2 >= 5
AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id) AND EXISTS (SELECT user_id FROM events_table WHERE event_type > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id)
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id); AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id)
ORDER BY 2 DESC, 1 DESC
LIMIT 4;
------------------------------------ ------------------------------------
-- Customers who have done X more than 2 times, and satisfy other customer specific criteria -- Customers who have done X more than 2 times, and satisfy other customer specific criteria
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, SELECT user_id,
value_2 avg(value_2)
FROM users_table FROM users_table
WHERE value_1 > 100 WHERE value_1 > 100
AND value_1 < 124 AND value_1 < 124
@ -520,7 +529,12 @@ SELECT user_id,
AND value_3 > 100 AND value_3 > 100
AND user_id = users_table.user_id AND user_id = users_table.user_id
GROUP BY user_id GROUP BY user_id
HAVING Count(*) > 2); HAVING Count(*) > 2)
GROUP BY
user_id
ORDER BY
1 DESC, 2 DESC
LIMIT 5;
------------------------------------ ------------------------------------
-- Find me all users_table who logged in more than once -- Find me all users_table who logged in more than once
@ -555,7 +569,6 @@ ORDER BY
------------------------------------ ------------------------------------
-- Find me all users_table who has done some event and has filters -- Find me all users_table who has done some event and has filters
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id SELECT user_id
FROM events_table FROM events_table
@ -568,15 +581,20 @@ WHERE
users_table users_table
WHERE WHERE
value_1 = 15 AND value_2 > 25 value_1 = 15 AND value_2 > 25
); )
ORDER BY 1;
------------------------------------ ------------------------------------
-- Which events_table did people who has done some specific events_table -- Which events_table did people who has done some specific events_table
-- This query will be supported when we have subqueries in where clauses.
------------------------------------ ------------------------------------
SELECT user_id, event_type FROM events_table SELECT
WHERE user_id in (SELECT user_id from events_table WHERE event_type > 500 and event_type < 505) user_id, event_type FROM events_table
GROUP BY user_id, event_type; WHERE
user_id in (SELECT user_id from events_table WHERE event_type > 500 and event_type < 505)
GROUP BY
user_id, event_type
ORDER BY 2 DESC, 1
LIMIT 3;
------------------------------------ ------------------------------------
-- Find me all the users_table who has done some event more than three times -- Find me all the users_table who has done some event more than three times
@ -1600,5 +1618,47 @@ FROM (
SET client_min_messages TO DEFAULT; SET client_min_messages TO DEFAULT;
DROP FUNCTION volatile_func_test(); DROP FUNCTION volatile_func_test();
SET citus.subquery_pushdown to OFF; CREATE FUNCTION test_join_function_2(integer, integer) RETURNS bool
AS 'select $1 > $2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- we don't support joins via functions
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 test_join_function_2(u.user_id, e.user_id)
) t
GROUP BY user_id
) q
ORDER BY 2 DESC, 1;
-- note that the following query has joins on the partition keys
-- however we fail to push down it due to the function call on the
-- where clause. We probably need to relax that check
SELECT
users_table.user_id, users_table.value_1, prob
FROM
users_table
JOIN
(SELECT
ma.user_id, (GREATEST(coalesce(ma.value_4 / 250, 0.0) + GREATEST(1.0))) / 2 AS prob
FROM
users_table AS ma, events_table as short_list
WHERE
short_list.user_id = ma.user_id and ma.value_1 < 50 and short_list.event_type < 50
) temp
ON users_table.user_id = temp.user_id
WHERE
users_table.value_1 < 50 AND test_join_function_2(users_table.user_id, temp.user_id);
DROP FUNCTION test_join_function_2(integer, integer);
SET citus.enable_router_execution TO TRUE; SET citus.enable_router_execution TO TRUE;
SET citus.subquery_pushdown to OFF;

View File

@ -2216,7 +2216,7 @@ FROM
FROM FROM
(SELECT (SELECT
1 as user_id, now(), 3 AS event 1 as user_id, now(), 3 AS event
) events_subquery_4) OFFSET 3) t1 ) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q" GROUP BY "t1"."user_id") AS t) "q"
INNER JOIN INNER JOIN
(SELECT (SELECT
@ -2275,7 +2275,7 @@ FROM
FROM FROM
(SELECT (SELECT
1 as user_id, now(), 3 AS event 1 as user_id, now(), 3 AS event
) events_subquery_4) OFFSET 3) t1 ) events_subquery_4)) t1
GROUP BY "t1"."user_id") AS t) "q" GROUP BY "t1"."user_id") AS t) "q"
INNER JOIN INNER JOIN
(SELECT random()::int as user_id) AS t (SELECT random()::int as user_id) AS t

View File

@ -0,0 +1,512 @@
--
-- multi subquery in where queries aims to expand existing subquery pushdown
-- regression tests to cover more cases specifically subqueries in WHERE clause
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
--
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
-- subqueries in WHERE with greater operator
SELECT
user_id
FROM
users_table
WHERE
value_2 >
(SELECT
max(value_2)
FROM
events_table
WHERE
users_table.user_id = events_table.user_id AND event_type = 50
GROUP BY
user_id
)
GROUP BY user_id
HAVING count(*) > 66
ORDER BY user_id
LIMIT 5;
-- subqueries in where with ALL operator
SELECT
user_id
FROM
users_table
WHERE
value_2 > 545 AND
value_2 < ALL (SELECT avg(value_3) FROM events_table WHERE users_table.user_id = events_table.user_id GROUP BY user_id)
GROUP BY
1
ORDER BY
1 DESC
LIMIT 3;
-- IN operator on non-partition key
SELECT
user_id
FROM
events_table as e1
WHERE
event_type IN
(SELECT
event_type
FROM
events_table as e2
WHERE
value_2 = 15 AND value_3 > 25 AND
e1.user_id = e2.user_id
)
ORDER BY 1;
-- NOT IN on non-partition key
SELECT
user_id
FROM
events_table as e1
WHERE
event_type NOT IN
(SELECT
event_type
FROM
events_table as e2
WHERE
value_2 = 15 AND value_3 > 25 AND
e1.user_id = e2.user_id
)
GROUP BY 1
HAVING count(*) > 122
ORDER BY 1;
-- non-correlated query with =ANY on partition keys
SELECT
user_id, count(*)
FROM
users_table
WHERE
user_id =ANY(SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20) GROUP BY 1 ORDER BY 2 DESC LIMIT 5;
-- users that appeared more than 118 times
SELECT
user_id
FROM
users_table
WHERE 118 <=
(SELECT
count(*)
FROM
events_table
WHERE
users_table.user_id = events_table.user_id
GROUP BY
user_id)
GROUP BY
user_id
ORDER BY
user_id;
-- the following query doesn't have a meaningful result
-- but it is a valid query with an arbitrary subquery in
-- WHERE clause
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 101 AND value_1 < 110
AND value_2 >= 5
AND user_id IN
(
SELECT
e1.user_id
FROM (
-- Get the first time each user viewed the homepage.
SELECT
user_id,
1 AS view_homepage,
min(time) AS view_homepage_time
FROM events_table
WHERE
event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90)
GROUP BY user_id
) e1 LEFT JOIN LATERAL (
SELECT
user_id,
1 AS use_demo,
time AS use_demo_time
FROM events_table
WHERE
user_id = e1.user_id AND
event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91)
ORDER BY time
) e2 ON true LEFT JOIN LATERAL (
SELECT
user_id,
1 AS enter_credit_card,
time AS enter_credit_card_time
FROM events_table
WHERE
user_id = e2.user_id AND
event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92)
ORDER BY time
) e3 ON true LEFT JOIN LATERAL (
SELECT
1 AS submit_card_info,
user_id,
time AS enter_credit_card_time
FROM events_table
WHERE
user_id = e3.user_id AND
event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93)
ORDER BY time
) e4 ON true LEFT JOIN LATERAL (
SELECT
1 AS see_bought_screen
FROM events_table
WHERE
user_id = e4.user_id AND
event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94)
ORDER BY time
) e5 ON true
group by e1.user_id
HAVING sum(submit_card_info) > 0
)
ORDER BY 1, 2;
-- similar to the above query
-- the following query doesn't have a meaningful result
-- but it is a valid query with an arbitrary subquery in
-- WHERE clause
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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 10 AND events_table.event_type < 12
)
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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 12 AND events_table.event_type < 14
)
) AS subquery_1
LEFT JOIN
(SELECT
user_id,
COUNT(*) AS count_pay
FROM
users_table
WHERE
user_id >= 10 AND
user_id <= 70 AND
users_table.value_1 > 15 AND users_table.value_1 < 17
GROUP BY
user_id
HAVING
COUNT(*) > 1) 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(*) > 3 AND sum(value_2) > 49000
ORDER BY 1;
-- the following query doesn't have a meaningful result
-- but it is a valid query with an arbitrary subquery in
-- FROM clause involving a complex query in WHERE clause
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 > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id)
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id = users_table.user_id)
)
) t
GROUP BY user_id
) q
ORDER BY 2 DESC, 1;
--
-- below tests only aims for cases where all relations
-- are not joined on partition key
--
-- e4 is not joined on the partition key
SELECT user_id, value_2 FROM users_table WHERE
value_1 > 101 AND value_1 < 110
AND value_2 >= 5
AND user_id IN
(
SELECT
e1.user_id
FROM (
-- Get the first time each user viewed the homepage.
SELECT
user_id,
1 AS view_homepage,
min(time) AS view_homepage_time
FROM events_table
WHERE
event_type IN (10, 20, 30, 40, 50, 60, 70, 80, 90)
GROUP BY user_id
) e1 LEFT JOIN LATERAL (
SELECT
user_id,
1 AS use_demo,
time AS use_demo_time
FROM events_table
WHERE
user_id = e1.user_id AND
event_type IN (11, 21, 31, 41, 51, 61, 71, 81, 91)
ORDER BY time
) e2 ON true LEFT JOIN LATERAL (
SELECT
user_id,
1 AS enter_credit_card,
time AS enter_credit_card_time
FROM events_table
WHERE
user_id = e2.user_id AND
event_type IN (12, 22, 32, 42, 52, 62, 72, 82, 92)
ORDER BY time
) e3 ON true LEFT JOIN LATERAL (
SELECT
1 AS submit_card_info,
user_id,
time AS enter_credit_card_time
FROM events_table
WHERE
value_2 = e3.user_id AND
event_type IN (13, 23, 33, 43, 53, 63, 73, 83, 93)
ORDER BY time
) e4 ON true LEFT JOIN LATERAL (
SELECT
1 AS see_bought_screen
FROM events_table
WHERE
user_id = e4.user_id AND
event_type IN (14, 24, 34, 44, 54, 64, 74, 84, 94)
ORDER BY time
) e5 ON true
group by e1.user_id
HAVING sum(submit_card_info) > 0
);
-- left leaf query does not return partition key
SELECT
user_id
FROM
users_table
WHERE
user_id IN
(
SELECT
user_id
FROM (
SELECT
subquery_1.user_id, count_pay
FROM
(
(SELECT
2 * users_table.user_id as 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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 10 AND events_table.event_type < 12
)
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 >= 10 AND
users_table.user_id <= 70 AND
events_table.event_type > 12 AND events_table.event_type < 14
)
) AS subquery_1
LEFT JOIN
(SELECT
user_id,
COUNT(*) AS count_pay
FROM
users_table
WHERE
user_id >= 10 AND
user_id <= 70 AND
users_table.value_1 > 15 AND users_table.value_1 < 17
GROUP BY
user_id
HAVING
COUNT(*) > 1) 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(*) > 3 AND sum(value_2) > 49000
ORDER BY 1;
-- NOT EXISTS query has non-equi join
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 > 100 AND event_type <= 300 AND value_3 > 100 AND user_id = users_table.user_id)
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type > 300 AND event_type <= 350 AND value_3 > 100 AND user_id != users_table.user_id)
)
) t
GROUP BY user_id
) q
ORDER BY 2 DESC, 1;
-- subquery in where clause doesn't have a relation
SELECT
user_id
FROM
users_table
WHERE
value_2 >
(SELECT 1);
-- OFFSET is not supported in the subquey
SELECT
user_id
FROM
users_table
WHERE
value_2 >
(SELECT
max(value_2)
FROM
events_table
WHERE
users_table.user_id = events_table.user_id AND event_type = 50
GROUP BY
user_id
OFFSET 3
);
-- we can detect unsupported subquerues even if they appear
-- in WHERE subquery -> FROM subquery -> WHERE subquery
SELECT user_id
FROM users_table
WHERE user_id
IN (SELECT
f_inner.user_id
FROM
(
SELECT
e1.user_id
FROM
users_table u1, events_table e1
WHERE
e1.user_id = u1.user_id
) as f_inner,
(
SELECT
e1.user_id
FROM
users_table u1, events_table e1
WHERE
e1.user_id = u1.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
);
-- semi join is not on the partition key for the third subquery
SELECT user_id
FROM users_table
WHERE user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 10 AND value_1 <= 20)
AND user_id IN (SELECT user_id FROM users_table WHERE value_1 >= 30 AND value_1 <= 40)
AND value_2 IN (SELECT user_id FROM users_table WHERE value_1 >= 50 AND value_1 <= 60);
CREATE FUNCTION test_join_function(integer, integer) RETURNS bool
AS 'select $1 > $2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- we disallow JOINs via functions
SELECT user_id, value_2 FROM users_table WHERE
value_1 = 101
AND value_2 >= 5
AND NOT EXISTS (SELECT user_id FROM events_table WHERE event_type=101 AND value_3 > 100 AND test_join_function(events_table.user_id, users_table.user_id))
ORDER BY 1 DESC, 2 DESC
LIMIT 3;
DROP FUNCTION test_join_function(int,int);

View File

@ -1,22 +0,0 @@
--
-- MULTI_VERIFY_NO_SUBQUERY
--
-- This test checks that we simply emit an error message instead of trying to
-- process a distributed unsupported SQL subquery.
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1030000;
SELECT * FROM lineitem WHERE l_orderkey IN
(SELECT l_orderkey FROM lineitem WHERE l_quantity > 0);
SELECT l_quantity FROM lineitem WHERE EXISTS
(SELECT 1 FROM orders WHERE o_orderkey = l_orderkey);
SELECT l_quantity FROM lineitem WHERE l_orderkey IN (SELECT o_orderkey FROM orders);
SELECT l_orderkey FROM lineitem WHERE l_quantity > ALL(SELECT o_orderkey FROM orders);
SELECT l_quantity FROM lineitem WHERE l_orderkey = (SELECT min(o_orderkey) FROM orders);

View File

@ -259,7 +259,7 @@ CREATE VIEW recent_selected_users AS SELECT su.* FROM selected_users su JOIN rec
SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1; SELECT user_id FROM recent_selected_users GROUP BY 1 ORDER BY 1;
-- this would be supported when we implement where partition_key in (subquery) support -- this would be supported when we implement where partition_key in (subquery) support
SELECT et.* FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users); SELECT et.user_id, et.time FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users) GROUP BY 1,2 ORDER BY 1 DESC,2 DESC LIMIT 5;
-- it is supported when it is a router query -- it is supported when it is a router query
SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90); SELECT count(*) FROM events_table et WHERE et.user_id IN (SELECT user_id FROM recent_selected_users WHERE user_id = 90);
@ -351,21 +351,9 @@ CREATE VIEW router_view AS SELECT * FROM users_table WHERE user_id = 2;
-- router plannable -- router plannable
SELECT user_id FROM router_view GROUP BY 1; SELECT user_id FROM router_view GROUP BY 1;
-- There is a known issue with router plannable subqueries joined with non-router -- join a router view
-- plannable subqueries. Following tests should be uncommented when we fix it SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id) ORDER BY 2 LIMIT 3;
SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id) ORDER BY 2 LIMIT 3;
-- join a router view (not implement error)
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN recent_events USING (user_id);
-- it still does not work when converted to 2 subquery join
-- SELECT * FROM (SELECT user_id FROM router_view GROUP BY 1) rv JOIN (SELECT * FROM recent_events) re USING (user_id);
-- views are completely removed and still it does not work
-- SELECT * FROM
-- (SELECT user_id FROM (SELECT * FROM users_table WHERE user_id = 2) rv1 GROUP BY 1) rv2
-- JOIN (SELECT user_id, time FROM events_table
-- WHERE time > '2014-01-20 01:45:49.978738'::timestamp) re
-- USING (user_id);
-- views with limits -- views with limits
CREATE VIEW recent_10_users AS CREATE VIEW recent_10_users AS