diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index 11c6a4eee..7e0a3a994 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -911,9 +911,6 @@ CanUseBinaryCopyFormat(TupleDesc tupleDescription) { Form_pg_attribute currentColumn = TupleDescAttr(tupleDescription, columnIndex); Oid typeId = InvalidOid; - char typeCategory = '\0'; - bool typePreferred = false; - bool binaryOutputFunctionDefined = false; if (currentColumn->attisdropped) { @@ -921,31 +918,48 @@ CanUseBinaryCopyFormat(TupleDesc tupleDescription) } typeId = currentColumn->atttypid; - - /* built-in types may also don't have binary output function */ - binaryOutputFunctionDefined = BinaryOutputFunctionDefined(typeId); - if (!binaryOutputFunctionDefined) + if (!CanUseBinaryCopyFormatForType(typeId)) { useBinaryCopyFormat = false; break; } - - if (typeId >= FirstNormalObjectId) - { - get_type_category_preferred(typeId, &typeCategory, &typePreferred); - if (typeCategory == TYPCATEGORY_ARRAY || - typeCategory == TYPCATEGORY_COMPOSITE) - { - useBinaryCopyFormat = false; - break; - } - } } return useBinaryCopyFormat; } +/* + * CanUseBinaryCopyFormatForType determines whether it is safe to use the + * binary copy format for the given type. The binary copy format cannot + * be used for arrays or composite types that contain user-defined types, + * or when there is no binary output function defined. + */ +bool +CanUseBinaryCopyFormatForType(Oid typeId) +{ + if (!BinaryOutputFunctionDefined(typeId)) + { + return false; + } + + if (typeId >= FirstNormalObjectId) + { + char typeCategory = '\0'; + bool typePreferred = false; + + get_type_category_preferred(typeId, &typeCategory, &typePreferred); + if (typeCategory == TYPCATEGORY_ARRAY || + typeCategory == TYPCATEGORY_COMPOSITE) + { + return false; + } + } + + return true; +} + + /* * BinaryOutputFunctionDefined checks whether binary output function is defined * for the given type. diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 055ac476b..62379fc3a 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -18,6 +18,7 @@ #include "distributed/multi_server_executor.h" #include "distributed/multi_router_executor.h" #include "distributed/multi_router_planner.h" +#include "distributed/subplan_execution.h" #include "distributed/worker_protocol.h" #include "executor/executor.h" #include "nodes/makefuncs.h" diff --git a/src/backend/distributed/executor/intermediate_results.c b/src/backend/distributed/executor/intermediate_results.c index 54acd5918..e9ccb849f 100644 --- a/src/backend/distributed/executor/intermediate_results.c +++ b/src/backend/distributed/executor/intermediate_results.c @@ -76,10 +76,6 @@ typedef struct RemoteFileDestReceiver } RemoteFileDestReceiver; -static RemoteFileDestReceiver * CreateRemoteFileDestReceiver(char *resultId, - EState *executorState, - List *initialNodeList, - bool writeLocalFile); static void RemoteFileDestReceiverStartup(DestReceiver *dest, int operation, TupleDesc inputTupleDescriptor); static StringInfo ConstructCopyResultStatement(const char *resultId); @@ -123,8 +119,9 @@ broadcast_intermediate_result(PG_FUNCTION_ARGS) nodeList = ActivePrimaryNodeList(); estate = CreateExecutorState(); - resultDest = CreateRemoteFileDestReceiver(resultIdString, estate, nodeList, - writeLocalFile); + resultDest = (RemoteFileDestReceiver *) CreateRemoteFileDestReceiver(resultIdString, + estate, nodeList, + writeLocalFile); ExecuteQueryStringIntoDestReceiver(queryString, paramListInfo, (DestReceiver *) resultDest); @@ -155,8 +152,9 @@ create_intermediate_result(PG_FUNCTION_ARGS) CheckCitusVersion(ERROR); estate = CreateExecutorState(); - resultDest = CreateRemoteFileDestReceiver(resultIdString, estate, nodeList, - writeLocalFile); + resultDest = (RemoteFileDestReceiver *) CreateRemoteFileDestReceiver(resultIdString, + estate, nodeList, + writeLocalFile); ExecuteQueryStringIntoDestReceiver(queryString, paramListInfo, (DestReceiver *) resultDest); @@ -171,7 +169,7 @@ create_intermediate_result(PG_FUNCTION_ARGS) * CreateRemoteFileDestReceiver creates a DestReceiver that streams results * to a set of worker nodes. */ -static RemoteFileDestReceiver * +DestReceiver * CreateRemoteFileDestReceiver(char *resultId, EState *executorState, List *initialNodeList, bool writeLocalFile) { @@ -193,7 +191,7 @@ CreateRemoteFileDestReceiver(char *resultId, EState *executorState, resultDest->memoryContext = CurrentMemoryContext; resultDest->writeLocalFile = writeLocalFile; - return resultDest; + return (DestReceiver *) resultDest; } @@ -541,8 +539,14 @@ CreateIntermediateResultsDirectory(void) if (!CreatedResultsDirectory) { makeOK = mkdir(resultDirectory, S_IRWXU); - if (makeOK != 0 && errno != EEXIST) + if (makeOK != 0) { + if (errno == EEXIST) + { + /* someone else beat us to it, that's ok */ + return resultDirectory; + } + ereport(ERROR, (errcode_for_file_access(), errmsg("could not create intermediate results directory " "\"%s\": %m", diff --git a/src/backend/distributed/executor/multi_real_time_executor.c b/src/backend/distributed/executor/multi_real_time_executor.c index 4fd4c932e..735f75a34 100644 --- a/src/backend/distributed/executor/multi_real_time_executor.c +++ b/src/backend/distributed/executor/multi_real_time_executor.c @@ -33,6 +33,7 @@ #include "distributed/multi_router_executor.h" #include "distributed/multi_server_executor.h" #include "distributed/resource_lock.h" +#include "distributed/subplan_execution.h" #include "distributed/worker_protocol.h" #include "distributed/version_compat.h" #include "storage/fd.h" @@ -1046,6 +1047,8 @@ RealTimeExecScan(CustomScanState *node) LockPartitionsInRelationList(distributedPlan->relationIdList, AccessShareLock); PrepareMasterJobDirectory(workerJob); + + ExecuteSubPlans(distributedPlan); MultiRealTimeExecute(workerJob); LoadTuplesIntoTupleStore(scanState, workerJob); diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index 3261fcc4a..3963b2b93 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -41,6 +41,7 @@ #include "distributed/multi_router_planner.h" #include "distributed/multi_shard_transaction.h" #include "distributed/placement_connection.h" +#include "distributed/subplan_execution.h" #include "distributed/relay_utility.h" #include "distributed/remote_commands.h" #include "distributed/remote_transaction.h" @@ -543,6 +544,8 @@ RouterSelectExecScan(CustomScanState *node) /* we are taking locks on partitions of partitioned tables */ LockPartitionsInRelationList(distributedPlan->relationIdList, AccessShareLock); + ExecuteSubPlans(distributedPlan); + if (list_length(taskList) > 0) { Task *task = (Task *) linitial(taskList); @@ -615,6 +618,20 @@ ExecuteSingleSelectTask(CitusScanState *scanState, Task *task) connection = GetPlacementListConnection(connectionFlags, placementAccessList, NULL); + /* + * Make sure we open a transaction block and assign a distributed transaction + * ID if we are in a coordinated transaction. + * + * This can happen when the SELECT goes to a node that was not involved in + * the transaction so far, or when existing connections to the node are + * claimed exclusively, e.g. the connection might be claimed to copy the + * intermediate result of a CTE to the node. Especially in the latter case, + * we want to make sure that we open a transaction block and assign a + * distributed transaction ID, such that the query can read intermediate + * results. + */ + RemoteTransactionBeginIfNecessary(connection); + queryOK = SendQueryInSingleRowMode(connection, queryString, paramListInfo); if (!queryOK) { diff --git a/src/backend/distributed/executor/multi_task_tracker_executor.c b/src/backend/distributed/executor/multi_task_tracker_executor.c index 5d4d7ee0f..3dc4f3111 100644 --- a/src/backend/distributed/executor/multi_task_tracker_executor.c +++ b/src/backend/distributed/executor/multi_task_tracker_executor.c @@ -3025,6 +3025,13 @@ TaskTrackerExecScan(CustomScanState *node) { DistributedPlan *distributedPlan = scanState->distributedPlan; Job *workerJob = distributedPlan->workerJob; + Query *jobQuery = workerJob->jobQuery; + + if (ContainsReadIntermediateResultFunction((Node *) jobQuery)) + { + ereport(ERROR, (errmsg("Complex subqueries and CTEs are not supported when " + "task_executor_type is set to 'task-tracker'"))); + } /* we are taking locks on partitions of partitioned tables */ LockPartitionsInRelationList(distributedPlan->relationIdList, AccessShareLock); diff --git a/src/backend/distributed/executor/subplan_execution.c b/src/backend/distributed/executor/subplan_execution.c new file mode 100644 index 000000000..e61820d46 --- /dev/null +++ b/src/backend/distributed/executor/subplan_execution.c @@ -0,0 +1,55 @@ +/*------------------------------------------------------------------------- + * + * subplan_execution.c + * + * Functions for execution subplans prior to distributed table execution. + * + * Copyright (c) 2017, Citus Data, Inc. + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/intermediate_results.h" +#include "distributed/multi_executor.h" +#include "distributed/multi_physical_planner.h" +#include "distributed/recursive_planning.h" +#include "distributed/subplan_execution.h" +#include "distributed/worker_manager.h" +#include "executor/executor.h" + + +/* + * ExecuteSubPlans executes a list of subplans from a distributed plan + * by sequentially executing each plan from the top. + */ +void +ExecuteSubPlans(DistributedPlan *distributedPlan) +{ + uint64 planId = distributedPlan->planId; + List *subPlanList = distributedPlan->subPlanList; + ListCell *subPlanCell = NULL; + List *nodeList = ActiveReadableNodeList(); + bool writeLocalFile = false; + + foreach(subPlanCell, subPlanList) + { + DistributedSubPlan *subPlan = (DistributedSubPlan *) lfirst(subPlanCell); + PlannedStmt *plannedStmt = subPlan->plan; + uint32 subPlanId = subPlan->subPlanId; + DestReceiver *copyDest = NULL; + ParamListInfo params = NULL; + EState *estate = NULL; + + char *resultId = GenerateResultId(planId, subPlanId); + + estate = CreateExecutorState(); + copyDest = (DestReceiver *) CreateRemoteFileDestReceiver(resultId, estate, + nodeList, + writeLocalFile); + + ExecutePlanIntoDestReceiver(plannedStmt, params, copyDest); + + FreeExecutorState(estate); + } +} diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index b5e694918..a4c2f4034 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -26,26 +26,37 @@ #include "distributed/multi_physical_planner.h" #include "distributed/multi_master_planner.h" #include "distributed/multi_router_planner.h" +#include "distributed/recursive_planning.h" #include "executor/executor.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" #include "parser/parsetree.h" #include "optimizer/pathnode.h" #include "optimizer/planner.h" +#include "utils/datum.h" #include "utils/lsyscache.h" #include "utils/memutils.h" static List *plannerRestrictionContextList = NIL; int MultiTaskQueryLogLevel = MULTI_TASK_QUERY_INFO_OFF; /* multi-task query log level */ +static uint64 NextPlanId = 1; /* local function forward declarations */ static bool NeedsDistributedPlanningWalker(Node *node, void *context); -static PlannedStmt * CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, - Query *query, ParamListInfo boundParams, +static PlannedStmt * CreateDistributedPlan(uint64 planId, PlannedStmt *localPlan, + Query *originalQuery, Query *query, + ParamListInfo boundParams, PlannerRestrictionContext * plannerRestrictionContext); +static DistributedPlan * CreateDistributedSelectPlan(uint64 planId, Query *originalQuery, + Query *query, + ParamListInfo boundParams, + bool hasUnresolvedParams, + PlannerRestrictionContext * + plannerRestrictionContext); +static Node * ResolveExternalParams(Node *inputNode, ParamListInfo boundParams); static void AssignRTEIdentities(Query *queryTree); static void AssignRTEIdentity(RangeTblEntry *rangeTableEntry, int rteIdentifier); @@ -76,6 +87,11 @@ distributed_planner(Query *parse, int cursorOptions, ParamListInfo boundParams) PlannerRestrictionContext *plannerRestrictionContext = NULL; bool setPartitionedTablesInherited = false; + if (cursorOptions & CURSOR_OPT_FORCE_DISTRIBUTED) + { + needsDistributedPlanning = true; + } + if (needsDistributedPlanning) { /* @@ -121,7 +137,9 @@ distributed_planner(Query *parse, int cursorOptions, ParamListInfo boundParams) if (needsDistributedPlanning) { - result = CreateDistributedPlan(result, originalQuery, parse, + uint64 planId = NextPlanId++; + + result = CreateDistributedPlan(planId, result, originalQuery, parse, boundParams, plannerRestrictionContext); } } @@ -336,7 +354,6 @@ static void AssignRTEIdentity(RangeTblEntry *rangeTableEntry, int rteIdentifier) { Assert(rangeTableEntry->rtekind == RTE_RELATION); - Assert(rangeTableEntry->values_lists == NIL); rangeTableEntry->values_lists = list_make1_int(rteIdentifier); } @@ -447,8 +464,8 @@ IsModifyDistributedPlan(DistributedPlan *distributedPlan) * query into a distributed plan. */ static PlannedStmt * -CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query, - ParamListInfo boundParams, +CreateDistributedPlan(uint64 planId, PlannedStmt *localPlan, Query *originalQuery, + Query *query, ParamListInfo boundParams, PlannerRestrictionContext *plannerRestrictionContext) { DistributedPlan *distributedPlan = NULL; @@ -480,56 +497,10 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query } else { - /* - * For select queries we, if router executor is enabled, first try to - * plan the query as a router query. If not supported, otherwise try - * the full blown plan/optimize/physical planing process needed to - * produce distributed query plans. - */ - if (EnableRouterExecution) - { - RelationRestrictionContext *relationRestrictionContext = - plannerRestrictionContext->relationRestrictionContext; - - distributedPlan = CreateRouterPlan(originalQuery, query, - relationRestrictionContext); - - /* for debugging it's useful to display why query was not router plannable */ - if (distributedPlan && distributedPlan->planningError) - { - RaiseDeferredError(distributedPlan->planningError, DEBUG1); - } - } - - /* - * Router didn't yield a plan, try the full distributed planner. As - * real-time/task-tracker don't support prepared statement parameters, - * skip planning in that case (we'll later trigger an error in that - * case if necessary). - */ - if ((!distributedPlan || distributedPlan->planningError) && !hasUnresolvedParams) - { - MultiTreeRoot *logicalPlan = MultiLogicalPlanCreate(originalQuery, query, - plannerRestrictionContext, - boundParams); - MultiLogicalPlanOptimize(logicalPlan); - - /* - * This check is here to make it likely that all node types used in - * Citus are dumpable. Explain can dump logical and physical plans - * using the extended outfuncs infrastructure, but it's infeasible to - * test most plans. MultiQueryContainerNode always serializes the - * physical plan, so there's no need to check that separately. - */ - CheckNodeIsDumpable((Node *) logicalPlan); - - /* Create the physical plan */ - distributedPlan = CreatePhysicalDistributedPlan(logicalPlan, - plannerRestrictionContext); - - /* distributed plan currently should always succeed or error out */ - Assert(distributedPlan && distributedPlan->planningError == NULL); - } + distributedPlan = + CreateDistributedSelectPlan(planId, originalQuery, query, boundParams, + hasUnresolvedParams, + plannerRestrictionContext); } /* @@ -571,6 +542,9 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query RaiseDeferredError(distributedPlan->planningError, ERROR); } + /* remember the plan's identifier for identifying subplans */ + distributedPlan->planId = planId; + /* create final plan by combining local plan with distributed plan */ resultPlan = FinalizePlan(localPlan, distributedPlan); @@ -593,6 +567,258 @@ CreateDistributedPlan(PlannedStmt *localPlan, Query *originalQuery, Query *query } +/* + * CreateDistributedSelectPlan generates a distributed plan for a SELECT query. + * It goes through 3 steps: + * + * 1. Try router planner + * 2. Generate subplans for CTEs and complex subqueries + * - If any, go back to step 1 by calling itself recursively + * 3. Logical planner + */ +static DistributedPlan * +CreateDistributedSelectPlan(uint64 planId, Query *originalQuery, Query *query, + ParamListInfo boundParams, bool hasUnresolvedParams, + PlannerRestrictionContext *plannerRestrictionContext) +{ + RelationRestrictionContext *relationRestrictionContext = + plannerRestrictionContext->relationRestrictionContext; + + DistributedPlan *distributedPlan = NULL; + MultiTreeRoot *logicalPlan = NULL; + DeferredErrorMessage *error = NULL; + List *subPlanList = NIL; + + /* + * For select queries we, if router executor is enabled, first try to + * plan the query as a router query. If not supported, otherwise try + * the full blown plan/optimize/physical planing process needed to + * produce distributed query plans. + */ + + distributedPlan = CreateRouterPlan(originalQuery, query, + relationRestrictionContext); + if (distributedPlan != NULL) + { + if (distributedPlan->planningError == NULL) + { + /* successfully created a router plan */ + return distributedPlan; + } + else + { + /* + * For debugging it's useful to display why query was not + * router plannable. + */ + RaiseDeferredError(distributedPlan->planningError, DEBUG1); + } + } + + if (hasUnresolvedParams) + { + /* + * There are parameters that don't have a value in boundParams. + * + * The remainder of the planning logic cannot handle unbound + * parameters. We return a NULL plan, which will have an + * extremely high cost, such that postgres will replan with + * bound parameters. + */ + return NULL; + } + + /* + * If there are parameters that do have a value in boundParams, replace + * them in the original query. This allows us to more easily cut the + * query into pieces (during recursive planning) or deparse parts of + * the query (during subquery pushdown planning). + */ + originalQuery = (Query *) ResolveExternalParams((Node *) originalQuery, + boundParams); + + /* + * Plan subqueries and CTEs that cannot be pushed down by recursively + * calling the planner and add the resulting plans to subPlanList. + */ + error = RecursivelyPlanSubqueriesAndCTEs(originalQuery, plannerRestrictionContext, + planId, &subPlanList); + if (error != NULL) + { + RaiseDeferredError(error, ERROR); + } + + /* + * If subqueries were recursively planned then we need to replan the query + * to get the new planner restriction context and apply planner transformations. + * + * We could simplify this code if the logical planner was capable of dealing + * with an original query. In that case, we would only have to filter the + * planner restriction context. + */ + if (list_length(subPlanList) > 0) + { + Query *newQuery = copyObject(originalQuery); + bool setPartitionedTablesInherited = false; + + /* remove the pre-transformation planner restrictions context */ + PopPlannerRestrictionContext(); + + /* create a fresh new planner context */ + plannerRestrictionContext = CreateAndPushPlannerRestrictionContext(); + + /* + * We force standard_planner to treat partitioned tables as regular tables + * by clearing the inh flag on RTEs. We already did this at the start of + * distributed_planner, but on a copy of the original query, so we need + * to do it again here. + */ + AdjustPartitioningForDistributedPlanning(newQuery, setPartitionedTablesInherited); + + /* + * Some relations may have been removed from the query, but we can skip + * AssignRTEIdentities since we currently do not rely on RTE identities + * being contiguous. + */ + + standard_planner(newQuery, 0, boundParams); + + /* overwrite the old transformed query with the new transformed query */ + memcpy(query, newQuery, sizeof(Query)); + + /* recurse into CreateDistributedSelectPlan with subqueries/CTEs replaced */ + distributedPlan = CreateDistributedSelectPlan(planId, originalQuery, query, NULL, + false, plannerRestrictionContext); + distributedPlan->subPlanList = subPlanList; + + return distributedPlan; + } + + /* + * CTEs are stripped from the original query by RecursivelyPlanSubqueriesAndCTEs. + * If we get here and there are still CTEs that means that none of the CTEs are + * referenced. We therefore also strip the CTEs from the rewritten query. + */ + query->cteList = NIL; + Assert(originalQuery->cteList == NIL); + + logicalPlan = MultiLogicalPlanCreate(originalQuery, query, + plannerRestrictionContext); + MultiLogicalPlanOptimize(logicalPlan); + + /* + * This check is here to make it likely that all node types used in + * Citus are dumpable. Explain can dump logical and physical plans + * using the extended outfuncs infrastructure, but it's infeasible to + * test most plans. MultiQueryContainerNode always serializes the + * physical plan, so there's no need to check that separately + */ + CheckNodeIsDumpable((Node *) logicalPlan); + + /* Create the physical plan */ + distributedPlan = CreatePhysicalDistributedPlan(logicalPlan, + plannerRestrictionContext); + + /* distributed plan currently should always succeed or error out */ + Assert(distributedPlan && distributedPlan->planningError == NULL); + + return distributedPlan; +} + + +/* + * ResolveExternalParams replaces the external parameters that appears + * in the query with the corresponding entries in the boundParams. + * + * Note that this function is inspired by eval_const_expr() on Postgres. + * We cannot use that function because it requires access to PlannerInfo. + */ +static Node * +ResolveExternalParams(Node *inputNode, ParamListInfo boundParams) +{ + /* consider resolving external parameters only when boundParams exists */ + if (!boundParams) + { + return inputNode; + } + + if (inputNode == NULL) + { + return NULL; + } + + if (IsA(inputNode, Param)) + { + Param *paramToProcess = (Param *) inputNode; + ParamExternData *correspondingParameterData = NULL; + int numberOfParameters = boundParams->numParams; + int parameterId = paramToProcess->paramid; + int16 typeLength = 0; + bool typeByValue = false; + Datum constValue = 0; + bool paramIsNull = false; + int parameterIndex = 0; + + if (paramToProcess->paramkind != PARAM_EXTERN) + { + return inputNode; + } + + if (parameterId < 0) + { + return inputNode; + } + + /* parameterId starts from 1 */ + parameterIndex = parameterId - 1; + if (parameterIndex >= numberOfParameters) + { + return inputNode; + } + + correspondingParameterData = &boundParams->params[parameterIndex]; + + if (!(correspondingParameterData->pflags & PARAM_FLAG_CONST)) + { + return inputNode; + } + + get_typlenbyval(paramToProcess->paramtype, &typeLength, &typeByValue); + + paramIsNull = correspondingParameterData->isnull; + if (paramIsNull) + { + constValue = 0; + } + else if (typeByValue) + { + constValue = correspondingParameterData->value; + } + else + { + /* + * Out of paranoia ensure that datum lives long enough, + * although bind params currently should always live + * long enough. + */ + constValue = datumCopy(correspondingParameterData->value, typeByValue, + typeLength); + } + + return (Node *) makeConst(paramToProcess->paramtype, paramToProcess->paramtypmod, + paramToProcess->paramcollid, typeLength, constValue, + paramIsNull, typeByValue); + } + else if (IsA(inputNode, Query)) + { + return (Node *) query_tree_mutator((Query *) inputNode, ResolveExternalParams, + boundParams, 0); + } + + return expression_tree_mutator(inputNode, ResolveExternalParams, boundParams); +} + + /* * GetDistributedPlan returns the associated DistributedPlan for a CustomScan. */ diff --git a/src/backend/distributed/planner/multi_explain.c b/src/backend/distributed/planner/multi_explain.c index c5bc8d49f..25523b5eb 100644 --- a/src/backend/distributed/planner/multi_explain.c +++ b/src/backend/distributed/planner/multi_explain.c @@ -72,6 +72,7 @@ typedef struct RemoteExplainPlan /* Explain functions for distributed queries */ +static void ExplainSubPlans(List *subPlanList, ExplainState *es); static void ExplainJob(Job *job, ExplainState *es); static void ExplainMapMergeJob(MapMergeJob *mapMergeJob, ExplainState *es); static void ExplainTaskList(List *taskList, ExplainState *es); @@ -124,6 +125,11 @@ CitusExplainScan(CustomScanState *node, List *ancestors, struct ExplainState *es ExplainOpenGroup("Distributed Query", "Distributed Query", true, es); + if (distributedPlan->subPlanList != NIL) + { + ExplainSubPlans(distributedPlan->subPlanList, es); + } + ExplainJob(distributedPlan->workerJob, es); ExplainCloseGroup("Distributed Query", "Distributed Query", true, es); @@ -166,6 +172,55 @@ CoordinatorInsertSelectExplainScan(CustomScanState *node, List *ancestors, } +/* + * ExplainSubPlans generates EXPLAIN output for subplans for CTEs + * and complex subqueries. Because the planning for these queries + * is done along with the top-level plan, we cannot determine the + * planning time and set it to 0. + */ +static void +ExplainSubPlans(List *subPlanList, ExplainState *es) +{ + ListCell *subPlanCell = NULL; + + ExplainOpenGroup("Subplans", "Subplans", false, es); + + if (es->format == EXPLAIN_FORMAT_TEXT) + { + appendStringInfoSpaces(es->str, es->indent * 2); + appendStringInfo(es->str, "-> Distributed Subplan\n"); + es->indent += 3; + } + + foreach(subPlanCell, subPlanList) + { + DistributedSubPlan *subPlan = (DistributedSubPlan *) lfirst(subPlanCell); + PlannedStmt *plan = subPlan->plan; + IntoClause *into = NULL; + ParamListInfo params = NULL; + char *queryString = NULL; + instr_time planduration; + + /* set the planning time to 0 */ + INSTR_TIME_SET_CURRENT(planduration); + INSTR_TIME_SUBTRACT(planduration, planduration); + +#if (PG_VERSION_NUM >= 100000) + ExplainOnePlan(plan, into, es, queryString, params, NULL, &planduration); +#else + ExplainOnePlan(plan, into, es, queryString, params, &planduration); +#endif + } + + if (es->format == EXPLAIN_FORMAT_TEXT) + { + es->indent -= 3; + } + + ExplainCloseGroup("Subplans", "Subplans", false, es); +} + + /* * ExplainJob shows the EXPLAIN output for a Job in the physical plan of * a distributed query by showing the remote EXPLAIN for the first task, diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index 853c03930..e4a8054f0 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -131,7 +131,6 @@ static bool RelationInfoContainsRecurringTuples(PlannerInfo *plannerInfo, RelOptInfo *relationInfo, RecurringTuplesType *recurType); static bool HasRecurringTuples(Node *node, RecurringTuplesType *recurType); -static bool ContainsReadIntermediateResultFunction(Node *node); static bool IsReadIntermediateResultFunction(Node *node); static void ValidateClauseList(List *clauseList); static bool ExtractFromExpressionWalker(Node *node, @@ -174,7 +173,6 @@ static MultiNode * ApplyCartesianProduct(MultiNode *leftNode, MultiNode *rightNo static bool ShouldUseSubqueryPushDown(Query *originalQuery, Query *rewrittenQuery); static bool IsFunctionRTE(Node *node); static bool FindNodeCheck(Node *node, bool (*check)(Node *)); -static Node * ResolveExternalParams(Node *inputNode, ParamListInfo boundParams); static MultiNode * SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree, PlannerRestrictionContext * @@ -195,12 +193,6 @@ static MultiTable * MultiSubqueryPushdownTable(Query *subquery); * plan and adds a root node to top of it. The original query is only used for subquery * pushdown planning. * - * In order to support external parameters for the queries where planning - * is done on the original query, we need to replace the external parameters - * manually. To achive that for subquery pushdown planning, we pass boundParams - * to this function. We need to do that since Citus currently unable to send - * parameters to the workers on the execution. - * * We also pass queryTree and plannerRestrictionContext to the planner. They * are primarily used to decide whether the subquery is safe to pushdown. * If not, it helps to produce meaningful error messages for subquery @@ -208,16 +200,13 @@ static MultiTable * MultiSubqueryPushdownTable(Query *subquery); */ MultiTreeRoot * MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree, - PlannerRestrictionContext *plannerRestrictionContext, - ParamListInfo boundParams) + PlannerRestrictionContext *plannerRestrictionContext) { MultiNode *multiQueryNode = NULL; MultiTreeRoot *rootNode = NULL; if (ShouldUseSubqueryPushDown(originalQuery, queryTree)) { - originalQuery = (Query *) ResolveExternalParams((Node *) originalQuery, - boundParams); multiQueryNode = SubqueryMultiNodeTree(originalQuery, queryTree, plannerRestrictionContext); } @@ -329,99 +318,6 @@ FindNodeCheck(Node *node, bool (*check)(Node *)) } -/* - * ResolveExternalParams replaces the external parameters that appears - * in the query with the corresponding entries in the boundParams. - * - * Note that this function is inspired by eval_const_expr() on Postgres. - * We cannot use that function because it requires access to PlannerInfo. - */ -static Node * -ResolveExternalParams(Node *inputNode, ParamListInfo boundParams) -{ - /* consider resolving external parameters only when boundParams exists */ - if (!boundParams) - { - return inputNode; - } - - if (inputNode == NULL) - { - return NULL; - } - - if (IsA(inputNode, Param)) - { - Param *paramToProcess = (Param *) inputNode; - ParamExternData *correspondingParameterData = NULL; - int numberOfParameters = boundParams->numParams; - int parameterId = paramToProcess->paramid; - int16 typeLength = 0; - bool typeByValue = false; - Datum constValue = 0; - bool paramIsNull = false; - int parameterIndex = 0; - - if (paramToProcess->paramkind != PARAM_EXTERN) - { - return inputNode; - } - - if (parameterId < 0) - { - return inputNode; - } - - /* parameterId starts from 1 */ - parameterIndex = parameterId - 1; - if (parameterIndex >= numberOfParameters) - { - return inputNode; - } - - correspondingParameterData = &boundParams->params[parameterIndex]; - - if (!(correspondingParameterData->pflags & PARAM_FLAG_CONST)) - { - return inputNode; - } - - get_typlenbyval(paramToProcess->paramtype, &typeLength, &typeByValue); - - paramIsNull = correspondingParameterData->isnull; - if (paramIsNull) - { - constValue = 0; - } - else if (typeByValue) - { - constValue = correspondingParameterData->value; - } - else - { - /* - * Out of paranoia ensure that datum lives long enough, - * although bind params currently should always live - * long enough. - */ - constValue = datumCopy(correspondingParameterData->value, typeByValue, - typeLength); - } - - return (Node *) makeConst(paramToProcess->paramtype, paramToProcess->paramtypmod, - paramToProcess->paramcollid, typeLength, constValue, - paramIsNull, typeByValue); - } - else if (IsA(inputNode, Query)) - { - return (Node *) query_tree_mutator((Query *) inputNode, ResolveExternalParams, - boundParams, 0); - } - - return expression_tree_mutator(inputNode, ResolveExternalParams, boundParams); -} - - /* * 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 @@ -1341,7 +1237,7 @@ DeferErrorIfUnsupportedTableCombination(Query *queryTree) else if (rangeTableEntry->rtekind == RTE_CTE) { unsupportedTableCombination = true; - errorDetail = "CTEs in multi-shard queries are currently unsupported"; + errorDetail = "CTEs in subqueries are currently unsupported"; break; } else if (rangeTableEntry->rtekind == RTE_VALUES) @@ -2237,7 +2133,7 @@ HasRecurringTuples(Node *node, RecurringTuplesType *recurType) * ContainsReadIntermediateResultFunction determines whether an expresion tree contains * a call to the read_intermediate_results function. */ -static bool +bool ContainsReadIntermediateResultFunction(Node *node) { return FindNodeCheck(node, IsReadIntermediateResultFunction); diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 932b4b6ec..08230c87d 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -162,8 +162,6 @@ DistributedPlan * CreateRouterPlan(Query *originalQuery, Query *query, RelationRestrictionContext *restrictionContext) { - Assert(EnableRouterExecution); - if (MultiRouterPlannableQuery(query, restrictionContext)) { return CreateSingleTaskRouterPlan(originalQuery, query, @@ -1623,6 +1621,14 @@ SelectsFromDistributedTable(List *rangeTableList) * If the given query is not routable, it fills planningError with the related * DeferredErrorMessage. The caller can check this error message to see if query * is routable or not. + * + * Note: If the query prunes down to 0 shards due to filters (e.g. WHERE false), + * or the query has only read_intermediate_result calls (no relations left after + * recursively planning CTEs and subqueries), then it will be assigned to an + * arbitrary worker node in a round-robin fashion. + * + * Relations that prune down to 0 shards are replaced by subqueries returning + * 0 values in UpdateRelationToShardNames. */ DeferredErrorMessage * PlanRouterQuery(Query *originalQuery, RelationRestrictionContext *restrictionContext, diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c new file mode 100644 index 000000000..b0f765e71 --- /dev/null +++ b/src/backend/distributed/planner/recursive_planning.c @@ -0,0 +1,650 @@ +/*------------------------------------------------------------------------- + * + * recursive_planning.c + * + * Logic for calling the postgres planner recursively for CTEs and + * non-pushdownable subqueries in distributed queries. + * + * PostgreSQL with Citus can execute 4 types of queries: + * + * - Postgres queries on local tables and functions. + * + * These queries can use all SQL features, but they may not reference + * distributed tables. + * + * - Router queries that can be executed on a single by node by replacing + * table names with shard names. + * + * These queries can use nearly all SQL features, but only if they have + * a single-valued filter on the distribution column. + * + * - Real-time queries that can be executed by performing a task for each + * shard in a distributed table and performing a merge step. + * + * These queries have limited SQL support. They may only include + * subqueries if the subquery can be executed on each shard by replacing + * table names with shard names and concatenating the result. + * + * - Task-tracker queries that can be executed through a tree of + * re-partitioning operations. + * + * These queries have very limited SQL support and only support basic + * inner joins and subqueries without joins. + * + * To work around the limitations of these planners, we recursively call + * the planner for CTEs and unsupported subqueries to obtain a list of + * subplans. + * + * During execution, each subplan is executed separately through the method + * that is appropriate for that query. The results are written to temporary + * files on the workers. In the original query, the CTEs and subqueries are + * replaced by mini-subqueries that read from the temporary files. + * + * This allows almost all SQL to be directly or indirectly supported, + * because if all subqueries that contain distributed tables have been + * replaced then what remains is a router query which can use nearly all + * SQL features. + * + * Copyright (c) 2017, Citus Data, Inc. + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "catalog/pg_type.h" +#include "distributed/citus_nodes.h" +#include "distributed/citus_ruleutils.h" +#include "distributed/distributed_planner.h" +#include "distributed/errormessage.h" +#include "distributed/metadata_cache.h" +#include "distributed/multi_copy.h" +#include "distributed/multi_logical_planner.h" +#include "distributed/multi_router_planner.h" +#include "distributed/multi_physical_planner.h" +#include "distributed/recursive_planning.h" +#include "distributed/relation_restriction_equivalence.h" +#include "lib/stringinfo.h" +#include "optimizer/planner.h" +#include "nodes/nodeFuncs.h" +#include "nodes/nodes.h" +#include "nodes/pg_list.h" +#include "nodes/primnodes.h" +#include "utils/builtins.h" +#include "utils/guc.h" + + +/* + * RecursivePlanningContext is used to recursively plan subqueries + * and CTEs, pull results to the coordinator, and push it back into + * the workers. + */ +typedef struct RecursivePlanningContext +{ + int level; + uint64 planId; + List *subPlanList; + PlannerRestrictionContext *plannerRestrictionContext; +} RecursivePlanningContext; + +/* + * CteReferenceWalkerContext is used to collect CTE references in + * CteReferenceListWalker. + */ +typedef struct CteReferenceWalkerContext +{ + int level; + List *cteReferenceList; +} CteReferenceWalkerContext; + +/* + * VarLevelsUpWalkerContext is used to find Vars in a (sub)query that + * refer to upper levels and therefore cannot be planned separately. + */ +typedef struct VarLevelsUpWalkerContext +{ + int level; +} VarLevelsUpWalkerContext; + + +/* local function forward declarations */ +static DeferredErrorMessage * RecursivelyPlanCTEs(Query *query, + RecursivePlanningContext *context); +static DistributedSubPlan * CreateDistributedSubPlan(uint32 subPlanId, + Query *subPlanQuery); +static bool CteReferenceListWalker(Node *node, CteReferenceWalkerContext *context); +static bool ContainsReferencesToOuterQuery(Query *query); +static bool ContainsReferencesToOuterQueryWalker(Node *node, + VarLevelsUpWalkerContext *context); +static Query * BuildSubPlanResultQuery(Query *subquery, uint64 planId, uint32 subPlanId); + + +/* + * RecursivelyPlanSubqueriesAndCTEs finds subqueries and CTEs that cannot be pushed down to + * workers directly and instead plans them by recursively calling the planner and + * adding the subplan to subPlanList. + * + * Subplans are executed prior to the distributed plan and the results are written + * to temporary files on workers. + * + * CTE references are replaced by a subquery on the read_intermediate_result + * function, which reads from the temporary file. + * + * If recursive planning results in an error then the error is returned. Otherwise, the + * subplans will be added to subPlanList. + */ +DeferredErrorMessage * +RecursivelyPlanSubqueriesAndCTEs(Query *query, + PlannerRestrictionContext *plannerRestrictionContext, + uint64 planId, List **subPlanList) +{ + DeferredErrorMessage *error = NULL; + RecursivePlanningContext context; + + if (SubqueryPushdown) + { + /* + * When the subquery_pushdown flag is enabled we make some hacks + * to push down subqueries with LIMIT. Recursive planning would + * valiantly do the right thing and try to recursively plan the + * inner subqueries, but we don't really want it to because those + * subqueries might not be supported and would be much slower. + * + * Instead, we skip recursive planning altogether when + * subquery_pushdown is enabled. + */ + return NULL; + } + + context.level = 0; + context.planId = planId; + context.subPlanList = NIL; + context.plannerRestrictionContext = plannerRestrictionContext; + + error = RecursivelyPlanCTEs(query, &context); + if (error != NULL) + { + return error; + } + + /* XXX: plan subqueries */ + + *subPlanList = context.subPlanList; + + return NULL; +} + + +/* + * RecursivelyPlanCTEs plans all CTEs in the query by recursively calling the planner + * The resulting plan is added to planningContext->subPlanList and CTE references + * are replaced by subqueries that call read_intermediate_result, which reads the + * intermediate result of the CTE after it is executed. + * + * Recursive and modifying CTEs are not yet supported and return an error. + */ +static DeferredErrorMessage * +RecursivelyPlanCTEs(Query *query, RecursivePlanningContext *planningContext) +{ + ListCell *cteCell = NULL; + CteReferenceWalkerContext context = { -1, NIL }; + + if (query->cteList == NIL) + { + /* no CTEs, nothing to do */ + return NULL; + } + + if (query->hasModifyingCTE) + { + /* we could easily support these, but it's a little scary */ + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "data-modifying statements are not supported in " + "the WITH clauses of distributed queries", + NULL, NULL); + } + + if (query->hasRecursive) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "recursive CTEs are not supported in distributed " + "queries", + NULL, NULL); + } + + /* get all RTE_CTEs that point to CTEs from cteList */ + CteReferenceListWalker((Node *) query, &context); + + foreach(cteCell, query->cteList) + { + CommonTableExpr *cte = (CommonTableExpr *) lfirst(cteCell); + char *cteName = cte->ctename; + Query *subquery = (Query *) cte->ctequery; + uint64 planId = planningContext->planId; + uint32 subPlanId = 0; + Query *resultQuery = NULL; + DistributedSubPlan *subPlan = NULL; + ListCell *rteCell = NULL; + int replacedCtesCount = 0; + + if (ContainsReferencesToOuterQuery(subquery)) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "CTEs that refer to other subqueries are not " + "supported in multi-shard queries", + NULL, NULL); + } + + if (cte->cterefcount == 0) + { + /* + * CTEs that aren't referenced aren't executed in postgres. We + * don't need to generate a subplan for it and can take the rest + * of this iteration off. + */ + continue; + } + + subPlanId = list_length(planningContext->subPlanList) + 1; + + if (log_min_messages >= DEBUG1) + { + StringInfo subPlanString = makeStringInfo(); + pg_get_query_def(subquery, subPlanString); + ereport(DEBUG1, (errmsg("generating subplan " UINT64_FORMAT "_%u for " + "CTE %s: %s", + planId, subPlanId, cteName, subPlanString->data))); + } + + /* build a sub plan for the CTE */ + subPlan = CreateDistributedSubPlan(subPlanId, subquery); + planningContext->subPlanList = lappend(planningContext->subPlanList, subPlan); + + /* replace references to the CTE with a subquery that reads results */ + resultQuery = BuildSubPlanResultQuery(subquery, planId, subPlanId); + + foreach(rteCell, context.cteReferenceList) + { + RangeTblEntry *rangeTableEntry = (RangeTblEntry *) lfirst(rteCell); + + if (rangeTableEntry->rtekind != RTE_CTE) + { + /* + * This RTE pointed to a preceding CTE that was already replaced + * by a subplan. + */ + continue; + } + + if (strncmp(rangeTableEntry->ctename, cteName, NAMEDATALEN) == 0) + { + /* change the RTE_CTE into an RTE_SUBQUERY */ + rangeTableEntry->rtekind = RTE_SUBQUERY; + rangeTableEntry->ctename = NULL; + rangeTableEntry->ctelevelsup = 0; + + if (replacedCtesCount == 0) + { + /* + * Replace the first CTE reference with the result query directly. + */ + rangeTableEntry->subquery = resultQuery; + } + else + { + /* + * Replace subsequent CTE references with a copy of the result + * query. + */ + rangeTableEntry->subquery = copyObject(resultQuery); + } + + replacedCtesCount++; + } + } + + Assert(cte->cterefcount == replacedCtesCount); + } + + /* + * All CTEs are now executed through subplans and RTE_CTEs pointing + * to the CTE list have been replaced with subqueries. We can now + * clear the cteList. + */ + query->cteList = NIL; + + return NULL; +} + + +/* + * CreateDistributedSubPlan creates a distributed subplan by recursively calling + * the planner from the top, which may either generate a local plan or another + * distributed plan, which can itself contain subplans. + */ +static DistributedSubPlan * +CreateDistributedSubPlan(uint32 subPlanId, Query *subPlanQuery) +{ + DistributedSubPlan *subPlan = NULL; + int cursorOptions = 0; + + if (ContainsReadIntermediateResultFunction((Node *) subPlanQuery)) + { + /* + * Make sure we go through distributed planning if there are + * read_intermediate_result calls, even if there are no distributed + * tables in the query anymore. + * + * We cannot perform this check in the planner itself, since that + * would also cause the workers to attempt distributed planning. + */ + cursorOptions |= CURSOR_OPT_FORCE_DISTRIBUTED; + } + + subPlan = CitusMakeNode(DistributedSubPlan); + subPlan->plan = planner(subPlanQuery, cursorOptions, NULL); + subPlan->subPlanId = subPlanId; + + return subPlan; +} + + +/* + * CteReferenceListWalker finds all references to CTEs in the top level of a query + * and adds them to context->cteReferenceList. + */ +static bool +CteReferenceListWalker(Node *node, CteReferenceWalkerContext *context) +{ + if (node == NULL) + { + return false; + } + + if (IsA(node, RangeTblEntry)) + { + RangeTblEntry *rangeTableEntry = (RangeTblEntry *) node; + + if (rangeTableEntry->rtekind == RTE_CTE && + rangeTableEntry->ctelevelsup == context->level) + { + context->cteReferenceList = lappend(context->cteReferenceList, + rangeTableEntry); + } + + /* caller will descend into range table entry */ + return false; + } + else if (IsA(node, Query)) + { + Query *query = (Query *) node; + + context->level += 1; + query_tree_walker(query, CteReferenceListWalker, context, QTW_EXAMINE_RTES); + context->level -= 1; + + return false; + } + else + { + return expression_tree_walker(node, CteReferenceListWalker, context); + } +} + + +/* + * ContainsReferencesToOuterQuery determines whether the given query contains + * any Vars that point outside of the query itself. Such queries cannot be + * planned recursively. + */ +static bool +ContainsReferencesToOuterQuery(Query *query) +{ + VarLevelsUpWalkerContext context = { 0 }; + int flags = 0; + + return query_tree_walker(query, ContainsReferencesToOuterQueryWalker, + &context, flags); +} + + +/* + * ContainsReferencesToOuterQueryWalker determines whether the given query + * contains any Vars that point more than context->level levels up. + * + * ContainsReferencesToOuterQueryWalker recursively descends into subqueries + * and increases the level by 1 before recursing. + */ +static bool +ContainsReferencesToOuterQueryWalker(Node *node, VarLevelsUpWalkerContext *context) +{ + if (node == NULL) + { + return false; + } + + if (IsA(node, Var)) + { + if (((Var *) node)->varlevelsup > context->level) + { + return true; + } + + return false; + } + else if (IsA(node, Aggref)) + { + if (((Aggref *) node)->agglevelsup > context->level) + { + return true; + } + } + else if (IsA(node, GroupingFunc)) + { + if (((GroupingFunc *) node)->agglevelsup > context->level) + { + return true; + } + + return false; + } + else if (IsA(node, PlaceHolderVar)) + { + if (((PlaceHolderVar *) node)->phlevelsup > context->level) + { + return true; + } + } + else if (IsA(node, Query)) + { + Query *query = (Query *) node; + bool found = false; + int flags = 0; + + context->level += 1; + found = query_tree_walker(query, ContainsReferencesToOuterQueryWalker, + context, flags); + context->level -= 1; + + return found; + } + + return expression_tree_walker(node, ContainsReferencesToOuterQueryWalker, + context); +} + + +/* + * BuildSubPlanResultQuery returns a query of the form: + * + * SELECT + * + * FROM + * read_intermediate_result('_', ') + * AS res (); + * + * The target list and column definition list are derived from the given subquery. + * + * If any of the types in the target list cannot be used in the binary copy format, + * then the copy format 'text' is used, otherwise 'binary' is used. + */ +static Query * +BuildSubPlanResultQuery(Query *subquery, uint64 planId, uint32 subPlanId) +{ + Query *resultQuery = NULL; + char *resultIdString = NULL; + Const *resultIdConst = NULL; + Const *resultFormatConst = NULL; + FuncExpr *funcExpr = NULL; + Alias *funcAlias = NULL; + List *funcColNames = NIL; + List *funcColTypes = NIL; + List *funcColTypMods = NIL; + List *funcColCollations = NIL; + RangeTblFunction *rangeTableFunction = NULL; + RangeTblEntry *rangeTableEntry = NULL; + RangeTblRef *rangeTableRef = NULL; + FromExpr *joinTree = NULL; + ListCell *targetEntryCell = NULL; + List *targetList = NIL; + int columnNumber = 1; + bool useBinaryCopyFormat = true; + Oid copyFormatId = BinaryCopyFormatId(); + + /* build the target list and column definition list */ + foreach(targetEntryCell, subquery->targetList) + { + TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); + Node *targetExpr = (Node *) targetEntry->expr; + char *columnName = targetEntry->resname; + Oid columnType = exprType(targetExpr); + Oid columnTypMod = exprTypmod(targetExpr); + Oid columnCollation = exprCollation(targetExpr); + Var *functionColumnVar = NULL; + TargetEntry *newTargetEntry = NULL; + + if (targetEntry->resjunk) + { + continue; + } + + funcColNames = lappend(funcColNames, makeString(columnName)); + funcColTypes = lappend_int(funcColTypes, columnType); + funcColTypMods = lappend_int(funcColTypMods, columnTypMod); + funcColCollations = lappend_int(funcColCollations, columnCollation); + + functionColumnVar = makeNode(Var); + functionColumnVar->varno = 1; + functionColumnVar->varattno = columnNumber; + functionColumnVar->vartype = columnType; + functionColumnVar->vartypmod = columnTypMod; + functionColumnVar->varcollid = columnCollation; + functionColumnVar->varlevelsup = 0; + functionColumnVar->varnoold = 1; + functionColumnVar->varoattno = columnNumber; + functionColumnVar->location = -1; + + newTargetEntry = makeNode(TargetEntry); + newTargetEntry->expr = (Expr *) functionColumnVar; + newTargetEntry->resno = columnNumber; + newTargetEntry->resname = columnName; + newTargetEntry->resjunk = false; + + targetList = lappend(targetList, newTargetEntry); + + if (useBinaryCopyFormat && !CanUseBinaryCopyFormatForType(columnType)) + { + useBinaryCopyFormat = false; + } + + columnNumber++; + } + + /* build the result_id parameter for the call to read_intermediate_result */ + resultIdString = GenerateResultId(planId, subPlanId); + + resultIdConst = makeNode(Const); + resultIdConst->consttype = TEXTOID; + resultIdConst->consttypmod = -1; + resultIdConst->constlen = -1; + resultIdConst->constvalue = CStringGetTextDatum(resultIdString); + resultIdConst->constbyval = false; + resultIdConst->constisnull = false; + resultIdConst->location = -1; + + /* build the citus_copy_format parameter for the call to read_intermediate_result */ + if (!useBinaryCopyFormat) + { + copyFormatId = TextCopyFormatId(); + } + + resultFormatConst = makeNode(Const); + resultFormatConst->consttype = CitusCopyFormatTypeId(); + resultFormatConst->consttypmod = -1; + resultFormatConst->constlen = 4; + resultFormatConst->constvalue = ObjectIdGetDatum(copyFormatId); + resultFormatConst->constbyval = true; + resultFormatConst->constisnull = false; + resultFormatConst->location = -1; + + /* build the call to read_intermediate_result */ + funcExpr = makeNode(FuncExpr); + funcExpr->funcid = CitusReadIntermediateResultFuncId(); + funcExpr->funcretset = true; + funcExpr->funcvariadic = false; + funcExpr->funcformat = 0; + funcExpr->funccollid = 0; + funcExpr->inputcollid = 0; + funcExpr->location = -1; + funcExpr->args = list_make2(resultIdConst, resultFormatConst); + + /* build the RTE for the call to read_intermediate_result */ + rangeTableFunction = makeNode(RangeTblFunction); + rangeTableFunction->funccolcount = list_length(funcColNames); + rangeTableFunction->funccolnames = funcColNames; + rangeTableFunction->funccoltypes = funcColTypes; + rangeTableFunction->funccoltypmods = funcColTypMods; + rangeTableFunction->funccolcollations = funcColCollations; + rangeTableFunction->funcparams = NULL; + rangeTableFunction->funcexpr = (Node *) funcExpr; + + funcAlias = makeNode(Alias); + funcAlias->aliasname = "intermediate_result"; + funcAlias->colnames = funcColNames; + + rangeTableEntry = makeNode(RangeTblEntry); + rangeTableEntry->rtekind = RTE_FUNCTION; + rangeTableEntry->functions = list_make1(rangeTableFunction); + rangeTableEntry->inFromCl = true; + rangeTableEntry->eref = funcAlias; + + /* build the join tree using the read_intermediate_result RTE */ + rangeTableRef = makeNode(RangeTblRef); + rangeTableRef->rtindex = 1; + + joinTree = makeNode(FromExpr); + joinTree->fromlist = list_make1(rangeTableRef); + + /* build the SELECT query */ + resultQuery = makeNode(Query); + resultQuery->commandType = CMD_SELECT; + resultQuery->rtable = list_make1(rangeTableEntry); + resultQuery->jointree = joinTree; + resultQuery->targetList = targetList; + + return resultQuery; +} + + +/* + * GenerateResultId generates the result ID that is used to identify an intermediate + * result of the subplan with the given plan ID and subplan ID. + */ +char * +GenerateResultId(uint64 planId, uint32 subPlanId) +{ + StringInfo resultId = makeStringInfo(); + + appendStringInfo(resultId, UINT64_FORMAT "_%u", planId, subPlanId); + + return resultId->data; +} diff --git a/src/backend/distributed/utils/citus_copyfuncs.c b/src/backend/distributed/utils/citus_copyfuncs.c index d2ea8c9fb..5ee49a8b7 100644 --- a/src/backend/distributed/utils/citus_copyfuncs.c +++ b/src/backend/distributed/utils/citus_copyfuncs.c @@ -101,6 +101,7 @@ CopyNodeDistributedPlan(COPYFUNC_ARGS) { DECLARE_FROM_AND_NEW_NODE(DistributedPlan); + COPY_SCALAR_FIELD(planId); COPY_SCALAR_FIELD(operation); COPY_SCALAR_FIELD(hasReturning); @@ -112,10 +113,23 @@ CopyNodeDistributedPlan(COPYFUNC_ARGS) COPY_NODE_FIELD(insertSelectSubquery); COPY_NODE_FIELD(insertTargetList); COPY_SCALAR_FIELD(targetRelationId); + + COPY_NODE_FIELD(subPlanList); + COPY_NODE_FIELD(planningError); } +void +CopyNodeDistributedSubPlan(COPYFUNC_ARGS) +{ + DECLARE_FROM_AND_NEW_NODE(DistributedSubPlan); + + COPY_SCALAR_FIELD(subPlanId); + COPY_NODE_FIELD(plan); +} + + void CopyNodeShardInterval(COPYFUNC_ARGS) { diff --git a/src/backend/distributed/utils/citus_nodefuncs.c b/src/backend/distributed/utils/citus_nodefuncs.c index cfdd52846..dc6160b6f 100644 --- a/src/backend/distributed/utils/citus_nodefuncs.c +++ b/src/backend/distributed/utils/citus_nodefuncs.c @@ -32,6 +32,7 @@ static const char *CitusNodeTagNamesD[] = { "Job", "MapMergeJob", "DistributedPlan", + "DistributedSubPlan", "Task", "TaskExecution", "ShardInterval", @@ -384,6 +385,7 @@ EqualUnsupportedCitusNode(const struct ExtensibleNode *a, const ExtensibleNodeMethods nodeMethods[] = { DEFINE_NODE_METHODS(DistributedPlan), + DEFINE_NODE_METHODS(DistributedSubPlan), DEFINE_NODE_METHODS(Job), DEFINE_NODE_METHODS(ShardInterval), DEFINE_NODE_METHODS(MapMergeJob), diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index 45ae1ea31..433007770 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -175,6 +175,7 @@ OutDistributedPlan(OUTFUNC_ARGS) WRITE_NODE_TYPE("DISTRIBUTEDPLAN"); + WRITE_UINT64_FIELD(planId); WRITE_INT_FIELD(operation); WRITE_BOOL_FIELD(hasReturning); @@ -187,10 +188,24 @@ OutDistributedPlan(OUTFUNC_ARGS) WRITE_NODE_FIELD(insertTargetList); WRITE_OID_FIELD(targetRelationId); + WRITE_NODE_FIELD(subPlanList); + WRITE_NODE_FIELD(planningError); } +void +OutDistributedSubPlan(OUTFUNC_ARGS) +{ + WRITE_LOCALS(DistributedSubPlan); + + WRITE_NODE_TYPE("DISTRIBUTEDSUBPLAN"); + + WRITE_UINT_FIELD(subPlanId); + WRITE_NODE_FIELD(plan); +} + + void OutMultiProject(OUTFUNC_ARGS) { diff --git a/src/backend/distributed/utils/citus_readfuncs.c b/src/backend/distributed/utils/citus_readfuncs.c index f3f9adc5d..4e8ddb1ef 100644 --- a/src/backend/distributed/utils/citus_readfuncs.c +++ b/src/backend/distributed/utils/citus_readfuncs.c @@ -199,6 +199,7 @@ ReadDistributedPlan(READFUNC_ARGS) { READ_LOCALS(DistributedPlan); + READ_UINT64_FIELD(planId); READ_INT_FIELD(operation); READ_BOOL_FIELD(hasReturning); @@ -211,12 +212,26 @@ ReadDistributedPlan(READFUNC_ARGS) READ_NODE_FIELD(insertTargetList); READ_OID_FIELD(targetRelationId); + READ_NODE_FIELD(subPlanList); + READ_NODE_FIELD(planningError); READ_DONE(); } +READFUNC_RET +ReadDistributedSubPlan(READFUNC_ARGS) +{ + READ_LOCALS(DistributedSubPlan); + + READ_UINT_FIELD(subPlanId); + READ_NODE_FIELD(plan); + + READ_DONE(); +} + + READFUNC_RET ReadShardInterval(READFUNC_ARGS) { diff --git a/src/backend/distributed/utils/metadata_cache.c b/src/backend/distributed/utils/metadata_cache.c index a2546cddb..055d8ff1d 100644 --- a/src/backend/distributed/utils/metadata_cache.c +++ b/src/backend/distributed/utils/metadata_cache.c @@ -122,6 +122,8 @@ typedef struct MetadataCacheData Oid extraDataContainerFuncId; Oid workerHashFunctionId; Oid extensionOwner; + Oid binaryCopyFormatId; + Oid textCopyFormatId; Oid primaryNodeRoleId; Oid secondaryNodeRoleId; Oid unavailableNodeRoleId; @@ -197,6 +199,7 @@ static void CachedRelationLookup(const char *relationName, Oid *cachedOid); static ShardPlacement * ResolveGroupShardPlacement( GroupShardPlacement *groupShardPlacement, ShardCacheEntry *shardEntry); static WorkerNode * LookupNodeForGroup(uint32 groupid); +static Oid LookupEnumValueId(Oid typeId, char *valueName); /* exports for SQL callable functions */ @@ -1872,6 +1875,34 @@ CitusCopyFormatTypeId(void) } +/* return oid of the 'binary' citus_copy_format enum value */ +Oid +BinaryCopyFormatId(void) +{ + if (MetadataCache.binaryCopyFormatId == InvalidOid) + { + Oid copyFormatTypeId = CitusCopyFormatTypeId(); + MetadataCache.binaryCopyFormatId = LookupEnumValueId(copyFormatTypeId, "binary"); + } + + return MetadataCache.binaryCopyFormatId; +} + + +/* return oid of the 'text' citus_copy_format enum value */ +Oid +TextCopyFormatId(void) +{ + if (MetadataCache.textCopyFormatId == InvalidOid) + { + Oid copyFormatTypeId = CitusCopyFormatTypeId(); + MetadataCache.textCopyFormatId = LookupEnumValueId(copyFormatTypeId, "text"); + } + + return MetadataCache.textCopyFormatId; +} + + /* return oid of the citus_extradata_container(internal) function */ Oid CitusExtraDataContainerFuncId(void) @@ -2040,17 +2071,27 @@ LookupNodeRoleValueId(char *valueName) } else { - Datum nodeRoleIdDatum = ObjectIdGetDatum(nodeRoleTypId); - Datum valueDatum = CStringGetDatum(valueName); - - Datum valueIdDatum = DirectFunctionCall2(enum_in, valueDatum, nodeRoleIdDatum); - - Oid valueId = DatumGetObjectId(valueIdDatum); + Oid valueId = LookupEnumValueId(nodeRoleTypId, valueName); return valueId; } } +/* + * LookupEnumValueId looks up the OID of an enum value. + */ +static Oid +LookupEnumValueId(Oid typeId, char *valueName) +{ + Datum typeIdDatum = ObjectIdGetDatum(typeId); + Datum valueDatum = CStringGetDatum(valueName); + Datum valueIdDatum = DirectFunctionCall2(enum_in, valueDatum, typeIdDatum); + Oid valueId = DatumGetObjectId(valueIdDatum); + + return valueId; +} + + /* return the Oid of the 'primary' nodeRole enum value */ Oid PrimaryNodeRoleId(void) diff --git a/src/include/distributed/citus_nodefuncs.h b/src/include/distributed/citus_nodefuncs.h index 451803e2b..969140abd 100644 --- a/src/include/distributed/citus_nodefuncs.h +++ b/src/include/distributed/citus_nodefuncs.h @@ -44,6 +44,7 @@ extern void RegisterNodes(void); extern READFUNC_RET ReadJob(READFUNC_ARGS); extern READFUNC_RET ReadDistributedPlan(READFUNC_ARGS); +extern READFUNC_RET ReadDistributedSubPlan(READFUNC_ARGS); extern READFUNC_RET ReadShardInterval(READFUNC_ARGS); extern READFUNC_RET ReadMapMergeJob(READFUNC_ARGS); extern READFUNC_RET ReadShardPlacement(READFUNC_ARGS); @@ -57,6 +58,7 @@ extern READFUNC_RET ReadUnsupportedCitusNode(READFUNC_ARGS); extern void OutJob(OUTFUNC_ARGS); extern void OutDistributedPlan(OUTFUNC_ARGS); +extern void OutDistributedSubPlan(OUTFUNC_ARGS); extern void OutShardInterval(OUTFUNC_ARGS); extern void OutMapMergeJob(OUTFUNC_ARGS); extern void OutShardPlacement(OUTFUNC_ARGS); @@ -79,6 +81,7 @@ extern void OutMultiExtendedOp(OUTFUNC_ARGS); extern void CopyNodeJob(COPYFUNC_ARGS); extern void CopyNodeDistributedPlan(COPYFUNC_ARGS); +extern void CopyNodeDistributedSubPlan(COPYFUNC_ARGS); extern void CopyNodeShardInterval(COPYFUNC_ARGS); extern void CopyNodeMapMergeJob(COPYFUNC_ARGS); extern void CopyNodeShardPlacement(COPYFUNC_ARGS); diff --git a/src/include/distributed/citus_nodes.h b/src/include/distributed/citus_nodes.h index 4e057290a..b25082472 100644 --- a/src/include/distributed/citus_nodes.h +++ b/src/include/distributed/citus_nodes.h @@ -56,6 +56,7 @@ typedef enum CitusNodeTag T_Job, T_MapMergeJob, T_DistributedPlan, + T_DistributedSubPlan, T_Task, T_TaskExecution, T_ShardInterval, diff --git a/src/include/distributed/distributed_planner.h b/src/include/distributed/distributed_planner.h index 07d0aad18..996b3dd52 100644 --- a/src/include/distributed/distributed_planner.h +++ b/src/include/distributed/distributed_planner.h @@ -14,6 +14,7 @@ #include "nodes/relation.h" #include "distributed/citus_nodes.h" +#include "distributed/errormessage.h" /* values used by jobs and tasks which do not require identifiers */ @@ -21,6 +22,8 @@ #define INVALID_TASK_ID 0 #define MULTI_TASK_QUERY_INFO_OFF 0 /* do not log multi-task queries */ +#define CURSOR_OPT_FORCE_DISTRIBUTED 0x080000 + typedef struct RelationRestrictionContext { bool hasDistributedRelation; @@ -89,6 +92,7 @@ extern bool IsModifyDistributedPlan(struct DistributedPlan *distributedPlan); extern bool IsMultiTaskPlan(struct DistributedPlan *distributedPlan); extern bool IsMultiShardModifyPlan(struct DistributedPlan *distributedPlan); extern RangeTblEntry * RemoteScanRangeTableEntry(List *columnNameList); +extern char * GenerateResultId(uint64 planId, uint32 subPlanId); extern int GetRTEIdentity(RangeTblEntry *rte); diff --git a/src/include/distributed/intermediate_results.h b/src/include/distributed/intermediate_results.h index 139ab5d9b..d5be37c8c 100644 --- a/src/include/distributed/intermediate_results.h +++ b/src/include/distributed/intermediate_results.h @@ -16,12 +16,14 @@ #include "distributed/multi_copy.h" #include "nodes/execnodes.h" -#include "nodes/execnodes.h" #include "nodes/pg_list.h" #include "tcop/dest.h" #include "utils/palloc.h" +extern DestReceiver * CreateRemoteFileDestReceiver(char *resultId, EState *executorState, + List *initialNodeList, bool + writeLocalFile); extern void ReceiveQueryResultViaCopy(const char *resultId); extern void RemoveIntermediateResultsDirectory(void); diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index 3bd922b37..29350bbe7 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -131,10 +131,13 @@ extern Oid CitusReadIntermediateResultFuncId(void); extern Oid CitusExtraDataContainerFuncId(void); extern Oid CitusWorkerHashFunctionId(void); -/* nodeRole enum oids */ +/* enum oids */ extern Oid PrimaryNodeRoleId(void); extern Oid SecondaryNodeRoleId(void); extern Oid UnavailableNodeRoleId(void); +extern Oid CitusCopyFormatTypeId(void); +extern Oid TextCopyFormatId(void); +extern Oid BinaryCopyFormatId(void); /* user related functions */ extern Oid CitusExtensionOwner(void); diff --git a/src/include/distributed/multi_copy.h b/src/include/distributed/multi_copy.h index 0c9b4a2f8..5269a61f1 100644 --- a/src/include/distributed/multi_copy.h +++ b/src/include/distributed/multi_copy.h @@ -116,6 +116,7 @@ extern CitusCopyDestReceiver * CreateCitusCopyDestReceiver(Oid relationId, bool stopOnFailure); extern FmgrInfo * ColumnOutputFunctions(TupleDesc rowDescriptor, bool binaryFormat); extern bool CanUseBinaryCopyFormat(TupleDesc tupleDescription); +extern bool CanUseBinaryCopyFormatForType(Oid typeId); extern void AppendCopyRowData(Datum *valueArray, bool *isNullArray, TupleDesc rowDescriptor, CopyOutState rowOutputState, diff --git a/src/include/distributed/multi_logical_planner.h b/src/include/distributed/multi_logical_planner.h index 68b189d1f..bb6e25309 100644 --- a/src/include/distributed/multi_logical_planner.h +++ b/src/include/distributed/multi_logical_planner.h @@ -185,13 +185,13 @@ extern bool SubqueryPushdown; /* Function declarations for building logical plans */ extern MultiTreeRoot * MultiLogicalPlanCreate(Query *originalQuery, Query *queryTree, PlannerRestrictionContext * - plannerRestrictionContext, - ParamListInfo boundParams); + plannerRestrictionContext); extern PlannerRestrictionContext * FilterPlannerRestrictionForQuery( PlannerRestrictionContext *plannerRestrictionContext, Query *query); extern bool SafeToPushdownWindowFunction(Query *query, StringInfo *errorDetail); extern bool TargetListOnPartitionColumn(Query *query, List *targetEntryList); +extern bool ContainsReadIntermediateResultFunction(Node *node); extern MultiNode * ParentNode(MultiNode *multiNode); extern MultiNode * ChildNode(MultiUnaryNode *multiNode); extern MultiNode * GrandChildNode(MultiUnaryNode *multiNode); diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index ee183c3f9..cacd80d05 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -224,6 +224,9 @@ typedef struct DistributedPlan { CitusNode type; + /* unique identifier of the plan within the session */ + uint64 planId; + /* type of command to execute (SELECT/INSERT/...) */ CmdType operation; @@ -251,6 +254,9 @@ typedef struct DistributedPlan /* target relation of an INSERT ... SELECT via the coordinator */ Oid targetRelationId; + /* list of subplans to execute before the distributed query */ + List *subPlanList; + /* * NULL if this a valid plan, an error description otherwise. This will * e.g. be set if SQL features are present that a planner doesn't support, @@ -260,6 +266,21 @@ typedef struct DistributedPlan } DistributedPlan; +/* + * DistributedSubPlan contains a subplan of a distributed plan. Subplans are + * executed before the distributed query and their results are written to + * temporary files. This is used to execute CTEs and subquery joins that + * cannot be distributed. + */ +typedef struct DistributedSubPlan +{ + CitusNode type; + + uint32 subPlanId; + PlannedStmt *plan; +} DistributedSubPlan; + + /* OperatorCacheEntry contains information for each element in OperatorCache */ typedef struct OperatorCacheEntry { diff --git a/src/include/distributed/recursive_planning.h b/src/include/distributed/recursive_planning.h new file mode 100644 index 000000000..689729db1 --- /dev/null +++ b/src/include/distributed/recursive_planning.h @@ -0,0 +1,29 @@ +/*------------------------------------------------------------------------- + * + * recursive_planning.h + * General Citus planner code. + * + * Copyright (c) 2017, Citus Data, Inc. + *------------------------------------------------------------------------- + */ + +#ifndef RECURSIVE_PLANNING_H +#define RECURSIVE_PLANNING_H + + +#include "distributed/errormessage.h" +#include "distributed/relation_restriction_equivalence.h" +#include "nodes/pg_list.h" +#include "nodes/primnodes.h" +#include "nodes/relation.h" + + +extern DeferredErrorMessage * RecursivelyPlanSubqueriesAndCTEs(Query *query, + PlannerRestrictionContext * + plannerRestrictionContext, + uint64 planId, + List **subPlanList); +extern char * GenerateResultId(uint64 planId, uint32 subPlanId); + + +#endif /* RECURSIVE_PLANNING_H */ diff --git a/src/include/distributed/subplan_execution.h b/src/include/distributed/subplan_execution.h new file mode 100644 index 000000000..5f5b45114 --- /dev/null +++ b/src/include/distributed/subplan_execution.h @@ -0,0 +1,21 @@ +/*------------------------------------------------------------------------- + * + * subplan_execution.h + * + * Functions for execution subplans. + * + * Copyright (c) 2017, Citus Data, Inc. + *------------------------------------------------------------------------- + */ + +#ifndef SUBPLAN_EXECUTION_H +#define SUBPLAN_EXECUTION_H + + +#include "distributed/multi_physical_planner.h" + + +extern void ExecuteSubPlans(DistributedPlan *distributedPlan); + + +#endif /* SUBPLAN_EXECUTION_H */ diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index b64360297..ab5c97c65 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -679,9 +679,10 @@ SELECT user_id, value_1_agg FROM agg_events ORDER BY 1,2; 9 | 90 (9 rows) --- We do not support some CTEs +-- We support CTEs +BEGIN; WITH fist_table_agg AS - (SELECT sum(value_1) as v1_agg, user_id FROM raw_events_first GROUP BY user_id) + (SELECT max(value_1)+1 as v1_agg, user_id FROM raw_events_first GROUP BY user_id) INSERT INTO agg_events (value_1_agg, user_id) SELECT @@ -690,9 +691,11 @@ INSERT INTO agg_events fist_table_agg; DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: Collecting INSERT ... SELECT results on coordinator -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. --- We don't support CTEs that consist of const values as well +DEBUG: generating subplan 51_1 for CTE fist_table_agg: SELECT (max(value_1) + 1) AS v1_agg, user_id FROM public.raw_events_first GROUP BY user_id +DEBUG: Creating router plan +DEBUG: Plan is router executable +ROLLBACK; +-- We don't support CTEs that are referenced in the target list INSERT INTO agg_events WITH sub_cte AS (SELECT 1) SELECT @@ -701,7 +704,8 @@ INSERT INTO agg_events raw_events_first; DEBUG: Subqueries without relations are not allowed in distributed INSERT ... SELECT queries DEBUG: Collecting INSERT ... SELECT results on coordinator -ERROR: could not run distributed query with common table expressions +DEBUG: generating subplan 54_1 for CTE sub_cte: SELECT 1 +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. -- We support set operations via the coordinator BEGIN; diff --git a/src/test/regress/expected/multi_mx_router_planner.out b/src/test/regress/expected/multi_mx_router_planner.out index be69db7ce..62868945b 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -265,12 +265,22 @@ DEBUG: Plan is router executable ----+-----------+----+------- (0 rows) --- CTE joins are not supported if table shards are at different workers +-- CTE joins on different workers are supported because they are both planned recursively WITH id_author AS ( SELECT id, author_id FROM articles_hash_mx WHERE author_id = 1), id_title AS (SELECT id, title from articles_hash_mx WHERE author_id = 2) SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id; -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +DEBUG: generating subplan 66_1 for CTE id_author: SELECT id, author_id FROM public.articles_hash_mx WHERE (author_id = 1) +DEBUG: Creating router plan +DEBUG: Plan is router executable +DEBUG: generating subplan 66_2 for CTE id_title: SELECT id, title FROM public.articles_hash_mx WHERE (author_id = 2) +DEBUG: Creating router plan +DEBUG: Plan is router executable +DEBUG: Creating router plan +DEBUG: Plan is router executable + id | author_id | id | title +----+-----------+----+------- +(0 rows) + -- recursive CTEs are supported when filtered on partition column INSERT INTO company_employees_mx values(1, 1, 0); DEBUG: Creating router plan @@ -329,8 +339,7 @@ WITH RECURSIVE hierarchy as ( ON (h.employee_id = ce.manager_id AND h.company_id = ce.company_id)) SELECT * FROM hierarchy WHERE LEVEL <= 2; -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +ERROR: recursive CTEs are not supported in distributed queries -- logically wrong query, query involves different shards -- from the same table, but still router plannable due to -- shard being placed on the same worker. @@ -345,8 +354,7 @@ WITH RECURSIVE hierarchy as ( h.company_id = ce.company_id AND ce.company_id = 2)) SELECT * FROM hierarchy WHERE LEVEL <= 2; -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +ERROR: recursive CTEs are not supported in distributed queries -- grouping sets are supported on single shard SELECT id, substring(title, 2, 1) AS subtitle, count(*) @@ -584,15 +592,23 @@ DEBUG: Plan is router executable -- following join is not router plannable since there are no --- workers containing both shards, added a CTE to make this fail --- at logical planner +-- workers containing both shards, but will work through recursive +-- planning WITH single_shard as (SELECT * FROM articles_single_shard_hash_mx) SELECT a.author_id as first_author, b.word_count as second_word_count FROM articles_hash_mx a, single_shard b WHERE a.author_id = 2 and a.author_id = b.author_id LIMIT 3; DEBUG: Found no worker with all shard placements -ERROR: could not run distributed query with complex table expressions +DEBUG: generating subplan 94_1 for CTE single_shard: SELECT id, author_id, title, word_count FROM public.articles_single_shard_hash_mx +DEBUG: Creating router plan +DEBUG: Plan is router executable +DEBUG: Creating router plan +DEBUG: Plan is router executable + first_author | second_word_count +--------------+------------------- +(0 rows) + -- single shard select with limit is router plannable SELECT * FROM articles_hash_mx diff --git a/src/test/regress/expected/multi_router_planner.out b/src/test/regress/expected/multi_router_planner.out index d9e583bcf..0c7221d15 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -325,12 +325,22 @@ DEBUG: Plan is router executable ----+-----------+----+------- (0 rows) --- CTE joins are not supported if table shards are at different workers +-- CTE joins are supported because they are both planned recursively WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1), id_title AS (SELECT id, title from articles_hash WHERE author_id = 2) SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id; -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +DEBUG: generating subplan 67_1 for CTE id_author: SELECT id, author_id FROM public.articles_hash WHERE (author_id = 1) +DEBUG: Creating router plan +DEBUG: Plan is router executable +DEBUG: generating subplan 67_2 for CTE id_title: SELECT id, title FROM public.articles_hash WHERE (author_id = 2) +DEBUG: Creating router plan +DEBUG: Plan is router executable +DEBUG: Creating router plan +DEBUG: Plan is router executable + id | author_id | id | title +----+-----------+----+------- +(0 rows) + -- recursive CTEs are supported when filtered on partition column CREATE TABLE company_employees (company_id int, employee_id int, manager_id int); SELECT master_create_distributed_table('company_employees', 'company_id', 'hash'); @@ -402,8 +412,7 @@ WITH RECURSIVE hierarchy as ( ON (h.employee_id = ce.manager_id AND h.company_id = ce.company_id)) SELECT * FROM hierarchy WHERE LEVEL <= 2; -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +ERROR: recursive CTEs are not supported in distributed queries -- logically wrong query, query involves different shards -- from the same table WITH RECURSIVE hierarchy as ( @@ -417,16 +426,14 @@ WITH RECURSIVE hierarchy as ( h.company_id = ce.company_id AND ce.company_id = 2)) SELECT * FROM hierarchy WHERE LEVEL <= 2; -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +ERROR: recursive CTEs are not supported in distributed queries -- CTE with queries other than SELECT is not supported WITH new_article AS ( INSERT INTO articles_hash VALUES (1, 1, 'arsenous', 9572) RETURNING * ) SELECT * FROM new_article; DEBUG: data-modifying statements are not supported in the WITH clauses of distributed queries -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries -- Modifying statement in nested CTE case is covered by PostgreSQL itself WITH new_article AS ( WITH nested_cte AS ( @@ -700,16 +707,23 @@ DEBUG: Plan is router executable -- following join is not router plannable since there are no --- workers containing both shards, added a CTE to make this fail --- at logical planner +-- workers containing both shards, but will work through recursive +-- planning WITH single_shard as (SELECT * FROM articles_single_shard_hash) SELECT a.author_id as first_author, b.word_count as second_word_count FROM articles_hash a, single_shard b WHERE a.author_id = 2 and a.author_id = b.author_id LIMIT 3; DEBUG: Found no worker with all shard placements -ERROR: could not run distributed query with complex table expressions -HINT: Consider using an equality filter on the distributed table's partition column. +DEBUG: generating subplan 97_1 for CTE single_shard: SELECT id, author_id, title, word_count FROM public.articles_single_shard_hash +DEBUG: Creating router plan +DEBUG: Plan is router executable +DEBUG: Creating router plan +DEBUG: Plan is router executable + first_author | second_word_count +--------------+------------------- +(0 rows) + -- single shard select with limit is router plannable SELECT * FROM articles_hash diff --git a/src/test/regress/expected/multi_simple_queries.out b/src/test/regress/expected/multi_simple_queries.out index 17aaf100a..b997589ee 100644 --- a/src/test/regress/expected/multi_simple_queries.out +++ b/src/test/regress/expected/multi_simple_queries.out @@ -172,10 +172,18 @@ SELECT * FROM articles WHERE author_id = 10 UNION SELECT * FROM articles WHERE author_id = 2; ERROR: could not run distributed query with UNION, INTERSECT, or EXCEPT HINT: Consider using an equality filter on the distributed table's partition column. --- queries using CTEs are unsupported +-- queries using CTEs are supported WITH long_names AS ( SELECT id FROM authors WHERE char_length(name) > 15 ) -SELECT title FROM articles; -ERROR: relation authors is not distributed +SELECT title FROM articles ORDER BY 1 LIMIT 5; + title +----------- + abducing + abeyance + abhorring + abington + ablation +(5 rows) + -- queries which involve functions in FROM clause are unsupported. SELECT * FROM articles, position('om' in 'Thomas'); ERROR: could not run distributed query with complex table expressions diff --git a/src/test/regress/expected/multi_view.out b/src/test/regress/expected/multi_view.out index fab72b2cf..cb7c258ee 100644 --- a/src/test/regress/expected/multi_view.out +++ b/src/test/regress/expected/multi_view.out @@ -640,12 +640,12 @@ CREATE VIEW cte_view_1 AS WITH c1 AS (SELECT * FROM users_table WHERE value_1 = 3) SELECT * FROM c1 WHERE value_2 < 4; SELECT * FROM cte_view_1; ERROR: cannot push down this subquery -DETAIL: CTEs in multi-shard queries are currently unsupported +DETAIL: CTEs in subqueries are currently unsupported -- this is single shard query but still not supported since it has view + cte -- router planner can't detect it SELECT * FROM cte_view_1 WHERE user_id = 2; ERROR: cannot push down this subquery -DETAIL: CTEs in multi-shard queries are currently unsupported +DETAIL: CTEs in subqueries are currently unsupported -- if CTE itself prunes down to a single shard than the view is supported (router plannable) CREATE VIEW cte_view_2 AS WITH c1 AS (SELECT * FROM users_table WHERE user_id = 2) SELECT * FROM c1 WHERE value_1 = 3; diff --git a/src/test/regress/expected/with_basics.out b/src/test/regress/expected/with_basics.out new file mode 100644 index 000000000..5172ad302 --- /dev/null +++ b/src/test/regress/expected/with_basics.out @@ -0,0 +1,604 @@ +-- Test the basic CTE functionality and expected error messages +CREATE TYPE xy AS (x int, y int); +SELECT run_command_on_workers('CREATE TYPE xy AS (x int, y int)'); + run_command_on_workers +----------------------------------- + (localhost,57637,t,"CREATE TYPE") + (localhost,57638,t,"CREATE TYPE") +(2 rows) + +-- CTEs in FROM should work +WITH cte AS ( + SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 1,2 LIMIT 5 +) +SELECT * FROM cte; + user_id | value_2 +---------+--------- + 1 | 0 + 1 | 2 + 1 | 3 + 1 | 3 + 1 | 4 +(5 rows) + +-- CTEs in WHERE should work +WITH cte AS ( + SELECT user_id from users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + value_2 +FROM + users_table +WHERE + user_id IN (SELECT user_id FROM users_table) +ORDER BY + value_2 +LIMIT + 5; + value_2 +--------- + 0 + 0 + 0 + 0 + 0 +(5 rows) + +-- nested CTEs should work +WITH cte_1 AS ( + WITH cte_1_1 AS ( + SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 2 LIMIT 5 + ), + cte_1_2 AS ( + SELECT max(user_id) AS user_id FROM cte_1_1 + ) + SELECT user_id FROM cte_1_2 ORDER BY user_id +) +SELECT value_2 FROM users_table WHERE user_id IN (SELECT user_id FROM cte_1) ORDER BY value_2 LIMIT 1; + value_2 +--------- + 0 +(1 row) + +-- Mix of FROM/WHERE queries +WITH cte_from AS ( + SELECT max(user_id) AS user_id, value_2, value_1 FROM users_table GROUP BY value_2, value_1 +), +cte_where AS ( + SELECT value_2 FROM events_table +) +SELECT + * +FROM + (SELECT max(user_id), max(value_2) AS value_2 FROM cte_from GROUP BY value_1) f +WHERE + value_2 IN (SELECT * FROM cte_where) +ORDER BY + 1, 2 +LIMIT + 5; + max | value_2 +-----+--------- + 5 | 5 + 6 | 5 + 6 | 5 + 6 | 5 + 6 | 5 +(5 rows) + +-- CTE in subquery errors out +SELECT user_id FROM ( + WITH cte AS ( + SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 2 LIMIT 5 + ) + SELECT user_id FROM cte WHERE value_2 > 0 +) a; +ERROR: cannot push down this subquery +DETAIL: CTEs in subqueries are currently unsupported +-- CTE outside of FROM/WHERE errors out +WITH cte AS ( + SELECT user_id FROM users_table WHERE value_2 IN (1, 2) +) +SELECT (SELECT * FROM cte); +WARNING: more than one row returned by a subquery used as an expression +CONTEXT: while executing command on localhost:57638 +ERROR: could not receive query results +WITH cte_basic AS ( + SELECT user_id FROM users_table WHERE user_id = 1 +) +SELECT + (SELECT user_id FROM cte_basic), user_id +FROM + users_table; +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. +-- single-row sublink is acceptable when there is no FROM +WITH cte AS ( + SELECT user_id FROM users_table WHERE value_2 IN (1, 2) +) +SELECT (SELECT * FROM cte ORDER BY 1 LIMIT 1); + user_id +--------- + 1 +(1 row) + +-- group by partition column +WITH series AS ( + SELECT s AS once, s*2 AS twice FROM generate_series(1,10) s +) +SELECT user_id, count(*) +FROM + users_table +JOIN + series ON (user_id = once) +GROUP BY + user_id +ORDER BY + 1, 2 +LIMIT 5; + user_id | count +---------+------- + 1 | 7 + 2 | 18 + 3 | 17 + 4 | 23 + 5 | 26 +(5 rows) + +-- group by non-partition column +WITH series AS ( + SELECT s AS once, s*2 AS twice FROM generate_series(1,10) s +) +SELECT + twice, min(user_id) +FROM + users_table +JOIN + series ON (user_id = once) +GROUP BY + twice +HAVING + twice > 5 +ORDER BY + 1, 2 +LIMIT 5; + twice | min +-------+----- + 6 | 3 + 8 | 4 + 10 | 5 + 12 | 6 +(4 rows) + +-- distinct in subquery on CTE +WITH one_user AS ( + SELECT user_id from users_table WHERE user_id = 1 +) +SELECT + user_id +FROM + users_table +WHERE + value_2 IN (SELECT DISTINCT user_id FROM one_user) +ORDER BY + user_id +LIMIT + 1; + user_id +--------- + 2 +(1 row) + +-- having in subquery on CTE +WITH one_user AS ( + SELECT user_id from users_table WHERE user_id = 1 +) +SELECT + user_id +FROM + users_table +WHERE + value_2 IN (SELECT user_id FROM one_user GROUP BY user_id HAVING count(*) > 0) +ORDER BY + user_id +LIMIT + 1; + user_id +--------- + 2 +(1 row) + +-- aggregate in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT min(user_id) AS user_id FROM top_users) top_users +JOIN + users_table USING (user_id); +ERROR: cannot push down this subquery +DETAIL: Aggregates without group by are currently unsupported +-- FOR UPDATE in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users FOR UPDATE) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + user_id +--------- + 6 + 6 + 6 + 6 + 6 +(5 rows) + +-- LIMIT in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users LIMIT 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; +ERROR: cannot push down this subquery +DETAIL: Limit in subquery is currently unsupported +-- OFFSET in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users OFFSET 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; +ERROR: cannot push down this subquery +DETAIL: Offset clause is currently unsupported +-- Unsupported join in CTE +WITH top_users AS ( + SELECT DISTINCT e.user_id FROM users_table u JOIN events_table e ON (u.user_id = e.user_id AND u.value_1 > e.value_2) +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users WHERE user_id > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; +ERROR: unsupported clause type +-- Join can be supported with another CTE +WITH events_table AS ( + SELECT * FROM events_table +), +top_users AS ( + SELECT DISTINCT e.user_id FROM users_table u JOIN events_table e ON (u.user_id = e.user_id AND u.value_1 > e.value_2) +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users WHERE user_id > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + user_id +--------- + 6 + 6 + 6 + 6 + 6 +(5 rows) + +-- Window functions in CTE +WITH top_users AS ( + SELECT row_number() OVER(), user_id FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users WHERE user_id > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; +ERROR: could not run distributed query with window functions +HINT: Consider using an equality filter on the distributed table's partition column. +-- Window functions that partition by the distribution column in subqueries in CTEs are ok +WITH top_users AS + (SELECT * + FROM + (SELECT row_number() OVER(PARTITION BY user_id) AS row_number, + user_id + FROM users_table) AS foo +) +SELECT user_id +FROM + (SELECT user_id + FROM top_users + WHERE row_number > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + user_id +--------- + 1 + 1 + 1 + 1 + 1 +(5 rows) + +-- Unsupported aggregate in CTE +WITH top_users AS ( + SELECT array_agg(user_id ORDER BY value_2) user_ids FROM users_table +) +SELECT + user_id +FROM + (SELECT unnest(user_ids) user_id FROM top_users) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; +ERROR: array_agg with order by is unsupported +-- array_agg in CTE +WITH top_users AS ( + SELECT array_agg(user_id) user_ids FROM users_table +) +SELECT + user_id +FROM + (SELECT unnest(user_ids) user_id FROM top_users) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + user_id +--------- + 1 + 1 + 1 + 1 + 1 +(5 rows) + +-- composite type array +WITH top_users AS ( + SELECT array_agg((value_1,value_2)::xy) AS p FROM users_table WHERE user_id % 2 = 0 +) +SELECT + e.user_id, sum(y) +FROM + (SELECT (unnest(p)).* FROM top_users) tops +JOIN + events_table e ON (tops.x = e.user_id) +GROUP BY + e.user_id +ORDER BY + 2 DESC, 1 +LIMIT + 5; + user_id | sum +---------+----- + 3 | 651 + 2 | 552 + 4 | 544 + 5 | 126 + 1 | 90 +(5 rows) + +-- SELECT * FROM (SELECT * FROM cte UNION SELECT * FROM distributed_table) a; should error out +WITH cte AS ( + SELECT * FROM users_table +) +SELECT * FROM ( + SELECT * FROM cte UNION (SELECT * FROM events_table) + ) a +ORDER BY + 1,2,3,4,5,6 +LIMIT + 10; +ERROR: cannot push down this subquery +DETAIL: Complex subqueries and CTEs are not supported within a UNION +SELECT * FROM ( + SELECT * FROM (WITH cte AS ( + SELECT * FROM users_table + ) + SELECT * FROM cte + )b UNION (SELECT * FROM events_table)) a +ORDER BY +1,2,3,4,5,6 +LIMIT +10; +ERROR: cannot push down this subquery +DETAIL: CTEs in subqueries are currently unsupported +-- SELECT * FROM (SELECT * FROM cte UNION SELECT * FROM cte) a; should work +WITH cte AS ( + SELECT * FROM users_table WHERE user_id IN (1, 2) +) +SELECT + * +FROM + (SELECT * FROM cte UNION (SELECT * FROM cte)) a +ORDER BY + 1,2,3,4,5,6 +LIMIT + 5; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Wed Nov 22 22:51:43.132261 2017 | 4 | 0 | 3 | + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | +(5 rows) + +WITH cte AS ( + SELECT * FROM users_table WHERE user_id IN (1, 2) ORDER BY 1,2,3 LIMIT 5 +), +cte_2 AS ( + SELECT * FROM users_table WHERE user_id IN (3, 4) ORDER BY 1,2,3 LIMIT 5 +) +SELECT * FROM cte UNION ALL SELECT * FROM cte_2; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Wed Nov 22 22:51:43.132261 2017 | 4 | 0 | 3 | + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 3 | Wed Nov 22 18:43:51.450263 2017 | 1 | 1 | 4 | + 3 | Wed Nov 22 20:43:31.008625 2017 | 1 | 3 | 2 | + 3 | Wed Nov 22 23:24:32.080584 2017 | 3 | 2 | 5 | + 3 | Thu Nov 23 00:15:45.610845 2017 | 1 | 1 | 4 | + 3 | Thu Nov 23 03:23:24.702501 2017 | 1 | 2 | 5 | +(10 rows) + +-- basic recursive CTE which should all error out +WITH RECURSIVE basic_recursive(x) AS ( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 +) +SELECT sum(x) FROM basic_recursive; +ERROR: recursive CTEs are not supported in distributed queries +WITH RECURSIVE basic_recursive AS ( + SELECT -1 as user_id, '2017-11-22 20:16:16.614779'::timestamp, -1, -1, -1, -1 + UNION ALL + SELECT * FROM users_table WHERE user_id>1 +) +SELECT * FROM basic_recursive ORDER BY user_id LIMIT 1; +ERROR: recursive CTEs are not supported in distributed queries +-- basic_recursive in FROM should error out +SELECT + * +FROM +(WITH RECURSIVE basic_recursive AS ( + SELECT -1 as user_id, '2017-11-22 20:16:16.614779'::timestamp, -1, -1, -1, -1 + UNION ALL + SELECT * FROM users_table WHERE user_id>1 + ) + SELECT * FROM basic_recursive ORDER BY user_id LIMIT 1) cte_rec; +ERROR: cannot push down this subquery +DETAIL: CTEs in subqueries are currently unsupported +-- basic_recursive in WHERE with UNION ALL +SELECT + * +FROM + users_table +WHERE + user_id in +(WITH RECURSIVE basic_recursive AS ( + SELECT -1 as user_id + UNION ALL + SELECT user_id FROM users_table WHERE user_id>1 + ) + SELECT * FROM basic_recursive ORDER BY user_id LIMIT 1); +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- one recursive one regular CTE should error out +WITH RECURSIVE basic_recursive(x) AS( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 +), +basic AS ( + SELECT count(user_id) FROM users_table +) +SELECT x FROM basic, basic_recursive; +ERROR: recursive CTEs are not supported in distributed queries +-- one recursive one regular which SELECTs from the recursive CTE under a simple SELECT +WITH RECURSIVE basic_recursive(x) AS( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 +), +basic AS ( + SELECT count(x) FROM basic_recursive +) +SELECT * FROM basic; +ERROR: recursive CTEs are not supported in distributed queries +-- recursive CTE in a NESTED manner +WITH regular_cte AS ( + WITH regular_2 AS ( + WITH RECURSIVE recursive AS ( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 + ) + SELECT * FROM recursive + ) + SELECT * FROM regular_2 +) +SELECT * FROM regular_cte; +ERROR: recursive CTEs are not supported in distributed queries +-- CTEs should work with VIEWs as well +CREATE VIEW basic_view AS +SELECT * FROM users_table; +CREATE VIEW cte_view AS +WITH cte AS ( + SELECT * FROM users_table +) +SELECT user_id, max(value_1) as value_1 FROM cte GROUP BY 1; +WITH cte_user AS ( + SELECT basic_view.user_id,events_table.value_2 FROM basic_view join events_table on (basic_view.user_id = events_table.user_id) +) +SELECT user_id, sum(value_2) FROM cte_user GROUP BY 1 ORDER BY 1, 2; + user_id | sum +---------+------ + 1 | 294 + 2 | 1026 + 3 | 782 + 4 | 943 + 5 | 806 + 6 | 220 +(6 rows) + +SELECT * FROM cte_view; +ERROR: cannot push down this subquery +DETAIL: CTEs in subqueries are currently unsupported +WITH cte_user_with_view AS +( + SELECT * FROM cte_view WHERE user_id < 3 +) +SELECT user_id, value_1 FROM cte_user_with_view ORDER BY 1, 2 LIMIT 10 OFFSET 3; +ERROR: cannot push down this subquery +DETAIL: CTEs in subqueries are currently unsupported +DROP VIEW basic_view; +DROP VIEW cte_view; diff --git a/src/test/regress/expected/with_executors.out b/src/test/regress/expected/with_executors.out new file mode 100644 index 000000000..5e7955c94 --- /dev/null +++ b/src/test/regress/expected/with_executors.out @@ -0,0 +1,385 @@ +-- Confirm we can use local, router, real-time, and task-tracker execution +CREATE SCHEMA with_executors; +SET search_path TO with_executors, public; +SET citus.enable_repartition_joins TO on; +CREATE TABLE with_executors.local_table (id int); +INSERT INTO local_table VALUES (0), (1), (2), (3), (4), (5), (6), (7), (8), (9), (10); +-- CTEs should be able to use local queries +WITH cte AS ( + WITH local_cte AS ( + SELECT * FROM local_table + ), + dist_cte AS ( + SELECT user_id FROM events_table + ) + SELECT * FROM local_cte join dist_cte on dist_cte.user_id=local_cte.id +) +SELECT count(*) FROM cte; + count +------- + 101 +(1 row) + +WITH cte AS ( + WITH local_cte AS ( + SELECT * FROM local_table + ), + dist_cte AS ( + SELECT user_id FROM events_table + ), + merger_cte AS ( + SELECT id as user_id FROM local_cte UNION (SELECT * FROM dist_cte) + ) + SELECT * FROM merger_cte WHERE user_id IN (1, 2, 3) +) +SELECT * FROM cte ORDER BY 1; + user_id +--------- + 1 + 2 + 3 +(3 rows) + +WITH cte AS ( + WITH local_cte AS ( + SELECT * FROM local_table WHERE id < 5 + ), + local_cte_2 AS ( + SELECT * FROM local_table WHERE id > 5 + ) + SELECT local_cte.id as id_1, local_cte_2.id as id_2 FROM local_cte,local_cte_2 +) +SELECT + * +FROM + cte +join + users_table +on + cte.id_1 = users_table.user_id +WHERE + cte.id_1 IN (3, 4, 5) +ORDER BY + 1,2,3,4,5,6,7 +LIMIT + 10; + id_1 | id_2 | user_id | time | value_1 | value_2 | value_3 | value_4 +------+------+---------+---------------------------------+---------+---------+---------+--------- + 3 | 6 | 3 | Wed Nov 22 18:43:51.450263 2017 | 1 | 1 | 4 | + 3 | 6 | 3 | Wed Nov 22 20:43:31.008625 2017 | 1 | 3 | 2 | + 3 | 6 | 3 | Wed Nov 22 23:24:32.080584 2017 | 3 | 2 | 5 | + 3 | 6 | 3 | Thu Nov 23 00:15:45.610845 2017 | 1 | 1 | 4 | + 3 | 6 | 3 | Thu Nov 23 03:23:24.702501 2017 | 1 | 2 | 5 | + 3 | 6 | 3 | Thu Nov 23 03:52:32.008895 2017 | 4 | 2 | 0 | + 3 | 6 | 3 | Thu Nov 23 04:01:08.04806 2017 | 5 | 5 | 3 | + 3 | 6 | 3 | Thu Nov 23 05:01:44.885505 2017 | 3 | 5 | 4 | + 3 | 6 | 3 | Thu Nov 23 06:20:05.854857 2017 | 1 | 4 | 2 | + 3 | 6 | 3 | Thu Nov 23 09:57:41.540228 2017 | 2 | 2 | 3 | +(10 rows) + +-- CTEs should be able to use router queries +WITH cte AS ( + WITH router_cte AS ( + SELECT user_id, value_2 FROM users_table WHERE user_id = 1 + ), + router_cte_2 AS ( + SELECT user_id, event_type, value_2 FROM events_table WHERE user_id = 1 + ) + SELECT + router_cte.user_id as uid, event_type + FROM + router_cte, router_cte_2 +) +SELECT * FROM cte ORDER BY 2 LIMIT 5; + uid | event_type +-----+------------ + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 +(5 rows) + +-- CTEs should be able to use real-time queries +WITH real_time_cte AS ( + SELECT * FROM users_table WHERE value_2 IN (1, 2, 3) +) +SELECT * FROM real_time_cte ORDER BY 1, 2, 3, 4, 5, 6 LIMIT 10; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +-- router & real-time together +WITH cte AS ( + WITH router_cte AS ( + SELECT user_id, value_2 FROM users_table WHERE user_id = 1 + ), + real_time AS ( + SELECT user_id, event_type, value_2 FROM events_table + ) + SELECT + router_cte.user_id as uid, event_type + FROM + router_cte, real_time + WHERE + router_cte.user_id=real_time.user_id +) +SELECT * FROM cte WHERE uid=1 ORDER BY 2 LIMIT 5; + uid | event_type +-----+------------ + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 +(5 rows) + +-- CTEs should be able to use task-tracker queries +WITH cte AS ( + WITH task_tracker_1 AS ( + SELECT + users_table.user_id as uid_1, users_table.value_2 + FROM + users_table + JOIN + events_table + ON + users_table.value_2=events_table.value_2 + ), + task_tracker_2 AS ( + SELECT + users_table.user_id as uid_2, users_table.value_3 + FROM + users_table + JOIN + events_table + ON + users_table.value_3=events_table.value_3 + ) + SELECT + uid_1, uid_2, value_2, value_3 + FROM + task_tracker_1 + JOIN + task_tracker_2 + ON + value_2 = value_3 +) +SELECT + uid_1, uid_2, cte.value_2, cte.value_3 +FROM + cte +JOIN + events_table +ON + cte.value_2 = events_table.event_type +ORDER BY + 1, 2, 3, 4 +LIMIT 10; + uid_1 | uid_2 | value_2 | value_3 +-------+-------+---------+--------- + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 + 1 | 1 | 0 | 0 +(10 rows) + +-- All combined +WITH cte AS ( + WITH task_tracker AS ( + SELECT + users_table.user_id as uid_1, users_table.value_2 as val_2 + FROM + users_table + JOIN + events_table + ON + users_table.value_2=events_table.value_2 + ), + real_time AS ( + SELECT * FROM users_table + ), + router_exec AS ( + SELECT * FROM events_table WHERE user_id = 1 + ), + local_table AS ( + SELECT * FROM local_table + ), + join_first_two AS ( + SELECT uid_1, time, value_3 FROM task_tracker JOIN real_time ON val_2=value_3 + ), + join_last_two AS ( + SELECT + router_exec.user_id, local_table.id + FROM + router_exec + JOIN + local_table + ON + router_exec.user_id=local_table.id + ) + SELECT * FROM join_first_two JOIN join_last_two ON id = value_3 ORDER BY 1,2,3,4,5 LIMIT 10 +) +SELECT DISTINCT uid_1, time, value_3 FROM cte ORDER BY 1, 2, 3 LIMIT 20; + uid_1 | time | value_3 +-------+---------------------------------+--------- + 2 | Wed Nov 22 18:19:49.944985 2017 | 1 +(1 row) + +-- All combined with outer join +WITH cte AS ( + WITH task_tracker AS ( + SELECT + users_table.user_id as uid_1, users_table.value_2 as val_2 + FROM + users_table + JOIN + events_table + ON + users_table.value_2=events_table.value_2 + ), + real_time AS ( + SELECT * FROM users_table + ), + router_exec AS ( + SELECT * FROM events_table WHERE user_id = 1 + ), + local_table AS ( + SELECT * FROM local_table + ), + join_first_two AS ( + SELECT uid_1, time, value_3 FROM task_tracker JOIN real_time ON val_2=value_3 + ), + join_last_two AS ( + SELECT + router_exec.user_id, local_table.id + FROM + router_exec + JOIN + local_table + ON + router_exec.user_id=local_table.id + ) + SELECT uid_1, value_3 as val_3 FROM join_first_two JOIN join_last_two ON id = value_3 ORDER BY 1,2 LIMIT 10 +) +SELECT DISTINCT uid_1, val_3 FROM cte join events_table on cte.val_3=events_table.event_type ORDER BY 1, 2; + uid_1 | val_3 +-------+------- + 2 | 1 +(1 row) + +-- CTEs should not be able to terminate (the last SELECT) in a local query +WITH cte AS ( + SELECT * FROM users_table +) +SELECT count(*) FROM cte JOIN local_table ON (user_id = id); +ERROR: relation local_table is not distributed +-- CTEs should be able to terminate a router query +WITH cte AS ( + WITH cte_1 AS ( + SELECT * FROM local_table WHERE id < 7 + ), + cte_2 AS ( + SELECT * FROM local_table WHERE id > 3 + ), + cte_dist AS ( + SELECT count(*) as u_id FROM users_table + ), + cte_merge AS ( + SELECT cte_1.id as id FROM cte_1 join cte_2 on TRUE + ) + SELECT count(*) FROM users_table join cte_merge on id=user_id +) +SELECT + row_number() OVER (), count(*) +FROM + cte, users_table +WHERE + cte.count=user_id and user_id=5; + row_number | count +------------+------- + 1 | 0 +(1 row) + +-- CTEs should be able to terminate a real-time query +WITH cte AS ( + WITH cte_1 AS ( + SELECT * FROM local_table WHERE id < 7 + ), + cte_2 AS ( + SELECT * FROM local_table WHERE id > 3 + ), + cte_dist AS ( + SELECT count(*) as u_id FROM users_table + ), + cte_merge AS ( + SELECT cte_1.id as id FROM cte_1 join cte_2 on TRUE + ) + SELECT count(*) FROM users_table join cte_merge on id=user_id +) +SELECT count(*) FROM cte, users_table where cte.count=user_id; + count +------- + 0 +(1 row) + +SET citus.task_executor_type='task-tracker'; +-- CTEs shouldn't be able to terminate a task-tracker query +WITH cte_1 AS ( + SELECT + u_table.user_id as u_id, e_table.event_type + FROM + users_table as u_table + join + events_table as e_table + on + u_table.value_2=e_table.event_type + WHERE + u_table.user_id < 7 +), +cte_2 AS ( + SELECT + u_table.user_id as u_id, e_table.event_type + FROM + users_table as u_table + join + events_table as e_table + on + u_table.value_2=e_table.event_type + WHERE + u_table.user_id > 3 +), +cte_merge AS ( + SELECT + cte_1.u_id, cte_2.event_type + FROM + cte_1 + join + cte_2 + on cte_1.event_type=cte_2.u_id +) +SELECT + count(*) +FROM + users_table, cte_merge +WHERE + users_table.user_id = cte_merge.u_id; +ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker' +DROP SCHEMA with_executors CASCADE; +NOTICE: drop cascades to table local_table diff --git a/src/test/regress/expected/with_join.out b/src/test/regress/expected/with_join.out new file mode 100644 index 000000000..dc44d6042 --- /dev/null +++ b/src/test/regress/expected/with_join.out @@ -0,0 +1,263 @@ +CREATE SCHEMA with_join; +SET search_path TO with_join, public; +SET citus.next_shard_id TO 1501000; +CREATE TABLE with_join.reference_table(user_id int); +SELECT create_reference_table('with_join.reference_table'); + create_reference_table +------------------------ + +(1 row) + +INSERT INTO reference_table VALUES (6), (7); +SET citus.enable_repartition_joins TO on; +-- Two colocated CTE under a non-colocated join +WITH colocated_1 AS ( + SELECT + users_table.user_id, events_table.value_2 + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.user_id AND event_type IN (1, 2, 3) +), +colocated_2 AS ( + SELECT + users_table.user_id, events_table.value_2 + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.user_id AND event_type IN (4, 5, 6) +) +SELECT colocated_1.user_id, count(*) +FROM + colocated_1, colocated_2 +WHERE + colocated_1.value_2 = colocated_2.value_2 +GROUP BY + 1 +ORDER BY + 2 DESC, 1; + user_id | count +---------+------- + 3 | 26792 + 4 | 25024 + 5 | 22724 + 2 | 22554 + 6 | 5720 + 1 | 5593 +(6 rows) + +-- Two non-colocated CTE under a co-located join +WITH non_colocated_1 AS ( + SELECT + users_table.user_id + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.value_2 AND event_type IN (1, 2, 3) +), +non_colocated_2 AS ( + SELECT + users_table.user_id + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.value_2 AND event_type IN (4, 5, 6) +) +SELECT non_colocated_1.user_id, count(*) +FROM + non_colocated_1, non_colocated_2 +WHERE + non_colocated_1.user_id = non_colocated_2.user_id +GROUP BY + 1 +ORDER BY + 2 DESC, 1; + user_id | count +---------+------- + 2 | 60588 + 4 | 21160 + 3 | 13005 + 5 | 10140 + 1 | 4802 +(5 rows) + + +-- Subqueries in WHERE and FROM are mixed +-- In this query, only subquery in WHERE is not a colocated join +WITH users_events AS ( + WITH colocated_join AS ( + SELECT + users_table.user_id as uid, event_type + FROM + users_table + join + events_table + on (users_table.user_id = events_table.user_id) + WHERE + events_table.event_type IN (1, 2, 3) + ), + colocated_join_2 AS ( + SELECT + users_table.user_id, event_type + FROM + users_table + join + events_table + on (users_table.user_id = events_table.user_id) + WHERE + events_table.event_type IN (4, 5, 6) + ) + SELECT + uid, colocated_join.event_type + FROM + colocated_join, + colocated_join_2 + WHERE + colocated_join.uid = colocated_join_2.user_id AND + colocated_join.event_type IN ( + WITH some_events AS ( + SELECT + event_type + FROM + events_table + WHERE + user_id < 100 + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 + ) + SELECT + * + FROM + some_events + ) +) +SELECT + * +FROM + users_events +ORDER BY + 1, 2 +LIMIT + 20; +ERROR: cannot pushdown the subquery +DETAIL: Complex subqueries and CTEs are not allowed in the FROM clause when the query has subqueries in the WHERE clause +-- cte LEFT JOIN distributed_table should error out +WITH cte AS ( + SELECT * FROM users_table WHERE user_id = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + cte +LEFT JOIN + events_table ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; +ERROR: cannot pushdown the subquery +DETAIL: Complex subqueries and CTEs cannot be in the outer part of the outer join +-- cte RIGHT JOIN distributed_table should work +WITH cte AS ( + SELECT * FROM users_table WHERE user_id = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + cte +RIGHT JOIN + events_table ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; + user_id | time | event_type +---------+---------------------------------+------------ + 1 | Wed Nov 22 22:51:43.132261 2017 | 0 + 1 | Wed Nov 22 22:51:43.132261 2017 | 0 + 1 | Wed Nov 22 22:51:43.132261 2017 | 1 + 1 | Wed Nov 22 22:51:43.132261 2017 | 1 + 1 | Wed Nov 22 22:51:43.132261 2017 | 2 +(5 rows) + +-- distributed_table LEFT JOIN cte should work +WITH cte AS ( + SELECT * FROM users_table WHERE value_1 = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + events_table +LEFT JOIN + cte ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; + user_id | time | event_type +---------+---------------------------------+------------ + 1 | Thu Nov 23 09:26:42.145043 2017 | 0 + 1 | Thu Nov 23 09:26:42.145043 2017 | 0 + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 + 1 | Thu Nov 23 09:26:42.145043 2017 | 2 +(5 rows) + +-- distributed_table RIGHT JOIN cte should error out +WITH cte AS ( + SELECT * FROM users_table WHERE value_1 = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + events_table +RIGHT JOIN + cte ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; +ERROR: cannot pushdown the subquery +DETAIL: Complex subqueries and CTEs cannot be in the outer part of the outer join +-- cte FULL JOIN distributed_table should error out +WITH cte AS ( + SELECT * FROM users_table WHERE user_id = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + events_table +FULL JOIN + cte ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; +ERROR: cannot pushdown the subquery +DETAIL: Complex subqueries and CTEs cannot be in the outer part of the outer join +-- Joins with reference tables are planned as router queries +WITH cte AS ( + SELECT value_2, max(user_id) AS user_id FROM users_table WHERE value_2 = 1 GROUP BY value_2 HAVING count(*) > 1 +) +SELECT + row_number() OVER(), cte.user_id +FROM + cte +FULL JOIN + reference_table ON cte.user_id + 1 = reference_table.user_id +ORDER BY + user_id +LIMIT + 5; + row_number | user_id +------------+--------- + 2 | 6 + 1 | +(2 rows) + +RESET client_min_messages; +DROP SCHEMA with_join CASCADE; +NOTICE: drop cascades to table reference_table diff --git a/src/test/regress/expected/with_modifying.out b/src/test/regress/expected/with_modifying.out new file mode 100644 index 000000000..61a2f6371 --- /dev/null +++ b/src/test/regress/expected/with_modifying.out @@ -0,0 +1,127 @@ +-- Tests for modifying CTEs and CTEs in modifications +SET citus.next_shard_id TO 1502000; +CREATE SCHEMA with_modifying; +SET search_path TO with_modifying, public; +CREATE TABLE with_modifying.modify_table (id int, val int); +SELECT create_distributed_table('modify_table', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE with_modifying.users_table (LIKE public.users_table INCLUDING ALL); +SELECT create_distributed_table('with_modifying.users_table', 'user_id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO with_modifying.users_table SELECT * FROM public.users_table; +-- basic insert query in CTE +WITH basic_insert AS ( + INSERT INTO users_table VALUES (1), (2), (3) RETURNING * +) +SELECT + * +FROM + basic_insert; +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +-- single-shard UPDATE in CTE +WITH basic_update AS ( + UPDATE users_table SET value_3=42 WHERE user_id=0 RETURNING * +) +SELECT + * +FROM + basic_update; +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +-- multi-shard UPDATE in CTE +WITH basic_update AS ( + UPDATE users_table SET value_3=42 WHERE value_2=1 RETURNING * +) +SELECT + * +FROM + basic_update; +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +-- single-shard DELETE in CTE +WITH basic_delete AS ( + DELETE FROM users_table WHERE user_id=42 RETURNING * +) +SELECT + * +FROM + basic_delete; +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +-- multi-shard DELETE in CTE +WITH basic_delete AS ( + DELETE FROM users_table WHERE value_2=42 RETURNING * +) +SELECT + * +FROM + basic_delete; +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +-- INSERT...SELECT query in CTE +WITH copy_table AS ( + INSERT INTO users_table SELECT * FROM users_table RETURNING * +) +SELECT + * +FROM + copy_table; +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +-- CTEs prior to INSERT...SELECT via the coordinator should work +WITH cte AS ( + SELECT user_id FROM users_table WHERE value_2 IN (1, 2) +) +INSERT INTO modify_table (SELECT * FROM cte); +WITH cte_1 AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2, 3, 4) +), +cte_2 AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (3, 4, 5, 6) +) +INSERT INTO modify_table (SELECT cte_1.user_id FROM cte_1 join cte_2 on cte_1.value_2=cte_2.value_2); +-- even if this is an INSERT...SELECT, the CTE is under SELECT +WITH cte AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2) +) +INSERT INTO modify_table (SELECT (SELECT value_2 FROM cte GROUP BY value_2)); +WARNING: more than one row returned by a subquery used as an expression +CONTEXT: while executing command on localhost:57638 +ERROR: could not receive query results +-- CTEs prior to any other modification should error out +WITH cte AS ( + SELECT value_2 FROM users_table WHERE user_id IN (1, 2, 3) +) +DELETE FROM modify_table WHERE id IN (SELECT value_2 FROM cte); +ERROR: common table expressions are not supported in distributed modifications +WITH cte AS ( + SELECT value_2 FROM users_table WHERE user_id IN (1, 2, 3) +) +UPDATE modify_table SET val=-1 WHERE val IN (SELECT * FROM cte); +ERROR: common table expressions are not supported in distributed modifications +WITH cte AS ( + WITH basic AS ( + SELECT value_2 FROM users_table WHERE user_id IN (1, 2, 3) + ) + INSERT INTO modify_table (SELECT * FROM basic) RETURNING * +) +UPDATE modify_table SET val=-2 WHERE id IN (SELECT id FROM cte); +ERROR: common table expressions are not supported in distributed modifications +WITH cte AS ( + WITH basic AS ( + SELECT * FROM events_table WHERE event_type = 5 + ), + basic_2 AS ( + SELECT user_id FROM users_table + ) + INSERT INTO modify_table (SELECT user_id FROM events_table) RETURNING * +) +SELECT * FROM cte; +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +DROP SCHEMA with_modifying CASCADE; +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to table modify_table +drop cascades to table users_table diff --git a/src/test/regress/expected/with_nested.out b/src/test/regress/expected/with_nested.out new file mode 100644 index 000000000..cc158b62f --- /dev/null +++ b/src/test/regress/expected/with_nested.out @@ -0,0 +1,408 @@ +-- Complex nested CTEs +CREATE SCHEMA with_nested; +SET search_path tO with_nested, public; +CREATE TABLE with_nested.local_users (user_id int, event_type int); +INSERT INTO local_users VALUES (0, 0), (1, 4), (1, 7), (2, 1), (3, 3), (5, 4), (6, 2), (10, 7); +-- Can refer to outer CTE because it is recursively planned first +WITH cte_1 AS ( + SELECT DISTINCT user_id FROM users_table +), +cte_2 AS ( + WITH cte_1_1 AS ( + SELECT * FROM cte_1 WHERE user_id > 1 + ) + SELECT * FROM cte_1_1 WHERE user_id < 3 +) +SELECT user_id FROM cte_2 LIMIT 1; + user_id +--------- + 2 +(1 row) + +-- Nested CTEs +WITH users_events AS ( + WITH users_events_2 AS ( + SELECT + users_table.user_id as user_id, + events_table.event_type as event_type + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id + GROUP BY + users_table.user_id, + events_table.event_type + ) + SELECT + u_events.user_id, events_table.event_type + FROM + users_events_2 as u_events, + events_table + WHERE + u_events.user_id = events_table.user_id + ) +SELECT + * +FROM + users_events +ORDER BY + 1, 2 +LIMIT 20; + user_id | event_type +---------+------------ + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 1 + 1 | 1 + 1 | 1 + 1 | 1 + 1 | 1 + 1 | 1 + 1 | 1 + 1 | 1 +(20 rows) + +-- Nested CTEs +WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + SELECT + users_table.user_id as user_id, + events_table.event_type as event_type + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id + GROUP BY + users_table.user_id, + events_table.event_type + + ) + SELECT + uid, event_type, value_2, value_3 + FROM + ( + ( + SELECT + user_id as uid + FROM + users_events + ) users + join + events_table + on + users.uid = events_table.event_type + ) a + ) + SELECT + * + FROM + users_events + ORDER BY + 1, 3, 2, 4 + LIMIT 100 + ) + SELECT + * + FROM + users_events + LIMIT 90 + ) + SELECT + * + FROM + users_events + LIMIT 50 + ) + SELECT + uid, event_type, value_2, sum(value_3) as sum_3 + FROM + users_events + GROUP BY + 1, 2, 3 + LIMIT 40 + ) + SELECT + uid, event_type, sum(value_2) as sum_2, sum(sum_3) as sum_3 + FROM + users_events + GROUP BY + 1, 2 + LIMIT 30 +) +SELECT + uid, avg(event_type), sum(sum_2), sum(sum_3) +FROM + users_events +GROUP BY + 1; + uid | avg | sum | sum +-----+------------------------+-----+----- + 1 | 1.00000000000000000000 | 3 | 72 +(1 row) + +-- Nested CTEs +WITH users_events AS ( + -- router select query + WITH users_events_1 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 1 + ), + -- real-time select query + users_events_2_3 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 2 + OR + user_id = 3 + ), + -- router select query + -- sub CTE is a real-time executor query but the top level is router select + users_events_4 AS ( + WITH users_events_4_5 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 4 + OR + user_id = 5 + ) + SELECT + * + FROM + users_events_4_5 + WHERE + user_id = 4 + ), + -- merge all the results from CTEs + merged_users AS ( + SELECT + * + FROM + users_events_1 + UNION + SELECT + * + FROM + users_events_2_3 + UNION + SELECT + * + FROM + users_events_4 + ) + SELECT + * + FROM + merged_users +) +SELECT + * +FROM + users_events +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT + 20; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Wed Nov 22 22:51:43.132261 2017 | 4 | 0 | 3 | + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 | + 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 | + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:14:27.658529 2017 | 4 | 4 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:01:08.148777 2017 | 2 | 4 | 2 | + 2 | Thu Nov 23 06:23:53.572592 2017 | 4 | 4 | 5 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | + 2 | Thu Nov 23 09:59:08.975079 2017 | 2 | 2 | 4 | +(20 rows) + +-- Nested CTEs - joined with local table. Not supported yet. +WITH users_events AS ( + -- router select query + WITH users_events_1 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 1 + ), + -- real-time select query + users_events_2_3 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 2 + OR + user_id = 3 + ), + -- router select query + -- sub CTE is a real-time executor query but the top level is router select + users_events_4 AS ( + WITH users_events_4_5 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 4 + OR + user_id = 5 + ) + SELECT + * + FROM + users_events_4_5 + WHERE + user_id = 4 + ) + -- merge all the results from CTEs + SELECT + * + FROM + users_events_1 + UNION + SELECT + * + FROM + users_events_2_3 + UNION + SELECT + * + FROM + users_events_4 +) +SELECT + * +FROM + users_events +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT + 20; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Wed Nov 22 22:51:43.132261 2017 | 4 | 0 | 3 | + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 | + 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 | + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:14:27.658529 2017 | 4 | 4 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:01:08.148777 2017 | 2 | 4 | 2 | + 2 | Thu Nov 23 06:23:53.572592 2017 | 4 | 4 | 5 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | + 2 | Thu Nov 23 09:59:08.975079 2017 | 2 | 2 | 4 | +(20 rows) + +-- access to uncle, use window function, apply aggregates, use group by, LIMIT/OFFSET +WITH cte1 AS ( + WITH cte1_1 AS ( + WITH cte1_1_1 AS ( + SELECT user_id, time, value_2 FROM users_table + ), + cte1_1_2 AS ( + SELECT + user_id, count + FROM ( + SELECT + user_id, + count(value_2) OVER (PARTITION BY user_id) + FROM + users_table + GROUP BY 1, users_table.value_2 + )aa + GROUP BY + 1,2 + ORDER BY + 1,2 + LIMIT + 4 + OFFSET + 2 + ) + SELECT cte1_1_1.user_id, cte1_1_1.time, cte1_1_2.count FROM cte1_1_1 join cte1_1_2 on cte1_1_1.user_id=cte1_1_2.user_id + ), + cte1_2 AS ( + WITH cte1_2_1 AS ( + SELECT + user_id, time, avg(value_1)::real as value_1, min(value_2) as value_2 + FROM + users_table + GROUP BY + 1, 2 + ), + cte1_2_2 AS ( + SELECT cte1_2_1.user_id, cte1_1.time, cte1_2_1.value_1, cte1_1.count FROM cte1_2_1 join cte1_1 on cte1_2_1.time=cte1_1.time and cte1_2_1.user_id=cte1_1.user_id + ) + SELECT * FROM cte1_2_2 + ) + SELECT * FROM cte1_2 +), +cte2 AS ( + WITH cte2_1 AS ( + WITH cte2_1_1 AS ( + SELECT * FROM cte1 + ) + SELECT user_id, time, value_1, min(count) FROM cte2_1_1 GROUP BY 1, 2, 3 ORDER BY 1,2,3 + ) + SELECT * FROM cte2_1 LIMIT 3 OFFSET 2 +) +SELECT * FROM cte2; + user_id | time | value_1 | min +---------+---------------------------------+---------+----- + 3 | Wed Nov 22 23:24:32.080584 2017 | 3 | 5 + 3 | Thu Nov 23 00:15:45.610845 2017 | 1 | 5 + 3 | Thu Nov 23 03:23:24.702501 2017 | 1 | 5 +(3 rows) + +DROP SCHEMA with_nested CASCADE; +NOTICE: drop cascades to table local_users diff --git a/src/test/regress/expected/with_partitioning.out b/src/test/regress/expected/with_partitioning.out new file mode 100644 index 000000000..db812304d --- /dev/null +++ b/src/test/regress/expected/with_partitioning.out @@ -0,0 +1,94 @@ +CREATE SCHEMA with_partitioning; +SET search_path TO with_partitioning, public; +SET citus.shard_replication_factor TO 1; +CREATE TABLE with_partitioning.local_users_2 (user_id int, event_type int); +INSERT INTO local_users_2 VALUES (0, 0), (1, 4), (1, 7), (2, 1), (3, 3), (5, 4), (6, 2), (10, 7); +CREATE TABLE with_partitioning.partitioning_test(id int, time date) PARTITION BY RANGE (time); + +-- create its partitions +CREATE TABLE with_partitioning.partitioning_test_2017 PARTITION OF partitioning_test FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); +CREATE TABLE with_partitioning.partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +-- load some data and distribute tables +INSERT INTO partitioning_test VALUES (1, '2017-11-23'); +INSERT INTO partitioning_test VALUES (2, '2010-07-07'); +INSERT INTO partitioning_test_2017 VALUES (3, '2017-11-22'); +INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); +-- distribute partitioned table +SELECT create_distributed_table('with_partitioning.partitioning_test', 'id'); +NOTICE: Copying data from local table... +NOTICE: Copying data from local table... + create_distributed_table +-------------------------- + +(1 row) + +-- Join of a CTE on distributed table and then join with a partitioned table +WITH cte AS ( + SELECT * FROM users_table +) +SELECT DISTINCT ON (id) id, cte.time FROM cte join partitioning_test on cte.time::date=partitioning_test.time ORDER BY 1, 2 LIMIT 3; + id | time +----+--------------------------------- + 1 | Thu Nov 23 00:07:11.068353 2017 + 3 | Wed Nov 22 18:19:49.944985 2017 +(2 rows) + +-- Join of a CTE on distributed table and then join with a partitioned table hitting on only one partition +WITH cte AS ( + SELECT * FROM users_table +) +SELECT DISTINCT ON (id) id, cte.time FROM cte join partitioning_test on cte.time::date=partitioning_test.time WHERE partitioning_test.time >'2017-11-20' ORDER BY 1, 2 LIMIT 3; + id | time +----+--------------------------------- + 1 | Thu Nov 23 00:07:11.068353 2017 + 3 | Wed Nov 22 18:19:49.944985 2017 +(2 rows) + +-- Join with a distributed table and then join of two CTEs +WITH cte AS ( + SELECT id, time FROM partitioning_test +), +cte_2 AS ( + SELECT * FROM partitioning_test WHERE id > 2 +), +cte_joined AS ( + SELECT user_id, cte_2.time FROM users_table join cte_2 on (users_table.time::date = cte_2.time) +), +cte_joined_2 AS ( + SELECT user_id, cte_joined.time FROM cte_joined join cte on (cte_joined.time = cte.time) +) +SELECT DISTINCT ON (event_type) event_type, cte_joined_2.user_id FROM events_table join cte_joined_2 on (cte_joined_2.time=events_table.time::date) ORDER BY 1, 2 LIMIT 10 OFFSET 2; + event_type | user_id +------------+--------- + 2 | 1 + 3 | 1 + 4 | 1 + 5 | 1 +(4 rows) + +-- Join a partitioned table with a local table (both in CTEs) +-- and then with a distributed table. After all join with a +-- partitioned table again +WITH cte AS ( + SELECT id, time FROM partitioning_test +), +cte_2 AS ( + SELECT * FROM local_users_2 +), +cte_joined AS ( + SELECT user_id, cte.time FROM cte join cte_2 on (cte.id = cte_2.user_id) +), +cte_joined_2 AS ( + SELECT users_table.user_id, cte_joined.time FROM cte_joined join users_table on (cte_joined.time = users_table.time::date) +) +SELECT DISTINCT ON (id) id, cte_joined_2.time FROM cte_joined_2 join partitioning_test on (cte_joined_2.time=partitioning_test.time) ORDER BY 1, 2; + id | time +----+------------ + 1 | 11-23-2017 + 3 | 11-22-2017 +(2 rows) + +DROP SCHEMA with_partitioning CASCADE; +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to table local_users_2 +drop cascades to table partitioning_test diff --git a/src/test/regress/expected/with_partitioning_0.out b/src/test/regress/expected/with_partitioning_0.out new file mode 100644 index 000000000..c7b95adb8 --- /dev/null +++ b/src/test/regress/expected/with_partitioning_0.out @@ -0,0 +1,95 @@ +CREATE SCHEMA with_partitioning; +SET search_path TO with_partitioning, public; +SET citus.shard_replication_factor TO 1; +CREATE TABLE with_partitioning.local_users_2 (user_id int, event_type int); +INSERT INTO local_users_2 VALUES (0, 0), (1, 4), (1, 7), (2, 1), (3, 3), (5, 4), (6, 2), (10, 7); +CREATE TABLE with_partitioning.partitioning_test(id int, time date) PARTITION BY RANGE (time); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...partitioning.partitioning_test(id int, time date) PARTITION ... + ^ + +-- create its partitions +CREATE TABLE with_partitioning.partitioning_test_2017 PARTITION OF partitioning_test FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...TE TABLE with_partitioning.partitioning_test_2017 PARTITION ... + ^ +CREATE TABLE with_partitioning.partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +ERROR: syntax error at or near "PARTITION" +LINE 1: ...TE TABLE with_partitioning.partitioning_test_2010 PARTITION ... + ^ +-- load some data and distribute tables +INSERT INTO partitioning_test VALUES (1, '2017-11-23'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES (1, '2017-11-23'); + ^ +INSERT INTO partitioning_test VALUES (2, '2010-07-07'); +ERROR: relation "partitioning_test" does not exist +LINE 1: INSERT INTO partitioning_test VALUES (2, '2010-07-07'); + ^ +INSERT INTO partitioning_test_2017 VALUES (3, '2017-11-22'); +ERROR: relation "partitioning_test_2017" does not exist +LINE 1: INSERT INTO partitioning_test_2017 VALUES (3, '2017-11-22'); + ^ +INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); +ERROR: relation "partitioning_test_2010" does not exist +LINE 1: INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); + ^ +-- distribute partitioned table +SELECT create_distributed_table('with_partitioning.partitioning_test', 'id'); +ERROR: relation "with_partitioning.partitioning_test" does not exist +LINE 1: SELECT create_distributed_table('with_partitioning.partition... + ^ +-- Join of a CTE on distributed table and then join with a partitioned table +WITH cte AS ( + SELECT * FROM users_table +) +SELECT DISTINCT ON (id) id, cte.time FROM cte join partitioning_test on cte.time::date=partitioning_test.time ORDER BY 1, 2 LIMIT 3; +ERROR: relation "partitioning_test" does not exist +LINE 4: ...ELECT DISTINCT ON (id) id, cte.time FROM cte join partitioni... + ^ +-- Join of a CTE on distributed table and then join with a partitioned table hitting on only one partition +WITH cte AS ( + SELECT * FROM users_table +) +SELECT DISTINCT ON (id) id, cte.time FROM cte join partitioning_test on cte.time::date=partitioning_test.time WHERE partitioning_test.time >'2017-11-20' ORDER BY 1, 2 LIMIT 3; +ERROR: relation "partitioning_test" does not exist +LINE 4: ...ELECT DISTINCT ON (id) id, cte.time FROM cte join partitioni... + ^ +-- Join with a distributed table and then join of two CTEs +WITH cte AS ( + SELECT id, time FROM partitioning_test +), +cte_2 AS ( + SELECT * FROM partitioning_test WHERE id > 2 +), +cte_joined AS ( + SELECT user_id, cte_2.time FROM users_table join cte_2 on (users_table.time::date = cte_2.time) +), +cte_joined_2 AS ( + SELECT user_id, cte_joined.time FROM cte_joined join cte on (cte_joined.time = cte.time) +) +SELECT DISTINCT ON (event_type) event_type, cte_joined_2.user_id FROM events_table join cte_joined_2 on (cte_joined_2.time=events_table.time::date) ORDER BY 1, 2 LIMIT 10 OFFSET 2; +ERROR: relation "partitioning_test" does not exist +LINE 2: SELECT id, time FROM partitioning_test + ^ +-- Join a partitioned table with a local table (both in CTEs) +-- and then with a distributed table. After all join with a +-- partitioned table again +WITH cte AS ( + SELECT id, time FROM partitioning_test +), +cte_2 AS ( + SELECT * FROM local_users_2 +), +cte_joined AS ( + SELECT user_id, cte.time FROM cte join cte_2 on (cte.id = cte_2.user_id) +), +cte_joined_2 AS ( + SELECT users_table.user_id, cte_joined.time FROM cte_joined join users_table on (cte_joined.time = users_table.time::date) +) +SELECT DISTINCT ON (id) id, cte_joined_2.time FROM cte_joined_2 join partitioning_test on (cte_joined_2.time=partitioning_test.time) ORDER BY 1, 2; +ERROR: relation "partitioning_test" does not exist +LINE 2: SELECT id, time FROM partitioning_test + ^ +DROP SCHEMA with_partitioning CASCADE; +NOTICE: drop cascades to table local_users_2 diff --git a/src/test/regress/expected/with_prepare.out b/src/test/regress/expected/with_prepare.out new file mode 100644 index 000000000..629607c72 --- /dev/null +++ b/src/test/regress/expected/with_prepare.out @@ -0,0 +1,547 @@ +-- prepared statements +PREPARE prepared_test_1 AS +WITH basic AS( + SELECT * FROM users_table +) +SELECT + * +FROM + basic +WHERE + basic.value_2 IN (1, 2, 3) +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT 10; +PREPARE prepared_test_2 AS +WITH users_events AS( + SELECT + users_table.user_id as user_id, + events_table.event_type as event_type + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id + GROUP BY + users_table.user_id, + events_table.event_type +), +event_attendee_count AS( + SELECT + event_type, count(user_id) + FROM + users_events + GROUP BY + 1 +), +user_coolness AS( + SELECT + user_id, + sum(count) + FROM + users_events + join + event_attendee_count + on (users_events.event_type = event_attendee_count.event_type) + GROUP BY + user_id +) +SELECT + * +FROM + user_coolness +ORDER BY + 2, 1 +LIMIT + 10; +PREPARE prepared_test_3(integer) AS +WITH users_events AS( + -- events 1 and 2 only + WITH spec_events AS( + SELECT + * + FROM + events_table + WHERE + event_type IN (1, 2) + ) + -- users who have done 1 or 2 + SELECT + users_table.user_id, + spec_events.event_type + FROM + users_table + join + spec_events + on (users_table.user_id = spec_events.user_id) + ORDER BY + 1, + event_type +), +event_attendee_count AS( + -- distinct attendee count of each event in users_event + WITH event_attendee_count AS( + SELECT + event_type, count(user_id) + FROM + users_events + GROUP BY + 1 + ) + -- distinct attendee count of first 3 events + SELECT + * + FROM + event_attendee_count + ORDER BY + event_type + LIMIT 3 +), +-- measure how cool an attendee is by checking the number of events he attended +user_coolness AS( + SELECT + user_id, + sum(count) + FROM + users_events + join + event_attendee_count + on (users_events.event_type = $1) + GROUP BY + user_id +) +SELECT + * +FROM + user_coolness +ORDER BY + 2, 1 +LIMIT + 10; +PREPARE prepared_test_4(integer, integer, integer) AS +WITH basic AS( + SELECT * FROM users_table WHERE value_2 IN ($1, $2, $3) +) +SELECT + * +FROM + basic +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT 10; +-- prepared statement which inserts in a CTE should fail +PREPARE prepared_partition_column_insert(integer) AS +WITH prepared_insert AS ( + INSERT INTO users_table VALUES ($1) RETURNING * +) +SELECT * FROM prepared_insert; +PREPARE prepared_test_5(integer, integer, integer) AS +-- router select query +WITH users_events_1 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = $1 +), +-- real-time select query +users_events_2_3 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = $2 + OR + user_id = $3 +), +merged_users AS ( + SELECT + * + FROM + users_events_1 + UNION + SELECT + * + FROM + users_events_2_3 +) +SELECT + * +FROM + merged_users +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT 10; +EXECUTE prepared_test_1; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_1; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_1; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_1; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_1; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_1; + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_2; + user_id | sum +---------+----- + 1 | 29 + 3 | 29 + 6 | 29 + 2 | 33 + 4 | 33 + 5 | 33 +(6 rows) + +EXECUTE prepared_test_2; + user_id | sum +---------+----- + 1 | 29 + 3 | 29 + 6 | 29 + 2 | 33 + 4 | 33 + 5 | 33 +(6 rows) + +EXECUTE prepared_test_2; + user_id | sum +---------+----- + 1 | 29 + 3 | 29 + 6 | 29 + 2 | 33 + 4 | 33 + 5 | 33 +(6 rows) + +EXECUTE prepared_test_2; + user_id | sum +---------+----- + 1 | 29 + 3 | 29 + 6 | 29 + 2 | 33 + 4 | 33 + 5 | 33 +(6 rows) + +EXECUTE prepared_test_2; + user_id | sum +---------+----- + 1 | 29 + 3 | 29 + 6 | 29 + 2 | 33 + 4 | 33 + 5 | 33 +(6 rows) + +EXECUTE prepared_test_2; + user_id | sum +---------+----- + 1 | 29 + 3 | 29 + 6 | 29 + 2 | 33 + 4 | 33 + 5 | 33 +(6 rows) + +EXECUTE prepared_test_3(1); + user_id | sum +---------+-------- + 1 | 10850 + 6 | 15500 + 3 | 52700 + 4 | 71300 + 2 | 97650 + 5 | 100750 +(6 rows) + +EXECUTE prepared_test_3(2); + user_id | sum +---------+------- + 1 | 10850 + 6 | 15500 + 5 | 20150 + 2 | 41850 + 4 | 71300 + 3 | 92225 +(6 rows) + +EXECUTE prepared_test_3(3); + user_id | sum +---------+----- +(0 rows) + +EXECUTE prepared_test_3(4); + user_id | sum +---------+----- +(0 rows) + +EXECUTE prepared_test_3(5); + user_id | sum +---------+----- +(0 rows) + +EXECUTE prepared_test_3(6); + user_id | sum +---------+----- +(0 rows) + +EXECUTE prepared_test_4(1, 2, 3); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | + 2 | Thu Nov 23 08:49:47.029236 2017 | 4 | 2 | 4 | + 2 | Thu Nov 23 09:54:28.13665 2017 | 0 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_4(2, 3, 4); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 | + 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 | + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:14:27.658529 2017 | 4 | 4 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | +(10 rows) + +EXECUTE prepared_test_4(3, 4, 5); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 | + 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 | + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:14:27.658529 2017 | 4 | 4 | 4 | + 2 | Thu Nov 23 06:01:08.148777 2017 | 2 | 4 | 2 | +(10 rows) + +EXECUTE prepared_test_4(4, 5, 6); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 | + 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 | + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 01:14:27.658529 2017 | 4 | 4 | 4 | + 2 | Thu Nov 23 06:01:08.148777 2017 | 2 | 4 | 2 | + 2 | Thu Nov 23 06:23:53.572592 2017 | 4 | 4 | 5 | + 2 | Thu Nov 23 11:41:04.042936 2017 | 3 | 4 | 1 | + 2 | Thu Nov 23 11:48:24.943542 2017 | 0 | 5 | 5 | +(10 rows) + +EXECUTE prepared_test_4(5, 6, 7); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 11:48:24.943542 2017 | 0 | 5 | 5 | + 3 | Thu Nov 23 04:01:08.04806 2017 | 5 | 5 | 3 | + 3 | Thu Nov 23 05:01:44.885505 2017 | 3 | 5 | 4 | + 3 | Thu Nov 23 11:31:17.403189 2017 | 4 | 5 | 3 | + 3 | Thu Nov 23 11:41:21.157066 2017 | 3 | 5 | 3 | + 3 | Thu Nov 23 12:56:49.29191 2017 | 0 | 5 | 1 | + 3 | Thu Nov 23 17:18:51.048758 2017 | 1 | 5 | 5 | + 4 | Thu Nov 23 07:28:42.537255 2017 | 3 | 5 | 3 | + 4 | Thu Nov 23 11:45:39.744961 2017 | 4 | 5 | 4 | +(10 rows) + +EXECUTE prepared_test_4(6, 7, 8); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+------+---------+---------+---------+--------- +(0 rows) + +EXECUTE prepared_test_5(1, 2, 3); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 1 | Wed Nov 22 22:51:43.132261 2017 | 4 | 0 | 3 | + 1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 | + 1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 | + 1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 | + 1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 | + 1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 | + 1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 | + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | +(10 rows) + +EXECUTE prepared_test_5(2, 3, 4); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 | + 2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 | + 2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 | + 2 | Thu Nov 23 01:14:27.658529 2017 | 4 | 4 | 4 | + 2 | Thu Nov 23 03:27:50.327051 2017 | 2 | 2 | 0 | + 2 | Thu Nov 23 06:01:08.148777 2017 | 2 | 4 | 2 | + 2 | Thu Nov 23 06:23:53.572592 2017 | 4 | 4 | 5 | + 2 | Thu Nov 23 06:50:30.797805 2017 | 1 | 1 | 1 | + 2 | Thu Nov 23 06:56:38.46819 2017 | 0 | 1 | 3 | + 2 | Thu Nov 23 08:22:22.169158 2017 | 4 | 2 | 5 | +(10 rows) + +EXECUTE prepared_test_5(3, 4, 5); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 3 | Wed Nov 22 18:43:51.450263 2017 | 1 | 1 | 4 | + 3 | Wed Nov 22 20:43:31.008625 2017 | 1 | 3 | 2 | + 3 | Wed Nov 22 23:24:32.080584 2017 | 3 | 2 | 5 | + 3 | Thu Nov 23 00:15:45.610845 2017 | 1 | 1 | 4 | + 3 | Thu Nov 23 03:23:24.702501 2017 | 1 | 2 | 5 | + 3 | Thu Nov 23 03:52:32.008895 2017 | 4 | 2 | 0 | + 3 | Thu Nov 23 04:01:08.04806 2017 | 5 | 5 | 3 | + 3 | Thu Nov 23 05:01:44.885505 2017 | 3 | 5 | 4 | + 3 | Thu Nov 23 06:20:05.854857 2017 | 1 | 4 | 2 | + 3 | Thu Nov 23 09:57:41.540228 2017 | 2 | 2 | 3 | +(10 rows) + +EXECUTE prepared_test_5(4, 5, 6); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 4 | Wed Nov 22 21:33:03.616802 2017 | 5 | 4 | 1 | + 4 | Wed Nov 22 23:48:11.949567 2017 | 2 | 0 | 0 | + 4 | Wed Nov 22 23:59:46.493416 2017 | 3 | 1 | 3 | + 4 | Thu Nov 23 00:28:45.060248 2017 | 4 | 4 | 3 | + 4 | Thu Nov 23 01:55:21.824618 2017 | 3 | 1 | 4 | + 4 | Thu Nov 23 02:14:35.047974 2017 | 4 | 4 | 1 | + 4 | Thu Nov 23 03:34:40.419294 2017 | 1 | 0 | 4 | + 4 | Thu Nov 23 05:42:12.89386 2017 | 2 | 3 | 3 | + 4 | Thu Nov 23 06:39:06.287818 2017 | 3 | 3 | 2 | + 4 | Thu Nov 23 06:50:08.101207 2017 | 2 | 1 | 5 | +(10 rows) + +EXECUTE prepared_test_5(5, 6, 7); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 5 | Wed Nov 22 20:43:18.667473 2017 | 0 | 3 | 2 | + 5 | Wed Nov 22 21:02:07.575129 2017 | 2 | 0 | 2 | + 5 | Wed Nov 22 22:10:24.315371 2017 | 1 | 2 | 1 | + 5 | Wed Nov 22 22:31:47.62577 2017 | 3 | 1 | 4 | + 5 | Wed Nov 22 23:10:42.777699 2017 | 3 | 4 | 5 | + 5 | Thu Nov 23 00:46:13.498577 2017 | 3 | 2 | 2 | + 5 | Thu Nov 23 00:54:44.192608 2017 | 1 | 3 | 2 | + 5 | Thu Nov 23 02:09:42.27857 2017 | 3 | 2 | 4 | + 5 | Thu Nov 23 02:50:32.678074 2017 | 4 | 2 | 4 | + 5 | Thu Nov 23 06:35:05.166535 2017 | 5 | 5 | 1 | +(10 rows) + +EXECUTE prepared_test_5(6, 7, 8); + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+---------------------------------+---------+---------+---------+--------- + 6 | Wed Nov 22 20:15:53.317797 2017 | 1 | 1 | 1 | + 6 | Wed Nov 22 23:01:24.82289 2017 | 2 | 4 | 1 | + 6 | Thu Nov 23 00:07:11.068353 2017 | 1 | 1 | 4 | + 6 | Thu Nov 23 00:09:44.19812 2017 | 5 | 2 | 0 | + 6 | Thu Nov 23 01:13:50.526322 2017 | 2 | 4 | 1 | + 6 | Thu Nov 23 01:14:55.769581 2017 | 0 | 0 | 5 | + 6 | Thu Nov 23 10:22:11.02918 2017 | 5 | 0 | 5 | + 6 | Thu Nov 23 11:08:04.244582 2017 | 2 | 3 | 2 | + 6 | Thu Nov 23 13:51:16.92838 2017 | 0 | 4 | 2 | + 6 | Thu Nov 23 14:43:18.024104 2017 | 3 | 2 | 5 | +(10 rows) + +EXECUTE prepared_partition_column_insert(1); +ERROR: data-modifying statements are not supported in the WITH clauses of distributed queries +DEALLOCATE ALL; diff --git a/src/test/regress/expected/with_where.out b/src/test/regress/expected/with_where.out new file mode 100644 index 000000000..03fd85f8a --- /dev/null +++ b/src/test/regress/expected/with_where.out @@ -0,0 +1,168 @@ +-- More complex CTEs in WHERE +SET citus.enable_repartition_joins TO on; +-- CTE in WHERE basic +WITH events AS ( + SELECT + event_type + FROM + events_table + WHERE + user_id < 5 + GROUP BY + event_type + ORDER BY + event_type + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + event_type +IN + (SELECT + event_type + FROM + events); + count +------- + 101 +(1 row) + +WITH users AS ( + SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.user_id = users_table.user_id + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + ( + SELECT + * + FROM + users + ); + count +------- + 101 +(1 row) + +WITH users AS ( + SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.user_id = users_table.user_id + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + ( + SELECT + * + FROM + users + ); + count +------- + 101 +(1 row) + +-- CTE with non-colocated join in WHERE +WITH users AS ( + SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.value_2 = users_table.value_2 + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + ( + SELECT + * + FROM + users + ); + count +------- + 101 +(1 row) + +-- CTE in WHERE basic +SELECT + count(*) +FROM + events_table +WHERE + event_type +IN + (WITH events AS ( + SELECT + event_type + FROM + events_table + WHERE user_id < 5 + GROUP BY + 1 + ORDER BY + 1) + SELECT * FROM events LIMIT 10 + ); +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. +-- CTE with non-colocated join in WHERE +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + (WITH users AS + (SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.value_2 = users_table.value_2 + GROUP BY + 1 + ORDER BY + 1 + ) + SELECT * FROM users LIMIT 10 + ); +ERROR: cannot pushdown the subquery since all relations are not joined using distribution keys +DETAIL: Each relation should be joined with at least one another relation using distribution keys and equality operator. diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index bef6fb31b..6cd6a6a80 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -83,6 +83,12 @@ test: multi_join_order_tpch_large # ---------- test: multi_large_table_join_planning multi_large_table_pruning multi_large_table_task_assignment +# --------- +# Tests for recursive planning. +# --------- + +test: with_nested with_where with_basics with_prepare with_modifying with_executors with_join with_partitioning + # ---------- # Tests to check our large record loading and shard deletion behavior # ---------- diff --git a/src/test/regress/sql/multi_insert_select.sql b/src/test/regress/sql/multi_insert_select.sql index 618f3f971..ee80bd4f3 100644 --- a/src/test/regress/sql/multi_insert_select.sql +++ b/src/test/regress/sql/multi_insert_select.sql @@ -505,17 +505,19 @@ INSERT INTO agg_events (value_1_agg, user_id) SELECT user_id, value_1_agg FROM agg_events ORDER BY 1,2; --- We do not support some CTEs +-- We support CTEs +BEGIN; WITH fist_table_agg AS - (SELECT sum(value_1) as v1_agg, user_id FROM raw_events_first GROUP BY user_id) + (SELECT max(value_1)+1 as v1_agg, user_id FROM raw_events_first GROUP BY user_id) INSERT INTO agg_events (value_1_agg, user_id) SELECT v1_agg, user_id FROM fist_table_agg; +ROLLBACK; --- We don't support CTEs that consist of const values as well +-- We don't support CTEs that are referenced in the target list INSERT INTO agg_events WITH sub_cte AS (SELECT 1) SELECT diff --git a/src/test/regress/sql/multi_mx_router_planner.sql b/src/test/regress/sql/multi_mx_router_planner.sql index 6801e543b..7c0d840ad 100644 --- a/src/test/regress/sql/multi_mx_router_planner.sql +++ b/src/test/regress/sql/multi_mx_router_planner.sql @@ -137,7 +137,7 @@ WITH id_author AS ( SELECT id, author_id FROM articles_hash_mx WHERE author_id = id_title AS (SELECT id, title from articles_hash_mx WHERE author_id = 3) SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id; --- CTE joins are not supported if table shards are at different workers +-- CTE joins on different workers are supported because they are both planned recursively WITH id_author AS ( SELECT id, author_id FROM articles_hash_mx WHERE author_id = 1), id_title AS (SELECT id, title from articles_hash_mx WHERE author_id = 2) SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id; @@ -269,8 +269,8 @@ SELECT a.author_id as first_author, b.word_count as second_word_count LIMIT 3; -- following join is not router plannable since there are no --- workers containing both shards, added a CTE to make this fail --- at logical planner +-- workers containing both shards, but will work through recursive +-- planning WITH single_shard as (SELECT * FROM articles_single_shard_hash_mx) SELECT a.author_id as first_author, b.word_count as second_word_count FROM articles_hash_mx a, single_shard b diff --git a/src/test/regress/sql/multi_router_planner.sql b/src/test/regress/sql/multi_router_planner.sql index 66620d1c5..1e1707dc9 100644 --- a/src/test/regress/sql/multi_router_planner.sql +++ b/src/test/regress/sql/multi_router_planner.sql @@ -176,7 +176,7 @@ WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1) id_title AS (SELECT id, title from articles_hash WHERE author_id = 3) SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id; --- CTE joins are not supported if table shards are at different workers +-- CTE joins are supported because they are both planned recursively WITH id_author AS ( SELECT id, author_id FROM articles_hash WHERE author_id = 1), id_title AS (SELECT id, title from articles_hash WHERE author_id = 2) SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id; @@ -338,8 +338,8 @@ SELECT a.author_id as first_author, b.word_count as second_word_count LIMIT 3; -- following join is not router plannable since there are no --- workers containing both shards, added a CTE to make this fail --- at logical planner +-- workers containing both shards, but will work through recursive +-- planning WITH single_shard as (SELECT * FROM articles_single_shard_hash) SELECT a.author_id as first_author, b.word_count as second_word_count FROM articles_hash a, single_shard b diff --git a/src/test/regress/sql/multi_simple_queries.sql b/src/test/regress/sql/multi_simple_queries.sql index 5827331cb..4d8396885 100644 --- a/src/test/regress/sql/multi_simple_queries.sql +++ b/src/test/regress/sql/multi_simple_queries.sql @@ -120,9 +120,9 @@ SELECT author_id, sum(word_count) AS corpus_size FROM articles SELECT * FROM articles WHERE author_id = 10 UNION SELECT * FROM articles WHERE author_id = 2; --- queries using CTEs are unsupported +-- queries using CTEs are supported WITH long_names AS ( SELECT id FROM authors WHERE char_length(name) > 15 ) -SELECT title FROM articles; +SELECT title FROM articles ORDER BY 1 LIMIT 5; -- queries which involve functions in FROM clause are unsupported. SELECT * FROM articles, position('om' in 'Thomas'); diff --git a/src/test/regress/sql/with_basics.sql b/src/test/regress/sql/with_basics.sql new file mode 100644 index 000000000..5d5e80e0a --- /dev/null +++ b/src/test/regress/sql/with_basics.sql @@ -0,0 +1,471 @@ +-- Test the basic CTE functionality and expected error messages + +CREATE TYPE xy AS (x int, y int); +SELECT run_command_on_workers('CREATE TYPE xy AS (x int, y int)'); + +-- CTEs in FROM should work +WITH cte AS ( + SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 1,2 LIMIT 5 +) +SELECT * FROM cte; + +-- CTEs in WHERE should work +WITH cte AS ( + SELECT user_id from users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + value_2 +FROM + users_table +WHERE + user_id IN (SELECT user_id FROM users_table) +ORDER BY + value_2 +LIMIT + 5; + +-- nested CTEs should work +WITH cte_1 AS ( + WITH cte_1_1 AS ( + SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 2 LIMIT 5 + ), + cte_1_2 AS ( + SELECT max(user_id) AS user_id FROM cte_1_1 + ) + SELECT user_id FROM cte_1_2 ORDER BY user_id +) +SELECT value_2 FROM users_table WHERE user_id IN (SELECT user_id FROM cte_1) ORDER BY value_2 LIMIT 1; + +-- Mix of FROM/WHERE queries +WITH cte_from AS ( + SELECT max(user_id) AS user_id, value_2, value_1 FROM users_table GROUP BY value_2, value_1 +), +cte_where AS ( + SELECT value_2 FROM events_table +) +SELECT + * +FROM + (SELECT max(user_id), max(value_2) AS value_2 FROM cte_from GROUP BY value_1) f +WHERE + value_2 IN (SELECT * FROM cte_where) +ORDER BY + 1, 2 +LIMIT + 5; + +-- CTE in subquery errors out +SELECT user_id FROM ( + WITH cte AS ( + SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 2 LIMIT 5 + ) + SELECT user_id FROM cte WHERE value_2 > 0 +) a; + +-- CTE outside of FROM/WHERE errors out +WITH cte AS ( + SELECT user_id FROM users_table WHERE value_2 IN (1, 2) +) +SELECT (SELECT * FROM cte); + +WITH cte_basic AS ( + SELECT user_id FROM users_table WHERE user_id = 1 +) +SELECT + (SELECT user_id FROM cte_basic), user_id +FROM + users_table; + +-- single-row sublink is acceptable when there is no FROM +WITH cte AS ( + SELECT user_id FROM users_table WHERE value_2 IN (1, 2) +) +SELECT (SELECT * FROM cte ORDER BY 1 LIMIT 1); + +-- group by partition column +WITH series AS ( + SELECT s AS once, s*2 AS twice FROM generate_series(1,10) s +) +SELECT user_id, count(*) +FROM + users_table +JOIN + series ON (user_id = once) +GROUP BY + user_id +ORDER BY + 1, 2 +LIMIT 5; + +-- group by non-partition column +WITH series AS ( + SELECT s AS once, s*2 AS twice FROM generate_series(1,10) s +) +SELECT + twice, min(user_id) +FROM + users_table +JOIN + series ON (user_id = once) +GROUP BY + twice +HAVING + twice > 5 +ORDER BY + 1, 2 +LIMIT 5; + +-- distinct in subquery on CTE +WITH one_user AS ( + SELECT user_id from users_table WHERE user_id = 1 +) +SELECT + user_id +FROM + users_table +WHERE + value_2 IN (SELECT DISTINCT user_id FROM one_user) +ORDER BY + user_id +LIMIT + 1; + +-- having in subquery on CTE +WITH one_user AS ( + SELECT user_id from users_table WHERE user_id = 1 +) +SELECT + user_id +FROM + users_table +WHERE + value_2 IN (SELECT user_id FROM one_user GROUP BY user_id HAVING count(*) > 0) +ORDER BY + user_id +LIMIT + 1; + +-- aggregate in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT min(user_id) AS user_id FROM top_users) top_users +JOIN + users_table USING (user_id); + +-- FOR UPDATE in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users FOR UPDATE) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- LIMIT in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users LIMIT 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- OFFSET in subquery on CTE +WITH top_users AS ( + SELECT user_id, value_2 FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users OFFSET 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- Unsupported join in CTE +WITH top_users AS ( + SELECT DISTINCT e.user_id FROM users_table u JOIN events_table e ON (u.user_id = e.user_id AND u.value_1 > e.value_2) +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users WHERE user_id > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- Join can be supported with another CTE +WITH events_table AS ( + SELECT * FROM events_table +), +top_users AS ( + SELECT DISTINCT e.user_id FROM users_table u JOIN events_table e ON (u.user_id = e.user_id AND u.value_1 > e.value_2) +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users WHERE user_id > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- Window functions in CTE +WITH top_users AS ( + SELECT row_number() OVER(), user_id FROM users_table ORDER BY user_id DESC LIMIT 10 +) +SELECT + user_id +FROM + (SELECT user_id FROM top_users WHERE user_id > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- Window functions that partition by the distribution column in subqueries in CTEs are ok +WITH top_users AS + (SELECT * + FROM + (SELECT row_number() OVER(PARTITION BY user_id) AS row_number, + user_id + FROM users_table) AS foo +) +SELECT user_id +FROM + (SELECT user_id + FROM top_users + WHERE row_number > 5) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- Unsupported aggregate in CTE +WITH top_users AS ( + SELECT array_agg(user_id ORDER BY value_2) user_ids FROM users_table +) +SELECT + user_id +FROM + (SELECT unnest(user_ids) user_id FROM top_users) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- array_agg in CTE +WITH top_users AS ( + SELECT array_agg(user_id) user_ids FROM users_table +) +SELECT + user_id +FROM + (SELECT unnest(user_ids) user_id FROM top_users) top_users +JOIN + users_table USING (user_id) +ORDER BY + user_id +LIMIT + 5; + +-- composite type array +WITH top_users AS ( + SELECT array_agg((value_1,value_2)::xy) AS p FROM users_table WHERE user_id % 2 = 0 +) +SELECT + e.user_id, sum(y) +FROM + (SELECT (unnest(p)).* FROM top_users) tops +JOIN + events_table e ON (tops.x = e.user_id) +GROUP BY + e.user_id +ORDER BY + 2 DESC, 1 +LIMIT + 5; + +-- SELECT * FROM (SELECT * FROM cte UNION SELECT * FROM distributed_table) a; should error out +WITH cte AS ( + SELECT * FROM users_table +) +SELECT * FROM ( + SELECT * FROM cte UNION (SELECT * FROM events_table) + ) a +ORDER BY + 1,2,3,4,5,6 +LIMIT + 10; + +SELECT * FROM ( + SELECT * FROM (WITH cte AS ( + SELECT * FROM users_table + ) + SELECT * FROM cte + )b UNION (SELECT * FROM events_table)) a +ORDER BY +1,2,3,4,5,6 +LIMIT +10; + +-- SELECT * FROM (SELECT * FROM cte UNION SELECT * FROM cte) a; should work +WITH cte AS ( + SELECT * FROM users_table WHERE user_id IN (1, 2) +) +SELECT + * +FROM + (SELECT * FROM cte UNION (SELECT * FROM cte)) a +ORDER BY + 1,2,3,4,5,6 +LIMIT + 5; + +WITH cte AS ( + SELECT * FROM users_table WHERE user_id IN (1, 2) ORDER BY 1,2,3 LIMIT 5 +), +cte_2 AS ( + SELECT * FROM users_table WHERE user_id IN (3, 4) ORDER BY 1,2,3 LIMIT 5 +) +SELECT * FROM cte UNION ALL SELECT * FROM cte_2; + +-- basic recursive CTE which should all error out +WITH RECURSIVE basic_recursive(x) AS ( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 +) +SELECT sum(x) FROM basic_recursive; + +WITH RECURSIVE basic_recursive AS ( + SELECT -1 as user_id, '2017-11-22 20:16:16.614779'::timestamp, -1, -1, -1, -1 + UNION ALL + SELECT * FROM users_table WHERE user_id>1 +) +SELECT * FROM basic_recursive ORDER BY user_id LIMIT 1; + + +-- basic_recursive in FROM should error out +SELECT + * +FROM +(WITH RECURSIVE basic_recursive AS ( + SELECT -1 as user_id, '2017-11-22 20:16:16.614779'::timestamp, -1, -1, -1, -1 + UNION ALL + SELECT * FROM users_table WHERE user_id>1 + ) + SELECT * FROM basic_recursive ORDER BY user_id LIMIT 1) cte_rec; + + +-- basic_recursive in WHERE with UNION ALL +SELECT + * +FROM + users_table +WHERE + user_id in +(WITH RECURSIVE basic_recursive AS ( + SELECT -1 as user_id + UNION ALL + SELECT user_id FROM users_table WHERE user_id>1 + ) + SELECT * FROM basic_recursive ORDER BY user_id LIMIT 1); + + +-- one recursive one regular CTE should error out +WITH RECURSIVE basic_recursive(x) AS( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 +), +basic AS ( + SELECT count(user_id) FROM users_table +) +SELECT x FROM basic, basic_recursive; + + +-- one recursive one regular which SELECTs from the recursive CTE under a simple SELECT +WITH RECURSIVE basic_recursive(x) AS( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 +), +basic AS ( + SELECT count(x) FROM basic_recursive +) +SELECT * FROM basic; + + +-- recursive CTE in a NESTED manner +WITH regular_cte AS ( + WITH regular_2 AS ( + WITH RECURSIVE recursive AS ( + VALUES (1) + UNION ALL + SELECT user_id + 1 FROM users_table WHERE user_id < 100 + ) + SELECT * FROM recursive + ) + SELECT * FROM regular_2 +) +SELECT * FROM regular_cte; + +-- CTEs should work with VIEWs as well +CREATE VIEW basic_view AS +SELECT * FROM users_table; + + +CREATE VIEW cte_view AS +WITH cte AS ( + SELECT * FROM users_table +) +SELECT user_id, max(value_1) as value_1 FROM cte GROUP BY 1; + + +WITH cte_user AS ( + SELECT basic_view.user_id,events_table.value_2 FROM basic_view join events_table on (basic_view.user_id = events_table.user_id) +) +SELECT user_id, sum(value_2) FROM cte_user GROUP BY 1 ORDER BY 1, 2; + +SELECT * FROM cte_view; + + +WITH cte_user_with_view AS +( + SELECT * FROM cte_view WHERE user_id < 3 +) +SELECT user_id, value_1 FROM cte_user_with_view ORDER BY 1, 2 LIMIT 10 OFFSET 3; + +DROP VIEW basic_view; +DROP VIEW cte_view; diff --git a/src/test/regress/sql/with_executors.sql b/src/test/regress/sql/with_executors.sql new file mode 100644 index 000000000..c1a89bd4c --- /dev/null +++ b/src/test/regress/sql/with_executors.sql @@ -0,0 +1,316 @@ +-- Confirm we can use local, router, real-time, and task-tracker execution + +CREATE SCHEMA with_executors; +SET search_path TO with_executors, public; +SET citus.enable_repartition_joins TO on; + +CREATE TABLE with_executors.local_table (id int); +INSERT INTO local_table VALUES (0), (1), (2), (3), (4), (5), (6), (7), (8), (9), (10); + +-- CTEs should be able to use local queries +WITH cte AS ( + WITH local_cte AS ( + SELECT * FROM local_table + ), + dist_cte AS ( + SELECT user_id FROM events_table + ) + SELECT * FROM local_cte join dist_cte on dist_cte.user_id=local_cte.id +) +SELECT count(*) FROM cte; + + +WITH cte AS ( + WITH local_cte AS ( + SELECT * FROM local_table + ), + dist_cte AS ( + SELECT user_id FROM events_table + ), + merger_cte AS ( + SELECT id as user_id FROM local_cte UNION (SELECT * FROM dist_cte) + ) + SELECT * FROM merger_cte WHERE user_id IN (1, 2, 3) +) +SELECT * FROM cte ORDER BY 1; + + +WITH cte AS ( + WITH local_cte AS ( + SELECT * FROM local_table WHERE id < 5 + ), + local_cte_2 AS ( + SELECT * FROM local_table WHERE id > 5 + ) + SELECT local_cte.id as id_1, local_cte_2.id as id_2 FROM local_cte,local_cte_2 +) +SELECT + * +FROM + cte +join + users_table +on + cte.id_1 = users_table.user_id +WHERE + cte.id_1 IN (3, 4, 5) +ORDER BY + 1,2,3,4,5,6,7 +LIMIT + 10; + + +-- CTEs should be able to use router queries +WITH cte AS ( + WITH router_cte AS ( + SELECT user_id, value_2 FROM users_table WHERE user_id = 1 + ), + router_cte_2 AS ( + SELECT user_id, event_type, value_2 FROM events_table WHERE user_id = 1 + ) + SELECT + router_cte.user_id as uid, event_type + FROM + router_cte, router_cte_2 +) +SELECT * FROM cte ORDER BY 2 LIMIT 5; + + +-- CTEs should be able to use real-time queries +WITH real_time_cte AS ( + SELECT * FROM users_table WHERE value_2 IN (1, 2, 3) +) +SELECT * FROM real_time_cte ORDER BY 1, 2, 3, 4, 5, 6 LIMIT 10; + + +-- router & real-time together +WITH cte AS ( + WITH router_cte AS ( + SELECT user_id, value_2 FROM users_table WHERE user_id = 1 + ), + real_time AS ( + SELECT user_id, event_type, value_2 FROM events_table + ) + SELECT + router_cte.user_id as uid, event_type + FROM + router_cte, real_time + WHERE + router_cte.user_id=real_time.user_id +) +SELECT * FROM cte WHERE uid=1 ORDER BY 2 LIMIT 5; + + +-- CTEs should be able to use task-tracker queries +WITH cte AS ( + WITH task_tracker_1 AS ( + SELECT + users_table.user_id as uid_1, users_table.value_2 + FROM + users_table + JOIN + events_table + ON + users_table.value_2=events_table.value_2 + ), + task_tracker_2 AS ( + SELECT + users_table.user_id as uid_2, users_table.value_3 + FROM + users_table + JOIN + events_table + ON + users_table.value_3=events_table.value_3 + ) + SELECT + uid_1, uid_2, value_2, value_3 + FROM + task_tracker_1 + JOIN + task_tracker_2 + ON + value_2 = value_3 +) +SELECT + uid_1, uid_2, cte.value_2, cte.value_3 +FROM + cte +JOIN + events_table +ON + cte.value_2 = events_table.event_type +ORDER BY + 1, 2, 3, 4 +LIMIT 10; + + +-- All combined +WITH cte AS ( + WITH task_tracker AS ( + SELECT + users_table.user_id as uid_1, users_table.value_2 as val_2 + FROM + users_table + JOIN + events_table + ON + users_table.value_2=events_table.value_2 + ), + real_time AS ( + SELECT * FROM users_table + ), + router_exec AS ( + SELECT * FROM events_table WHERE user_id = 1 + ), + local_table AS ( + SELECT * FROM local_table + ), + join_first_two AS ( + SELECT uid_1, time, value_3 FROM task_tracker JOIN real_time ON val_2=value_3 + ), + join_last_two AS ( + SELECT + router_exec.user_id, local_table.id + FROM + router_exec + JOIN + local_table + ON + router_exec.user_id=local_table.id + ) + SELECT * FROM join_first_two JOIN join_last_two ON id = value_3 ORDER BY 1,2,3,4,5 LIMIT 10 +) +SELECT DISTINCT uid_1, time, value_3 FROM cte ORDER BY 1, 2, 3 LIMIT 20; + +-- All combined with outer join +WITH cte AS ( + WITH task_tracker AS ( + SELECT + users_table.user_id as uid_1, users_table.value_2 as val_2 + FROM + users_table + JOIN + events_table + ON + users_table.value_2=events_table.value_2 + ), + real_time AS ( + SELECT * FROM users_table + ), + router_exec AS ( + SELECT * FROM events_table WHERE user_id = 1 + ), + local_table AS ( + SELECT * FROM local_table + ), + join_first_two AS ( + SELECT uid_1, time, value_3 FROM task_tracker JOIN real_time ON val_2=value_3 + ), + join_last_two AS ( + SELECT + router_exec.user_id, local_table.id + FROM + router_exec + JOIN + local_table + ON + router_exec.user_id=local_table.id + ) + SELECT uid_1, value_3 as val_3 FROM join_first_two JOIN join_last_two ON id = value_3 ORDER BY 1,2 LIMIT 10 +) +SELECT DISTINCT uid_1, val_3 FROM cte join events_table on cte.val_3=events_table.event_type ORDER BY 1, 2; + + +-- CTEs should not be able to terminate (the last SELECT) in a local query +WITH cte AS ( + SELECT * FROM users_table +) +SELECT count(*) FROM cte JOIN local_table ON (user_id = id); + +-- CTEs should be able to terminate a router query +WITH cte AS ( + WITH cte_1 AS ( + SELECT * FROM local_table WHERE id < 7 + ), + cte_2 AS ( + SELECT * FROM local_table WHERE id > 3 + ), + cte_dist AS ( + SELECT count(*) as u_id FROM users_table + ), + cte_merge AS ( + SELECT cte_1.id as id FROM cte_1 join cte_2 on TRUE + ) + SELECT count(*) FROM users_table join cte_merge on id=user_id +) +SELECT + row_number() OVER (), count(*) +FROM + cte, users_table +WHERE + cte.count=user_id and user_id=5; + + +-- CTEs should be able to terminate a real-time query +WITH cte AS ( + WITH cte_1 AS ( + SELECT * FROM local_table WHERE id < 7 + ), + cte_2 AS ( + SELECT * FROM local_table WHERE id > 3 + ), + cte_dist AS ( + SELECT count(*) as u_id FROM users_table + ), + cte_merge AS ( + SELECT cte_1.id as id FROM cte_1 join cte_2 on TRUE + ) + SELECT count(*) FROM users_table join cte_merge on id=user_id +) +SELECT count(*) FROM cte, users_table where cte.count=user_id; + + +SET citus.task_executor_type='task-tracker'; +-- CTEs shouldn't be able to terminate a task-tracker query +WITH cte_1 AS ( + SELECT + u_table.user_id as u_id, e_table.event_type + FROM + users_table as u_table + join + events_table as e_table + on + u_table.value_2=e_table.event_type + WHERE + u_table.user_id < 7 +), +cte_2 AS ( + SELECT + u_table.user_id as u_id, e_table.event_type + FROM + users_table as u_table + join + events_table as e_table + on + u_table.value_2=e_table.event_type + WHERE + u_table.user_id > 3 +), +cte_merge AS ( + SELECT + cte_1.u_id, cte_2.event_type + FROM + cte_1 + join + cte_2 + on cte_1.event_type=cte_2.u_id +) +SELECT + count(*) +FROM + users_table, cte_merge +WHERE + users_table.user_id = cte_merge.u_id; + +DROP SCHEMA with_executors CASCADE; diff --git a/src/test/regress/sql/with_join.sql b/src/test/regress/sql/with_join.sql new file mode 100644 index 000000000..f1d72a2df --- /dev/null +++ b/src/test/regress/sql/with_join.sql @@ -0,0 +1,219 @@ +CREATE SCHEMA with_join; +SET search_path TO with_join, public; +SET citus.next_shard_id TO 1501000; + +CREATE TABLE with_join.reference_table(user_id int); +SELECT create_reference_table('with_join.reference_table'); +INSERT INTO reference_table VALUES (6), (7); + +SET citus.enable_repartition_joins TO on; + +-- Two colocated CTE under a non-colocated join +WITH colocated_1 AS ( + SELECT + users_table.user_id, events_table.value_2 + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.user_id AND event_type IN (1, 2, 3) +), +colocated_2 AS ( + SELECT + users_table.user_id, events_table.value_2 + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.user_id AND event_type IN (4, 5, 6) +) +SELECT colocated_1.user_id, count(*) +FROM + colocated_1, colocated_2 +WHERE + colocated_1.value_2 = colocated_2.value_2 +GROUP BY + 1 +ORDER BY + 2 DESC, 1; + +-- Two non-colocated CTE under a co-located join +WITH non_colocated_1 AS ( + SELECT + users_table.user_id + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.value_2 AND event_type IN (1, 2, 3) +), +non_colocated_2 AS ( + SELECT + users_table.user_id + FROM + users_table, events_table + WHERE + users_table.user_id = events_table.value_2 AND event_type IN (4, 5, 6) +) + +SELECT non_colocated_1.user_id, count(*) +FROM + non_colocated_1, non_colocated_2 +WHERE + non_colocated_1.user_id = non_colocated_2.user_id +GROUP BY + 1 +ORDER BY + 2 DESC, 1; + + +-- Subqueries in WHERE and FROM are mixed +-- In this query, only subquery in WHERE is not a colocated join +WITH users_events AS ( + WITH colocated_join AS ( + SELECT + users_table.user_id as uid, event_type + FROM + users_table + join + events_table + on (users_table.user_id = events_table.user_id) + WHERE + events_table.event_type IN (1, 2, 3) + ), + colocated_join_2 AS ( + SELECT + users_table.user_id, event_type + FROM + users_table + join + events_table + on (users_table.user_id = events_table.user_id) + WHERE + events_table.event_type IN (4, 5, 6) + ) + SELECT + uid, colocated_join.event_type + FROM + colocated_join, + colocated_join_2 + WHERE + colocated_join.uid = colocated_join_2.user_id AND + colocated_join.event_type IN ( + WITH some_events AS ( + SELECT + event_type + FROM + events_table + WHERE + user_id < 100 + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 + ) + SELECT + * + FROM + some_events + ) +) +SELECT + * +FROM + users_events +ORDER BY + 1, 2 +LIMIT + 20; + +-- cte LEFT JOIN distributed_table should error out +WITH cte AS ( + SELECT * FROM users_table WHERE user_id = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + cte +LEFT JOIN + events_table ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; + +-- cte RIGHT JOIN distributed_table should work +WITH cte AS ( + SELECT * FROM users_table WHERE user_id = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + cte +RIGHT JOIN + events_table ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; + +-- distributed_table LEFT JOIN cte should work +WITH cte AS ( + SELECT * FROM users_table WHERE value_1 = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + events_table +LEFT JOIN + cte ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; + +-- distributed_table RIGHT JOIN cte should error out +WITH cte AS ( + SELECT * FROM users_table WHERE value_1 = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + events_table +RIGHT JOIN + cte ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; + +-- cte FULL JOIN distributed_table should error out +WITH cte AS ( + SELECT * FROM users_table WHERE user_id = 1 ORDER BY value_1 +) +SELECT + cte.user_id, cte.time, events_table.event_type +FROM + events_table +FULL JOIN + cte ON cte.user_id = events_table.user_id +ORDER BY + 1,2,3 +LIMIT + 5; + +-- Joins with reference tables are planned as router queries +WITH cte AS ( + SELECT value_2, max(user_id) AS user_id FROM users_table WHERE value_2 = 1 GROUP BY value_2 HAVING count(*) > 1 +) +SELECT + row_number() OVER(), cte.user_id +FROM + cte +FULL JOIN + reference_table ON cte.user_id + 1 = reference_table.user_id +ORDER BY + user_id +LIMIT + 5; + +RESET client_min_messages; +DROP SCHEMA with_join CASCADE; diff --git a/src/test/regress/sql/with_modifying.sql b/src/test/regress/sql/with_modifying.sql new file mode 100644 index 000000000..60ced959a --- /dev/null +++ b/src/test/regress/sql/with_modifying.sql @@ -0,0 +1,124 @@ +-- Tests for modifying CTEs and CTEs in modifications +SET citus.next_shard_id TO 1502000; + +CREATE SCHEMA with_modifying; +SET search_path TO with_modifying, public; + +CREATE TABLE with_modifying.modify_table (id int, val int); +SELECT create_distributed_table('modify_table', 'id'); + +CREATE TABLE with_modifying.users_table (LIKE public.users_table INCLUDING ALL); +SELECT create_distributed_table('with_modifying.users_table', 'user_id'); +INSERT INTO with_modifying.users_table SELECT * FROM public.users_table; + +-- basic insert query in CTE +WITH basic_insert AS ( + INSERT INTO users_table VALUES (1), (2), (3) RETURNING * +) +SELECT + * +FROM + basic_insert; + +-- single-shard UPDATE in CTE +WITH basic_update AS ( + UPDATE users_table SET value_3=42 WHERE user_id=0 RETURNING * +) +SELECT + * +FROM + basic_update; + +-- multi-shard UPDATE in CTE +WITH basic_update AS ( + UPDATE users_table SET value_3=42 WHERE value_2=1 RETURNING * +) +SELECT + * +FROM + basic_update; + +-- single-shard DELETE in CTE +WITH basic_delete AS ( + DELETE FROM users_table WHERE user_id=42 RETURNING * +) +SELECT + * +FROM + basic_delete; + +-- multi-shard DELETE in CTE +WITH basic_delete AS ( + DELETE FROM users_table WHERE value_2=42 RETURNING * +) +SELECT + * +FROM + basic_delete; + +-- INSERT...SELECT query in CTE +WITH copy_table AS ( + INSERT INTO users_table SELECT * FROM users_table RETURNING * +) +SELECT + * +FROM + copy_table; + +-- CTEs prior to INSERT...SELECT via the coordinator should work +WITH cte AS ( + SELECT user_id FROM users_table WHERE value_2 IN (1, 2) +) +INSERT INTO modify_table (SELECT * FROM cte); + + +WITH cte_1 AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2, 3, 4) +), +cte_2 AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (3, 4, 5, 6) +) +INSERT INTO modify_table (SELECT cte_1.user_id FROM cte_1 join cte_2 on cte_1.value_2=cte_2.value_2); + + +-- even if this is an INSERT...SELECT, the CTE is under SELECT +WITH cte AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2) +) +INSERT INTO modify_table (SELECT (SELECT value_2 FROM cte GROUP BY value_2)); + + +-- CTEs prior to any other modification should error out +WITH cte AS ( + SELECT value_2 FROM users_table WHERE user_id IN (1, 2, 3) +) +DELETE FROM modify_table WHERE id IN (SELECT value_2 FROM cte); + + +WITH cte AS ( + SELECT value_2 FROM users_table WHERE user_id IN (1, 2, 3) +) +UPDATE modify_table SET val=-1 WHERE val IN (SELECT * FROM cte); + + +WITH cte AS ( + WITH basic AS ( + SELECT value_2 FROM users_table WHERE user_id IN (1, 2, 3) + ) + INSERT INTO modify_table (SELECT * FROM basic) RETURNING * +) +UPDATE modify_table SET val=-2 WHERE id IN (SELECT id FROM cte); + + +WITH cte AS ( + WITH basic AS ( + SELECT * FROM events_table WHERE event_type = 5 + ), + basic_2 AS ( + SELECT user_id FROM users_table + ) + INSERT INTO modify_table (SELECT user_id FROM events_table) RETURNING * +) +SELECT * FROM cte; + +DROP SCHEMA with_modifying CASCADE; diff --git a/src/test/regress/sql/with_nested.sql b/src/test/regress/sql/with_nested.sql new file mode 100644 index 000000000..4a3a9b392 --- /dev/null +++ b/src/test/regress/sql/with_nested.sql @@ -0,0 +1,328 @@ +-- Complex nested CTEs +CREATE SCHEMA with_nested; +SET search_path tO with_nested, public; + +CREATE TABLE with_nested.local_users (user_id int, event_type int); +INSERT INTO local_users VALUES (0, 0), (1, 4), (1, 7), (2, 1), (3, 3), (5, 4), (6, 2), (10, 7); + +-- Can refer to outer CTE because it is recursively planned first +WITH cte_1 AS ( + SELECT DISTINCT user_id FROM users_table +), +cte_2 AS ( + WITH cte_1_1 AS ( + SELECT * FROM cte_1 WHERE user_id > 1 + ) + SELECT * FROM cte_1_1 WHERE user_id < 3 +) +SELECT user_id FROM cte_2 LIMIT 1; + +-- Nested CTEs +WITH users_events AS ( + WITH users_events_2 AS ( + SELECT + users_table.user_id as user_id, + events_table.event_type as event_type + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id + GROUP BY + users_table.user_id, + events_table.event_type + ) + SELECT + u_events.user_id, events_table.event_type + FROM + users_events_2 as u_events, + events_table + WHERE + u_events.user_id = events_table.user_id + ) +SELECT + * +FROM + users_events +ORDER BY + 1, 2 +LIMIT 20; + +-- Nested CTEs +WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + WITH users_events AS ( + SELECT + users_table.user_id as user_id, + events_table.event_type as event_type + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id + GROUP BY + users_table.user_id, + events_table.event_type + + ) + SELECT + uid, event_type, value_2, value_3 + FROM + ( + ( + SELECT + user_id as uid + FROM + users_events + ) users + join + events_table + on + users.uid = events_table.event_type + ) a + ) + SELECT + * + FROM + users_events + ORDER BY + 1, 3, 2, 4 + LIMIT 100 + ) + SELECT + * + FROM + users_events + LIMIT 90 + ) + SELECT + * + FROM + users_events + LIMIT 50 + ) + SELECT + uid, event_type, value_2, sum(value_3) as sum_3 + FROM + users_events + GROUP BY + 1, 2, 3 + LIMIT 40 + ) + SELECT + uid, event_type, sum(value_2) as sum_2, sum(sum_3) as sum_3 + FROM + users_events + GROUP BY + 1, 2 + LIMIT 30 +) +SELECT + uid, avg(event_type), sum(sum_2), sum(sum_3) +FROM + users_events +GROUP BY + 1; + + +-- Nested CTEs +WITH users_events AS ( + -- router select query + WITH users_events_1 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 1 + ), + -- real-time select query + users_events_2_3 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 2 + OR + user_id = 3 + ), + -- router select query + -- sub CTE is a real-time executor query but the top level is router select + users_events_4 AS ( + WITH users_events_4_5 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 4 + OR + user_id = 5 + ) + SELECT + * + FROM + users_events_4_5 + WHERE + user_id = 4 + ), + -- merge all the results from CTEs + merged_users AS ( + SELECT + * + FROM + users_events_1 + UNION + SELECT + * + FROM + users_events_2_3 + UNION + SELECT + * + FROM + users_events_4 + ) + SELECT + * + FROM + merged_users +) +SELECT + * +FROM + users_events +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT + 20; + + +-- Nested CTEs - joined with local table. Not supported yet. +WITH users_events AS ( + -- router select query + WITH users_events_1 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 1 + ), + -- real-time select query + users_events_2_3 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 2 + OR + user_id = 3 + ), + -- router select query + -- sub CTE is a real-time executor query but the top level is router select + users_events_4 AS ( + WITH users_events_4_5 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = 4 + OR + user_id = 5 + ) + SELECT + * + FROM + users_events_4_5 + WHERE + user_id = 4 + ) + -- merge all the results from CTEs + SELECT + * + FROM + users_events_1 + UNION + SELECT + * + FROM + users_events_2_3 + UNION + SELECT + * + FROM + users_events_4 +) +SELECT + * +FROM + users_events +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT + 20; + +-- access to uncle, use window function, apply aggregates, use group by, LIMIT/OFFSET +WITH cte1 AS ( + WITH cte1_1 AS ( + WITH cte1_1_1 AS ( + SELECT user_id, time, value_2 FROM users_table + ), + cte1_1_2 AS ( + SELECT + user_id, count + FROM ( + SELECT + user_id, + count(value_2) OVER (PARTITION BY user_id) + FROM + users_table + GROUP BY 1, users_table.value_2 + )aa + GROUP BY + 1,2 + ORDER BY + 1,2 + LIMIT + 4 + OFFSET + 2 + ) + SELECT cte1_1_1.user_id, cte1_1_1.time, cte1_1_2.count FROM cte1_1_1 join cte1_1_2 on cte1_1_1.user_id=cte1_1_2.user_id + ), + cte1_2 AS ( + WITH cte1_2_1 AS ( + SELECT + user_id, time, avg(value_1)::real as value_1, min(value_2) as value_2 + FROM + users_table + GROUP BY + 1, 2 + ), + cte1_2_2 AS ( + SELECT cte1_2_1.user_id, cte1_1.time, cte1_2_1.value_1, cte1_1.count FROM cte1_2_1 join cte1_1 on cte1_2_1.time=cte1_1.time and cte1_2_1.user_id=cte1_1.user_id + ) + SELECT * FROM cte1_2_2 + ) + SELECT * FROM cte1_2 +), +cte2 AS ( + WITH cte2_1 AS ( + WITH cte2_1_1 AS ( + SELECT * FROM cte1 + ) + SELECT user_id, time, value_1, min(count) FROM cte2_1_1 GROUP BY 1, 2, 3 ORDER BY 1,2,3 + ) + SELECT * FROM cte2_1 LIMIT 3 OFFSET 2 +) +SELECT * FROM cte2; + +DROP SCHEMA with_nested CASCADE; diff --git a/src/test/regress/sql/with_partitioning.sql b/src/test/regress/sql/with_partitioning.sql new file mode 100644 index 000000000..d21faeba6 --- /dev/null +++ b/src/test/regress/sql/with_partitioning.sql @@ -0,0 +1,73 @@ +CREATE SCHEMA with_partitioning; +SET search_path TO with_partitioning, public; + +SET citus.shard_replication_factor TO 1; + +CREATE TABLE with_partitioning.local_users_2 (user_id int, event_type int); +INSERT INTO local_users_2 VALUES (0, 0), (1, 4), (1, 7), (2, 1), (3, 3), (5, 4), (6, 2), (10, 7); + +CREATE TABLE with_partitioning.partitioning_test(id int, time date) PARTITION BY RANGE (time); + +-- create its partitions +CREATE TABLE with_partitioning.partitioning_test_2017 PARTITION OF partitioning_test FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); +CREATE TABLE with_partitioning.partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); + +-- load some data and distribute tables +INSERT INTO partitioning_test VALUES (1, '2017-11-23'); +INSERT INTO partitioning_test VALUES (2, '2010-07-07'); + +INSERT INTO partitioning_test_2017 VALUES (3, '2017-11-22'); +INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); + +-- distribute partitioned table +SELECT create_distributed_table('with_partitioning.partitioning_test', 'id'); + + +-- Join of a CTE on distributed table and then join with a partitioned table +WITH cte AS ( + SELECT * FROM users_table +) +SELECT DISTINCT ON (id) id, cte.time FROM cte join partitioning_test on cte.time::date=partitioning_test.time ORDER BY 1, 2 LIMIT 3; + + +-- Join of a CTE on distributed table and then join with a partitioned table hitting on only one partition +WITH cte AS ( + SELECT * FROM users_table +) +SELECT DISTINCT ON (id) id, cte.time FROM cte join partitioning_test on cte.time::date=partitioning_test.time WHERE partitioning_test.time >'2017-11-20' ORDER BY 1, 2 LIMIT 3; + + +-- Join with a distributed table and then join of two CTEs +WITH cte AS ( + SELECT id, time FROM partitioning_test +), +cte_2 AS ( + SELECT * FROM partitioning_test WHERE id > 2 +), +cte_joined AS ( + SELECT user_id, cte_2.time FROM users_table join cte_2 on (users_table.time::date = cte_2.time) +), +cte_joined_2 AS ( + SELECT user_id, cte_joined.time FROM cte_joined join cte on (cte_joined.time = cte.time) +) +SELECT DISTINCT ON (event_type) event_type, cte_joined_2.user_id FROM events_table join cte_joined_2 on (cte_joined_2.time=events_table.time::date) ORDER BY 1, 2 LIMIT 10 OFFSET 2; + + +-- Join a partitioned table with a local table (both in CTEs) +-- and then with a distributed table. After all join with a +-- partitioned table again +WITH cte AS ( + SELECT id, time FROM partitioning_test +), +cte_2 AS ( + SELECT * FROM local_users_2 +), +cte_joined AS ( + SELECT user_id, cte.time FROM cte join cte_2 on (cte.id = cte_2.user_id) +), +cte_joined_2 AS ( + SELECT users_table.user_id, cte_joined.time FROM cte_joined join users_table on (cte_joined.time = users_table.time::date) +) +SELECT DISTINCT ON (id) id, cte_joined_2.time FROM cte_joined_2 join partitioning_test on (cte_joined_2.time=partitioning_test.time) ORDER BY 1, 2; + +DROP SCHEMA with_partitioning CASCADE; diff --git a/src/test/regress/sql/with_prepare.sql b/src/test/regress/sql/with_prepare.sql new file mode 100644 index 000000000..ac26508f1 --- /dev/null +++ b/src/test/regress/sql/with_prepare.sql @@ -0,0 +1,227 @@ +-- prepared statements +PREPARE prepared_test_1 AS +WITH basic AS( + SELECT * FROM users_table +) +SELECT + * +FROM + basic +WHERE + basic.value_2 IN (1, 2, 3) +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT 10; + + +PREPARE prepared_test_2 AS +WITH users_events AS( + SELECT + users_table.user_id as user_id, + events_table.event_type as event_type + FROM + users_table, + events_table + WHERE + users_table.user_id = events_table.user_id + GROUP BY + users_table.user_id, + events_table.event_type +), +event_attendee_count AS( + SELECT + event_type, count(user_id) + FROM + users_events + GROUP BY + 1 +), +user_coolness AS( + SELECT + user_id, + sum(count) + FROM + users_events + join + event_attendee_count + on (users_events.event_type = event_attendee_count.event_type) + GROUP BY + user_id +) +SELECT + * +FROM + user_coolness +ORDER BY + 2, 1 +LIMIT + 10; + + +PREPARE prepared_test_3(integer) AS +WITH users_events AS( + -- events 1 and 2 only + WITH spec_events AS( + SELECT + * + FROM + events_table + WHERE + event_type IN (1, 2) + ) + -- users who have done 1 or 2 + SELECT + users_table.user_id, + spec_events.event_type + FROM + users_table + join + spec_events + on (users_table.user_id = spec_events.user_id) + ORDER BY + 1, + event_type +), +event_attendee_count AS( + -- distinct attendee count of each event in users_event + WITH event_attendee_count AS( + SELECT + event_type, count(user_id) + FROM + users_events + GROUP BY + 1 + ) + -- distinct attendee count of first 3 events + SELECT + * + FROM + event_attendee_count + ORDER BY + event_type + LIMIT 3 +), +-- measure how cool an attendee is by checking the number of events he attended +user_coolness AS( + SELECT + user_id, + sum(count) + FROM + users_events + join + event_attendee_count + on (users_events.event_type = $1) + GROUP BY + user_id +) +SELECT + * +FROM + user_coolness +ORDER BY + 2, 1 +LIMIT + 10; + + +PREPARE prepared_test_4(integer, integer, integer) AS +WITH basic AS( + SELECT * FROM users_table WHERE value_2 IN ($1, $2, $3) +) +SELECT + * +FROM + basic +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT 10; + + +-- prepared statement which inserts in a CTE should fail +PREPARE prepared_partition_column_insert(integer) AS +WITH prepared_insert AS ( + INSERT INTO users_table VALUES ($1) RETURNING * +) +SELECT * FROM prepared_insert; + + +PREPARE prepared_test_5(integer, integer, integer) AS +-- router select query +WITH users_events_1 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = $1 +), +-- real-time select query +users_events_2_3 AS ( + SELECT + * + FROM + users_table + WHERE + user_id = $2 + OR + user_id = $3 +), +merged_users AS ( + SELECT + * + FROM + users_events_1 + UNION + SELECT + * + FROM + users_events_2_3 +) +SELECT + * +FROM + merged_users +ORDER BY + 1, 2, 3, 4, 5, 6 +LIMIT 10; + + + +EXECUTE prepared_test_1; +EXECUTE prepared_test_1; +EXECUTE prepared_test_1; +EXECUTE prepared_test_1; +EXECUTE prepared_test_1; +EXECUTE prepared_test_1; + +EXECUTE prepared_test_2; +EXECUTE prepared_test_2; +EXECUTE prepared_test_2; +EXECUTE prepared_test_2; +EXECUTE prepared_test_2; +EXECUTE prepared_test_2; + +EXECUTE prepared_test_3(1); +EXECUTE prepared_test_3(2); +EXECUTE prepared_test_3(3); +EXECUTE prepared_test_3(4); +EXECUTE prepared_test_3(5); +EXECUTE prepared_test_3(6); + +EXECUTE prepared_test_4(1, 2, 3); +EXECUTE prepared_test_4(2, 3, 4); +EXECUTE prepared_test_4(3, 4, 5); +EXECUTE prepared_test_4(4, 5, 6); +EXECUTE prepared_test_4(5, 6, 7); +EXECUTE prepared_test_4(6, 7, 8); + +EXECUTE prepared_test_5(1, 2, 3); +EXECUTE prepared_test_5(2, 3, 4); +EXECUTE prepared_test_5(3, 4, 5); +EXECUTE prepared_test_5(4, 5, 6); +EXECUTE prepared_test_5(5, 6, 7); +EXECUTE prepared_test_5(6, 7, 8); + +EXECUTE prepared_partition_column_insert(1); + +DEALLOCATE ALL; diff --git a/src/test/regress/sql/with_where.sql b/src/test/regress/sql/with_where.sql new file mode 100644 index 000000000..930c51474 --- /dev/null +++ b/src/test/regress/sql/with_where.sql @@ -0,0 +1,153 @@ +-- More complex CTEs in WHERE + +SET citus.enable_repartition_joins TO on; + +-- CTE in WHERE basic +WITH events AS ( + SELECT + event_type + FROM + events_table + WHERE + user_id < 5 + GROUP BY + event_type + ORDER BY + event_type + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + event_type +IN + (SELECT + event_type + FROM + events); + +WITH users AS ( + SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.user_id = users_table.user_id + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + ( + SELECT + * + FROM + users + ); + + +WITH users AS ( + SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.user_id = users_table.user_id + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + ( + SELECT + * + FROM + users + ); + + +-- CTE with non-colocated join in WHERE +WITH users AS ( + SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.value_2 = users_table.value_2 + GROUP BY + 1 + ORDER BY + 1 + LIMIT 10 +) +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + ( + SELECT + * + FROM + users + ); + +-- CTE in WHERE basic +SELECT + count(*) +FROM + events_table +WHERE + event_type +IN + (WITH events AS ( + SELECT + event_type + FROM + events_table + WHERE user_id < 5 + GROUP BY + 1 + ORDER BY + 1) + SELECT * FROM events LIMIT 10 + ); + +-- CTE with non-colocated join in WHERE +SELECT + count(*) +FROM + events_table +WHERE + user_id IN + (WITH users AS + (SELECT + events_table.user_id + FROM + events_table, users_table + WHERE + events_table.value_2 = users_table.value_2 + GROUP BY + 1 + ORDER BY + 1 + ) + SELECT * FROM users LIMIT 10 + );