diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 4a59512f2..de809848c 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -326,6 +326,7 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags) scanState->distributedPlan = currentPlan; Job *workerJob = currentPlan->workerJob; + Query *jobQuery = workerJob->jobQuery; if (ModifyJobNeedsEvaluation(workerJob)) @@ -367,16 +368,22 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags) RebuildQueryStrings(workerJob); } - /* - * Now that we know the shard ID(s) we can acquire the necessary shard metadata - * locks. Once we have the locks it's safe to load the placement metadata. - */ - /* prevent concurrent placement changes */ - AcquireMetadataLocks(workerJob->taskList); + /* We skip shard related things if the job contains only local tables */ + if (!ModifyLocalTableJob(workerJob)) + { + /* + * Now that we know the shard ID(s) we can acquire the necessary shard metadata + * locks. Once we have the locks it's safe to load the placement metadata. + */ + + /* prevent concurrent placement changes */ + AcquireMetadataLocks(workerJob->taskList); + + /* modify tasks are always assigned using first-replica policy */ + workerJob->taskList = FirstReplicaAssignTaskList(workerJob->taskList); + } - /* modify tasks are always assigned using first-replica policy */ - workerJob->taskList = FirstReplicaAssignTaskList(workerJob->taskList); /* * Now that we have populated the task placements we can determine whether @@ -537,9 +544,12 @@ RegenerateTaskForFasthPathQuery(Job *workerJob) shardId = GetAnchorShardId(shardIntervalList); } + bool isLocalTableModification = false; GenerateSingleShardRouterTaskList(workerJob, relationShardList, - placementList, shardId); + placementList, + shardId, + isLocalTableModification); } diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index 35dc01cf4..af1563129 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -78,7 +78,6 @@ static bool DistributedTableSizeOnWorker(WorkerNode *workerNode, Oid relationId, uint64 *tableSize); static List * ShardIntervalsOnWorkerGroup(WorkerNode *workerNode, Oid relationId); static void ErrorIfNotSuitableToGetSize(Oid relationId); -static ShardPlacement * ShardPlacementOnGroup(uint64 shardId, int groupId); /* exports for SQL callable functions */ @@ -1295,7 +1294,7 @@ UpdatePartitionShardPlacementStates(ShardPlacement *parentShardPlacement, char s * of the shard on the given group. If no such placement exists, the function * return NULL. */ -static ShardPlacement * +ShardPlacement * ShardPlacementOnGroup(uint64 shardId, int groupId) { List *placementList = ShardPlacementList(shardId); diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 21de545e2..a0c27fc66 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -410,11 +410,10 @@ AdjustPartitioningForDistributedPlanning(List *rangeTableList, /* * We want Postgres to behave partitioned tables as regular relations * (i.e. we do not want to expand them to their partitions). To do this - * we set each distributed partitioned table's inh flag to appropriate + * we set each partitioned table's inh flag to appropriate * value before and after dropping to the standart_planner. */ if (rangeTableEntry->rtekind == RTE_RELATION && - IsCitusTable(rangeTableEntry->relid) && PartitionedTable(rangeTableEntry->relid)) { rangeTableEntry->inh = setPartitionedTablesInherited; diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 5949f1b18..f2da9ef90 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -789,7 +789,8 @@ RouterModifyTaskForShardInterval(Query *originalQuery, &relationShardList, &prunedShardIntervalListList, replacePrunedQueryWithDummy, - &multiShardModifyQuery, NULL); + &multiShardModifyQuery, NULL, + false); Assert(!multiShardModifyQuery); diff --git a/src/backend/distributed/planner/local_distributed_join_planner.c b/src/backend/distributed/planner/local_distributed_join_planner.c new file mode 100644 index 000000000..17eee2724 --- /dev/null +++ b/src/backend/distributed/planner/local_distributed_join_planner.c @@ -0,0 +1,573 @@ +/*------------------------------------------------------------------------- + * + * local_distributed_join_planner.c + * + * This file contains functions to convert convert local-distributed + * tables to subqueries so that they can be planned by the router planner. + * + * + * The current algorithm checks if there is any table in the `jointree` that + * should be converted, if so it creates conversion candidates. + * With conversion candidates, it will convert either a distributed table or a local table to a + * subquery until it is plannable by router planner. It will choose a distributed table if we + * expect it to return few rows, such as a constant equality filter on a unique column. + * + * ```sql + * -- assuming dist.a is a unique column, this will convert distributed table + * SELECT * FROM dist join local ON(a) where dist.a = 5; + * ``` + * + * If the uniqueness is defined on multiple columns such as `dist.a, dist.b` + * then distributed table will only be chosen if there is a constant equality in all of the columns such as: + * + * ```sql + * SELECT * FROM dist join local ON(a) where dist.a = 5 AND dist.b =10; -- this will choose distributed table + * SELECT * FROM dist join local ON(a) where dist.a = 5 AND dist.b >10; -- this won't since no equality on dist.b + * SELECT * FROM dist join local ON(a) where dist.a = 5; -- this won't since no equality on dist.b + * ``` + * + * The algorithm will also not favor distributed tables if there exists a + * distributed table which is expected to return many rows, because in that + * case we will already plan local tables hence there is no point in converting some distributed tables. + * + * ```sql + * -- here only the local table will be chosen + * SELECT * FROM dist_without_unique JOIN dist_with_unique USING(a) join local USING (a); + * ``` + * + * this also makes the algorithm consistent. + * + * The algorithm can understand `OR` and `AND` expressions in the filters. + * + * There is a GUC called `local_table_join_policy` consisting of 4 modes: + * `none`: don't do any conversion + * `prefer-local`: prefer converting local tables if there is + * `prefer-distributed`: prefer converting distributed tables if there is + * `auto`: use the above mechanism to decide (constant equality on unique column) + * + * `auto` mode is the default. + * + * While converting to a subquery, we use a trick to avoid unnecessary network bandwidth, + * if there are columns that are not required in a table that will be converted to a subquery, We do: + * + * ```sql + * SELECT t.a, NULL, NULL (SELECT a FROM table) t + * ``` + * + * instead of + * + * ```sql + * SELECT a, NULL, NULL FROM table + * ``` + * + * There are NULLs in the query because we currently don't have an easy way to update the Vars + * that reference the non-required ones and we don't want to break the postgres query. + * + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/pg_version_constants.h" + +#include "funcapi.h" + +#include "catalog/pg_type.h" +#include "catalog/pg_class.h" +#include "catalog/pg_index.h" +#include "distributed/citus_nodes.h" +#include "distributed/citus_ruleutils.h" +#include "distributed/commands.h" +#include "distributed/commands/multi_copy.h" +#include "distributed/distributed_planner.h" +#include "distributed/errormessage.h" +#include "distributed/local_distributed_join_planner.h" +#include "distributed/listutils.h" +#include "distributed/log_utils.h" +#include "distributed/metadata_cache.h" +#include "distributed/multi_logical_planner.h" +#include "distributed/multi_logical_optimizer.h" +#include "distributed/multi_router_planner.h" +#include "distributed/multi_physical_planner.h" +#include "distributed/multi_server_executor.h" +#include "distributed/query_colocation_checker.h" +#include "distributed/query_pushdown_planning.h" +#include "distributed/recursive_planning.h" +#include "distributed/relation_restriction_equivalence.h" +#include "distributed/log_utils.h" +#include "distributed/shard_pruning.h" +#include "distributed/version_compat.h" +#include "lib/stringinfo.h" +#include "optimizer/clauses.h" +#if PG_VERSION_NUM >= PG_VERSION_12 +#include "optimizer/optimizer.h" +#else +#include "optimizer/var.h" +#endif +#include "optimizer/planner.h" +#include "optimizer/prep.h" +#include "parser/parsetree.h" +#include "nodes/makefuncs.h" +#include "nodes/nodeFuncs.h" +#include "nodes/nodes.h" +#include "nodes/nodeFuncs.h" +#include "nodes/pg_list.h" +#include "nodes/primnodes.h" +#if PG_VERSION_NUM >= PG_VERSION_12 +#include "nodes/pathnodes.h" +#else +#include "nodes/relation.h" +#endif +#include "utils/builtins.h" +#include "utils/guc.h" +#include "utils/lsyscache.h" + +#define INVALID_RTE_IDENTITY -1 + +/* + * Managed via a GUC + */ +int LocalTableJoinPolicy = LOCAL_JOIN_POLICY_AUTO; + +/* + * RangeTableEntryDetails contains some information about + * a range table entry so that we don't need to calculate + * them over and over. + */ +typedef struct RangeTableEntryDetails +{ + RangeTblEntry *rangeTableEntry; + List *requiredAttributeNumbers; + bool hasConstantFilterOnUniqueColumn; +} RangeTableEntryDetails; + +/* + * ConversionCandidates contains candidates that could + * be converted to a subquery. This is used as a convenience to + * first generate all the candidates and then choose which ones to convert. + */ +typedef struct ConversionCandidates +{ + List *distributedTableList; /* reference or distributed table */ + List *localTableList; /* local or citus local table */ +}ConversionCandidates; + + +/* + * IndexColumns contains the column numbers for an index. + * For example if there is an index on (a, b) then it will contain + * their column numbers (1,2). + */ +typedef struct IndexColumns +{ + List *indexColumnNos; +}IndexColumns; + +/* + * ConversionChoice represents which conversion group + * to convert to a subquery. Currently we either convert all + * local tables, or distributed tables. + */ +typedef enum ConversionChoice +{ + CONVERT_LOCAL_TABLES = 1, + CONVERT_DISTRIBUTED_TABLES = 2 +}ConversionChoice; + +static bool HasConstantFilterOnUniqueColumn(RangeTblEntry *rangeTableEntry, + RelationRestriction *relationRestriction); +static List * RequiredAttrNumbersForRelation(RangeTblEntry *relationRte, + PlannerRestrictionContext * + plannerRestrictionContext); +static ConversionCandidates * CreateConversionCandidates(PlannerRestrictionContext * + plannerRestrictionContext, + List *rangeTableList, + int resultRTEIdentity); +static void AppendUniqueIndexColumnsToList(Form_pg_index indexForm, List **uniqueIndexes); +static ConversionChoice GetConversionChoice(ConversionCandidates * + conversionCandidates, + PlannerRestrictionContext * + plannerRestrictionContext); +static bool AllRangeTableEntriesHaveUniqueIndex(List *rangeTableEntryDetailsList); +static bool FirstIsSuperSetOfSecond(List *firstIntList, List *secondIntList); +static void ConvertRTEsToSubquery(List *rangeTableEntryDetailsList, + RecursivePlanningContext *context); +static int ResultRTEIdentity(Query *query); +static List * RTEListToConvert(ConversionCandidates *conversionCandidates, + ConversionChoice conversionChoice); + + +/* + * RecursivelyPlanLocalTableJoins gets a query and the planner + * restrictions. As long as the query is not plannable by router planner, + * it converts either a local or distributed table to a subquery. + */ +void +RecursivelyPlanLocalTableJoins(Query *query, + RecursivePlanningContext *context) +{ + PlannerRestrictionContext *plannerRestrictionContext = + GetPlannerRestrictionContext(context); + + List *rangeTableList = query->rtable; + int resultRTEIdentity = ResultRTEIdentity(query); + ConversionCandidates *conversionCandidates = + CreateConversionCandidates(plannerRestrictionContext, + rangeTableList, resultRTEIdentity); + + ConversionChoice conversionChoise = + GetConversionChoice(conversionCandidates, plannerRestrictionContext); + + + List *rteListToConvert = RTEListToConvert(conversionCandidates, conversionChoise); + ConvertRTEsToSubquery(rteListToConvert, context); +} + + +/* + * ResultRTEIdentity returns the result RTE's identity if it exists, + * otherwise it returns INVALID_RTE_INDENTITY + */ +static int +ResultRTEIdentity(Query *query) +{ + int resultRTEIdentity = INVALID_RTE_IDENTITY; + if (IsModifyCommand(query)) + { + RangeTblEntry *resultRTE = ExtractResultRelationRTE(query); + resultRTEIdentity = GetRTEIdentity(resultRTE); + } + return resultRTEIdentity; +} + + +/* + * RTEListToConvert to converts returns a list of RTEs that should + * be converted to a subquery. + */ +static List * +RTEListToConvert(ConversionCandidates *conversionCandidates, ConversionChoice + conversionChoice) +{ + List *rtesToConvert = NIL; + if (conversionChoice == CONVERT_LOCAL_TABLES) + { + rtesToConvert = list_concat(rtesToConvert, conversionCandidates->localTableList); + } + else + { + rtesToConvert = list_concat(rtesToConvert, + conversionCandidates->distributedTableList); + } + return rtesToConvert; +} + + +/* + * GetConversionChoice returns the conversion choice considering the local table + * join policy. + */ +static ConversionChoice +GetConversionChoice(ConversionCandidates *conversionCandidates, + PlannerRestrictionContext *plannerRestrictionContext) +{ + RangeTableEntryDetails *localRTECandidate = NULL; + RangeTableEntryDetails *distributedRTECandidate = NULL; + + if (list_length(conversionCandidates->localTableList) > 0) + { + localRTECandidate = linitial(conversionCandidates->localTableList); + } + if (list_length(conversionCandidates->distributedTableList) > 0) + { + distributedRTECandidate = linitial(conversionCandidates->distributedTableList); + } + + if (LocalTableJoinPolicy == LOCAL_JOIN_POLICY_PREFER_LOCAL) + { + return localRTECandidate ? CONVERT_LOCAL_TABLES : CONVERT_DISTRIBUTED_TABLES; + } + else if (LocalTableJoinPolicy == LOCAL_JOIN_POLICY_PREFER_DISTRIBUTED) + { + return distributedRTECandidate ? CONVERT_DISTRIBUTED_TABLES : + CONVERT_LOCAL_TABLES; + } + else + { + /* + * We want to convert distributed tables only if all the distributed tables + * have a constant filter on a unique index, otherwise we would be redundantly + * converting a distributed table as we will convert all the other local tables. + */ + bool allRangeTableEntriesHaveUniqueIndex = AllRangeTableEntriesHaveUniqueIndex( + conversionCandidates->distributedTableList); + + if (allRangeTableEntriesHaveUniqueIndex) + { + return distributedRTECandidate ? CONVERT_DISTRIBUTED_TABLES : + CONVERT_LOCAL_TABLES; + } + else + { + return localRTECandidate ? CONVERT_LOCAL_TABLES : CONVERT_DISTRIBUTED_TABLES; + } + } +} + + +/* + * ConvertRTEsToSubquery converts all the given range table entries + * to a subquery. + */ +static void +ConvertRTEsToSubquery(List *rangeTableEntryDetailsList, RecursivePlanningContext *context) +{ + RangeTableEntryDetails *rangeTableEntryDetails = NULL; + foreach_ptr(rangeTableEntryDetails, rangeTableEntryDetailsList) + { + RangeTblEntry *rangeTableEntry = rangeTableEntryDetails->rangeTableEntry; + List *requiredAttributeNumbers = rangeTableEntryDetails->requiredAttributeNumbers; + ReplaceRTERelationWithRteSubquery(rangeTableEntry, + requiredAttributeNumbers, context); + } +} + + +/* + * AllRangeTableEntriesHaveUniqueIndex returns true if all of the RTE's in the given + * list have a unique index. + */ +static bool +AllRangeTableEntriesHaveUniqueIndex(List *rangeTableEntryDetailsList) +{ + RangeTableEntryDetails *rangeTableEntryDetails = NULL; + foreach_ptr(rangeTableEntryDetails, rangeTableEntryDetailsList) + { + if (!rangeTableEntryDetails->hasConstantFilterOnUniqueColumn) + { + return false; + } + } + return true; +} + + +/* + * ShouldConvertLocalTableJoinsToSubqueries returns true if we should + * convert local-dist table joins to subqueries. + */ +bool +ShouldConvertLocalTableJoinsToSubqueries(List *rangeTableList) +{ + if (LocalTableJoinPolicy == LOCAL_JOIN_POLICY_NEVER) + { + /* user doesn't want Citus to enable local table joins */ + return false; + } + + if (!ContainsLocalTableDistributedTableJoin(rangeTableList)) + { + return false; + } + return true; +} + + +/* + * HasConstantFilterOnUniqueColumn returns true if the given rangeTableEntry has a constant + * filter on a unique column. + */ +static bool +HasConstantFilterOnUniqueColumn(RangeTblEntry *rangeTableEntry, + RelationRestriction *relationRestriction) +{ + if (rangeTableEntry == NULL) + { + return false; + } + List *baseRestrictionList = relationRestriction->relOptInfo->baserestrictinfo; + List *restrictClauseList = get_all_actual_clauses(baseRestrictionList); + if (ContainsFalseClause(restrictClauseList)) + { + /* If there is a WHERE FALSE, we consider it as a constant filter. */ + return true; + } + List *rteEqualityColumnsNos = + FetchEqualityAttrNumsForRTE((Node *) restrictClauseList); + + List *uniqueIndexColumnsList = ExecuteFunctionOnEachTableIndex(rangeTableEntry->relid, + AppendUniqueIndexColumnsToList); + IndexColumns *indexColumns = NULL; + foreach_ptr(indexColumns, uniqueIndexColumnsList) + { + List *uniqueIndexColumnNos = indexColumns->indexColumnNos; + if (FirstIsSuperSetOfSecond(rteEqualityColumnsNos, + uniqueIndexColumnNos)) + { + return true; + } + } + return false; +} + + +/* + * FirstIsSuperSetOfSecond returns true if the first int List + * contains every element of the second int List. + */ +static bool +FirstIsSuperSetOfSecond(List *firstIntList, List *secondIntList) +{ + int curInt = 0; + foreach_int(curInt, secondIntList) + { + if (!list_member_int(firstIntList, curInt)) + { + return false; + } + } + return true; +} + + +/* + * AppendUniqueIndexColumnsToList adds the given index's column numbers if it is a + * unique index. + */ +static void +AppendUniqueIndexColumnsToList(Form_pg_index indexForm, List **uniqueIndexGroups) +{ + if (indexForm->indisunique || indexForm->indisprimary) + { + IndexColumns *indexColumns = palloc0(sizeof(IndexColumns)); + List *uniqueIndexes = NIL; + for (int i = 0; i < indexForm->indkey.dim1; i++) + { + uniqueIndexes = list_append_unique_int(uniqueIndexes, + indexForm->indkey.values[i]); + } + if (list_length(uniqueIndexes) == 0) + { + return; + } + indexColumns->indexColumnNos = uniqueIndexes; + *uniqueIndexGroups = lappend(*uniqueIndexGroups, indexColumns); + } +} + + +/* + * RequiredAttrNumbersForRelation returns the required attribute numbers for + * the input RTE relation in order for the planning to succeed. + * + * The function could be optimized by not adding the columns that only appear + * WHERE clause as a filter (e.g., not a join clause). + */ +static List * +RequiredAttrNumbersForRelation(RangeTblEntry *rangeTableEntry, + PlannerRestrictionContext *plannerRestrictionContext) +{ + RelationRestriction *relationRestriction = + RelationRestrictionForRelation(rangeTableEntry, plannerRestrictionContext); + + if (relationRestriction == NULL) + { + return NIL; + } + + PlannerInfo *plannerInfo = relationRestriction->plannerInfo; + + /* + * Here we used the query from plannerInfo because it has the optimizations + * so that it doesn't have unnecessary columns. The original query doesn't have + * some of these optimizations hence if we use it here, we don't get the + * 'required' attributes. + */ + Query *queryToProcess = plannerInfo->parse; + int rteIndex = relationRestriction->index; + + List *allVarsInQuery = pull_vars_of_level((Node *) queryToProcess, 0); + + List *requiredAttrNumbers = NIL; + + Var *var = NULL; + foreach_ptr(var, allVarsInQuery) + { + if (var->varno == rteIndex) + { + requiredAttrNumbers = list_append_unique_int(requiredAttrNumbers, + var->varattno); + } + } + + return requiredAttrNumbers; +} + + +/* + * CreateConversionCandidates creates the conversion candidates that might + * be converted to a subquery so that citus planners can work. + */ +static ConversionCandidates * +CreateConversionCandidates(PlannerRestrictionContext *plannerRestrictionContext, + List *rangeTableList, int resultRTEIdentity) +{ + ConversionCandidates *conversionCandidates = + palloc0(sizeof(ConversionCandidates)); + + + RangeTblEntry *rangeTableEntry = NULL; + foreach_ptr(rangeTableEntry, rangeTableList) + { + /* we're only interested in tables */ + if (!IsRecursivelyPlannableRelation(rangeTableEntry)) + { + continue; + } + + int rteIdentity = GetRTEIdentity(rangeTableEntry); + + /* result relation cannot converted to a subquery */ + if (resultRTEIdentity == rteIdentity) + { + continue; + } + + RelationRestriction *relationRestriction = + RelationRestrictionForRelation(rangeTableEntry, plannerRestrictionContext); + if (relationRestriction == NULL) + { + continue; + } + + + RangeTableEntryDetails *rangeTableEntryDetails = + palloc0(sizeof(RangeTableEntryDetails)); + + rangeTableEntryDetails->rangeTableEntry = rangeTableEntry; + rangeTableEntryDetails->requiredAttributeNumbers = + RequiredAttrNumbersForRelation(rangeTableEntry, plannerRestrictionContext); + rangeTableEntryDetails->hasConstantFilterOnUniqueColumn = + HasConstantFilterOnUniqueColumn(rangeTableEntry, relationRestriction); + + bool referenceOrDistributedTable = + IsCitusTableType(rangeTableEntry->relid, REFERENCE_TABLE) || + IsCitusTableType(rangeTableEntry->relid, DISTRIBUTED_TABLE); + if (referenceOrDistributedTable) + { + conversionCandidates->distributedTableList = + lappend(conversionCandidates->distributedTableList, + rangeTableEntryDetails); + } + else + { + conversionCandidates->localTableList = + lappend(conversionCandidates->localTableList, + rangeTableEntryDetails); + } + } + return conversionCandidates; +} diff --git a/src/backend/distributed/planner/multi_logical_optimizer.c b/src/backend/distributed/planner/multi_logical_optimizer.c index 6e36c0f9f..5a3f78745 100644 --- a/src/backend/distributed/planner/multi_logical_optimizer.c +++ b/src/backend/distributed/planner/multi_logical_optimizer.c @@ -291,8 +291,9 @@ static SortGroupClause * CreateSortGroupClause(Var *column); /* Local functions forward declarations for count(distinct) approximations */ static const char * CountDistinctHashFunctionName(Oid argumentType); static int CountDistinctStorageSize(double approximationErrorRate); -static Const * MakeIntegerConst(int32 integerValue); static Const * MakeIntegerConstInt64(int64 integerValue); +static Const * MakeIntegerConst(int32 integerValue); + /* Local functions forward declarations for aggregate expression checks */ static bool HasNonDistributableAggregates(MultiNode *logicalPlanNode); diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index b74b7a352..8d0289d7c 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -333,6 +333,23 @@ IsCitusTableRTE(Node *node) } +/* + * IsDistributedOrReferenceTableRTE returns true if the given node + * is eeither a distributed(hash/range/append) or reference table. + */ +bool +IsDistributedOrReferenceTableRTE(Node *node) +{ + Oid relationId = NodeTryGetRteRelid(node); + if (!OidIsValid(relationId)) + { + return false; + } + return IsCitusTableType(relationId, DISTRIBUTED_TABLE) || + IsCitusTableType(relationId, REFERENCE_TABLE); +} + + /* * IsDistributedTableRTE gets a node and returns true if the node * is a range table relation entry that points to a distributed relation, diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 1dd5c661a..1ef7554ac 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -171,7 +171,8 @@ static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, RelationRestrictionContext *restrictionContext, uint32 taskId, TaskType taskType, - bool modifyRequiresCoordinatorEvaluation); + bool modifyRequiresCoordinatorEvaluation, + DeferredErrorMessage **planningError); static bool ShardIntervalsEqual(FmgrInfo *comparisonFunction, Oid collation, ShardInterval *firstInterval, @@ -233,6 +234,9 @@ static StringInfo ColumnTypeArrayString(List *targetEntryList); static bool CoPlacedShardIntervals(ShardInterval *firstInterval, ShardInterval *secondInterval); +static List * FetchEqualityAttrNumsForRTEOpExpr(OpExpr *opExpr); +static List * FetchEqualityAttrNumsForRTEBoolExpr(BoolExpr *boolExpr); +static List * FetchEqualityAttrNumsForList(List *nodeList); #if PG_VERSION_NUM >= PG_VERSION_13 static List * GetColumnOriginalIndexes(Oid relationId); #endif @@ -267,6 +271,27 @@ CreatePhysicalDistributedPlan(MultiTreeRoot *multiTree, } +/* + * ModifyLocalTableJob returns true if the given task contains + * a modification of local table. + */ +bool +ModifyLocalTableJob(Job *job) +{ + if (job == NULL) + { + return false; + } + List *taskList = job->taskList; + if (list_length(taskList) != 1) + { + return false; + } + Task *singleTask = (Task *) linitial(taskList); + return singleTask->isLocalTableModification; +} + + /* * BuildJobTree builds the physical job tree from the given logical plan tree. * The function walks over the logical plan from the bottom up, finds boundaries @@ -2102,11 +2127,17 @@ BuildJobTreeTaskList(Job *jobTree, PlannerRestrictionContext *plannerRestriction relationRestrictionContext, &isMultiShardQuery, NULL); + DeferredErrorMessage *deferredErrorMessage = NULL; sqlTaskList = QueryPushdownSqlTaskList(job->jobQuery, job->jobId, plannerRestrictionContext-> relationRestrictionContext, prunedRelationShardList, READ_TASK, - false); + false, + &deferredErrorMessage); + if (deferredErrorMessage != NULL) + { + RaiseDeferredErrorInternal(deferredErrorMessage, ERROR); + } } else { @@ -2184,7 +2215,8 @@ List * QueryPushdownSqlTaskList(Query *query, uint64 jobId, RelationRestrictionContext *relationRestrictionContext, List *prunedRelationShardList, TaskType taskType, bool - modifyRequiresCoordinatorEvaluation) + modifyRequiresCoordinatorEvaluation, + DeferredErrorMessage **planningError) { List *sqlTaskList = NIL; ListCell *restrictionCell = NULL; @@ -2198,8 +2230,11 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, if (list_length(relationRestrictionContext->relationRestrictionList) == 0) { - ereport(ERROR, (errmsg("cannot handle complex subqueries when the " - "router executor is disabled"))); + *planningError = DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot handle complex subqueries when the " + "router executor is disabled", + NULL, NULL); + return NIL; } /* defaults to be used if this is a reference table-only query */ @@ -2224,8 +2259,11 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, /* we expect distributed tables to have the same shard count */ if (shardCount > 0 && shardCount != cacheEntry->shardIntervalArrayLength) { - ereport(ERROR, (errmsg("shard counts of co-located tables do not " - "match"))); + *planningError = DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "shard counts of co-located tables do not " + "match", + NULL, NULL); + return NIL; } if (taskRequiredForShardIndex == NULL) @@ -2288,7 +2326,12 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, relationRestrictionContext, taskIdIndex, taskType, - modifyRequiresCoordinatorEvaluation); + modifyRequiresCoordinatorEvaluation, + planningError); + if (*planningError != NULL) + { + return NIL; + } subqueryTask->jobId = jobId; sqlTaskList = lappend(sqlTaskList, subqueryTask); @@ -2464,7 +2507,8 @@ ErrorIfUnsupportedShardDistribution(Query *query) static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, RelationRestrictionContext *restrictionContext, uint32 taskId, - TaskType taskType, bool modifyRequiresCoordinatorEvaluation) + TaskType taskType, bool modifyRequiresCoordinatorEvaluation, + DeferredErrorMessage **planningError) { Query *taskQuery = copyObject(originalQuery); @@ -2543,8 +2587,12 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, List *taskPlacementList = PlacementsForWorkersContainingAllShards(taskShardList); if (list_length(taskPlacementList) == 0) { - ereport(ERROR, (errmsg("cannot find a worker that has active placements for all " - "shards in the query"))); + *planningError = DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "cannot find a worker that has active placements for all " + "shards in the query", + NULL, NULL); + + return NULL; } /* @@ -3589,6 +3637,122 @@ NodeIsRangeTblRefReferenceTable(Node *node, List *rangeTableList) } +/* + * FetchEqualityAttrNumsForRTE fetches the attribute numbers from quals + * which have an equality operator + */ +List * +FetchEqualityAttrNumsForRTE(Node *node) +{ + if (node == NULL) + { + return NIL; + } + if (IsA(node, List)) + { + return FetchEqualityAttrNumsForList((List *) node); + } + else if (IsA(node, OpExpr)) + { + return FetchEqualityAttrNumsForRTEOpExpr((OpExpr *) node); + } + else if (IsA(node, BoolExpr)) + { + return FetchEqualityAttrNumsForRTEBoolExpr((BoolExpr *) node); + } + return NIL; +} + + +/* + * FetchEqualityAttrNumsForList fetches the attribute numbers of expression + * of the form "= constant" from the given node list. + */ +static List * +FetchEqualityAttrNumsForList(List *nodeList) +{ + List *attributeNums = NIL; + Node *node = NULL; + bool hasAtLeastOneEquality = false; + foreach_ptr(node, nodeList) + { + List *fetchedEqualityAttrNums = + FetchEqualityAttrNumsForRTE(node); + hasAtLeastOneEquality |= list_length(fetchedEqualityAttrNums) > 0; + attributeNums = list_concat(attributeNums, fetchedEqualityAttrNums); + } + + /* + * the given list is in the form of AND'ed expressions + * hence if we have one equality then it is enough. + * E.g: dist.a = 5 AND dist.a > 10 + */ + if (hasAtLeastOneEquality) + { + return attributeNums; + } + return NIL; +} + + +/* + * FetchEqualityAttrNumsForRTEOpExpr fetches the attribute numbers of expression + * of the form "= constant" from the given opExpr. + */ +static List * +FetchEqualityAttrNumsForRTEOpExpr(OpExpr *opExpr) +{ + if (!OperatorImplementsEquality(opExpr->opno)) + { + return NIL; + } + + List *attributeNums = NIL; + Var *var = NULL; + if (VarConstOpExprClause(opExpr, &var, NULL)) + { + attributeNums = lappend_int(attributeNums, var->varattno); + } + return attributeNums; +} + + +/* + * FetchEqualityAttrNumsForRTEBoolExpr fetches the attribute numbers of expression + * of the form "= constant" from the given boolExpr. + */ +static List * +FetchEqualityAttrNumsForRTEBoolExpr(BoolExpr *boolExpr) +{ + if (boolExpr->boolop != AND_EXPR && boolExpr->boolop != OR_EXPR) + { + return NIL; + } + + List *attributeNums = NIL; + bool hasEquality = true; + Node *arg = NULL; + foreach_ptr(arg, boolExpr->args) + { + List *attributeNumsInSubExpression = FetchEqualityAttrNumsForRTE(arg); + if (boolExpr->boolop == AND_EXPR) + { + hasEquality |= list_length(attributeNumsInSubExpression) > 0; + } + else if (boolExpr->boolop == OR_EXPR) + { + hasEquality &= list_length(attributeNumsInSubExpression) > 0; + } + attributeNums = list_concat(attributeNums, attributeNumsInSubExpression); + } + if (hasEquality) + { + return attributeNums; + } + return NIL; +} + + /* * JoinSequenceArray walks over the join nodes in the job query and constructs a join * sequence containing an entry for each joined table. The function then returns an @@ -5352,7 +5516,6 @@ ActiveShardPlacementLists(List *taskList) { Task *task = (Task *) lfirst(taskCell); uint64 anchorShardId = task->anchorShardId; - List *shardPlacementList = ActiveShardPlacementList(anchorShardId); /* filter out shard placements that reside in inactive nodes */ diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index eaba0ff02..768e9b171 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -49,6 +49,7 @@ #include "distributed/reference_table_utils.h" #include "distributed/relation_restriction_equivalence.h" #include "distributed/relay_utility.h" +#include "distributed/recursive_planning.h" #include "distributed/resource_lock.h" #include "distributed/shardinterval_utils.h" #include "distributed/shard_pruning.h" @@ -131,12 +132,6 @@ static DeferredErrorMessage * ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery, Oid *distributedTableId); static bool NodeIsFieldStore(Node *node); -static DeferredErrorMessage * DeferErrorIfUnsupportedModifyQueryWithLocalTable( - Query *query); -static DeferredErrorMessage * DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable( - RTEListProperties *rteListProperties, Oid targetRelationId); -static DeferredErrorMessage * DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable( - RTEListProperties *rteListProperties, Oid targetRelationId); static DeferredErrorMessage * MultiShardUpdateDeleteSupported(Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext); @@ -171,17 +166,20 @@ static DeferredErrorMessage * ErrorIfQueryHasUnroutableModifyingCTE(Query *query static bool SelectsFromDistributedTable(List *rangeTableList, Query *query); static ShardPlacement * CreateDummyPlacement(bool hasLocalRelation); static ShardPlacement * CreateLocalDummyPlacement(); -static List * get_all_actual_clauses(List *restrictinfo_list); static int CompareInsertValuesByShardId(const void *leftElement, const void *rightElement); static List * SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, List *placementList, - uint64 shardId, bool parametersInQueryResolved); + uint64 shardId, bool parametersInQueryResolved, + bool isLocalTableModification); static bool RowLocksOnRelations(Node *node, List **rtiLockList); static void ReorderTaskPlacementsByTaskAssignmentPolicy(Job *job, TaskAssignmentPolicyType taskAssignmentPolicy, List *placementList); +static bool ModifiesLocalTableWithRemoteCitusLocalTable(List *rangeTableList); +static DeferredErrorMessage * DeferErrorIfUnsupportedLocalTableJoin(List *rangeTableList); +static bool IsTableLocallyAccessible(Oid relationId); /* @@ -230,6 +228,7 @@ CreateModifyPlan(Query *originalQuery, Query *query, distributedPlan->planningError = ModifyQuerySupported(query, originalQuery, multiShardQuery, plannerRestrictionContext); + if (distributedPlan->planningError != NULL) { return distributedPlan; @@ -525,8 +524,9 @@ ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery, { return deferredError; } + CmdType commandType = queryTree->commandType; - deferredError = DeferErrorIfUnsupportedModifyQueryWithLocalTable(queryTree); + deferredError = DeferErrorIfUnsupportedLocalTableJoin(queryTree->rtable); if (deferredError != NULL) { return deferredError; @@ -617,11 +617,17 @@ ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery, } } - Oid distributedTableId = ModifyQueryResultRelationId(queryTree); - uint32 rangeTableId = 1; - Var *partitionColumn = PartitionColumn(distributedTableId, rangeTableId); - CmdType commandType = queryTree->commandType; + Oid resultRelationId = ModifyQueryResultRelationId(queryTree); + *distributedTableIdOutput = resultRelationId; + uint32 rangeTableId = 1; + + Var *partitionColumn = NULL; + if (IsCitusTable(resultRelationId)) + { + partitionColumn = PartitionColumn(resultRelationId, rangeTableId); + } + commandType = queryTree->commandType; if (commandType == CMD_INSERT || commandType == CMD_UPDATE || commandType == CMD_DELETE) { @@ -749,12 +755,97 @@ ModifyPartialQuerySupported(Query *queryTree, bool multiShardQuery, /* set it for caller to use when we don't return any errors */ - *distributedTableIdOutput = distributedTableId; + *distributedTableIdOutput = resultRelationId; return NULL; } +/* + * DeferErrorIfUnsupportedLocalTableJoin returns an error message + * if there is an unsupported join in the given range table list. + */ +static DeferredErrorMessage * +DeferErrorIfUnsupportedLocalTableJoin(List *rangeTableList) +{ + if (ModifiesLocalTableWithRemoteCitusLocalTable(rangeTableList)) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "Modifying local tables with citus local tables is " + "supported only from the coordinator.", + NULL, + "Consider wrapping citus local table to a CTE, or subquery"); + } + return NULL; +} + + +/* + * ModifiesLocalTableWithRemoteCitusLocalTable returns true if a local + * table is modified with a remote citus local table. This could be a case with + * MX structure. + */ +static bool +ModifiesLocalTableWithRemoteCitusLocalTable(List *rangeTableList) +{ + bool containsLocalResultRelation = false; + bool containsRemoteCitusLocalTable = false; + + RangeTblEntry *rangeTableEntry = NULL; + foreach_ptr(rangeTableEntry, rangeTableList) + { + if (!IsRecursivelyPlannableRelation(rangeTableEntry)) + { + continue; + } + if (IsCitusTableType(rangeTableEntry->relid, CITUS_LOCAL_TABLE)) + { + if (!IsTableLocallyAccessible(rangeTableEntry->relid)) + { + containsRemoteCitusLocalTable = true; + } + } + else if (!IsCitusTable(rangeTableEntry->relid)) + { + containsLocalResultRelation = true; + } + } + return containsLocalResultRelation && containsRemoteCitusLocalTable; +} + + +/* + * IsTableLocallyAccessible returns true if the given table + * can be accessed in local. + */ +static bool +IsTableLocallyAccessible(Oid relationId) +{ + if (!IsCitusTable(relationId)) + { + /* local tables are locally accessible */ + return true; + } + + List *shardIntervalList = LoadShardIntervalList(relationId); + if (list_length(shardIntervalList) != 1) + { + return false; + } + + ShardInterval *shardInterval = linitial(shardIntervalList); + uint64 shardId = shardInterval->shardId; + ShardPlacement *localShardPlacement = + ShardPlacementOnGroup(shardId, GetLocalGroupId()); + if (localShardPlacement != NULL) + { + /* the table has a placement on this node */ + return true; + } + return false; +} + + /* * NodeIsFieldStore returns true if given Node is a FieldStore object. */ @@ -765,93 +856,6 @@ NodeIsFieldStore(Node *node) } -/* - * DeferErrorIfUnsupportedModifyQueryWithLocalTable returns DeferredErrorMessage - * for unsupported modify queries that cannot be planned by router planner due to - * unsupported usage of postgres local or citus local tables. - */ -static DeferredErrorMessage * -DeferErrorIfUnsupportedModifyQueryWithLocalTable(Query *query) -{ - RTEListProperties *rteListProperties = GetRTEListPropertiesForQuery(query); - Oid targetRelationId = ModifyQueryResultRelationId(query); - - DeferredErrorMessage *deferredErrorMessage = - DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable(rteListProperties, - targetRelationId); - if (deferredErrorMessage) - { - return deferredErrorMessage; - } - - deferredErrorMessage = DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable( - rteListProperties, - targetRelationId); - return deferredErrorMessage; -} - - -/* - * DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable is a helper function - * that takes RTEListProperties & targetRelationId and returns deferred error - * if query is not supported due to unsupported usage of citus local tables. - */ -static DeferredErrorMessage * -DeferErrorIfUnsupportedModifyQueryWithCitusLocalTable( - RTEListProperties *rteListProperties, Oid targetRelationId) -{ - if (rteListProperties->hasDistributedTable && rteListProperties->hasCitusLocalTable) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "cannot plan modifications with citus local tables and " - "distributed tables", NULL, - LOCAL_TABLE_SUBQUERY_CTE_HINT); - } - - if (IsCitusTableType(targetRelationId, REFERENCE_TABLE) && - rteListProperties->hasCitusLocalTable) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "cannot plan modifications of reference tables with citus " - "local tables", NULL, - LOCAL_TABLE_SUBQUERY_CTE_HINT); - } - - return NULL; -} - - -/* - * DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable is a helper - * function that takes RTEListProperties & targetRelationId and returns - * deferred error if query is not supported due to unsupported usage of - * postgres local tables. - */ -static DeferredErrorMessage * -DeferErrorIfUnsupportedModifyQueryWithPostgresLocalTable( - RTEListProperties *rteListProperties, Oid targetRelationId) -{ - if (rteListProperties->hasPostgresLocalTable && - rteListProperties->hasCitusTable) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "cannot plan modifications with local tables involving " - "citus tables", NULL, - LOCAL_TABLE_SUBQUERY_CTE_HINT); - } - - if (!IsCitusTable(targetRelationId)) - { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "cannot plan modifications of local tables involving " - "distributed tables", - NULL, NULL); - } - - return NULL; -} - - /* * ModifyQuerySupported returns NULL if the query only contains supported * features, otherwise it returns an error description. @@ -915,6 +919,8 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer { ExtractRangeTableEntryWalker((Node *) originalQuery, &rangeTableList); } + bool containsLocalTableDistributedTableJoin = + ContainsLocalTableDistributedTableJoin(queryTree->rtable); RangeTblEntry *rangeTableEntry = NULL; foreach_ptr(rangeTableEntry, rangeTableList) @@ -939,16 +945,22 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer /* for other kinds of relations, check if its distributed */ else { - Oid relationId = rangeTableEntry->relid; - - if (!IsCitusTable(relationId)) + if (IsRelationLocalTableOrMatView(rangeTableEntry->relid) && + containsLocalTableDistributedTableJoin) { StringInfo errorMessage = makeStringInfo(); char *relationName = get_rel_name(rangeTableEntry->relid); - - appendStringInfo(errorMessage, "relation %s is not distributed", - relationName); - + if (IsCitusTable(rangeTableEntry->relid)) + { + appendStringInfo(errorMessage, + "citus local table %s cannot be joined with these distributed tables", + relationName); + } + else + { + appendStringInfo(errorMessage, "relation %s is not distributed", + relationName); + } return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, errorMessage->data, NULL, NULL); } @@ -1076,7 +1088,8 @@ DeferErrorIfModifyView(Query *queryTree) firstRangeTableElement->inFromCl == false) { return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "cannot modify views over distributed tables", NULL, + "cannot modify views when the query contains citus tables", + NULL, NULL); } } @@ -1750,6 +1763,8 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon /* router planner should create task even if it doesn't hit a shard at all */ bool replacePrunedQueryWithDummy = true; + bool isLocalTableModification = false; + /* check if this query requires coordinator evaluation */ bool requiresCoordinatorEvaluation = RequiresCoordinatorEvaluation(originalQuery); FastPathRestrictionContext *fastPathRestrictionContext = @@ -1777,7 +1792,8 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon &prunedShardIntervalListList, replacePrunedQueryWithDummy, &isMultiShardModifyQuery, - &partitionKeyValue); + &partitionKeyValue, + &isLocalTableModification); } if (*planningError) @@ -1813,12 +1829,18 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon relationRestrictionContext, prunedShardIntervalListList, MODIFY_TASK, - requiresCoordinatorEvaluation); + requiresCoordinatorEvaluation, + planningError); + if (*planningError) + { + return NULL; + } } else { GenerateSingleShardRouterTaskList(job, relationShardList, - placementList, shardId); + placementList, shardId, + isLocalTableModification); } job->requiresCoordinatorEvaluation = requiresCoordinatorEvaluation; @@ -1834,17 +1856,18 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon */ void GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, - List *placementList, uint64 shardId) + List *placementList, uint64 shardId, bool + isLocalTableModification) { Query *originalQuery = job->jobQuery; - if (originalQuery->commandType == CMD_SELECT) { job->taskList = SingleShardTaskList(originalQuery, job->jobId, relationShardList, placementList, shardId, - job->parametersInJobQueryResolved); + job->parametersInJobQueryResolved, + isLocalTableModification); /* * Queries to reference tables, or distributed tables with multiple replica's have @@ -1861,7 +1884,7 @@ GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, placementList); } } - else if (shardId == INVALID_SHARD_ID) + else if (shardId == INVALID_SHARD_ID && !isLocalTableModification) { /* modification that prunes to 0 shards */ job->taskList = NIL; @@ -1871,7 +1894,8 @@ GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, job->taskList = SingleShardTaskList(originalQuery, job->jobId, relationShardList, placementList, shardId, - job->parametersInJobQueryResolved); + job->parametersInJobQueryResolved, + isLocalTableModification); } } @@ -1964,7 +1988,8 @@ RemoveCoordinatorPlacementIfNotSingleNode(List *placementList) static List * SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, List *placementList, uint64 shardId, - bool parametersInQueryResolved) + bool parametersInQueryResolved, + bool isLocalTableModification) { TaskType taskType = READ_TASK; char replicationModel = 0; @@ -1977,10 +2002,14 @@ SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, RangeTblEntry *updateOrDeleteRTE = ExtractResultRelationRTE(query); Assert(updateOrDeleteRTE != NULL); - CitusTableCacheEntry *modificationTableCacheEntry = GetCitusTableCacheEntry( - updateOrDeleteRTE->relid); + CitusTableCacheEntry *modificationTableCacheEntry = NULL; + if (IsCitusTable(updateOrDeleteRTE->relid)) + { + modificationTableCacheEntry = GetCitusTableCacheEntry( + updateOrDeleteRTE->relid); + } - if (IsCitusTableTypeCacheEntry(modificationTableCacheEntry, REFERENCE_TABLE) && + if (IsCitusTableType(updateOrDeleteRTE->relid, REFERENCE_TABLE) && SelectsFromDistributedTable(rangeTableList, query)) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), @@ -1989,7 +2018,10 @@ SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, } taskType = MODIFY_TASK; - replicationModel = modificationTableCacheEntry->replicationModel; + if (modificationTableCacheEntry) + { + replicationModel = modificationTableCacheEntry->replicationModel; + } } if (taskType == READ_TASK && query->hasModifyingCTE) @@ -2016,6 +2048,7 @@ SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList, } Task *task = CreateTask(taskType); + task->isLocalTableModification = isLocalTableModification; List *relationRowLockList = NIL; RowLocksOnRelations((Node *) query, &relationRowLockList); @@ -2127,6 +2160,8 @@ SelectsFromDistributedTable(List *rangeTableList, Query *query) } +static bool ContainsOnlyLocalTables(RTEListProperties *rteProperties); + /* * RouterQuery runs router pruning logic for SELECT, UPDATE and DELETE queries. * If there are shards present and query is routable, all RTEs have been updated @@ -2154,7 +2189,8 @@ PlanRouterQuery(Query *originalQuery, List **placementList, uint64 *anchorShardId, List **relationShardList, List **prunedShardIntervalListList, bool replacePrunedQueryWithDummy, bool *multiShardModifyQuery, - Const **partitionValueConst) + Const **partitionValueConst, + bool *isLocalTableModification) { bool isMultiShardQuery = false; DeferredErrorMessage *planningError = NULL; @@ -2270,6 +2306,13 @@ PlanRouterQuery(Query *originalQuery, /* both Postgres tables and materialized tables are locally avaliable */ RTEListProperties *rteProperties = GetRTEListPropertiesForQuery(originalQuery); + if (shardId == INVALID_SHARD_ID && ContainsOnlyLocalTables(rteProperties)) + { + if (commandType != CMD_SELECT) + { + *isLocalTableModification = true; + } + } bool hasPostgresLocalRelation = rteProperties->hasPostgresLocalTable || rteProperties->hasMaterializedView; List *taskPlacementList = @@ -2303,6 +2346,17 @@ PlanRouterQuery(Query *originalQuery, } +/* + * ContainsOnlyLocalTables returns true if there is only + * local tables and not any distributed or reference table. + */ +static bool +ContainsOnlyLocalTables(RTEListProperties *rteProperties) +{ + return !rteProperties->hasDistributedTable && !rteProperties->hasReferenceTable; +} + + /* * CreateTaskPlacementListForShardIntervals returns a list of shard placements * on which it can access all shards in shardIntervalListList, which contains @@ -3679,7 +3733,7 @@ ErrorIfQueryHasUnroutableModifyingCTE(Query *queryTree) * This loses the distinction between regular and pseudoconstant clauses, * so be careful what you use it for. */ -static List * +List * get_all_actual_clauses(List *restrictinfo_list) { List *result = NIL; diff --git a/src/backend/distributed/planner/query_colocation_checker.c b/src/backend/distributed/planner/query_colocation_checker.c index 0a84ea49e..b20d6664d 100644 --- a/src/backend/distributed/planner/query_colocation_checker.c +++ b/src/backend/distributed/planner/query_colocation_checker.c @@ -21,6 +21,13 @@ #include "postgres.h" +#include "distributed/pg_version_constants.h" + +#if PG_VERSION_NUM >= PG_VERSION_12 +#include "access/relation.h" +#else +#include "access/heapam.h" +#endif #include "distributed/multi_logical_planner.h" #include "distributed/query_colocation_checker.h" #include "distributed/pg_dist_partition.h" @@ -30,15 +37,24 @@ #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" #include "parser/parsetree.h" +#include "distributed/listutils.h" #include "parser/parse_relation.h" #include "optimizer/planner.h" #include "optimizer/prep.h" +#include "utils/rel.h" static RangeTblEntry * AnchorRte(Query *subquery); -static Query * WrapRteRelationIntoSubquery(RangeTblEntry *rteRelation); static List * UnionRelationRestrictionLists(List *firstRelationList, List *secondRelationList); +static List * CreateFilteredTargetListForRelation(Oid relationId, + List *requiredAttributes); +static List * CreateDummyTargetList(Oid relationId, List *requiredAttributes); +static TargetEntry * CreateTargetEntryForColumn(Form_pg_attribute attributeTuple, Index + rteIndex, + int attributeNumber, int resno); +static TargetEntry * CreateTargetEntryForNullCol(Form_pg_attribute attributeTuple, int + resno); /* @@ -69,7 +85,7 @@ CreateColocatedJoinChecker(Query *subquery, PlannerRestrictionContext *restricti * functions (i.e., FilterPlannerRestrictionForQuery()) rely on queries * not relations. */ - anchorSubquery = WrapRteRelationIntoSubquery(anchorRangeTblEntry); + anchorSubquery = WrapRteRelationIntoSubquery(anchorRangeTblEntry, NIL); } else if (anchorRangeTblEntry->rtekind == RTE_SUBQUERY) { @@ -252,8 +268,8 @@ SubqueryColocated(Query *subquery, ColocatedJoinChecker *checker) * projections. The returned query should be used cautiosly and it is mostly * designed for generating a stub query. */ -static Query * -WrapRteRelationIntoSubquery(RangeTblEntry *rteRelation) +Query * +WrapRteRelationIntoSubquery(RangeTblEntry *rteRelation, List *requiredAttributes) { Query *subquery = makeNode(Query); RangeTblRef *newRangeTableRef = makeNode(RangeTblRef); @@ -266,23 +282,158 @@ WrapRteRelationIntoSubquery(RangeTblEntry *rteRelation) /* set the FROM expression to the subquery */ newRangeTableRef = makeNode(RangeTblRef); - newRangeTableRef->rtindex = 1; + newRangeTableRef->rtindex = SINGLE_RTE_INDEX; subquery->jointree = makeFromExpr(list_make1(newRangeTableRef), NULL); - /* Need the whole row as a junk var */ - Var *targetColumn = makeWholeRowVar(newRangeTableEntry, newRangeTableRef->rtindex, 0, - false); + subquery->targetList = + CreateFilteredTargetListForRelation(rteRelation->relid, requiredAttributes); - /* create a dummy target entry */ - TargetEntry *targetEntry = makeTargetEntry((Expr *) targetColumn, 1, "wholerow", - true); - - subquery->targetList = lappend(subquery->targetList, targetEntry); + if (list_length(subquery->targetList) == 0) + { + /* + * in case there is no required column, we assign one dummy NULL target entry + * to the subquery targetList so that it has at least one target. + * (targetlist should have at least one element) + */ + subquery->targetList = CreateDummyTargetList(rteRelation->relid, + requiredAttributes); + } return subquery; } +/* + * CreateAllTargetListForRelation creates a target list which contains all the columns + * of the given relation. If the column is not in required columns, then it is added + * as a NULL column. + */ +List * +CreateAllTargetListForRelation(Oid relationId, List *requiredAttributes) +{ + Relation relation = relation_open(relationId, AccessShareLock); + int numberOfAttributes = RelationGetNumberOfAttributes(relation); + + List *targetList = NIL; + int varAttrNo = 1; + for (int attrNum = 1; attrNum <= numberOfAttributes; attrNum++) + { + Form_pg_attribute attributeTuple = + TupleDescAttr(relation->rd_att, attrNum - 1); + + if (attributeTuple->attisdropped) + { + continue; + } + + int resNo = attrNum; + if (!list_member_int(requiredAttributes, attrNum)) + { + TargetEntry *nullTargetEntry = + CreateTargetEntryForNullCol(attributeTuple, resNo); + targetList = lappend(targetList, nullTargetEntry); + } + else + { + TargetEntry *targetEntry = + CreateTargetEntryForColumn(attributeTuple, SINGLE_RTE_INDEX, varAttrNo++, + resNo); + targetList = lappend(targetList, targetEntry); + } + } + + relation_close(relation, NoLock); + return targetList; +} + + +/* + * CreateFilteredTargetListForRelation creates a target list which contains + * only the required columns of the given relation. If there is not required + * columns then a dummy NULL column is put as the only entry. + */ +static List * +CreateFilteredTargetListForRelation(Oid relationId, List *requiredAttributes) +{ + Relation relation = relation_open(relationId, AccessShareLock); + int numberOfAttributes = RelationGetNumberOfAttributes(relation); + + List *targetList = NIL; + int resultNo = 1; + for (int attrNum = 1; attrNum <= numberOfAttributes; attrNum++) + { + Form_pg_attribute attributeTuple = + TupleDescAttr(relation->rd_att, attrNum - 1); + + if (list_member_int(requiredAttributes, attrNum)) + { + /* In the subquery with only required attribute numbers, the result no + * corresponds to the ordinal index of it in targetList. + */ + TargetEntry *targetEntry = + CreateTargetEntryForColumn(attributeTuple, SINGLE_RTE_INDEX, attrNum, + resultNo++); + targetList = lappend(targetList, targetEntry); + } + } + relation_close(relation, NoLock); + return targetList; +} + + +/* + * CreateDummyTargetList creates a target list which contains only a + * NULL entry. + */ +static List * +CreateDummyTargetList(Oid relationId, List *requiredAttributes) +{ + Relation relation = relation_open(relationId, AccessShareLock); + + Form_pg_attribute attributeTuple = + TupleDescAttr(relation->rd_att, 0); + TargetEntry *nullTargetEntry = + CreateTargetEntryForNullCol(attributeTuple, 1); + + relation_close(relation, NoLock); + return list_make1(nullTargetEntry); +} + + +/* + * CreateTargetEntryForColumn creates a target entry for the given + * column. + */ +static TargetEntry * +CreateTargetEntryForColumn(Form_pg_attribute attributeTuple, Index rteIndex, + int attributeNumber, int resno) +{ + Var *targetColumn = + makeVar(rteIndex, attributeNumber, attributeTuple->atttypid, + attributeTuple->atttypmod, attributeTuple->attcollation, 0); + TargetEntry *targetEntry = + makeTargetEntry((Expr *) targetColumn, resno, + strdup(attributeTuple->attname.data), false); + return targetEntry; +} + + +/* + * CreateTargetEntryForNullCol creates a target entry that has a NULL expression. + */ +static TargetEntry * +CreateTargetEntryForNullCol(Form_pg_attribute attributeTuple, int resno) +{ + Expr *nullExpr = (Expr *) makeNullConst(attributeTuple->atttypid, + attributeTuple->atttypmod, + attributeTuple->attcollation); + TargetEntry *targetEntry = + makeTargetEntry(nullExpr, resno, + strdup(attributeTuple->attname.data), false); + return targetEntry; +} + + /* * UnionRelationRestrictionLists merges two relation restriction lists * and returns a newly allocated list. The merged relation restriction diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index c6dce836e..f8cc45cc7 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -59,10 +59,12 @@ #include "distributed/commands/multi_copy.h" #include "distributed/distributed_planner.h" #include "distributed/errormessage.h" +#include "distributed/local_distributed_join_planner.h" #include "distributed/listutils.h" #include "distributed/log_utils.h" #include "distributed/metadata_cache.h" #include "distributed/multi_logical_planner.h" +#include "distributed/multi_logical_optimizer.h" #include "distributed/multi_router_planner.h" #include "distributed/multi_physical_planner.h" #include "distributed/multi_server_executor.h" @@ -71,14 +73,22 @@ #include "distributed/recursive_planning.h" #include "distributed/relation_restriction_equivalence.h" #include "distributed/log_utils.h" +#include "distributed/shard_pruning.h" #include "distributed/version_compat.h" #include "lib/stringinfo.h" +#include "optimizer/clauses.h" +#if PG_VERSION_NUM >= PG_VERSION_12 +#include "optimizer/optimizer.h" +#else +#include "optimizer/var.h" +#endif #include "optimizer/planner.h" #include "optimizer/prep.h" #include "parser/parsetree.h" #include "nodes/makefuncs.h" #include "nodes/nodeFuncs.h" #include "nodes/nodes.h" +#include "nodes/nodeFuncs.h" #include "nodes/pg_list.h" #include "nodes/primnodes.h" #if PG_VERSION_NUM >= PG_VERSION_12 @@ -88,25 +98,24 @@ #endif #include "utils/builtins.h" #include "utils/guc.h" - - -/* track depth of current recursive planner query */ -static int recursivePlanningDepth = 0; +#include "utils/lsyscache.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 +struct RecursivePlanningContextInternal { int level; uint64 planId; bool allDistributionKeysInQueryAreEqual; /* used for some optimizations */ List *subPlanList; PlannerRestrictionContext *plannerRestrictionContext; -} RecursivePlanningContext; +}; +/* track depth of current recursive planner query */ +static int recursivePlanningDepth = 0; /* * CteReferenceWalkerContext is used to collect CTE references in @@ -163,11 +172,11 @@ static bool AllDistributionKeysInSubqueryAreEqual(Query *subquery, restrictionContext); static bool ShouldRecursivelyPlanSetOperation(Query *query, RecursivePlanningContext *context); +static bool RecursivelyPlanSubquery(Query *subquery, + RecursivePlanningContext *planningContext); static void RecursivelyPlanSetOperations(Query *query, Node *node, RecursivePlanningContext *context); static bool IsLocalTableRteOrMatView(Node *node); -static void RecursivelyPlanSubquery(Query *subquery, - RecursivePlanningContext *planningContext); static DistributedSubPlan * CreateDistributedSubPlan(uint32 subPlanId, Query *subPlanQuery); static bool CteReferenceListWalker(Node *node, CteReferenceWalkerContext *context); @@ -182,6 +191,11 @@ static Query * BuildReadIntermediateResultsQuery(List *targetEntryList, List *columnAliasList, Const *resultIdConst, Oid functionOid, bool useBinaryCopyFormat); +static void UpdateVarNosInNode(Node *node, Index newVarNo); +static Query * CreateOuterSubquery(RangeTblEntry *rangeTableEntry, + List *outerSubqueryTargetList); +static List * GenerateRequiredColNamesFromTargetList(List *targetList); +static char * GetRelationNameAndAliasName(RangeTblEntry *rangeTablentry); /* * GenerateSubplansForSubqueriesAndCTEs is a wrapper around RecursivelyPlanSubqueriesAndCTEs. @@ -337,10 +351,33 @@ RecursivelyPlanSubqueriesAndCTEs(Query *query, RecursivePlanningContext *context RecursivelyPlanNonColocatedSubqueries(query, context); } + + if (ShouldConvertLocalTableJoinsToSubqueries(query->rtable)) + { + /* + * Logical planner cannot handle "local_table" [OUTER] JOIN "dist_table", or + * a query with local table/citus local table and subquery. We convert local/citus local + * tables to a subquery until they can be planned. + */ + RecursivelyPlanLocalTableJoins(query, context); + } + + return NULL; } +/* + * GetPlannerRestrictionContext returns the planner restriction context + * from the given context. + */ +PlannerRestrictionContext * +GetPlannerRestrictionContext(RecursivePlanningContext *recursivePlanningContext) +{ + return recursivePlanningContext->plannerRestrictionContext; +} + + /* * ShouldRecursivelyPlanNonColocatedSubqueries returns true if the input query contains joins * that are not on the distribution key. @@ -1076,6 +1113,17 @@ IsLocalTableRteOrMatView(Node *node) } Oid relationId = rangeTableEntry->relid; + return IsRelationLocalTableOrMatView(relationId); +} + + +/* + * IsRelationLocalTableOrMatView returns true if the given relation + * is a citus local, local, or materialized view. + */ +bool +IsRelationLocalTableOrMatView(Oid relationId) +{ if (!IsCitusTable(relationId)) { /* postgres local table or a materialized view */ @@ -1102,7 +1150,7 @@ IsLocalTableRteOrMatView(Node *node) * and immediately returns. Later, the planner decides on what to do * with the query. */ -static void +static bool RecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *planningContext) { uint64 planId = planningContext->planId; @@ -1113,7 +1161,7 @@ RecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *planningConte elog(DEBUG2, "skipping recursive planning for the subquery since it " "contains references to outer queries"); - return; + return false; } /* @@ -1125,6 +1173,7 @@ RecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *planningConte debugQuery = copyObject(subquery); } + /* * Create the subplan and append it to the list in the planning context. */ @@ -1155,6 +1204,7 @@ RecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *planningConte /* finally update the input subquery to point the result query */ *subquery = *resultQuery; + return true; } @@ -1337,6 +1387,221 @@ NodeContainsSubqueryReferencingOuterQuery(Node *node) } +/* + * ReplaceRTERelationWithRteSubquery replaces the input rte relation target entry + * with a subquery. The function also pushes down the filters to the subquery. + * + * It then recursively plans the subquery. This subquery is wrapped with another subquery + * as a trick to reduce network cost, because we currently don't have an easy way to + * skip generating NULL's for non-required columns, and if we create (SELECT a, NULL, NULL FROM table) + * then this will be sent over network and NULL's also occupy some space. Instead of this we generate: + * (SELECT t.a, NULL, NULL FROM (SELECT a FROM table) t). The inner subquery will be recursively planned + * but the outer part will not be yet it will still have the NULL columns so that the query is correct. + */ +void +ReplaceRTERelationWithRteSubquery(RangeTblEntry *rangeTableEntry, + List *requiredAttrNumbers, + RecursivePlanningContext *context) +{ + Query *subquery = WrapRteRelationIntoSubquery(rangeTableEntry, requiredAttrNumbers); + List *outerQueryTargetList = CreateAllTargetListForRelation(rangeTableEntry->relid, + requiredAttrNumbers); + + List *restrictionList = + GetRestrictInfoListForRelation(rangeTableEntry, + context->plannerRestrictionContext); + List *copyRestrictionList = copyObject(restrictionList); + Expr *andedBoundExpressions = make_ands_explicit(copyRestrictionList); + subquery->jointree->quals = (Node *) andedBoundExpressions; + + /* + * Originally the quals were pointing to the RTE and its varno + * was pointing to its index in rtable. However now we converted the RTE + * to a subquery and the quals should be pointing to that subquery, which + * is the only RTE in its rtable, hence we update the varnos so that they + * point to the subquery RTE. + * Originally: rtable: [rte1, current_rte, rte3...] + * Now: rtable: [rte1, subquery[current_rte], rte3...] --subquery[current_rte] refers to its rtable. + */ + Node *quals = subquery->jointree->quals; + UpdateVarNosInNode(quals, SINGLE_RTE_INDEX); + + /* replace the function with the constructed subquery */ + rangeTableEntry->rtekind = RTE_SUBQUERY; + rangeTableEntry->subquery = subquery; + + /* + * If the relation is inherited, it'll still be inherited as + * we've copied it earlier. This is to prevent the newly created + * subquery being treated as inherited. + */ + rangeTableEntry->inh = false; + + if (IsLoggableLevel(DEBUG1)) + { + char *relationAndAliasName = GetRelationNameAndAliasName(rangeTableEntry); + ereport(DEBUG1, (errmsg("Wrapping relation %s to a subquery", + relationAndAliasName))); + } + + /* as we created the subquery, now forcefully recursively plan it */ + bool recursivelyPlanned = RecursivelyPlanSubquery(subquery, context); + if (!recursivelyPlanned) + { + ereport(ERROR, (errmsg( + "unexpected state: query should have been recursively planned"))); + } + + Query *outerSubquery = CreateOuterSubquery(rangeTableEntry, outerQueryTargetList); + rangeTableEntry->subquery = outerSubquery; +} + + +/* + * GetRelationNameAndAliasName returns the relname + alias name if + * alias name exists otherwise only the relname is returned. + */ +static char * +GetRelationNameAndAliasName(RangeTblEntry *rangeTableEntry) +{ + StringInfo str = makeStringInfo(); + appendStringInfo(str, "\"%s\"", get_rel_name(rangeTableEntry->relid)); + + char *aliasName = NULL; + if (rangeTableEntry->alias) + { + aliasName = rangeTableEntry->alias->aliasname; + } + + if (aliasName) + { + appendStringInfo(str, " \"%s\"", aliasName); + } + return str->data; +} + + +/* + * CreateOuterSubquery creates outer subquery which contains + * the given range table entry in its rtable. + */ +static Query * +CreateOuterSubquery(RangeTblEntry *rangeTableEntry, List *outerSubqueryTargetList) +{ + List *innerSubqueryColNames = GenerateRequiredColNamesFromTargetList( + outerSubqueryTargetList); + + Query *outerSubquery = makeNode(Query); + outerSubquery->commandType = CMD_SELECT; + + /* we copy the input rteRelation to preserve the rteIdentity */ + RangeTblEntry *innerSubqueryRTE = copyObject(rangeTableEntry); + + innerSubqueryRTE->eref->colnames = innerSubqueryColNames; + outerSubquery->rtable = list_make1(innerSubqueryRTE); + + /* set the FROM expression to the subquery */ + RangeTblRef *newRangeTableRef = makeNode(RangeTblRef); + newRangeTableRef->rtindex = 1; + outerSubquery->jointree = makeFromExpr(list_make1(newRangeTableRef), NULL); + + outerSubquery->targetList = outerSubqueryTargetList; + return outerSubquery; +} + + +/* + * GenerateRequiredColNamesFromTargetList generates the required colnames + * from the given target list. + */ +static List * +GenerateRequiredColNamesFromTargetList(List *targetList) +{ + TargetEntry *entry = NULL; + List *innerSubqueryColNames = NIL; + foreach_ptr(entry, targetList) + { + if (IsA(entry->expr, Var)) + { + /* + * column names of the inner subquery should only contain the + * required columns, as in if we choose 'b' from ('a','b') colnames + * should be 'a' not ('a','b') + */ + innerSubqueryColNames = lappend(innerSubqueryColNames, makeString( + entry->resname)); + } + } + return innerSubqueryColNames; +} + + +/* + * UpdateVarNosInNode iterates the Vars in the + * given node and updates the varno's as the newVarNo. + */ +static void +UpdateVarNosInNode(Node *node, Index newVarNo) +{ + List *varList = pull_var_clause(node, PVC_RECURSE_AGGREGATES | + PVC_RECURSE_PLACEHOLDERS); + Var *var = NULL; + foreach_ptr(var, varList) + { + var->varno = newVarNo; + } +} + + +/* + * IsRecursivelyPlannableRelation returns true if the given range table entry + * is a relation type that can be converted to a subquery. + */ +bool +IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry) +{ + if (rangeTableEntry->rtekind != RTE_RELATION) + { + return false; + } + return rangeTableEntry->relkind == RELKIND_PARTITIONED_TABLE || + rangeTableEntry->relkind == RELKIND_RELATION || + rangeTableEntry->relkind == RELKIND_MATVIEW || + rangeTableEntry->relkind == RELKIND_FOREIGN_TABLE; +} + + +/* + * ContainsLocalTableDistributedTableJoin returns true if the input range table list + * contains a direct join between local RTE and an RTE that contains a distributed + * or reference table. + */ +bool +ContainsLocalTableDistributedTableJoin(List *rangeTableList) +{ + bool containsLocalTable = false; + bool containsDistributedTable = false; + + RangeTblEntry *rangeTableEntry = NULL; + foreach_ptr(rangeTableEntry, rangeTableList) + { + if (FindNodeMatchingCheckFunctionInRangeTableList(list_make1(rangeTableEntry), + IsDistributedOrReferenceTableRTE)) + { + containsDistributedTable = true; + } + else if (IsRecursivelyPlannableRelation(rangeTableEntry) && + IsLocalTableRteOrMatView((Node *) rangeTableEntry)) + { + /* we consider citus local tables as local table */ + containsLocalTable = true; + } + } + + return containsLocalTable && containsDistributedTable; +} + + /* * WrapFunctionsInSubqueries iterates over all the immediate Range Table Entries * of a query and wraps the functions inside (SELECT * FROM fnc() f) diff --git a/src/backend/distributed/planner/relation_restriction_equivalence.c b/src/backend/distributed/planner/relation_restriction_equivalence.c index 565e17c11..cf1c5a59d 100644 --- a/src/backend/distributed/planner/relation_restriction_equivalence.c +++ b/src/backend/distributed/planner/relation_restriction_equivalence.c @@ -21,14 +21,19 @@ #include "distributed/pg_dist_partition.h" #include "distributed/query_utils.h" #include "distributed/relation_restriction_equivalence.h" +#include "distributed/shard_pruning.h" #include "nodes/nodeFuncs.h" #include "nodes/pg_list.h" #include "nodes/primnodes.h" #if PG_VERSION_NUM >= PG_VERSION_12 #include "nodes/pathnodes.h" +#include "optimizer/optimizer.h" #else +#include "optimizer/cost.h" #include "nodes/relation.h" +#include "optimizer/var.h" #endif +#include "optimizer/paths.h" #include "parser/parsetree.h" #include "optimizer/pathnode.h" @@ -139,10 +144,7 @@ static Index RelationRestrictionPartitionKeyIndex(RelationRestriction * relationRestriction); static bool AllRelationsInRestrictionContextColocated(RelationRestrictionContext * restrictionContext); -static RelationRestrictionContext * FilterRelationRestrictionContext( - RelationRestrictionContext *relationRestrictionContext, - Relids - queryRteIdentities); +static bool IsParam(Node *node); static JoinRestrictionContext * FilterJoinRestrictionContext( JoinRestrictionContext *joinRestrictionContext, Relids queryRteIdentities); @@ -483,9 +485,13 @@ FindUnionAllVar(PlannerInfo *root, List *appendRelList, Oid relationOid, bool RestrictionEquivalenceForPartitionKeys(PlannerRestrictionContext *restrictionContext) { - /* there is a single distributed relation, no need to continue */ - if (!ContainsMultipleDistributedRelations(restrictionContext)) + if (ContextContainsLocalRelation(restrictionContext->relationRestrictionContext)) { + return false; + } + else if (!ContainsMultipleDistributedRelations(restrictionContext)) + { + /* there is a single distributed relation, no need to continue */ return true; } @@ -1808,8 +1814,8 @@ FilterPlannerRestrictionForQuery(PlannerRestrictionContext *plannerRestrictionCo /* allocate the filtered planner restriction context and set all the fields */ PlannerRestrictionContext *filteredPlannerRestrictionContext = palloc0( sizeof(PlannerRestrictionContext)); - filteredPlannerRestrictionContext->fastPathRestrictionContext = - palloc0(sizeof(FastPathRestrictionContext)); + filteredPlannerRestrictionContext->fastPathRestrictionContext = palloc0( + sizeof(FastPathRestrictionContext)); filteredPlannerRestrictionContext->memoryContext = plannerRestrictionContext->memoryContext; @@ -1832,13 +1838,128 @@ FilterPlannerRestrictionForQuery(PlannerRestrictionContext *plannerRestrictionCo } +/* + * GetRestrictInfoListForRelation gets a range table entry and planner + * restriction context. The function returns a list of expressions that + * appear in the restriction context for only the given relation. And, + * all the varnos are set to 1. + */ +List * +GetRestrictInfoListForRelation(RangeTblEntry *rangeTblEntry, + PlannerRestrictionContext *plannerRestrictionContext) +{ + RelationRestriction *relationRestriction = + RelationRestrictionForRelation(rangeTblEntry, plannerRestrictionContext); + if (relationRestriction == NULL) + { + return NIL; + } + + RelOptInfo *relOptInfo = relationRestriction->relOptInfo; + List *joinRestrictInfo = relOptInfo->joininfo; + List *baseRestrictInfo = relOptInfo->baserestrictinfo; + + List *joinRestrictClauseList = get_all_actual_clauses(joinRestrictInfo); + if (ContainsFalseClause(joinRestrictClauseList)) + { + /* found WHERE false, no need to continue */ + return copyObject((List *) joinRestrictClauseList); + } + + + List *restrictExprList = NIL; + RestrictInfo *restrictInfo = NULL; + foreach_ptr(restrictInfo, baseRestrictInfo) + { + Expr *restrictionClause = restrictInfo->clause; + + /* we cannot process Params beacuse they are not known at this point */ + if (FindNodeMatchingCheckFunction((Node *) restrictionClause, IsParam)) + { + continue; + } + + /* + * If the restriction involves multiple tables, we cannot add it to + * input relation's expression list. + */ + Relids varnos = pull_varnos((Node *) restrictionClause); + if (bms_num_members(varnos) != 1) + { + continue; + } + + /* + * We're going to add this restriction expression to a subquery + * which consists of only one relation in its jointree. Thus, + * simply set the varnos accordingly. + */ + Expr *copyOfRestrictClause = (Expr *) copyObject((Node *) restrictionClause); + List *varClauses = pull_var_clause_default((Node *) copyOfRestrictClause); + Var *column = NULL; + foreach_ptr(column, varClauses) + { + column->varno = SINGLE_RTE_INDEX; + column->varnosyn = SINGLE_RTE_INDEX; + } + + restrictExprList = lappend(restrictExprList, copyOfRestrictClause); + } + + return restrictExprList; +} + + +/* + * RelationRestrictionForRelation gets the relation restriction for the given + * range table entry. + */ +RelationRestriction * +RelationRestrictionForRelation(RangeTblEntry *rangeTableEntry, + PlannerRestrictionContext *plannerRestrictionContext) +{ + int rteIdentity = GetRTEIdentity(rangeTableEntry); + RelationRestrictionContext *relationRestrictionContext = + plannerRestrictionContext->relationRestrictionContext; + Relids queryRteIdentities = bms_make_singleton(rteIdentity); + RelationRestrictionContext *filteredRelationRestrictionContext = + FilterRelationRestrictionContext(relationRestrictionContext, queryRteIdentities); + List *filteredRelationRestrictionList = + filteredRelationRestrictionContext->relationRestrictionList; + + if (list_length(filteredRelationRestrictionList) != 1) + { + return NULL; + } + + RelationRestriction *relationRestriction = + (RelationRestriction *) linitial(filteredRelationRestrictionList); + return relationRestriction; +} + + +/* + * IsParam determines whether the given node is a param. + */ +static bool +IsParam(Node *node) +{ + if (IsA(node, Param)) + { + return true; + } + + return false; +} + + /* * FilterRelationRestrictionContext gets a relation restriction context and * set of rte identities. It returns the relation restrictions that that appear * in the queryRteIdentities and returns a newly allocated * RelationRestrictionContext. */ -static RelationRestrictionContext * +RelationRestrictionContext * FilterRelationRestrictionContext(RelationRestrictionContext *relationRestrictionContext, Relids queryRteIdentities) { diff --git a/src/backend/distributed/planner/shard_pruning.c b/src/backend/distributed/planner/shard_pruning.c index b7133561d..8d606cc14 100644 --- a/src/backend/distributed/planner/shard_pruning.c +++ b/src/backend/distributed/planner/shard_pruning.c @@ -254,8 +254,6 @@ static bool IsValidPartitionKeyRestriction(OpExpr *opClause); static void AddPartitionKeyRestrictionToInstance(ClauseWalkerContext *context, OpExpr *opClause, Var *varClause, Const *constantClause); -static bool VarConstOpExprClause(OpExpr *opClause, Var **varClause, - Const **constantClause); static void AddSAOPartitionKeyRestrictionToInstance(ClauseWalkerContext *context, ScalarArrayOpExpr * arrayOperatorExpression); @@ -896,7 +894,7 @@ PrunableExpressionsWalker(PruningTreeNode *node, ClauseWalkerContext *context) * VarConstOpExprClause check whether an expression is a valid comparison of a Var to a Const. * Also obtaining the var with constant when valid. */ -static bool +bool VarConstOpExprClause(OpExpr *opClause, Var **varClause, Const **constantClause) { Var *foundVarClause = NULL; diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 6c4de8fe5..d54548b4c 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -38,6 +38,7 @@ #include "distributed/insert_select_executor.h" #include "distributed/intermediate_result_pruning.h" #include "distributed/local_executor.h" +#include "distributed/local_distributed_join_planner.h" #include "distributed/locally_reserved_shared_connections.h" #include "distributed/maintenanced.h" #include "distributed/metadata_utility.h" @@ -55,6 +56,7 @@ #include "distributed/multi_server_executor.h" #include "distributed/pg_dist_partition.h" #include "distributed/placement_connection.h" +#include "distributed/recursive_planning.h" #include "distributed/reference_table_utils.h" #include "distributed/relation_access_tracking.h" #include "distributed/run_from_same_connection.h" @@ -196,6 +198,16 @@ static const struct config_enum_entry log_level_options[] = { { NULL, 0, false} }; + +static const struct config_enum_entry local_table_join_policies[] = { + { "never", LOCAL_JOIN_POLICY_NEVER, false}, + { "prefer-local", LOCAL_JOIN_POLICY_PREFER_LOCAL, false}, + { "prefer-distributed", LOCAL_JOIN_POLICY_PREFER_DISTRIBUTED, false}, + { "auto", LOCAL_JOIN_POLICY_AUTO, false}, + { NULL, 0, false} +}; + + static const struct config_enum_entry multi_shard_modify_connection_options[] = { { "parallel", PARALLEL_CONNECTION, false }, { "sequential", SEQUENTIAL_CONNECTION, false }, @@ -709,6 +721,24 @@ RegisterCitusConfigVariables(void) GUC_SUPERUSER_ONLY, NULL, NULL, LocalPoolSizeGucShowHook); + DefineCustomEnumVariable( + "citus.local_table_join_policy", + gettext_noop("defines the behaviour when a distributed table " + "is joined with a local table"), + gettext_noop( + "There are 4 values available. The default, 'auto' will recursively plan" + "distributed tables if there is a constant filter on a unique index." + "'prefer-local' will choose local tables if possible." + "'prefer-distributed' will choose distributed tables if possible" + "'never' will basically skip local table joins." + ), + &LocalTableJoinPolicy, + LOCAL_JOIN_POLICY_AUTO, + local_table_join_policies, + PGC_USERSET, + GUC_STANDARD, + NULL, NULL, NULL); + DefineCustomBoolVariable( "citus.log_multi_join_order", gettext_noop("Logs the distributed join order to the server log."), diff --git a/src/backend/distributed/utils/citus_copyfuncs.c b/src/backend/distributed/utils/citus_copyfuncs.c index 24caf6a8d..8e725a7d0 100644 --- a/src/backend/distributed/utils/citus_copyfuncs.c +++ b/src/backend/distributed/utils/citus_copyfuncs.c @@ -327,6 +327,7 @@ CopyNodeTask(COPYFUNC_ARGS) COPY_SCALAR_FIELD(fetchedExplainAnalyzePlacementIndex); COPY_STRING_FIELD(fetchedExplainAnalyzePlan); COPY_SCALAR_FIELD(fetchedExplainAnalyzeExecutionDuration); + COPY_SCALAR_FIELD(isLocalTableModification); } diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index 9adbca31a..abf18aa27 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -540,6 +540,7 @@ OutTask(OUTFUNC_ARGS) WRITE_INT_FIELD(fetchedExplainAnalyzePlacementIndex); WRITE_STRING_FIELD(fetchedExplainAnalyzePlan); WRITE_FLOAT_FIELD(fetchedExplainAnalyzeExecutionDuration, "%.2f"); + WRITE_BOOL_FIELD(isLocalTableModification); } diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index 603b68b81..3461e2246 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -70,6 +70,7 @@ extern List * PreprocessClusterStmt(Node *node, const char *clusterCommand); /* index.c */ typedef void (*PGIndexProcessor)(Form_pg_index, List **); + /* call.c */ extern bool CallDistributedProcedureRemotely(CallStmt *callStmt, DestReceiver *dest); diff --git a/src/include/distributed/local_distributed_join_planner.h b/src/include/distributed/local_distributed_join_planner.h new file mode 100644 index 000000000..dd74c8fb1 --- /dev/null +++ b/src/include/distributed/local_distributed_join_planner.h @@ -0,0 +1,34 @@ + +/*------------------------------------------------------------------------- + * + * local_distributed_join_planner.h + * + * Declarations for functions to handle local-distributed table joins. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#ifndef LOCAL_DISTRIBUTED_JOIN_PLANNER_H +#define LOCAL_DISTRIBUTED_JOIN_PLANNER_H + +#include "postgres.h" +#include "distributed/recursive_planning.h" + +/* managed via guc.c */ +typedef enum +{ + LOCAL_JOIN_POLICY_NEVER = 0, + LOCAL_JOIN_POLICY_PREFER_LOCAL = 1, + LOCAL_JOIN_POLICY_PREFER_DISTRIBUTED = 2, + LOCAL_JOIN_POLICY_AUTO = 3, +} LocalJoinPolicy; + +extern int LocalTableJoinPolicy; + +extern bool ShouldConvertLocalTableJoinsToSubqueries(List *rangeTableList); +extern void RecursivelyPlanLocalTableJoins(Query *query, + RecursivePlanningContext *context); + +#endif /* LOCAL_DISTRIBUTED_JOIN_PLANNER_H */ diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index 7303e55fe..36a0a5d23 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -134,6 +134,7 @@ typedef enum ANY_CITUS_TABLE_TYPE } CitusTableType; + extern bool IsCitusTableType(Oid relationId, CitusTableType tableType); extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry, CitusTableType tableType); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 5db5b0b69..a34a6a9ff 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -114,6 +114,8 @@ extern List * GroupShardPlacementsForTableOnGroup(Oid relationId, int32 groupId) extern StringInfo GenerateSizeQueryOnMultiplePlacements(List *shardIntervalList, char *sizeQuery); extern List * RemoveCoordinatorPlacementIfNotSingleNode(List *placementList); +extern ShardPlacement * ShardPlacementOnGroup(uint64 shardId, int groupId); + /* Function declarations to modify shard and shard placement data */ extern void InsertShardRow(Oid relationId, uint64 shardId, char storageType, diff --git a/src/include/distributed/multi_logical_planner.h b/src/include/distributed/multi_logical_planner.h index 993e8b819..f8d1811a2 100644 --- a/src/include/distributed/multi_logical_planner.h +++ b/src/include/distributed/multi_logical_planner.h @@ -195,6 +195,7 @@ extern bool TargetListOnPartitionColumn(Query *query, List *targetEntryList); extern bool FindNodeMatchingCheckFunctionInRangeTableList(List *rtable, bool (*check)( Node *)); extern bool IsCitusTableRTE(Node *node); +extern bool IsDistributedOrReferenceTableRTE(Node *node); extern bool IsDistributedTableRTE(Node *node); extern bool IsReferenceTableRTE(Node *node); extern bool QueryContainsDistributedTableRTE(Query *query); diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 9f90798da..6dd9153b8 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -328,6 +328,11 @@ typedef struct Task * ExplainTaskList(). */ double fetchedExplainAnalyzeExecutionDuration; + + /* + * isLocalTableModification is true if the task is on modifying a local table. + */ + bool isLocalTableModification; } Task; @@ -575,7 +580,10 @@ extern List * QueryPushdownSqlTaskList(Query *query, uint64 jobId, RelationRestrictionContext * relationRestrictionContext, List *prunedRelationShardList, TaskType taskType, - bool modifyRequiresCoordinatorEvaluation); + bool modifyRequiresCoordinatorEvaluation, + DeferredErrorMessage **planningError); + +extern bool ModifyLocalTableJob(Job *job); /* function declarations for managing jobs */ extern uint64 UniqueJobId(void); @@ -589,5 +597,6 @@ extern RangeTblEntry * DerivedRangeTableEntry(MultiNode *multiNode, List *column List *funcColumnTypeMods, List *funcCollations); +extern List * FetchEqualityAttrNumsForRTE(Node *quals); #endif /* MULTI_PHYSICAL_PLANNER_H */ diff --git a/src/include/distributed/multi_router_planner.h b/src/include/distributed/multi_router_planner.h index 18278bda5..40cbaf447 100644 --- a/src/include/distributed/multi_router_planner.h +++ b/src/include/distributed/multi_router_planner.h @@ -42,7 +42,8 @@ extern DeferredErrorMessage * PlanRouterQuery(Query *originalQuery, List **prunedShardIntervalListList, bool replacePrunedQueryWithDummy, bool *multiShardModifyQuery, - Const **partitionValueConst); + Const **partitionValueConst, + bool *containOnlyLocalTable); extern List * RelationShardListForShardIntervalList(List *shardIntervalList, bool *shardsPresent); extern List * CreateTaskPlacementListForShardIntervals(List *shardIntervalList, @@ -84,7 +85,9 @@ extern List * TargetShardIntervalForFastPathQuery(Query *query, Const **outGoingPartitionValueConst); extern void GenerateSingleShardRouterTaskList(Job *job, List *relationShardList, - List *placementList, uint64 shardId); + List *placementList, + uint64 shardId, + bool isLocalTableModification); /* * FastPathPlanner is a subset of router planner, that's why we prefer to diff --git a/src/include/distributed/query_colocation_checker.h b/src/include/distributed/query_colocation_checker.h index 0c8c7292b..969ecbcf9 100644 --- a/src/include/distributed/query_colocation_checker.h +++ b/src/include/distributed/query_colocation_checker.h @@ -34,6 +34,8 @@ extern ColocatedJoinChecker CreateColocatedJoinChecker(Query *subquery, PlannerRestrictionContext * restrictionContext); extern bool SubqueryColocated(Query *subquery, ColocatedJoinChecker *context); - +extern Query * WrapRteRelationIntoSubquery(RangeTblEntry *rteRelation, + List *requiredAttributes); +extern List * CreateAllTargetListForRelation(Oid relationId, List *requiredAttributes); #endif /* QUERY_COLOCATION_CHECKER_H */ diff --git a/src/include/distributed/recursive_planning.h b/src/include/distributed/recursive_planning.h index e1017bd70..0a64f6845 100644 --- a/src/include/distributed/recursive_planning.h +++ b/src/include/distributed/recursive_planning.h @@ -22,6 +22,16 @@ #include "nodes/relation.h" #endif +typedef struct RecursivePlanningContextInternal RecursivePlanningContext; + +typedef struct RangeTblEntryIndex +{ + RangeTblEntry *rangeTableEntry; + Index rteIndex; +}RangeTblEntryIndex; + +extern PlannerRestrictionContext * GetPlannerRestrictionContext( + RecursivePlanningContext *recursivePlanningContext); extern List * GenerateSubplansForSubqueriesAndCTEs(uint64 planId, Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext); @@ -33,5 +43,10 @@ extern Query * BuildReadIntermediateResultsArrayQuery(List *targetEntryList, List *resultIdList, bool useBinaryCopyFormat); extern bool GeneratingSubplans(void); - +extern bool ContainsLocalTableDistributedTableJoin(List *rangeTableList); +extern void ReplaceRTERelationWithRteSubquery(RangeTblEntry *rangeTableEntry, + List *requiredAttrNumbers, + RecursivePlanningContext *context); +extern bool IsRecursivelyPlannableRelation(RangeTblEntry *rangeTableEntry); +extern bool IsRelationLocalTableOrMatView(Oid relationId); #endif /* RECURSIVE_PLANNING_H */ diff --git a/src/include/distributed/relation_restriction_equivalence.h b/src/include/distributed/relation_restriction_equivalence.h index 028b0587a..bfa650c0e 100644 --- a/src/include/distributed/relation_restriction_equivalence.h +++ b/src/include/distributed/relation_restriction_equivalence.h @@ -15,6 +15,8 @@ #include "distributed/distributed_planner.h" #include "distributed/metadata_cache.h" +#define SINGLE_RTE_INDEX 1 + extern bool AllDistributionKeysInQueryAreEqual(Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext); @@ -36,8 +38,21 @@ extern List * DistributedRelationIdList(Query *query); extern PlannerRestrictionContext * FilterPlannerRestrictionForQuery( PlannerRestrictionContext *plannerRestrictionContext, Query *query); +extern List * GetRestrictInfoListForRelation(RangeTblEntry *rangeTblEntry, + PlannerRestrictionContext * + plannerRestrictionContext); +extern RelationRestriction * RelationRestrictionForRelation( + RangeTblEntry *rangeTableEntry, + PlannerRestrictionContext * + plannerRestrictionContext); +extern JoinRestrictionContext * RemoveDuplicateJoinRestrictions(JoinRestrictionContext * + joinRestrictionContext); + extern bool EquivalenceListContainsRelationsEquality(List *attributeEquivalenceList, RelationRestrictionContext * restrictionContext); - +extern RelationRestrictionContext * FilterRelationRestrictionContext( + RelationRestrictionContext *relationRestrictionContext, + Relids + queryRteIdentities); #endif /* RELATION_RESTRICTION_EQUIVALENCE_H */ diff --git a/src/include/distributed/shard_pruning.h b/src/include/distributed/shard_pruning.h index a780a7336..04176314e 100644 --- a/src/include/distributed/shard_pruning.h +++ b/src/include/distributed/shard_pruning.h @@ -20,7 +20,10 @@ extern List * PruneShards(Oid relationId, Index rangeTableId, List *whereClauseList, Const **partitionValueConst); extern bool ContainsFalseClause(List *whereClauseList); +extern List * get_all_actual_clauses(List *restrictinfo_list); extern Const * TransformPartitionRestrictionValue(Var *partitionColumn, Const *restrictionValue, bool missingOk); +bool VarConstOpExprClause(OpExpr *opClause, Var **varClause, Const **constantClause); + #endif /* SHARD_PRUNING_H_ */ diff --git a/src/test/regress/expected/citus_local_dist_joins.out b/src/test/regress/expected/citus_local_dist_joins.out new file mode 100644 index 000000000..56d40ff23 --- /dev/null +++ b/src/test/regress/expected/citus_local_dist_joins.out @@ -0,0 +1,509 @@ +CREATE SCHEMA citus_local_dist_joins; +SET search_path TO citus_local_dist_joins; +SET client_min_messages to ERROR; +SELECT master_add_node('localhost', :master_port, groupId => 0) AS coordinator_nodeid \gset +CREATE TABLE citus_local(key int, value text); +SELECT create_citus_local_table('citus_local'); + create_citus_local_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE postgres_table (key int, value text, value_2 jsonb); +CREATE TABLE reference_table (key int, value text, value_2 jsonb); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table (key int, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_pkey (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_pkey', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_windex (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_windex', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE UNIQUE INDEX key_index ON distributed_table_windex (key); +CREATE TABLE distributed_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE distributed_partitioned_table_1 PARTITION OF distributed_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE distributed_partitioned_table_2 PARTITION OF distributed_partitioned_table FOR VALUES FROM (50) TO (200); +SELECT create_distributed_table('distributed_partitioned_table', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE local_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE local_partitioned_table_1 PARTITION OF local_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE local_partitioned_table_2 PARTITION OF local_partitioned_table FOR VALUES FROM (50) TO (200); +CREATE TABLE distributed_table_composite (key int, value text, value_2 jsonb, primary key (key, value)); +SELECT create_distributed_table('distributed_table_composite', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM postgres_table; +CREATE MATERIALIZED VIEW mv2 AS SELECT * FROM distributed_table; +-- set log messages to debug1 so that we can see which tables are recursively planned. +SET client_min_messages TO DEBUG1; +INSERT INTO postgres_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO reference_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_windex SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_pkey SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_composite SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO local_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO citus_local SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- a unique index on key so dist table should be recursively planned +SELECT count(*) FROM citus_local JOIN distributed_table_windex USING(key); +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table_windex USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM citus_local JOIN distributed_table_windex USING(value); +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, citus_local_1.value FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table_windex USING (value)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM citus_local JOIN distributed_table_windex ON citus_local.key = distributed_table_windex.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table_windex ON ((citus_local.key OPERATOR(pg_catalog.=) distributed_table_windex.key))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM citus_local JOIN distributed_table_windex ON distributed_table_windex.key = 10; +DEBUG: Wrapping relation "distributed_table_windex" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.distributed_table_windex WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (citus_local_dist_joins.citus_local JOIN (SELECT distributed_table_windex_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_windex_1) distributed_table_windex ON ((distributed_table_windex.key OPERATOR(pg_catalog.=) 10))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- no unique index, citus local table should be recursively planned +SELECT count(*) FROM citus_local JOIN distributed_table USING(key); +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM citus_local JOIN distributed_table USING(value); +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, citus_local_1.value FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table USING (value)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM citus_local JOIN distributed_table ON citus_local.key = distributed_table.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table ON ((citus_local.key OPERATOR(pg_catalog.=) distributed_table.key))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM citus_local JOIN distributed_table ON distributed_table.key = 10; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table ON ((distributed_table.key OPERATOR(pg_catalog.=) 10))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM citus_local JOIN distributed_table USING(key) JOIN postgres_table USING (key) JOIN reference_table USING(key); +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM citus_local_dist_joins.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((((SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local JOIN citus_local_dist_joins.distributed_table USING (key)) JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) JOIN citus_local_dist_joins.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) JOIN reference_table USING (key) + JOIN citus_local USING(key) WHERE distributed_partitioned_table.key > 10 and distributed_partitioned_table.key = 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((citus_local_dist_joins.distributed_partitioned_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) JOIN citus_local_dist_joins.reference_table USING (key)) JOIN (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local USING (key)) WHERE ((distributed_partitioned_table.key OPERATOR(pg_catalog.>) 10) AND (distributed_partitioned_table.key OPERATOR(pg_catalog.=) 10)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- update +BEGIN; +SELECT COUNT(DISTINCT value) FROM citus_local; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + citus_local +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = citus_local.key; +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE citus_local_dist_joins.citus_local SET value = 'test'::text FROM (SELECT distributed_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_1) distributed_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM citus_local; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table +SET + value = 'test' +FROM + citus_local +WHERE + distributed_table.key = citus_local.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE citus_local_dist_joins.distributed_table SET value = 'test'::text FROM (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local WHERE (distributed_table.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + citus_local +WHERE + distributed_table_pkey.key = citus_local.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE citus_local_dist_joins.distributed_table_pkey SET value = 'test'::text FROM (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + citus_local +WHERE + distributed_table_windex.key = citus_local.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE citus_local_dist_joins.distributed_table_windex SET value = 'test'::text FROM (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local WHERE (distributed_table_windex.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +UPDATE + mv1 +SET + value = 'test' +FROM + citus_local +WHERE + mv1.key = citus_local.key; +ERROR: materialized views in modify queries are not supported +ROLLBACK; +BEGIN; +UPDATE + citus_local +SET + value = 'test' +FROM + mv1 +WHERE + mv1.key = citus_local.key; +ERROR: materialized views in modify queries are not supported +ROLLBACK; +BEGIN; +UPDATE + citus_local +SET + value = 'test' +FROM + mv2 +WHERE + mv2.key = citus_local.key; +ERROR: materialized views in modify queries are not supported +ROLLBACK; +-- DELETE operations +BEGIN; +SELECT COUNT(DISTINCT value) FROM citus_local; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + citus_local +USING + distributed_table +WHERE + distributed_table.key = citus_local.key; +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM citus_local_dist_joins.citus_local USING (SELECT distributed_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_1) distributed_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM citus_local; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + distributed_table +USING + citus_local +WHERE + distributed_table.key = citus_local.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM citus_local_dist_joins.distributed_table USING (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local WHERE (distributed_table.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + distributed_table_pkey +USING + citus_local +WHERE + distributed_table_pkey.key = citus_local.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM citus_local_dist_joins.distributed_table_pkey USING (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + distributed_table_windex +USING + citus_local +WHERE + distributed_table_windex.key = citus_local.key; +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM citus_local_dist_joins.distributed_table_windex USING (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local WHERE (distributed_table_windex.key OPERATOR(pg_catalog.=) citus_local.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +DELETE FROM + mv1 +USING + citus_local +WHERE + mv1.key = citus_local.key; +ERROR: materialized views in modify queries are not supported +DELETE FROM + citus_local +USING + mv1 +WHERE + mv1.key = citus_local.key; +ERROR: materialized views in modify queries are not supported +DELETE FROM + citus_local +USING + mv2 +WHERE + mv2.key = citus_local.key; +ERROR: materialized views in modify queries are not supported +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM (SELECT * FROM distributed_table LIMIT 1) d1) d2 using (key) JOIN reference_table USING(key) JOIN citus_local USING (key) JOIN (SELECT * FROM citus_local) c1 USING (key) WHERE d2.key > 10 AND d2.key = 10; +DEBUG: push down of limit count: 1 +DEBUG: generating subplan XXX_1 for subquery SELECT key, value, value_2 FROM citus_local_dist_joins.distributed_table LIMIT 1 +DEBUG: generating subplan XXX_2 for subquery SELECT key, value FROM citus_local_dist_joins.citus_local +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT key FROM citus_local_dist_joins.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_4 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN (SELECT d1.key, d1.value, d1.value_2 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb)) d1) d2 USING (key)) JOIN citus_local_dist_joins.reference_table USING (key)) JOIN (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local USING (key)) JOIN (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) c1 USING (key)) WHERE ((d2.key OPERATOR(pg_catalog.>) 10) AND (d2.key OPERATOR(pg_catalog.=) 10)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM (SELECT * FROM distributed_table LIMIT 1) d1) d2 using (key) JOIN reference_table USING(key) JOIN citus_local USING (key) JOIN (SELECT * FROM citus_local) c1 USING (key) WHERE d2.key > 10 AND d2.key = 10; +DEBUG: push down of limit count: 1 +DEBUG: generating subplan XXX_1 for subquery SELECT key, value, value_2 FROM citus_local_dist_joins.distributed_table LIMIT 1 +DEBUG: generating subplan XXX_2 for subquery SELECT key, value FROM citus_local_dist_joins.citus_local +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT key FROM citus_local_dist_joins.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Wrapping relation "citus_local" to a subquery +DEBUG: generating subplan XXX_4 for subquery SELECT key FROM citus_local_dist_joins.citus_local WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN (SELECT d1.key, d1.value, d1.value_2 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb)) d1) d2 USING (key)) JOIN citus_local_dist_joins.reference_table USING (key)) JOIN (SELECT citus_local_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) citus_local_1) citus_local USING (key)) JOIN (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) c1 USING (key)) WHERE ((d2.key OPERATOR(pg_catalog.>) 10) AND (d2.key OPERATOR(pg_catalog.=) 10)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT + COUNT(*) +FROM + postgres_table p1 +JOIN + distributed_partitioned_table dp1 +USING (key) +JOIN + distributed_table d1 +USING (key) +JOIN + citus_local c1 +USING (key) +JOIN + postgres_table p2 +USING (key) +JOIN + reference_table r1 +USING (key) +JOIN + distributed_table d2 +USING (key) +JOIN + citus_local c2 +USING (key); +DEBUG: Wrapping relation "postgres_table" "p1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM citus_local_dist_joins.postgres_table p1 WHERE true +DEBUG: Wrapping relation "citus_local" "c1" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM citus_local_dist_joins.citus_local c1 WHERE true +DEBUG: Wrapping relation "postgres_table" "p2" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT key FROM citus_local_dist_joins.postgres_table p2 WHERE true +DEBUG: Wrapping relation "citus_local" "c2" to a subquery +DEBUG: generating subplan XXX_4 for subquery SELECT key FROM citus_local_dist_joins.citus_local c2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((((((((SELECT p1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p1_1) p1 JOIN citus_local_dist_joins.distributed_partitioned_table dp1 USING (key)) JOIN citus_local_dist_joins.distributed_table d1 USING (key)) JOIN (SELECT c1_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) c1_1) c1 USING (key)) JOIN (SELECT p2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p2_1) p2 USING (key)) JOIN citus_local_dist_joins.reference_table r1 USING (key)) JOIN citus_local_dist_joins.distributed_table d2 USING (key)) JOIN (SELECT c2_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) c2_1) c2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SET client_min_messages to ERROR; +DROP TABLE citus_local; +SELECT master_remove_node('localhost', :master_port); + master_remove_node +--------------------------------------------------------------------- + +(1 row) + +\set VERBOSITY terse +DROP SCHEMA citus_local_dist_joins CASCADE; diff --git a/src/test/regress/expected/citus_local_tables_queries.out b/src/test/regress/expected/citus_local_tables_queries.out index aabd2fe07..1f6ec90a6 100644 --- a/src/test/regress/expected/citus_local_tables_queries.out +++ b/src/test/regress/expected/citus_local_tables_queries.out @@ -120,7 +120,7 @@ NOTICE: executing the command locally: SELECT a, b FROM citus_local_table_queri -- should fail as we don't support direct joins between distributed/local tables SELECT count(*) FROM distributed_table d1, distributed_table d2, citus_local_table; -ERROR: direct joins between distributed and local tables are not supported +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns -- local table inside subquery should just work SELECT count(*) FROM ( @@ -154,7 +154,12 @@ SELECT count(*) FROM ( SELECT *, random() FROM (SELECT *, random() FROM citus_local_table, distributed_table) as subquery_inner ) as subquery_top; -ERROR: direct joins between distributed and local tables are not supported +NOTICE: executing the command locally: SELECT NULL::integer AS a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true + count +--------------------------------------------------------------------- + 36 +(1 row) + -- should fail as we don't support direct joins between distributed/local tables SELECT count(*) FROM ( @@ -162,7 +167,13 @@ SELECT count(*) FROM FROM ( WITH cte_1 AS (SELECT *, random() FROM citus_local_table, distributed_table) SELECT * FROM cte_1) as subquery_inner ) as subquery_top; -ERROR: direct joins between distributed and local tables are not supported +NOTICE: executing the command locally: SELECT a, b FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT subquery_inner.a, subquery_inner.b, subquery_inner.a_1 AS a, subquery_inner.b_1 AS b, subquery_inner.random, random() AS random FROM (SELECT cte_1.a, cte_1.b, cte_1.a_1 AS a, cte_1.b_1 AS b, cte_1.random FROM (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.a_1 AS a, intermediate_result.b_1 AS b, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, a_1 integer, b_1 integer, random double precision)) cte_1(a, b, a_1, b_1, random)) subquery_inner(a, b, a_1, b_1, random)) subquery_top(a, b, a_1, b_1, random, random_1) + count +--------------------------------------------------------------------- + 36 +(1 row) + -- should be fine SELECT count(*) FROM ( @@ -474,7 +485,7 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (((citus_l -- not supported direct outer join SELECT count(*) FROM citus_local_table LEFT JOIN distributed_table ON (true); -ERROR: direct joins between distributed and local tables are not supported +ERROR: cannot pushdown the subquery -- distinct in subquery on CTE WITH one_row AS ( SELECT a from citus_local_table WHERE b = 1 @@ -516,9 +527,14 @@ NOTICE: executing the command locally: SELECT a, b FROM citus_local_table_queri -- join between citus local tables and distributed tables would fail SELECT count(*) FROM citus_local_table, distributed_table; -ERROR: direct joins between distributed and local tables are not supported +NOTICE: executing the command locally: SELECT NULL::integer AS a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true + count +--------------------------------------------------------------------- + 36 +(1 row) + SELECT * FROM citus_local_table, distributed_table ORDER BY 1,2,3,4 FOR UPDATE; -ERROR: direct joins between distributed and local tables are not supported +ERROR: could not run distributed query with FOR UPDATE/SHARE commands -- join between citus local tables and postgres local tables are okey SELECT count(citus_local_table.b), count(postgres_local_table.a) FROM citus_local_table, postgres_local_table @@ -638,7 +654,8 @@ NOTICE: executing the copy locally for shard xxxxx INSERT INTO citus_local_table SELECT distributed_table.* FROM distributed_table JOIN citus_local_table ON (true); -ERROR: direct joins between distributed and local tables are not supported +NOTICE: executing the command locally: SELECT NULL::integer AS a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true +NOTICE: executing the copy locally for shard xxxxx -- .. but when wrapped into a CTE, join works fine INSERT INTO citus_local_table SELECT distributed_table.* FROM distributed_table @@ -662,63 +679,67 @@ SELECT clear_and_init_test_tables(); DELETE FROM citus_local_table USING postgres_local_table WHERE citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table USING citus_local_table_queries.postgres_local_table WHERE (citus_local_table.b OPERATOR(pg_catalog.=) postgres_local_table.b) UPDATE citus_local_table SET b = 5 FROM postgres_local_table WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +NOTICE: executing the command locally: UPDATE citus_local_table_queries.citus_local_table_1509000 citus_local_table SET b = 5 FROM citus_local_table_queries.postgres_local_table WHERE ((citus_local_table.a OPERATOR(pg_catalog.=) 3) AND (citus_local_table.b OPERATOR(pg_catalog.=) postgres_local_table.b)) DELETE FROM postgres_local_table USING citus_local_table WHERE citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.postgres_local_table USING citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE (citus_local_table.b OPERATOR(pg_catalog.=) postgres_local_table.b) UPDATE postgres_local_table SET b = 5 FROM citus_local_table WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +NOTICE: executing the command locally: UPDATE citus_local_table_queries.postgres_local_table SET b = 5 FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE ((citus_local_table.a OPERATOR(pg_catalog.=) 3) AND (citus_local_table.b OPERATOR(pg_catalog.=) postgres_local_table.b)) -- no direct joins supported UPDATE distributed_table SET b = 6 FROM citus_local_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true UPDATE reference_table SET b = 6 FROM citus_local_table WHERE citus_local_table.a = reference_table.a; -ERROR: cannot plan modifications of reference tables with citus local tables +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true +NOTICE: executing the command locally: UPDATE citus_local_table_queries.reference_table_1509002 reference_table SET b = 6 FROM (SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) reference_table.a) -- should not work, add HINT use CTEs UPDATE citus_local_table SET b = 6 FROM distributed_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables +NOTICE: executing the command locally: UPDATE citus_local_table_queries.citus_local_table_1509000 citus_local_table SET b = 6 FROM (SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) distributed_table.a) -- should work, add HINT use CTEs UPDATE citus_local_table SET b = 6 FROM reference_table WHERE citus_local_table.a = reference_table.a; -NOTICE: executing the command locally: UPDATE citus_local_table_queries.citus_local_table_1509000 citus_local_table SET b = 6 FROM citus_local_table_queries.reference_table_1509002 reference_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) reference_table.a) +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.reference_table_1509002 reference_table WHERE true +NOTICE: executing the command locally: UPDATE citus_local_table_queries.citus_local_table_1509000 citus_local_table SET b = 6 FROM (SELECT reference_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) reference_table_1) reference_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) reference_table.a) -- should not work, add HINT use CTEs DELETE FROM distributed_table USING citus_local_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true -- should not work, add HINT use CTEs DELETE FROM citus_local_table USING distributed_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables +NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table USING (SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) distributed_table.a) DELETE FROM reference_table USING citus_local_table WHERE citus_local_table.a = reference_table.a; -ERROR: cannot plan modifications of reference tables with citus local tables +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true +NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.reference_table_1509002 reference_table USING (SELECT citus_local_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) reference_table.a) -- should work, add HINT use CTEs DELETE FROM citus_local_table USING reference_table WHERE citus_local_table.a = reference_table.a; -NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table USING citus_local_table_queries.reference_table_1509002 reference_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) reference_table.a) +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.reference_table_1509002 reference_table WHERE true +NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table USING (SELECT reference_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) reference_table_1) reference_table WHERE (citus_local_table.a OPERATOR(pg_catalog.=) reference_table.a) -- just works DELETE FROM citus_local_table WHERE citus_local_table.a IN (SELECT a FROM distributed_table); @@ -726,7 +747,8 @@ NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.ci -- just works DELETE FROM citus_local_table WHERE citus_local_table.a IN (SELECT a FROM reference_table); -NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE (a OPERATOR(pg_catalog.=) ANY (SELECT reference_table.a FROM citus_local_table_queries.reference_table_1509002 reference_table)) +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.reference_table_1509002 reference_table +NOTICE: executing the command locally: DELETE FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE (a OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer))) -- just works WITH distributed_table_cte AS (SELECT * FROM distributed_table) UPDATE citus_local_table @@ -734,13 +756,14 @@ SET b = 6 FROM distributed_table_cte WHERE citus_local_table.a = distributed_table_cte.a; NOTICE: executing the command locally: UPDATE citus_local_table_queries.citus_local_table_1509000 citus_local_table SET b = 6 FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) distributed_table_cte WHERE (citus_local_table.a OPERATOR(pg_catalog.=) distributed_table_cte.a) +SET citus.log_local_commands to off; -- just works WITH reference_table_cte AS (SELECT * FROM reference_table) UPDATE citus_local_table SET b = 6 FROM reference_table_cte WHERE citus_local_table.a = reference_table_cte.a; -NOTICE: executing the command locally: WITH reference_table_cte AS (SELECT reference_table.a, reference_table.b FROM citus_local_table_queries.reference_table_1509002 reference_table) UPDATE citus_local_table_queries.citus_local_table_1509000 citus_local_table SET b = 6 FROM reference_table_cte WHERE (citus_local_table.a OPERATOR(pg_catalog.=) reference_table_cte.a) +set citus.log_local_commands to on; --------------------------------------------------------------------- ----- VIEW QUERIES ----- --------------------------------------------------------------------- @@ -772,7 +795,14 @@ JOIN citus_local_table_2 USING (a) JOIN distributed_table USING (a); -- should fail as view contains direct local dist join SELECT count(*) FROM view_2; -ERROR: direct joins between distributed and local tables are not supported +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.citus_local_table_1509000 citus_local_table WHERE true +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.citus_local_table_2_1509001 citus_local_table_2 WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT intermediate_result.count FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)) view_2 + count +--------------------------------------------------------------------- + 1 +(1 row) + CREATE VIEW view_3 AS SELECT count(*) FROM citus_local_table_2 @@ -788,7 +818,8 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT co -- view treated as subquery, so should work SELECT count(*) FROM view_3, distributed_table; -NOTICE: executing the command locally: SELECT count(*) AS count FROM (citus_local_table_queries.citus_local_table_2_1509001 citus_local_table_2(a, b) JOIN citus_local_table_queries.reference_table_1509002 reference_table(a, b) USING (a)) +NOTICE: executing the command locally: SELECT a FROM citus_local_table_queries.citus_local_table_2_1509001 citus_local_table_2 WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM ((SELECT citus_local_table_2_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_2_1) citus_local_table_2 JOIN citus_local_table_queries.reference_table_1509002 reference_table(a, b) USING (a)) count --------------------------------------------------------------------- 6 @@ -826,7 +857,17 @@ UPDATE citus_local_table lt SET a = mt.a FROM distributed_table mt WHERE mt.b = lt.b RETURNING lt.b, lt.a ) SELECT * FROM cte JOIN distributed_table mt ON mt.b = cte.b ORDER BY 1,2,3,4; -ERROR: cannot plan modifications with citus local tables and distributed tables +NOTICE: executing the command locally: UPDATE citus_local_table_queries.citus_local_table_1509000 lt SET a = mt.a FROM (SELECT mt_1.a, mt_1.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) mt_1) mt WHERE (mt.b OPERATOR(pg_catalog.=) lt.b) RETURNING lt.b, lt.a + b | a | a | b +--------------------------------------------------------------------- + 0 | 0 | 0 | 0 + 1 | 1 | 1 | 1 + 2 | 2 | 2 | 2 + 3 | 3 | 3 | 3 + 4 | 4 | 4 | 4 + 5 | 5 | 5 | 5 +(6 rows) + -- join with CTE just works UPDATE citus_local_table SET a=5 @@ -852,7 +893,7 @@ UPDATE citus_local_table SET a=5 FROM (SELECT b FROM distributed_table) AS foo WHERE foo.b = citus_local_table.b; -ERROR: cannot plan modifications with citus local tables and distributed tables +ERROR: citus local table citus_local_table cannot be joined with these distributed tables --------------------------------------------------------------------- -- test different execution paths -- --------------------------------------------------------------------- diff --git a/src/test/regress/expected/citus_local_tables_queries_mx.out b/src/test/regress/expected/citus_local_tables_queries_mx.out index 833699555..ce61c7fdb 100644 --- a/src/test/regress/expected/citus_local_tables_queries_mx.out +++ b/src/test/regress/expected/citus_local_tables_queries_mx.out @@ -126,7 +126,7 @@ WHERE cte_citus_local_table.a = 1 AND cte_distributed_table.a = 1; -- should fail as we don't support direct joins between distributed/local tables SELECT count(*) FROM distributed_table d1, distributed_table d2, citus_local_table; -ERROR: direct joins between distributed and local tables are not supported +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns -- local table inside subquery should just work SELECT count(*) FROM ( @@ -154,7 +154,11 @@ SELECT count(*) FROM ( SELECT *, random() FROM (SELECT *, random() FROM citus_local_table, distributed_table) as subquery_inner ) as subquery_top; -ERROR: direct joins between distributed and local tables are not supported + count +--------------------------------------------------------------------- + 36 +(1 row) + -- should fail as we don't support direct joins between distributed/local tables SELECT count(*) FROM ( @@ -162,7 +166,11 @@ SELECT count(*) FROM FROM ( WITH cte_1 AS (SELECT *, random() FROM citus_local_table, distributed_table) SELECT * FROM cte_1) as subquery_inner ) as subquery_top; -ERROR: direct joins between distributed and local tables are not supported + count +--------------------------------------------------------------------- + 36 +(1 row) + -- should be fine SELECT count(*) FROM ( @@ -415,10 +423,14 @@ SELECT count(*) FROM reference_table LEFT JOIN citus_local_table ON (true) LEFT JOIN postgres_local_table ON (true) LEFT JOIN reference_table r2 ON (true); -ERROR: direct joins between distributed and local tables are not supported + count +--------------------------------------------------------------------- + 1296 +(1 row) + -- not supported direct outer join SELECT count(*) FROM citus_local_table LEFT JOIN distributed_table ON (true); -ERROR: direct joins between distributed and local tables are not supported +ERROR: cannot pushdown the subquery -- distinct in subquery on CTE WITH one_row AS ( SELECT a from citus_local_table WHERE b = 1 @@ -458,9 +470,13 @@ LIMIT -- join between citus local tables and distributed tables would fail SELECT count(*) FROM citus_local_table, distributed_table; -ERROR: direct joins between distributed and local tables are not supported + count +--------------------------------------------------------------------- + 36 +(1 row) + SELECT * FROM citus_local_table, distributed_table ORDER BY 1,2,3,4 FOR UPDATE; -ERROR: direct joins between distributed and local tables are not supported +ERROR: could not run distributed query with FOR UPDATE/SHARE commands -- join between citus local table and postgres local table would fail -- as citus local table is on the coordinator SELECT count(citus_local_table.b), count(postgres_local_table.a) @@ -523,7 +539,6 @@ JOIN citus_local_table ON (true); INSERT INTO reference_table SELECT reference_table.* FROM reference_table, postgres_local_table JOIN citus_local_table ON (true); -ERROR: direct joins between distributed and local tables are not supported SELECT clear_and_init_test_tables(); clear_and_init_test_tables --------------------------------------------------------------------- @@ -536,7 +551,6 @@ JOIN citus_local_table ON (true); INSERT INTO distributed_table SELECT reference_table.* FROM reference_table, postgres_local_table JOIN citus_local_table ON (true); -ERROR: direct joins between distributed and local tables are not supported INSERT INTO postgres_local_table SELECT reference_table.* FROM reference_table JOIN citus_local_table ON (true); @@ -557,7 +571,6 @@ FROM reference_table, distributed_table; INSERT INTO citus_local_table SELECT distributed_table.* FROM distributed_table JOIN citus_local_table ON (true); -ERROR: direct joins between distributed and local tables are not supported -- .. but when wrapped into a CTE, join works fine INSERT INTO citus_local_table SELECT distributed_table.* FROM distributed_table @@ -578,38 +591,35 @@ SELECT clear_and_init_test_tables(); DELETE FROM citus_local_table USING postgres_local_table WHERE citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +ERROR: Modifying local tables with citus local tables is supported only from the coordinator. UPDATE citus_local_table SET b = 5 FROM postgres_local_table WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +ERROR: Modifying local tables with citus local tables is supported only from the coordinator. DELETE FROM postgres_local_table USING citus_local_table WHERE citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +ERROR: Modifying local tables with citus local tables is supported only from the coordinator. UPDATE postgres_local_table SET b = 5 FROM citus_local_table WHERE citus_local_table.a = 3 AND citus_local_table.b = postgres_local_table.b; -ERROR: cannot plan modifications with local tables involving citus tables +ERROR: Modifying local tables with citus local tables is supported only from the coordinator. -- no direct joins supported UPDATE distributed_table SET b = 6 FROM citus_local_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables UPDATE reference_table SET b = 6 FROM citus_local_table WHERE citus_local_table.a = reference_table.a; -ERROR: cannot plan modifications of reference tables with citus local tables -- should not work, add HINT use CTEs UPDATE citus_local_table SET b = 6 FROM distributed_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables -- should work, add HINT use CTEs UPDATE citus_local_table SET b = 6 @@ -619,16 +629,13 @@ WHERE citus_local_table.a = reference_table.a; DELETE FROM distributed_table USING citus_local_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables -- should not work, add HINT use CTEs DELETE FROM citus_local_table USING distributed_table WHERE citus_local_table.a = distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables DELETE FROM reference_table USING citus_local_table WHERE citus_local_table.a = reference_table.a; -ERROR: cannot plan modifications of reference tables with citus local tables -- should work, add HINT use CTEs DELETE FROM citus_local_table USING reference_table @@ -671,7 +678,7 @@ SELECT count(*) FROM distributed_table WHERE b in (SELECT count FROM mat_view_4); count --------------------------------------------------------------------- - 1 + 0 (1 row) CREATE VIEW view_2 AS @@ -681,7 +688,11 @@ JOIN citus_local_table_2 USING (a) JOIN distributed_table USING (a); -- should fail as view contains direct local dist join SELECT count(*) FROM view_2; -ERROR: direct joins between distributed and local tables are not supported + count +--------------------------------------------------------------------- + 1 +(1 row) + CREATE VIEW view_3 AS SELECT count(*) FROM citus_local_table_2 @@ -698,7 +709,7 @@ SELECT count(*) FROM view_3; SELECT count(*) FROM view_3, distributed_table; count --------------------------------------------------------------------- - 6 + 0 (1 row) --------------------------------------------------------------------- @@ -732,7 +743,16 @@ UPDATE citus_local_table lt SET a = mt.a FROM distributed_table mt WHERE mt.b = lt.b RETURNING lt.b, lt.a ) SELECT * FROM cte JOIN distributed_table mt ON mt.b = cte.b ORDER BY 1,2,3,4; -ERROR: cannot plan modifications with citus local tables and distributed tables + b | a | a | b +--------------------------------------------------------------------- + 0 | 0 | 0 | 0 + 1 | 1 | 1 | 1 + 2 | 2 | 2 | 2 + 3 | 3 | 3 | 3 + 4 | 4 | 4 | 4 + 5 | 5 | 5 | 5 +(6 rows) + -- join with CTE just works UPDATE citus_local_table SET a=5 @@ -756,7 +776,7 @@ UPDATE citus_local_table SET a=5 FROM (SELECT b FROM distributed_table) AS foo WHERE foo.b = citus_local_table.b; -ERROR: cannot plan modifications with citus local tables and distributed tables +ERROR: citus local table citus_local_table cannot be joined with these distributed tables --------------------------------------------------------------------- -- test different execution paths -- --------------------------------------------------------------------- diff --git a/src/test/regress/expected/coordinator_shouldhaveshards.out b/src/test/regress/expected/coordinator_shouldhaveshards.out index 300c390d0..c847efb87 100644 --- a/src/test/regress/expected/coordinator_shouldhaveshards.out +++ b/src/test/regress/expected/coordinator_shouldhaveshards.out @@ -140,8 +140,8 @@ NOTICE: executing the command locally: SELECT y FROM coordinator_shouldhaveshar -- this should be run locally SELECT create_distributed_table('dist_table', 'a', colocate_with := 'none'); -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503004, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer)');SELECT worker_apply_shard_ddl_command (1503004, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503007, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer)');SELECT worker_apply_shard_ddl_command (1503007, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503004, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer) ');SELECT worker_apply_shard_ddl_command (1503004, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503007, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer) ');SELECT worker_apply_shard_ddl_command (1503007, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') NOTICE: executing the copy locally for shard xxxxx NOTICE: Copying data from local table... NOTICE: executing the copy locally for shard xxxxx @@ -176,8 +176,8 @@ NOTICE: executing the command locally: SELECT y FROM coordinator_shouldhaveshar -- this should be run locally SELECT create_distributed_table('dist_table', 'a', colocate_with := 'none'); -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503010, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer)');SELECT worker_apply_shard_ddl_command (1503010, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503013, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer)');SELECT worker_apply_shard_ddl_command (1503013, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503010, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer) ');SELECT worker_apply_shard_ddl_command (1503010, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503013, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table (a integer) ');SELECT worker_apply_shard_ddl_command (1503013, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table OWNER TO postgres') NOTICE: executing the copy locally for shard xxxxx NOTICE: Copying data from local table... NOTICE: executing the copy locally for shard xxxxx @@ -417,18 +417,28 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut INSERT INTO dist_table VALUES(1); NOTICE: executing the command locally: INSERT INTO coordinator_shouldhaveshards.dist_table_1503017 (a) VALUES (1) -SELECT * FROM local JOIN dist_table ON (a = x); -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins -SELECT * FROM local JOIN dist_table ON (a = x) WHERE a = 1;; -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins +SELECT * FROM local JOIN dist_table ON (a = x) ORDER BY 1,2,3; + x | y | a +--------------------------------------------------------------------- + 1 | 2 | 1 + 1 | 2 | 1 + 3 | 2 | 3 +(3 rows) + +SELECT * FROM local JOIN dist_table ON (a = x) WHERE a = 1 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT local.x, local.y, dist_table.a FROM ((SELECT local_1.x, local_1.y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) local_1) local JOIN coordinator_shouldhaveshards.dist_table_1503017 dist_table ON ((dist_table.a OPERATOR(pg_catalog.=) local.x))) WHERE (dist_table.a OPERATOR(pg_catalog.=) 1) ORDER BY local.x, local.y, dist_table.a + x | y | a +--------------------------------------------------------------------- + 1 | 2 | 1 + 1 | 2 | 1 +(2 rows) + -- intermediate results are allowed WITH cte_1 AS (SELECT * FROM dist_table ORDER BY 1 LIMIT 1) SELECT * FROM ref JOIN local ON (a = x) JOIN cte_1 ON (local.x = cte_1.a); NOTICE: executing the command locally: SELECT a FROM coordinator_shouldhaveshards.dist_table_1503017 dist_table WHERE true ORDER BY a LIMIT '1'::bigint NOTICE: executing the command locally: SELECT a FROM coordinator_shouldhaveshards.dist_table_1503020 dist_table WHERE true ORDER BY a LIMIT '1'::bigint -NOTICE: executing the command locally: SELECT ref.a, ref.b, local.x, local.y, cte_1.a FROM ((coordinator_shouldhaveshards.ref_1503016 ref JOIN coordinator_shouldhaveshards.local ON ((ref.a OPERATOR(pg_catalog.=) local.x))) JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) cte_1 ON ((local.x OPERATOR(pg_catalog.=) cte_1.a))) +NOTICE: executing the command locally: SELECT ref.a, ref.b, local.x, local.y, cte_1.a FROM ((coordinator_shouldhaveshards.ref_1503016 ref JOIN (SELECT local_1.x, local_1.y FROM (SELECT intermediate_result.x, intermediate_result.y FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(x integer, y integer)) local_1) local ON ((ref.a OPERATOR(pg_catalog.=) local.x))) JOIN (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) cte_1 ON ((local.x OPERATOR(pg_catalog.=) cte_1.a))) a | b | x | y | a --------------------------------------------------------------------- 1 | 2 | 1 | 2 | 1 @@ -465,10 +475,10 @@ CREATE TABLE dist_table1(a int); -- this will use queryStringList, make sure it works correctly with -- copying task SELECT create_distributed_table('dist_table1', 'a'); -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503029, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer)');SELECT worker_apply_shard_ddl_command (1503029, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503031, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer)');SELECT worker_apply_shard_ddl_command (1503031, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503032, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer)');SELECT worker_apply_shard_ddl_command (1503032, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') -NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503034, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer)');SELECT worker_apply_shard_ddl_command (1503034, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503029, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer) ');SELECT worker_apply_shard_ddl_command (1503029, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503031, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer) ');SELECT worker_apply_shard_ddl_command (1503031, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503032, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer) ');SELECT worker_apply_shard_ddl_command (1503032, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') +NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1503034, 'coordinator_shouldhaveshards', 'CREATE TABLE coordinator_shouldhaveshards.dist_table1 (a integer) ');SELECT worker_apply_shard_ddl_command (1503034, 'coordinator_shouldhaveshards', 'ALTER TABLE coordinator_shouldhaveshards.dist_table1 OWNER TO postgres') create_distributed_table --------------------------------------------------------------------- diff --git a/src/test/regress/expected/dml_recursive.out b/src/test/regress/expected/dml_recursive.out index e5a720618..bf2da3866 100644 --- a/src/test/regress/expected/dml_recursive.out +++ b/src/test/regress/expected/dml_recursive.out @@ -340,8 +340,10 @@ FROM cte_1 WHERE distributed_table.tenant_id < cte_1.tenant_id; DEBUG: generating subplan XXX_1 for CTE cte_1: WITH cte_2 AS (SELECT second_distributed_table.tenant_id AS cte2_id FROM recursive_dml_queries.second_distributed_table WHERE (second_distributed_table.dept OPERATOR(pg_catalog.>=) 2)) UPDATE recursive_dml_queries.distributed_table SET dept = 10 RETURNING tenant_id, dept, info DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE recursive_dml_queries.distributed_table SET dept = 5 FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept, intermediate_result.info FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer, info jsonb)) cte_1 WHERE (distributed_table.tenant_id OPERATOR(pg_catalog.<) cte_1.tenant_id) --- we don't support updating local table with a join with --- distributed tables +-- we support updating local table with a join with +-- distributed tables, though as the local table +-- is target here, distributed table is recursively +-- planned UPDATE local_table SET @@ -350,8 +352,9 @@ FROM distributed_table WHERE distributed_table.tenant_id = local_table.id; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT tenant_id FROM recursive_dml_queries.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE recursive_dml_queries.local_table SET id = 'citus_test'::text FROM (SELECT distributed_table_1.tenant_id, NULL::integer AS dept, NULL::jsonb AS info FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text)) distributed_table_1) distributed_table WHERE (distributed_table.tenant_id OPERATOR(pg_catalog.=) local_table.id) RESET client_min_messages; DROP SCHEMA recursive_dml_queries CASCADE; NOTICE: drop cascades to 5 other objects diff --git a/src/test/regress/expected/local_dist_join_mixed.out b/src/test/regress/expected/local_dist_join_mixed.out new file mode 100644 index 000000000..141509dfd --- /dev/null +++ b/src/test/regress/expected/local_dist_join_mixed.out @@ -0,0 +1,1609 @@ +CREATE SCHEMA local_dist_join_mixed; +SET search_path TO local_dist_join_mixed; +CREATE TABLE distributed (id bigserial PRIMARY KEY, + name text, + created_at timestamptz DEFAULT now()); +CREATE TABLE reference (id bigserial PRIMARY KEY, + title text); +CREATE TABLE local (id bigserial PRIMARY KEY, + title text); +-- these above restrictions brought us to the following schema +SELECT create_reference_table('reference'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('distributed', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO distributed SELECT i, i::text, now() FROM generate_series(0,100)i; +INSERT INTO reference SELECT i, i::text FROM generate_series(0,100)i; +INSERT INTO local SELECT i, i::text FROM generate_series(0,100)i; +SET client_min_messages to DEBUG1; +-- very simple 1-1 Joins +SELECT count(*) FROM distributed JOIN local USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM distributed JOIN local ON (name = title); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON ((distributed.name OPERATOR(pg_catalog.=) local.title))) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed d1 JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local ON ((d1.name OPERATOR(pg_catalog.=) (d1.id)::text))) + count +--------------------------------------------------------------------- + 10201 +(1 row) + +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) + count +--------------------------------------------------------------------- + 5050 +(1 row) + +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +DEBUG: Wrapping relation "distributed" "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, name FROM local_dist_join_mixed.distributed d1 WHERE ((name OPERATOR(pg_catalog.=) (id)::text) AND (id OPERATOR(pg_catalog.=) 1)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT d1_1.id, d1_1.name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id, intermediate_result.name FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text)) d1_1) d1 JOIN local_dist_join_mixed.local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE (d1.id OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 99 +(1 row) + +SELECT count(*) FROM distributed JOIN local USING (id) WHERE false; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE false +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) WHERE false + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 OR True; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE ((d1.id OPERATOR(pg_catalog.=) 1) OR true) + count +--------------------------------------------------------------------- + 5050 +(1 row) + +SELECT count(*) FROM distributed d1 JOIN local ON (name::int + local.id > d1.id AND d1.id < local.title::int) WHERE d1.id = 1; +DEBUG: Wrapping relation "distributed" "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, name FROM local_dist_join_mixed.distributed d1 WHERE (id OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT d1_1.id, d1_1.name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id, intermediate_result.name FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text)) d1_1) d1 JOIN local_dist_join_mixed.local ON (((((d1.name)::integer OPERATOR(pg_catalog.+) local.id) OPERATOR(pg_catalog.>) d1.id) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE (d1.id OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 99 +(1 row) + +SELECT count(*) FROM distributed JOIN local ON (hashtext(name) = hashtext(title)); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON ((hashtext(distributed.name) OPERATOR(pg_catalog.=) hashtext(local.title)))) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT hashtext(local.id::text) FROM distributed JOIN local ON (hashtext(name) = hashtext(title)) ORDER BY 1 LIMIT 4; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT hashtext((local.id)::text) AS hashtext FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, local_1.title FROM (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_1) local ON ((hashtext(distributed.name) OPERATOR(pg_catalog.=) hashtext(local.title)))) ORDER BY (hashtext((local.id)::text)) LIMIT 4 +DEBUG: push down of limit count: 4 + hashtext +--------------------------------------------------------------------- + -2114455578 + -2097988278 + -1997006946 + -1985772843 +(4 rows) + +SELECT '' as "xxx", local.*, 'xxx' as "test" FROM distributed JOIN local ON (hashtext(name) = hashtext(title)) ORDER BY 1,2,3 LIMIT 4; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ''::text AS xxx, local.id, local.title, 'xxx'::text AS test FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, local_1.title FROM (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_1) local ON ((hashtext(distributed.name) OPERATOR(pg_catalog.=) hashtext(local.title)))) ORDER BY ''::text, local.id, local.title LIMIT 4 +DEBUG: push down of limit count: 4 + xxx | id | title | test +--------------------------------------------------------------------- + | 0 | 0 | xxx + | 1 | 1 | xxx + | 2 | 2 | xxx + | 3 | 3 | xxx +(4 rows) + +SELECT local.title, count(*) FROM distributed JOIN local USING (id) GROUP BY 1 ORDER BY 1, 2 DESC LIMIT 5; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT local.title, count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, local_1.title FROM (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_1) local USING (id)) GROUP BY local.title ORDER BY local.title, (count(*)) DESC LIMIT 5 + title | count +--------------------------------------------------------------------- + 0 | 1 + 1 | 1 + 10 | 1 + 100 | 1 + 11 | 1 +(5 rows) + +SELECT distributed.id as id1, local.id as id2 FROM distributed JOIN local USING(id) ORDER BY distributed.id + local.id LIMIT 5; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT distributed.id AS id1, local.id AS id2 FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) ORDER BY (distributed.id OPERATOR(pg_catalog.+) local.id) LIMIT 5 +DEBUG: push down of limit count: 5 + id1 | id2 +--------------------------------------------------------------------- + 0 | 0 + 1 | 1 + 2 | 2 + 3 | 3 + 4 | 4 +(5 rows) + +SELECT distributed.id as id1, local.id as id2, count(*) FROM distributed JOIN local USING(id) GROUP BY distributed.id, local.id ORDER BY 1,2 LIMIT 5; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT distributed.id AS id1, local.id AS id2, count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) GROUP BY distributed.id, local.id ORDER BY distributed.id, local.id LIMIT 5 +DEBUG: push down of limit count: 5 + id1 | id2 | count +--------------------------------------------------------------------- + 0 | 0 | 1 + 1 | 1 | 1 + 2 | 2 | 1 + 3 | 3 | 1 + 4 | 4 | 1 +(5 rows) + +-- basic subqueries that cannot be pulled up +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = title); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON ((d1.name OPERATOR(pg_catalog.=) local.title))) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local ON ((d1.name OPERATOR(pg_catalog.=) (d1.id)::text))) + count +--------------------------------------------------------------------- + 10201 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) + count +--------------------------------------------------------------------- + 5050 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE (d1.id OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 99 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 AND false; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE ((id OPERATOR(pg_catalog.<) (title)::integer) AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE ((d1.id OPERATOR(pg_catalog.=) 1) AND false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 OR true; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE ((d1.id OPERATOR(pg_catalog.=) 1) OR true) + count +--------------------------------------------------------------------- + 5050 +(1 row) + +-- pull up subqueries as they are pretty simple, local table should be recursively planned +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = title); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON ((d1.name OPERATOR(pg_catalog.=) local.title))) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local ON ((d1.name OPERATOR(pg_catalog.=) (d1.id)::text))) + count +--------------------------------------------------------------------- + 10201 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) + count +--------------------------------------------------------------------- + 5050 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE (d1.id OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 99 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 AND false; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE ((id OPERATOR(pg_catalog.<) (title)::integer) AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE ((d1.id OPERATOR(pg_catalog.=) 1) AND false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 OR true; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE ((d1.id OPERATOR(pg_catalog.=) 1) OR true) + count +--------------------------------------------------------------------- + 5050 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed WHERE id = 2) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE ((id OPERATOR(pg_catalog.<) (title)::integer) AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed WHERE (distributed.id OPERATOR(pg_catalog.=) 2)) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE (d1.id OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed WHERE false) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT title FROM local_dist_join_mixed.local WHERE (false AND (id OPERATOR(pg_catalog.<) (title)::integer)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed WHERE false) d1 JOIN (SELECT NULL::bigint AS id, local_1.title FROM (SELECT intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(title text)) local_1) local ON (((d1.name OPERATOR(pg_catalog.=) (d1.id)::text) AND (d1.id OPERATOR(pg_catalog.<) (local.title)::integer)))) WHERE (d1.id OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- TEMPORARY table +CREATE TEMPORARY TABLE temp_local AS SELECT * FROM local; +SELECT count(*) FROM distributed JOIN temp_local USING (id); +DEBUG: Wrapping relation "temp_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM temp_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT temp_local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) temp_local_1) temp_local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +-- UNLOGGED table +CREATE UNLOGGED TABLE unlogged_local AS SELECT * FROM local; +SELECT count(*) FROM distributed JOIN unlogged_local USING (id); +DEBUG: Wrapping relation "unlogged_local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.unlogged_local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT unlogged_local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) unlogged_local_1) unlogged_local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +-- mat view +CREATE MATERIALIZED VIEW mat_view AS SELECT * FROM local; +SELECT count(*) FROM distributed JOIN mat_view USING (id); +DEBUG: Wrapping relation "mat_view" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.mat_view WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT mat_view_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) mat_view_1) mat_view USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +CREATE VIEW local_regular_view AS SELECT * FROM local; +CREATE VIEW dist_regular_view AS SELECT * FROM distributed; +SELECT count(*) FROM distributed JOIN local_regular_view USING (id); +DEBUG: generating subplan XXX_1 for subquery SELECT local.id, local.title FROM local_dist_join_mixed.local +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_regular_view USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM local JOIN dist_regular_view USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN (SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) dist_regular_view USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM dist_regular_view JOIN local_regular_view USING (id); +DEBUG: generating subplan XXX_1 for subquery SELECT local.id, local.title FROM local_dist_join_mixed.local +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) dist_regular_view JOIN (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_regular_view USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +-- join alias/table alias +SELECT COUNT(*) FROM (distributed JOIN local USING (id)) AS t(a,b,c,d) ORDER BY d,c,a,b LIMIT 3; +ERROR: column "local.title" must appear in the GROUP BY clause or be used in an aggregate function +SELECT COUNT(*) FROM (distributed d1(x,y,y1) JOIN local l1(x,t) USING (x)) AS t(a,b,c,d) ORDER BY d,c,a,b LIMIT 3; +ERROR: column "l1.t" must appear in the GROUP BY clause or be used in an aggregate function +-- final queries are pushdown queries +SELECT sum(d1.id + local.id) FROM distributed d1 JOIN local USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum((d1.id OPERATOR(pg_catalog.+) local.id)) AS sum FROM (local_dist_join_mixed.distributed d1 JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + sum +--------------------------------------------------------------------- + 10100 +(1 row) + +SELECT sum(d1.id + local.id) OVER (PARTITION BY d1.id) FROM distributed d1 JOIN local USING (id) ORDER BY 1 DESC LIMIT 4; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum((d1.id OPERATOR(pg_catalog.+) local.id)) OVER (PARTITION BY d1.id) AS sum FROM (local_dist_join_mixed.distributed d1 JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) ORDER BY (sum((d1.id OPERATOR(pg_catalog.+) local.id)) OVER (PARTITION BY d1.id)) DESC LIMIT 4 +DEBUG: push down of limit count: 4 + sum +--------------------------------------------------------------------- + 200 + 198 + 196 + 194 +(4 rows) + +SELECT count(*) FROM distributed d1 JOIN local USING (id) LEFT JOIN distributed d2 USING (id) ORDER BY 1 DESC LIMIT 4; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_dist_join_mixed.distributed d1 JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) LEFT JOIN local_dist_join_mixed.distributed d2 USING (id)) ORDER BY (count(*)) DESC LIMIT 4 +DEBUG: push down of limit count: 4 + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(DISTINCT d1.name::int * local.id) FROM distributed d1 JOIN local USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(DISTINCT ((d1.name)::integer OPERATOR(pg_catalog.*) local.id)) AS count FROM (local_dist_join_mixed.distributed d1 JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +-- final queries are router queries +SELECT sum(d1.id + local.id) FROM distributed d1 JOIN local USING (id) WHERE d1.id = 1; +DEBUG: Wrapping relation "distributed" "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.distributed d1 WHERE (id OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum((d1.id OPERATOR(pg_catalog.+) local.id)) AS sum FROM ((SELECT d1_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) d1_1) d1 JOIN local_dist_join_mixed.local USING (id)) WHERE (d1.id OPERATOR(pg_catalog.=) 1) + sum +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT sum(d1.id + local.id) OVER (PARTITION BY d1.id) FROM distributed d1 JOIN local USING (id) WHERE d1.id = 1 ORDER BY 1 DESC LIMIT 4; +DEBUG: Wrapping relation "distributed" "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.distributed d1 WHERE (id OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum((d1.id OPERATOR(pg_catalog.+) local.id)) OVER (PARTITION BY d1.id) AS sum FROM ((SELECT d1_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) d1_1) d1 JOIN local_dist_join_mixed.local USING (id)) WHERE (d1.id OPERATOR(pg_catalog.=) 1) ORDER BY (sum((d1.id OPERATOR(pg_catalog.+) local.id)) OVER (PARTITION BY d1.id)) DESC LIMIT 4 + sum +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT count(*) FROM distributed d1 JOIN local USING (id) LEFT JOIN distributed d2 USING (id) WHERE d2.id = 1 ORDER BY 1 DESC LIMIT 4; +DEBUG: Wrapping relation "distributed" "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.distributed d1 WHERE (id OPERATOR(pg_catalog.=) 1) +DEBUG: Wrapping relation "distributed" "d2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM local_dist_join_mixed.distributed d2 WHERE (id OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((SELECT d1_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) d1_1) d1 JOIN local_dist_join_mixed.local USING (id)) LEFT JOIN (SELECT d2_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) d2_1) d2 USING (id)) WHERE (d2.id OPERATOR(pg_catalog.=) 1) ORDER BY (count(*)) DESC LIMIT 4 + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- final queries are pull to coordinator queries +SELECT sum(d1.id + local.id) OVER (PARTITION BY d1.id + local.id) FROM distributed d1 JOIN local USING (id) ORDER BY 1 DESC LIMIT 4; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum((d1.id OPERATOR(pg_catalog.+) local.id)) OVER (PARTITION BY (d1.id OPERATOR(pg_catalog.+) local.id)) AS sum FROM (local_dist_join_mixed.distributed d1 JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) ORDER BY (sum((d1.id OPERATOR(pg_catalog.+) local.id)) OVER (PARTITION BY (d1.id OPERATOR(pg_catalog.+) local.id))) DESC LIMIT 4 + sum +--------------------------------------------------------------------- + 200 + 198 + 196 + 194 +(4 rows) + +-- nested subqueries +SELECT + count(*) +FROM + (SELECT * FROM (SELECT * FROM distributed) as foo) as bar + JOIN + local + USING(id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.id, foo.name, foo.created_at FROM (SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed) foo) bar JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + local + USING(id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.id, foo.name, foo.created_at, foo.random, random() AS random FROM (SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) foo) bar(id, name, created_at, random, random_1) JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + local + USING(id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.id, foo.name, foo.created_at, foo.random, random() AS random FROM (SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) foo) bar(id, name, created_at, random, random_1) JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id); +DEBUG: generating subplan XXX_1 for subquery SELECT id, title, random() AS random FROM local_dist_join_mixed.local +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.id, foo.name, foo.created_at, foo.random, random() AS random FROM (SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) foo) bar(id, name, created_at, random, random_1) JOIN (SELECT foo2.id, foo2.title, foo2.random, random() AS random FROM (SELECT intermediate_result.id, intermediate_result.title, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text, random double precision)) foo2) bar2(id, title, random, random_1) USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +-- TODO: Unnecessary recursive planning for local +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed LIMIT 1) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id); +DEBUG: push down of limit count: 1 +DEBUG: generating subplan XXX_1 for subquery SELECT id, name, created_at, random() AS random FROM local_dist_join_mixed.distributed LIMIT 1 +DEBUG: generating subplan XXX_2 for subquery SELECT id, title, random() AS random FROM local_dist_join_mixed.local +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo.id, foo.name, foo.created_at, foo.random, random() AS random FROM (SELECT intermediate_result.id, intermediate_result.name, intermediate_result.created_at, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text, created_at timestamp with time zone, random double precision)) foo) bar(id, name, created_at, random, random_1) JOIN (SELECT foo2.id, foo2.title, foo2.random, random() AS random FROM (SELECT intermediate_result.id, intermediate_result.title, intermediate_result.random FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text, random double precision)) foo2) bar2(id, title, random, random_1) USING (id)) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- subqueries in WHERE clause +-- is not colocated, and the JOIN inside as well. +-- so should be recursively planned twice +SELECT + count(*) +FROM + distributed +WHERE + id > (SELECT + count(*) + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id) + ); +DEBUG: generating subplan XXX_1 for subquery SELECT id, title, random() AS random FROM local_dist_join_mixed.local +DEBUG: generating subplan XXX_2 for subquery SELECT count(*) AS count FROM ((SELECT foo.id, foo.name, foo.created_at, foo.random, random() AS random FROM (SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed) foo) bar(id, name, created_at, random, random_1) JOIN (SELECT foo2.id, foo2.title, foo2.random, random() AS random FROM (SELECT intermediate_result.id, intermediate_result.title, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text, random double precision)) foo2) bar2(id, title, random, random_1) USING (id)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM local_dist_join_mixed.distributed WHERE (id OPERATOR(pg_catalog.>) (SELECT intermediate_result.count FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(count bigint))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- two distributed tables are co-located and JOINed on distribution +-- key, so should be fine to pushdown +SELECT + count(*) +FROM + distributed d_upper +WHERE + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id) + ) IS NOT NULL; +DEBUG: generating subplan XXX_1 for subquery SELECT id, title, random() AS random FROM local_dist_join_mixed.local +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM local_dist_join_mixed.distributed d_upper WHERE ((SELECT bar.id FROM ((SELECT foo.id, foo.name, foo.created_at, foo.random, random() AS random FROM (SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed WHERE (distributed.id OPERATOR(pg_catalog.=) d_upper.id)) foo) bar(id, name, created_at, random, random_1) JOIN (SELECT foo2.id, foo2.title, foo2.random, random() AS random FROM (SELECT intermediate_result.id, intermediate_result.title, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text, random double precision)) foo2) bar2(id, title, random, random_1) USING (id))) IS NOT NULL) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT + count(*) +FROM + distributed d_upper +WHERE + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + local as foo + USING(id) + ) IS NOT NULL; +DEBUG: Wrapping relation "local" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local foo WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM local_dist_join_mixed.distributed d_upper WHERE ((SELECT bar.id FROM ((SELECT foo_1.id, foo_1.name, foo_1.created_at, foo_1.random, random() AS random FROM (SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed WHERE (distributed.id OPERATOR(pg_catalog.=) d_upper.id)) foo_1) bar(id, name, created_at, random, random_1) JOIN (SELECT foo_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) foo_1) foo USING (id))) IS NOT NULL) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT + count(*) +FROM + distributed d_upper +WHERE d_upper.id > + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + local as foo + USING(id) + ); +DEBUG: Wrapping relation "local" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local foo WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM local_dist_join_mixed.distributed d_upper WHERE (id OPERATOR(pg_catalog.>) (SELECT bar.id FROM ((SELECT foo_1.id, foo_1.name, foo_1.created_at, foo_1.random, random() AS random FROM (SELECT distributed.id, distributed.name, distributed.created_at, random() AS random FROM local_dist_join_mixed.distributed WHERE (distributed.id OPERATOR(pg_catalog.=) d_upper.id)) foo_1) bar(id, name, created_at, random, random_1) JOIN (SELECT foo_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) foo_1) foo USING (id)))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT + count(*) +FROM + distributed d_upper +WHERE + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local WHERE d_upper.id = id) as foo2) as bar2 + USING(id) + ) IS NOT NULL; +ERROR: direct joins between distributed and local tables are not supported +HINT: Use CTE's or subqueries to select from local tables and use them in joins +-- subqueries in the target list +-- router, should work +select (SELECT local.id) FROM local, distributed WHERE distributed.id = 1 LIMIT 1; +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE (id OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT (SELECT local.id) AS id FROM local_dist_join_mixed.local, (SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed WHERE (distributed.id OPERATOR(pg_catalog.=) 1) LIMIT 1 + id +--------------------------------------------------------------------- + 0 +(1 row) + +-- should fail +select (SELECT local.id) FROM local, distributed WHERE distributed.id != 1 LIMIT 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT (SELECT local.id) AS id FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (distributed.id OPERATOR(pg_catalog.<>) 1) LIMIT 1 +DEBUG: push down of limit count: 1 + id +--------------------------------------------------------------------- + 0 +(1 row) + +-- currently not supported, but should work with https://github.com/citusdata/citus/pull/4360/files +SELECT + name, (SELECT id FROM local WHERE id = e.id) +FROM + distributed e +ORDER BY 1,2 LIMIT 1; +ERROR: direct joins between distributed and local tables are not supported +HINT: Use CTE's or subqueries to select from local tables and use them in joins +-- set operations +SELECT local.* FROM distributed JOIN local USING (id) + EXCEPT +SELECT local.* FROM distributed JOIN local USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id, title FROM local_dist_join_mixed.local WHERE true +DEBUG: generating subplan XXX_3 for subquery SELECT local.id, local.title FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, local_1.title FROM (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_1) local USING (id)) +DEBUG: generating subplan XXX_4 for subquery SELECT local.id, local.title FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, local_1.title FROM (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_1) local USING (id)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text) EXCEPT SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text) + id | title +--------------------------------------------------------------------- +(0 rows) + +SELECT distributed.* FROM distributed JOIN local USING (id) + EXCEPT +SELECT distributed.* FROM distributed JOIN local USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: generating subplan XXX_3 for subquery SELECT distributed.id, distributed.name, distributed.created_at FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) +DEBUG: generating subplan XXX_4 for subquery SELECT distributed.id, distributed.name, distributed.created_at FROM (local_dist_join_mixed.distributed JOIN (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local USING (id)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT intermediate_result.id, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text, created_at timestamp with time zone) EXCEPT SELECT intermediate_result.id, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text, created_at timestamp with time zone) + id | name | created_at +--------------------------------------------------------------------- +(0 rows) + +SELECT count(*) FROM +( + (SELECT * FROM (SELECT * FROM local) as f JOIN distributed USING (id)) + UNION ALL + (SELECT * FROM (SELECT * FROM local) as f2 JOIN distributed USING (id)) +) bar; +DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM local_dist_join_mixed.local +DEBUG: generating subplan XXX_2 for subquery SELECT id, title FROM local_dist_join_mixed.local +DEBUG: generating subplan XXX_3 for subquery SELECT f.id, f.title, distributed.name, distributed.created_at FROM ((SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) f JOIN local_dist_join_mixed.distributed USING (id)) +DEBUG: generating subplan XXX_4 for subquery SELECT f2.id, f2.title, distributed.name, distributed.created_at FROM ((SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) f2 JOIN local_dist_join_mixed.distributed USING (id)) +DEBUG: generating subplan XXX_5 for subquery SELECT intermediate_result.id, intermediate_result.title, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text, name text, created_at timestamp with time zone) UNION ALL SELECT intermediate_result.id, intermediate_result.title, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text, name text, created_at timestamp with time zone) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.id, intermediate_result.title, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text, name text, created_at timestamp with time zone)) bar + count +--------------------------------------------------------------------- + 202 +(1 row) + +SELECT count(*) FROM +( + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as fo) + UNION ALL + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as ba) +) bar; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: generating subplan XXX_3 for subquery SELECT id, name, created_at FROM (SELECT distributed.id, distributed.name, distributed.created_at FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed USING (id))) fo +DEBUG: generating subplan XXX_4 for subquery SELECT id, name, created_at FROM (SELECT distributed.id, distributed.name, distributed.created_at FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed USING (id))) ba +DEBUG: generating subplan XXX_5 for subquery SELECT intermediate_result.id, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text, created_at timestamp with time zone) UNION ALL SELECT intermediate_result.id, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text, created_at timestamp with time zone) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.id, intermediate_result.name, intermediate_result.created_at FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text, created_at timestamp with time zone)) bar + count +--------------------------------------------------------------------- + 202 +(1 row) + +select count(DISTINCT id) +FROM +( + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as fo) + UNION ALL + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as ba) +) bar; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(DISTINCT id) AS count FROM (SELECT fo.id, fo.name, fo.created_at FROM (SELECT distributed.id, distributed.name, distributed.created_at FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed USING (id))) fo UNION ALL SELECT ba.id, ba.name, ba.created_at FROM (SELECT distributed.id, distributed.name, distributed.created_at FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed USING (id))) ba) bar + count +--------------------------------------------------------------------- + 101 +(1 row) + +-- 25 Joins +select ' select count(*) from distributed ' || string_Agg('INNER +JOIN local u'|| x::text || ' USING (id)',' ') from +generate_Series(1,25)x; + ?column? +--------------------------------------------------------------------- + select count(*) from distributed INNER+ + JOIN local u1 USING (id) INNER + + JOIN local u2 USING (id) INNER + + JOIN local u3 USING (id) INNER + + JOIN local u4 USING (id) INNER + + JOIN local u5 USING (id) INNER + + JOIN local u6 USING (id) INNER + + JOIN local u7 USING (id) INNER + + JOIN local u8 USING (id) INNER + + JOIN local u9 USING (id) INNER + + JOIN local u10 USING (id) INNER + + JOIN local u11 USING (id) INNER + + JOIN local u12 USING (id) INNER + + JOIN local u13 USING (id) INNER + + JOIN local u14 USING (id) INNER + + JOIN local u15 USING (id) INNER + + JOIN local u16 USING (id) INNER + + JOIN local u17 USING (id) INNER + + JOIN local u18 USING (id) INNER + + JOIN local u19 USING (id) INNER + + JOIN local u20 USING (id) INNER + + JOIN local u21 USING (id) INNER + + JOIN local u22 USING (id) INNER + + JOIN local u23 USING (id) INNER + + JOIN local u24 USING (id) INNER + + JOIN local u25 USING (id) +(1 row) + +\gexec + select count(*) from distributed INNER +JOIN local u1 USING (id) INNER +JOIN local u2 USING (id) INNER +JOIN local u3 USING (id) INNER +JOIN local u4 USING (id) INNER +JOIN local u5 USING (id) INNER +JOIN local u6 USING (id) INNER +JOIN local u7 USING (id) INNER +JOIN local u8 USING (id) INNER +JOIN local u9 USING (id) INNER +JOIN local u10 USING (id) INNER +JOIN local u11 USING (id) INNER +JOIN local u12 USING (id) INNER +JOIN local u13 USING (id) INNER +JOIN local u14 USING (id) INNER +JOIN local u15 USING (id) INNER +JOIN local u16 USING (id) INNER +JOIN local u17 USING (id) INNER +JOIN local u18 USING (id) INNER +JOIN local u19 USING (id) INNER +JOIN local u20 USING (id) INNER +JOIN local u21 USING (id) INNER +JOIN local u22 USING (id) INNER +JOIN local u23 USING (id) INNER +JOIN local u24 USING (id) INNER +JOIN local u25 USING (id) +DEBUG: Wrapping relation "local" "u1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local u1 WHERE true +DEBUG: Wrapping relation "local" "u2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM local_dist_join_mixed.local u2 WHERE true +DEBUG: Wrapping relation "local" "u3" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT id FROM local_dist_join_mixed.local u3 WHERE true +DEBUG: Wrapping relation "local" "u4" to a subquery +DEBUG: generating subplan XXX_4 for subquery SELECT id FROM local_dist_join_mixed.local u4 WHERE true +DEBUG: Wrapping relation "local" "u5" to a subquery +DEBUG: generating subplan XXX_5 for subquery SELECT id FROM local_dist_join_mixed.local u5 WHERE true +DEBUG: Wrapping relation "local" "u6" to a subquery +DEBUG: generating subplan XXX_6 for subquery SELECT id FROM local_dist_join_mixed.local u6 WHERE true +DEBUG: Wrapping relation "local" "u7" to a subquery +DEBUG: generating subplan XXX_7 for subquery SELECT id FROM local_dist_join_mixed.local u7 WHERE true +DEBUG: Wrapping relation "local" "u8" to a subquery +DEBUG: generating subplan XXX_8 for subquery SELECT id FROM local_dist_join_mixed.local u8 WHERE true +DEBUG: Wrapping relation "local" "u9" to a subquery +DEBUG: generating subplan XXX_9 for subquery SELECT id FROM local_dist_join_mixed.local u9 WHERE true +DEBUG: Wrapping relation "local" "u10" to a subquery +DEBUG: generating subplan XXX_10 for subquery SELECT id FROM local_dist_join_mixed.local u10 WHERE true +DEBUG: Wrapping relation "local" "u11" to a subquery +DEBUG: generating subplan XXX_11 for subquery SELECT id FROM local_dist_join_mixed.local u11 WHERE true +DEBUG: Wrapping relation "local" "u12" to a subquery +DEBUG: generating subplan XXX_12 for subquery SELECT id FROM local_dist_join_mixed.local u12 WHERE true +DEBUG: Wrapping relation "local" "u13" to a subquery +DEBUG: generating subplan XXX_13 for subquery SELECT id FROM local_dist_join_mixed.local u13 WHERE true +DEBUG: Wrapping relation "local" "u14" to a subquery +DEBUG: generating subplan XXX_14 for subquery SELECT id FROM local_dist_join_mixed.local u14 WHERE true +DEBUG: Wrapping relation "local" "u15" to a subquery +DEBUG: generating subplan XXX_15 for subquery SELECT id FROM local_dist_join_mixed.local u15 WHERE true +DEBUG: Wrapping relation "local" "u16" to a subquery +DEBUG: generating subplan XXX_16 for subquery SELECT id FROM local_dist_join_mixed.local u16 WHERE true +DEBUG: Wrapping relation "local" "u17" to a subquery +DEBUG: generating subplan XXX_17 for subquery SELECT id FROM local_dist_join_mixed.local u17 WHERE true +DEBUG: Wrapping relation "local" "u18" to a subquery +DEBUG: generating subplan XXX_18 for subquery SELECT id FROM local_dist_join_mixed.local u18 WHERE true +DEBUG: Wrapping relation "local" "u19" to a subquery +DEBUG: generating subplan XXX_19 for subquery SELECT id FROM local_dist_join_mixed.local u19 WHERE true +DEBUG: Wrapping relation "local" "u20" to a subquery +DEBUG: generating subplan XXX_20 for subquery SELECT id FROM local_dist_join_mixed.local u20 WHERE true +DEBUG: Wrapping relation "local" "u21" to a subquery +DEBUG: generating subplan XXX_21 for subquery SELECT id FROM local_dist_join_mixed.local u21 WHERE true +DEBUG: Wrapping relation "local" "u22" to a subquery +DEBUG: generating subplan XXX_22 for subquery SELECT id FROM local_dist_join_mixed.local u22 WHERE true +DEBUG: Wrapping relation "local" "u23" to a subquery +DEBUG: generating subplan XXX_23 for subquery SELECT id FROM local_dist_join_mixed.local u23 WHERE true +DEBUG: Wrapping relation "local" "u24" to a subquery +DEBUG: generating subplan XXX_24 for subquery SELECT id FROM local_dist_join_mixed.local u24 WHERE true +DEBUG: Wrapping relation "local" "u25" to a subquery +DEBUG: generating subplan XXX_25 for subquery SELECT id FROM local_dist_join_mixed.local u25 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((((((((((((((((((((((((local_dist_join_mixed.distributed JOIN (SELECT u1_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u1_1) u1 USING (id)) JOIN (SELECT u2_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u2_1) u2 USING (id)) JOIN (SELECT u3_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u3_1) u3 USING (id)) JOIN (SELECT u4_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u4_1) u4 USING (id)) JOIN (SELECT u5_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u5_1) u5 USING (id)) JOIN (SELECT u6_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_6'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u6_1) u6 USING (id)) JOIN (SELECT u7_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_7'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u7_1) u7 USING (id)) JOIN (SELECT u8_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_8'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u8_1) u8 USING (id)) JOIN (SELECT u9_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_9'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u9_1) u9 USING (id)) JOIN (SELECT u10_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_10'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u10_1) u10 USING (id)) JOIN (SELECT u11_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_11'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u11_1) u11 USING (id)) JOIN (SELECT u12_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_12'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u12_1) u12 USING (id)) JOIN (SELECT u13_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_13'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u13_1) u13 USING (id)) JOIN (SELECT u14_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_14'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u14_1) u14 USING (id)) JOIN (SELECT u15_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_15'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u15_1) u15 USING (id)) JOIN (SELECT u16_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_16'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u16_1) u16 USING (id)) JOIN (SELECT u17_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_17'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u17_1) u17 USING (id)) JOIN (SELECT u18_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_18'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u18_1) u18 USING (id)) JOIN (SELECT u19_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_19'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u19_1) u19 USING (id)) JOIN (SELECT u20_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_20'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u20_1) u20 USING (id)) JOIN (SELECT u21_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_21'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u21_1) u21 USING (id)) JOIN (SELECT u22_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_22'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u22_1) u22 USING (id)) JOIN (SELECT u23_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_23'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u23_1) u23 USING (id)) JOIN (SELECT u24_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_24'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u24_1) u24 USING (id)) JOIN (SELECT u25_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_25'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u25_1) u25 USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +select ' select count(*) from distributed ' || string_Agg('INNER +JOIN local u'|| x::text || ' ON (false)',' ') from +generate_Series(1,25)x; + ?column? +--------------------------------------------------------------------- + select count(*) from distributed INNER+ + JOIN local u1 ON (false) INNER + + JOIN local u2 ON (false) INNER + + JOIN local u3 ON (false) INNER + + JOIN local u4 ON (false) INNER + + JOIN local u5 ON (false) INNER + + JOIN local u6 ON (false) INNER + + JOIN local u7 ON (false) INNER + + JOIN local u8 ON (false) INNER + + JOIN local u9 ON (false) INNER + + JOIN local u10 ON (false) INNER + + JOIN local u11 ON (false) INNER + + JOIN local u12 ON (false) INNER + + JOIN local u13 ON (false) INNER + + JOIN local u14 ON (false) INNER + + JOIN local u15 ON (false) INNER + + JOIN local u16 ON (false) INNER + + JOIN local u17 ON (false) INNER + + JOIN local u18 ON (false) INNER + + JOIN local u19 ON (false) INNER + + JOIN local u20 ON (false) INNER + + JOIN local u21 ON (false) INNER + + JOIN local u22 ON (false) INNER + + JOIN local u23 ON (false) INNER + + JOIN local u24 ON (false) INNER + + JOIN local u25 ON (false) +(1 row) + +\gexec + select count(*) from distributed INNER +JOIN local u1 ON (false) INNER +JOIN local u2 ON (false) INNER +JOIN local u3 ON (false) INNER +JOIN local u4 ON (false) INNER +JOIN local u5 ON (false) INNER +JOIN local u6 ON (false) INNER +JOIN local u7 ON (false) INNER +JOIN local u8 ON (false) INNER +JOIN local u9 ON (false) INNER +JOIN local u10 ON (false) INNER +JOIN local u11 ON (false) INNER +JOIN local u12 ON (false) INNER +JOIN local u13 ON (false) INNER +JOIN local u14 ON (false) INNER +JOIN local u15 ON (false) INNER +JOIN local u16 ON (false) INNER +JOIN local u17 ON (false) INNER +JOIN local u18 ON (false) INNER +JOIN local u19 ON (false) INNER +JOIN local u20 ON (false) INNER +JOIN local u21 ON (false) INNER +JOIN local u22 ON (false) INNER +JOIN local u23 ON (false) INNER +JOIN local u24 ON (false) INNER +JOIN local u25 ON (false) +DEBUG: Wrapping relation "local" "u1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u1 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u2 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u3" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u3 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u4" to a subquery +DEBUG: generating subplan XXX_4 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u4 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u5" to a subquery +DEBUG: generating subplan XXX_5 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u5 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u6" to a subquery +DEBUG: generating subplan XXX_6 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u6 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u7" to a subquery +DEBUG: generating subplan XXX_7 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u7 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u8" to a subquery +DEBUG: generating subplan XXX_8 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u8 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u9" to a subquery +DEBUG: generating subplan XXX_9 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u9 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u10" to a subquery +DEBUG: generating subplan XXX_10 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u10 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u11" to a subquery +DEBUG: generating subplan XXX_11 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u11 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u12" to a subquery +DEBUG: generating subplan XXX_12 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u12 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u13" to a subquery +DEBUG: generating subplan XXX_13 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u13 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u14" to a subquery +DEBUG: generating subplan XXX_14 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u14 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u15" to a subquery +DEBUG: generating subplan XXX_15 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u15 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u16" to a subquery +DEBUG: generating subplan XXX_16 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u16 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u17" to a subquery +DEBUG: generating subplan XXX_17 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u17 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u18" to a subquery +DEBUG: generating subplan XXX_18 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u18 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u19" to a subquery +DEBUG: generating subplan XXX_19 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u19 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u20" to a subquery +DEBUG: generating subplan XXX_20 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u20 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u21" to a subquery +DEBUG: generating subplan XXX_21 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u21 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u22" to a subquery +DEBUG: generating subplan XXX_22 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u22 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u23" to a subquery +DEBUG: generating subplan XXX_23 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u23 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u24" to a subquery +DEBUG: generating subplan XXX_24 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u24 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Wrapping relation "local" "u25" to a subquery +DEBUG: generating subplan XXX_25 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local u25 WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((((((((((((((((((((((((local_dist_join_mixed.distributed JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u1_1) u1 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u2_1) u2 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u3_1) u3 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u4_1) u4 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u5_1) u5 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_6'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u6_1) u6 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_7'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u7_1) u7 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_8'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u8_1) u8 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_9'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u9_1) u9 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_10'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u10_1) u10 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_11'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u11_1) u11 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_12'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u12_1) u12 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_13'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u13_1) u13 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_14'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u14_1) u14 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_15'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u15_1) u15 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_16'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u16_1) u16 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_17'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u17_1) u17 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_18'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u18_1) u18 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_19'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u19_1) u19 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_20'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u20_1) u20 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_21'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u21_1) u21 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_22'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u22_1) u22 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_23'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u23_1) u23 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_24'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u24_1) u24 ON (false)) JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_25'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) u25_1) u25 ON (false)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +select ' select count(*) from local ' || string_Agg('INNER +JOIN distributed u'|| x::text || ' USING (id)',' ') from +generate_Series(1,25)x; + ?column? +--------------------------------------------------------------------- + select count(*) from local INNER + + JOIN distributed u1 USING (id) INNER + + JOIN distributed u2 USING (id) INNER + + JOIN distributed u3 USING (id) INNER + + JOIN distributed u4 USING (id) INNER + + JOIN distributed u5 USING (id) INNER + + JOIN distributed u6 USING (id) INNER + + JOIN distributed u7 USING (id) INNER + + JOIN distributed u8 USING (id) INNER + + JOIN distributed u9 USING (id) INNER + + JOIN distributed u10 USING (id) INNER+ + JOIN distributed u11 USING (id) INNER+ + JOIN distributed u12 USING (id) INNER+ + JOIN distributed u13 USING (id) INNER+ + JOIN distributed u14 USING (id) INNER+ + JOIN distributed u15 USING (id) INNER+ + JOIN distributed u16 USING (id) INNER+ + JOIN distributed u17 USING (id) INNER+ + JOIN distributed u18 USING (id) INNER+ + JOIN distributed u19 USING (id) INNER+ + JOIN distributed u20 USING (id) INNER+ + JOIN distributed u21 USING (id) INNER+ + JOIN distributed u22 USING (id) INNER+ + JOIN distributed u23 USING (id) INNER+ + JOIN distributed u24 USING (id) INNER+ + JOIN distributed u25 USING (id) +(1 row) + +\gexec + select count(*) from local INNER +JOIN distributed u1 USING (id) INNER +JOIN distributed u2 USING (id) INNER +JOIN distributed u3 USING (id) INNER +JOIN distributed u4 USING (id) INNER +JOIN distributed u5 USING (id) INNER +JOIN distributed u6 USING (id) INNER +JOIN distributed u7 USING (id) INNER +JOIN distributed u8 USING (id) INNER +JOIN distributed u9 USING (id) INNER +JOIN distributed u10 USING (id) INNER +JOIN distributed u11 USING (id) INNER +JOIN distributed u12 USING (id) INNER +JOIN distributed u13 USING (id) INNER +JOIN distributed u14 USING (id) INNER +JOIN distributed u15 USING (id) INNER +JOIN distributed u16 USING (id) INNER +JOIN distributed u17 USING (id) INNER +JOIN distributed u18 USING (id) INNER +JOIN distributed u19 USING (id) INNER +JOIN distributed u20 USING (id) INNER +JOIN distributed u21 USING (id) INNER +JOIN distributed u22 USING (id) INNER +JOIN distributed u23 USING (id) INNER +JOIN distributed u24 USING (id) INNER +JOIN distributed u25 USING (id) +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((((((((((((((((((((((((((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed u1 USING (id)) JOIN local_dist_join_mixed.distributed u2 USING (id)) JOIN local_dist_join_mixed.distributed u3 USING (id)) JOIN local_dist_join_mixed.distributed u4 USING (id)) JOIN local_dist_join_mixed.distributed u5 USING (id)) JOIN local_dist_join_mixed.distributed u6 USING (id)) JOIN local_dist_join_mixed.distributed u7 USING (id)) JOIN local_dist_join_mixed.distributed u8 USING (id)) JOIN local_dist_join_mixed.distributed u9 USING (id)) JOIN local_dist_join_mixed.distributed u10 USING (id)) JOIN local_dist_join_mixed.distributed u11 USING (id)) JOIN local_dist_join_mixed.distributed u12 USING (id)) JOIN local_dist_join_mixed.distributed u13 USING (id)) JOIN local_dist_join_mixed.distributed u14 USING (id)) JOIN local_dist_join_mixed.distributed u15 USING (id)) JOIN local_dist_join_mixed.distributed u16 USING (id)) JOIN local_dist_join_mixed.distributed u17 USING (id)) JOIN local_dist_join_mixed.distributed u18 USING (id)) JOIN local_dist_join_mixed.distributed u19 USING (id)) JOIN local_dist_join_mixed.distributed u20 USING (id)) JOIN local_dist_join_mixed.distributed u21 USING (id)) JOIN local_dist_join_mixed.distributed u22 USING (id)) JOIN local_dist_join_mixed.distributed u23 USING (id)) JOIN local_dist_join_mixed.distributed u24 USING (id)) JOIN local_dist_join_mixed.distributed u25 USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +select ' select count(*) from local ' || string_Agg('INNER +JOIN distributed u'|| x::text || ' ON (false)',' ') from +generate_Series(1,25)x; + ?column? +--------------------------------------------------------------------- + select count(*) from local INNER + + JOIN distributed u1 ON (false) INNER + + JOIN distributed u2 ON (false) INNER + + JOIN distributed u3 ON (false) INNER + + JOIN distributed u4 ON (false) INNER + + JOIN distributed u5 ON (false) INNER + + JOIN distributed u6 ON (false) INNER + + JOIN distributed u7 ON (false) INNER + + JOIN distributed u8 ON (false) INNER + + JOIN distributed u9 ON (false) INNER + + JOIN distributed u10 ON (false) INNER+ + JOIN distributed u11 ON (false) INNER+ + JOIN distributed u12 ON (false) INNER+ + JOIN distributed u13 ON (false) INNER+ + JOIN distributed u14 ON (false) INNER+ + JOIN distributed u15 ON (false) INNER+ + JOIN distributed u16 ON (false) INNER+ + JOIN distributed u17 ON (false) INNER+ + JOIN distributed u18 ON (false) INNER+ + JOIN distributed u19 ON (false) INNER+ + JOIN distributed u20 ON (false) INNER+ + JOIN distributed u21 ON (false) INNER+ + JOIN distributed u22 ON (false) INNER+ + JOIN distributed u23 ON (false) INNER+ + JOIN distributed u24 ON (false) INNER+ + JOIN distributed u25 ON (false) +(1 row) + +\gexec + select count(*) from local INNER +JOIN distributed u1 ON (false) INNER +JOIN distributed u2 ON (false) INNER +JOIN distributed u3 ON (false) INNER +JOIN distributed u4 ON (false) INNER +JOIN distributed u5 ON (false) INNER +JOIN distributed u6 ON (false) INNER +JOIN distributed u7 ON (false) INNER +JOIN distributed u8 ON (false) INNER +JOIN distributed u9 ON (false) INNER +JOIN distributed u10 ON (false) INNER +JOIN distributed u11 ON (false) INNER +JOIN distributed u12 ON (false) INNER +JOIN distributed u13 ON (false) INNER +JOIN distributed u14 ON (false) INNER +JOIN distributed u15 ON (false) INNER +JOIN distributed u16 ON (false) INNER +JOIN distributed u17 ON (false) INNER +JOIN distributed u18 ON (false) INNER +JOIN distributed u19 ON (false) INNER +JOIN distributed u20 ON (false) INNER +JOIN distributed u21 ON (false) INNER +JOIN distributed u22 ON (false) INNER +JOIN distributed u23 ON (false) INNER +JOIN distributed u24 ON (false) INNER +JOIN distributed u25 ON (false) +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local WHERE (false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((((((((((((((((((((((((((SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed u1 ON (false)) JOIN local_dist_join_mixed.distributed u2 ON (false)) JOIN local_dist_join_mixed.distributed u3 ON (false)) JOIN local_dist_join_mixed.distributed u4 ON (false)) JOIN local_dist_join_mixed.distributed u5 ON (false)) JOIN local_dist_join_mixed.distributed u6 ON (false)) JOIN local_dist_join_mixed.distributed u7 ON (false)) JOIN local_dist_join_mixed.distributed u8 ON (false)) JOIN local_dist_join_mixed.distributed u9 ON (false)) JOIN local_dist_join_mixed.distributed u10 ON (false)) JOIN local_dist_join_mixed.distributed u11 ON (false)) JOIN local_dist_join_mixed.distributed u12 ON (false)) JOIN local_dist_join_mixed.distributed u13 ON (false)) JOIN local_dist_join_mixed.distributed u14 ON (false)) JOIN local_dist_join_mixed.distributed u15 ON (false)) JOIN local_dist_join_mixed.distributed u16 ON (false)) JOIN local_dist_join_mixed.distributed u17 ON (false)) JOIN local_dist_join_mixed.distributed u18 ON (false)) JOIN local_dist_join_mixed.distributed u19 ON (false)) JOIN local_dist_join_mixed.distributed u20 ON (false)) JOIN local_dist_join_mixed.distributed u21 ON (false)) JOIN local_dist_join_mixed.distributed u22 ON (false)) JOIN local_dist_join_mixed.distributed u23 ON (false)) JOIN local_dist_join_mixed.distributed u24 ON (false)) JOIN local_dist_join_mixed.distributed u25 ON (false)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- lateral joins +SELECT COUNT(*) FROM (VALUES (1), (2), (3)) as f(x) LATERAL JOIN (SELECT * FROM local WHERE id = x) as bar; +ERROR: syntax error at or near "LATERAL" +SELECT COUNT(*) FROM local JOIN LATERAL (SELECT * FROM distributed WHERE local.id = distributed.id) as foo ON (true); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN LATERAL (SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo ON (true)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT COUNT(*) FROM local JOIN LATERAL (SELECT * FROM distributed WHERE local.id > distributed.id) as foo ON (true); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN LATERAL (SELECT distributed.id, distributed.name, distributed.created_at FROM local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.>) distributed.id)) foo ON (true)) + count +--------------------------------------------------------------------- + 5050 +(1 row) + +SELECT COUNT(*) FROM distributed JOIN LATERAL (SELECT * FROM local WHERE local.id = distributed.id) as foo ON (true); +ERROR: direct joins between distributed and local tables are not supported +HINT: Use CTE's or subqueries to select from local tables and use them in joins +SELECT COUNT(*) FROM distributed JOIN LATERAL (SELECT * FROM local WHERE local.id > distributed.id) as foo ON (true); +ERROR: direct joins between distributed and local tables are not supported +HINT: Use CTE's or subqueries to select from local tables and use them in joins +SELECT count(*) FROM distributed CROSS JOIN local; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::bigint AS id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed CROSS JOIN (SELECT NULL::bigint AS id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local) + count +--------------------------------------------------------------------- + 10201 +(1 row) + +SELECT count(*) FROM distributed CROSS JOIN local WHERE distributed.id = 1; +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE (id OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed CROSS JOIN local_dist_join_mixed.local) WHERE (distributed.id OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 101 +(1 row) + +-- w count(*) it works fine as PG ignores the inner tables +SELECT count(*) FROM distributed LEFT JOIN local USING (id); +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed LEFT JOIN local_dist_join_mixed.local USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT count(*) FROM local LEFT JOIN distributed USING (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local LEFT JOIN local_dist_join_mixed.distributed USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT id, name FROM distributed LEFT JOIN local USING (id) ORDER BY 1 LIMIT 1; +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, name FROM local_dist_join_mixed.distributed WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT distributed.id, distributed.name FROM ((SELECT distributed_1.id, distributed_1.name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id, intermediate_result.name FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, name text)) distributed_1) distributed LEFT JOIN local_dist_join_mixed.local USING (id)) ORDER BY distributed.id LIMIT 1 + id | name +--------------------------------------------------------------------- + 0 | 0 +(1 row) + +SELECT id, name FROM local LEFT JOIN distributed USING (id) ORDER BY 1 LIMIT 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT local.id, distributed.name FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local LEFT JOIN local_dist_join_mixed.distributed USING (id)) ORDER BY local.id LIMIT 1 +ERROR: cannot pushdown the subquery +DETAIL: Complex subqueries and CTEs cannot be in the outer part of the outer join + SELECT + foo1.id + FROM + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo9, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo8, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo7, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo6, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo5, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo4, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo3, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo2, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo10, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo1 + WHERE + foo1.id = foo9.id AND + foo1.id = foo8.id AND + foo1.id = foo7.id AND + foo1.id = foo6.id AND + foo1.id = foo5.id AND + foo1.id = foo4.id AND + foo1.id = foo3.id AND + foo1.id = foo2.id AND + foo1.id = foo10.id AND + foo1.id = foo1.id +ORDER BY 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_4 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_5 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_6 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_7 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_8 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_9 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_10 for subquery SELECT id FROM local_dist_join_mixed.local WHERE (id IS NOT NULL) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo1.id FROM (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo9, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo8, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo7, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo6, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo5, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_6'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo4, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_7'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo3, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_8'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo2, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_9'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo10, (SELECT local.id, local.title FROM (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_10'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local, local_dist_join_mixed.distributed WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo1 WHERE ((foo1.id OPERATOR(pg_catalog.=) foo9.id) AND (foo1.id OPERATOR(pg_catalog.=) foo8.id) AND (foo1.id OPERATOR(pg_catalog.=) foo7.id) AND (foo1.id OPERATOR(pg_catalog.=) foo6.id) AND (foo1.id OPERATOR(pg_catalog.=) foo5.id) AND (foo1.id OPERATOR(pg_catalog.=) foo4.id) AND (foo1.id OPERATOR(pg_catalog.=) foo3.id) AND (foo1.id OPERATOR(pg_catalog.=) foo2.id) AND (foo1.id OPERATOR(pg_catalog.=) foo10.id) AND (foo1.id OPERATOR(pg_catalog.=) foo1.id)) ORDER BY foo1.id + id +--------------------------------------------------------------------- + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12 + 13 + 14 + 15 + 16 + 17 + 18 + 19 + 20 + 21 + 22 + 23 + 24 + 25 + 26 + 27 + 28 + 29 + 30 + 31 + 32 + 33 + 34 + 35 + 36 + 37 + 38 + 39 + 40 + 41 + 42 + 43 + 44 + 45 + 46 + 47 + 48 + 49 + 50 + 51 + 52 + 53 + 54 + 55 + 56 + 57 + 58 + 59 + 60 + 61 + 62 + 63 + 64 + 65 + 66 + 67 + 68 + 69 + 70 + 71 + 72 + 73 + 74 + 75 + 76 + 77 + 78 + 79 + 80 + 81 + 82 + 83 + 84 + 85 + 86 + 87 + 88 + 89 + 90 + 91 + 92 + 93 + 94 + 95 + 96 + 97 + 98 + 99 + 100 +(101 rows) + +SELECT + foo1.id +FROM + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo1, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo2, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo3, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo4, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo5 +WHERE + foo1.id = foo4.id AND + foo1.id = foo2.id AND + foo1.id = foo3.id AND + foo1.id = foo4.id AND + foo1.id = foo5.id +ORDER BY 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_4 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_5 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo1.id FROM (SELECT local.id FROM local_dist_join_mixed.distributed, (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo1, (SELECT local.id FROM local_dist_join_mixed.distributed, (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo2, (SELECT local.id FROM local_dist_join_mixed.distributed, (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo3, (SELECT local.id FROM local_dist_join_mixed.distributed, (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo4, (SELECT local.id FROM local_dist_join_mixed.distributed, (SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local WHERE (local.id OPERATOR(pg_catalog.=) distributed.id)) foo5 WHERE ((foo1.id OPERATOR(pg_catalog.=) foo4.id) AND (foo1.id OPERATOR(pg_catalog.=) foo2.id) AND (foo1.id OPERATOR(pg_catalog.=) foo3.id) AND (foo1.id OPERATOR(pg_catalog.=) foo4.id) AND (foo1.id OPERATOR(pg_catalog.=) foo5.id)) ORDER BY foo1.id + id +--------------------------------------------------------------------- + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12 + 13 + 14 + 15 + 16 + 17 + 18 + 19 + 20 + 21 + 22 + 23 + 24 + 25 + 26 + 27 + 28 + 29 + 30 + 31 + 32 + 33 + 34 + 35 + 36 + 37 + 38 + 39 + 40 + 41 + 42 + 43 + 44 + 45 + 46 + 47 + 48 + 49 + 50 + 51 + 52 + 53 + 54 + 55 + 56 + 57 + 58 + 59 + 60 + 61 + 62 + 63 + 64 + 65 + 66 + 67 + 68 + 69 + 70 + 71 + 72 + 73 + 74 + 75 + 76 + 77 + 78 + 79 + 80 + 81 + 82 + 83 + 84 + 85 + 86 + 87 + 88 + 89 + 90 + 91 + 92 + 93 + 94 + 95 + 96 + 97 + 98 + 99 + 100 +(101 rows) + +SELECT + foo1.id +FROM + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 1) as foo1, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 2) as foo2, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 3) as foo3, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 4) as foo4, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 5) as foo5 +WHERE + foo1.id = foo4.id AND + foo1.id = foo2.id AND + foo1.id = foo3.id AND + foo1.id = foo4.id AND + foo1.id = foo5.id +ORDER BY 1; +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE (false AND false AND false AND false) +DEBUG: generating subplan XXX_2 for subquery SELECT local.id FROM (SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed, local_dist_join_mixed.local WHERE ((local.id OPERATOR(pg_catalog.=) distributed.id) AND (distributed.id OPERATOR(pg_catalog.=) 1)) +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE (false AND false AND false AND false) +DEBUG: generating subplan XXX_4 for subquery SELECT local.id FROM (SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed, local_dist_join_mixed.local WHERE ((local.id OPERATOR(pg_catalog.=) distributed.id) AND (distributed.id OPERATOR(pg_catalog.=) 2)) +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_5 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE (false AND false AND false AND false) +DEBUG: generating subplan XXX_6 for subquery SELECT local.id FROM (SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_5'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed, local_dist_join_mixed.local WHERE ((local.id OPERATOR(pg_catalog.=) distributed.id) AND (distributed.id OPERATOR(pg_catalog.=) 3)) +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_7 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE (false AND false AND false AND false) +DEBUG: generating subplan XXX_8 for subquery SELECT local.id FROM (SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_7'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed, local_dist_join_mixed.local WHERE ((local.id OPERATOR(pg_catalog.=) distributed.id) AND (distributed.id OPERATOR(pg_catalog.=) 4)) +DEBUG: Wrapping relation "distributed" to a subquery +DEBUG: generating subplan XXX_9 for subquery SELECT id FROM local_dist_join_mixed.distributed WHERE (false AND false AND false AND false) +DEBUG: generating subplan XXX_10 for subquery SELECT local.id FROM (SELECT distributed_1.id, NULL::text AS name, NULL::timestamp with time zone AS created_at FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_9'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) distributed_1) distributed, local_dist_join_mixed.local WHERE ((local.id OPERATOR(pg_catalog.=) distributed.id) AND (distributed.id OPERATOR(pg_catalog.=) 5)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo1.id FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) foo1, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_4'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) foo2, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_6'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) foo3, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_8'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) foo4, (SELECT intermediate_result.id FROM read_intermediate_result('XXX_10'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) foo5 WHERE ((foo1.id OPERATOR(pg_catalog.=) foo4.id) AND (foo1.id OPERATOR(pg_catalog.=) foo2.id) AND (foo1.id OPERATOR(pg_catalog.=) foo3.id) AND (foo1.id OPERATOR(pg_catalog.=) foo4.id) AND (foo1.id OPERATOR(pg_catalog.=) foo5.id)) ORDER BY foo1.id + id +--------------------------------------------------------------------- +(0 rows) + +SELECT + count(*) +FROM + distributed +JOIN LATERAL + (SELECT + * + FROM + local + JOIN + distributed d2 + ON(true) + WHERE local.id = distributed.id AND d2.id = local.id) as foo +ON (true); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_dist_join_mixed.distributed JOIN LATERAL (SELECT local.id, local.title, d2.id, d2.name, d2.created_at FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed d2 ON (true)) WHERE ((local.id OPERATOR(pg_catalog.=) distributed.id) AND (d2.id OPERATOR(pg_catalog.=) local.id))) foo(id, title, id_1, name, created_at) ON (true)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +SELECT local.title, local.title FROM local JOIN distributed USING(id) ORDER BY 1,2 LIMIt 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT local.title, local.title FROM ((SELECT local_1.id, local_1.title FROM (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_1) local JOIN local_dist_join_mixed.distributed USING (id)) ORDER BY local.title, local.title LIMIT 1 +DEBUG: push down of limit count: 1 + title | title +--------------------------------------------------------------------- + 0 | 0 +(1 row) + +SELECT NULL FROM local JOIN distributed USING(id) ORDER BY 1 LIMIt 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT NULL::text FROM ((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed USING (id)) ORDER BY NULL::text LIMIT 1 +DEBUG: push down of limit count: 1 + ?column? +--------------------------------------------------------------------- + +(1 row) + +SELECT distributed.name, distributed.name, local.title, local.title FROM local JOIN distributed USING(id) ORDER BY 1,2,3,4 LIMIT 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id, title FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT distributed.name, distributed.name, local.title, local.title FROM ((SELECT local_1.id, local_1.title FROM (SELECT intermediate_result.id, intermediate_result.title FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint, title text)) local_1) local JOIN local_dist_join_mixed.distributed USING (id)) ORDER BY distributed.name, distributed.name, local.title, local.title LIMIT 1 +DEBUG: push down of limit count: 1 + name | name | title | title +--------------------------------------------------------------------- + 0 | 0 | 0 | 0 +(1 row) + +SELECT + COUNT(*) +FROM + local +JOIN + distributed +USING + (id) +JOIN + (SELECT id, NULL, NULL FROM distributed) foo +USING + (id); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT id FROM local_dist_join_mixed.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((SELECT local_1.id, NULL::text AS title FROM (SELECT intermediate_result.id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(id bigint)) local_1) local JOIN local_dist_join_mixed.distributed USING (id)) JOIN (SELECT distributed_1.id, NULL::text, NULL::text FROM local_dist_join_mixed.distributed distributed_1) foo(id, "?column?", "?column?_1") USING (id)) + count +--------------------------------------------------------------------- + 101 +(1 row) + +DROP SCHEMA local_dist_join_mixed CASCADE; +NOTICE: drop cascades to 7 other objects +DETAIL: drop cascades to table distributed +drop cascades to table reference +drop cascades to table local +drop cascades to table unlogged_local +drop cascades to materialized view mat_view +drop cascades to view local_regular_view +drop cascades to view dist_regular_view diff --git a/src/test/regress/expected/local_dist_join_modifications.out b/src/test/regress/expected/local_dist_join_modifications.out new file mode 100644 index 000000000..988fc243d --- /dev/null +++ b/src/test/regress/expected/local_dist_join_modifications.out @@ -0,0 +1,597 @@ +CREATE SCHEMA local_dist_join_modifications; +SET search_path TO local_dist_join_modifications; +CREATE TABLE postgres_table (key int, value text, value_2 jsonb); +CREATE TABLE reference_table (key int, value text, value_2 jsonb); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table (key int, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_pkey (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_pkey', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_windex (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_windex', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE UNIQUE INDEX key_index ON distributed_table_windex (key); +CREATE TABLE distributed_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE distributed_partitioned_table_1 PARTITION OF distributed_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE distributed_partitioned_table_2 PARTITION OF distributed_partitioned_table FOR VALUES FROM (50) TO (200); +SELECT create_distributed_table('distributed_partitioned_table', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE local_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE local_partitioned_table_1 PARTITION OF local_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE local_partitioned_table_2 PARTITION OF local_partitioned_table FOR VALUES FROM (50) TO (200); +CREATE TABLE distributed_table_composite (key int, value text, value_2 jsonb, primary key (key, value)); +SELECT create_distributed_table('distributed_table_composite', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM postgres_table; +CREATE MATERIALIZED VIEW mv2 AS SELECT * FROM distributed_table; +-- set log messages to debug1 so that we can see which tables are recursively planned. +SET client_min_messages TO DEBUG1; +INSERT INTO postgres_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO reference_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_windex SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_pkey SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_composite SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO local_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +SET citus.local_table_join_policy to 'auto'; +-- we can support modification queries as well +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.postgres_table SET value = 'test'::text FROM (SELECT distributed_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_1) distributed_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table_pkey SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table_windex SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_windex.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +UPDATE + mv1 +SET + value = 'test' +FROM + postgres_table +WHERE + mv1.key = postgres_table.key; +ERROR: cannot change materialized view "mv1" +ROLLBACK; +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + mv1 +WHERE + mv1.key = postgres_table.key; +ROLLBACK; +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + mv2 +WHERE + mv2.key = postgres_table.key; +ROLLBACK; +-- in case of update/delete we always recursively plan +-- the tables other than target table no matter what the policy is +SET citus.local_table_join_policy TO 'prefer-local'; +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.postgres_table SET value = 'test'::text FROM (SELECT distributed_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_1) distributed_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table_pkey SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table_windex SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_windex.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +SET citus.local_table_join_policy TO 'prefer-distributed'; +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.postgres_table SET value = 'test'::text FROM (SELECT distributed_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_1) distributed_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table_pkey SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 100 +(1 row) + +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table_windex SET value = 'test'::text FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_windex.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +SET citus.local_table_join_policy TO 'auto'; +-- modifications with multiple tables +BEGIN; +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table p1, postgres_table p2 +WHERE + distributed_table.key = p1.key AND p1.key = p2.key; +DEBUG: Wrapping relation "postgres_table" "p1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table p1 WHERE true +DEBUG: Wrapping relation "postgres_table" "p2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_dist_join_modifications.postgres_table p2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table SET value = 'test'::text FROM (SELECT p1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p1_1) p1, (SELECT p2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p2_1) p2 WHERE ((distributed_table.key OPERATOR(pg_catalog.=) p1.key) AND (p1.key OPERATOR(pg_catalog.=) p2.key)) +ROLLBACK; +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + (SELECT * FROM distributed_table) d1 +WHERE + d1.key = postgres_table.key; +ERROR: relation postgres_table is not distributed +ROLLBACK; +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + (SELECT * FROM distributed_table LIMIT 1) d1 +WHERE + d1.key = postgres_table.key; +DEBUG: push down of limit count: 1 +DEBUG: generating subplan XXX_1 for subquery SELECT key, value, value_2 FROM local_dist_join_modifications.distributed_table LIMIT 1 +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.postgres_table SET value = 'test'::text FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb)) d1 WHERE (d1.key OPERATOR(pg_catalog.=) postgres_table.key) +ROLLBACK; +BEGIN; +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table p1, distributed_table d2 +WHERE + distributed_table.key = p1.key AND p1.key = d2.key; +DEBUG: Wrapping relation "postgres_table" "p1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table p1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.distributed_table SET value = 'test'::text FROM (SELECT p1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p1_1) p1, local_dist_join_modifications.distributed_table d2 WHERE ((distributed_table.key OPERATOR(pg_catalog.=) p1.key) AND (p1.key OPERATOR(pg_catalog.=) d2.key)) +ROLLBACK; +-- pretty inefficient plan as it requires +-- recursive planninng of 2 distributed tables +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table d1, distributed_table d2 +WHERE + postgres_table.key = d1.key AND d1.key = d2.key; +DEBUG: Wrapping relation "distributed_table" "d1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.distributed_table d1 WHERE true +DEBUG: Wrapping relation "distributed_table" "d2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_dist_join_modifications.distributed_table d2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_dist_join_modifications.postgres_table SET value = 'test'::text FROM (SELECT d1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) d1_1) d1, (SELECT d2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) d2_1) d2 WHERE ((postgres_table.key OPERATOR(pg_catalog.=) d1.key) AND (d1.key OPERATOR(pg_catalog.=) d2.key)) +ROLLBACK; +-- DELETE operations +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + postgres_table +USING + distributed_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM local_dist_join_modifications.postgres_table USING (SELECT distributed_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_1) distributed_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM postgres_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + distributed_table +USING + postgres_table +WHERE + distributed_table.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM local_dist_join_modifications.distributed_table USING (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + distributed_table_pkey +USING + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM local_dist_join_modifications.distributed_table_pkey USING (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 100 +(1 row) + +DELETE FROM + distributed_table_windex +USING + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_dist_join_modifications.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM local_dist_join_modifications.distributed_table_windex USING (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (distributed_table_windex.key OPERATOR(pg_catalog.=) postgres_table.key) +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +DELETE FROM + mv1 +USING + postgres_table +WHERE + mv1.key = postgres_table.key; +ERROR: cannot change materialized view "mv1" +DELETE FROM + postgres_table +USING + mv1 +WHERE + mv1.key = postgres_table.key; +DELETE FROM + postgres_table +USING + mv2 +WHERE + mv2.key = postgres_table.key; +SET client_min_messages to ERROR; +DROP SCHEMA local_dist_join_modifications CASCADE; diff --git a/src/test/regress/expected/local_table_join.out b/src/test/regress/expected/local_table_join.out new file mode 100644 index 000000000..5749cb824 --- /dev/null +++ b/src/test/regress/expected/local_table_join.out @@ -0,0 +1,1285 @@ +CREATE SCHEMA local_table_join; +SET search_path TO local_table_join; +CREATE TABLE postgres_table (key int, value text, value_2 jsonb); +CREATE TABLE reference_table (key int, value text, value_2 jsonb); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table (key int, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_pkey (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_pkey', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_windex (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_windex', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE UNIQUE INDEX key_index ON distributed_table_windex (key); +CREATE TABLE distributed_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE distributed_partitioned_table_1 PARTITION OF distributed_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE distributed_partitioned_table_2 PARTITION OF distributed_partitioned_table FOR VALUES FROM (50) TO (200); +SELECT create_distributed_table('distributed_partitioned_table', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE local_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE local_partitioned_table_1 PARTITION OF local_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE local_partitioned_table_2 PARTITION OF local_partitioned_table FOR VALUES FROM (50) TO (200); +CREATE TABLE distributed_table_composite (key int, value text, value_2 jsonb, primary key (key, value)); +SELECT create_distributed_table('distributed_table_composite', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO postgres_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO reference_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_windex SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_pkey SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_composite SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO local_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +CREATE FUNCTION fake_fdw_handler() +RETURNS fdw_handler +AS 'citus' +LANGUAGE C STRICT; +CREATE FOREIGN DATA WRAPPER fake_fdw_1 HANDLER fake_fdw_handler; +CREATE SERVER fake_fdw_server_1 FOREIGN DATA WRAPPER fake_fdw_1; +CREATE FOREIGN TABLE foreign_table ( + key int, + value text +) SERVER fake_fdw_server_1 OPTIONS (encoding 'utf-8', compression 'true'); +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM postgres_table; +CREATE MATERIALIZED VIEW mv2 AS SELECT * FROM distributed_table; +SET client_min_messages TO DEBUG1; +-- the user doesn't allow local / distributed table joinn +SET citus.local_table_join_policy TO 'never'; +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key); +ERROR: direct joins between distributed and local tables are not supported +HINT: Use CTE's or subqueries to select from local tables and use them in joins +SELECT count(*) FROM postgres_table JOIN reference_table USING(key); +ERROR: direct joins between distributed and local tables are not supported +HINT: Use CTE's or subqueries to select from local tables and use them in joins +-- the user prefers local table recursively planned +SET citus.local_table_join_policy TO 'prefer-local'; +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN reference_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- the user prefers distributed table recursively planned +SET citus.local_table_join_policy TO 'prefer-distributed'; +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key); +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_1) distributed_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN reference_table USING(key); +DEBUG: Wrapping relation "reference_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.reference_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT reference_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) reference_table_1) reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- auto tests +-- switch back to the default policy, which is auto +SET citus.local_table_join_policy to 'auto'; +-- on the auto mode, the local tables should be recursively planned +-- unless a unique index exists in a column for distributed table +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM reference_table JOIN postgres_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.reference_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- partititoned local tables should work as well +SELECT count(*) FROM distributed_table JOIN local_partitioned_table USING(key); +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM reference_table JOIN local_partitioned_table USING(key); +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.reference_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN local_partitioned_table USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- materialized views should work too +SELECT count(*) FROM distributed_table JOIN mv1 USING(key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv1 USING(key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table.key, distributed_table.value, distributed_table.value_2 FROM local_table_join.distributed_table) d1 JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM reference_table JOIN mv1 USING(key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.reference_table JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN mv1 USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_table JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN mv2 USING(key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv2 USING(key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table.key, distributed_table.value, distributed_table.value_2 FROM local_table_join.distributed_table) d1 JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM reference_table JOIN mv2 USING(key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.reference_table JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN mv2 USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_table JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- foreign tables should work too +SELECT count(*) FROM foreign_table JOIN distributed_table USING(key); +DEBUG: Wrapping relation "foreign_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.foreign_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foreign_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) foreign_table_1) foreign_table JOIN local_table_join.distributed_table USING (key)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- partitioned tables should work as well +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) WHERE distributed_partitioned_table.key = 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (distributed_partitioned_table.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_partitioned_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key); +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) WHERE distributed_partitioned_table.key = 10; +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) WHERE (distributed_partitioned_table.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_partitioned_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- similar tests in transaction block should work fine +BEGIN; +-- materialized views should work too +SELECT count(*) FROM distributed_table JOIN mv1 USING(key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv1 USING(key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table.key, distributed_table.value, distributed_table.value_2 FROM local_table_join.distributed_table) d1 JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM reference_table JOIN mv1 USING(key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.reference_table JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN mv1 USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "mv1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv1 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_table JOIN (SELECT mv1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv1_1) mv1 USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN mv2 USING(key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv2 USING(key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table.key, distributed_table.value, distributed_table.value_2 FROM local_table_join.distributed_table) d1 JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM reference_table JOIN mv2 USING(key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.reference_table JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_table JOIN mv2 USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "mv2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.mv2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_table JOIN (SELECT mv2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) mv2_1) mv2 USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- foreign tables should work too +SELECT count(*) FROM foreign_table JOIN distributed_table USING(key); +DEBUG: Wrapping relation "foreign_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.foreign_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foreign_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) foreign_table_1) foreign_table JOIN local_table_join.distributed_table USING (key)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- partitioned tables should work as well +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) WHERE distributed_partitioned_table.key = 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (distributed_partitioned_table.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_partitioned_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key); +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) WHERE distributed_partitioned_table.key = 10; +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_partitioned_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) WHERE (distributed_partitioned_table.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) JOIN reference_table USING (key); +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((local_table_join.distributed_partitioned_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) JOIN local_table_join.reference_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +ROLLBACK; +-- the conversions should be independent from the order of table entries in the query +SELECT COUNT(*) FROM postgres_table join distributed_table_pkey using(key) join local_partitioned_table using(key) join distributed_table using(key) where distributed_table_pkey.key = 5; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey USING (key)) JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) JOIN local_table_join.distributed_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT COUNT(*) FROM postgres_table join local_partitioned_table using(key) join distributed_table_pkey using(key) join distributed_table using(key) where distributed_table_pkey.key = 5; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) JOIN local_table_join.distributed_table_pkey USING (key)) JOIN local_table_join.distributed_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT COUNT(*) FROM postgres_table join distributed_table using(key) join local_partitioned_table using(key) join distributed_table_pkey using(key) where distributed_table_pkey.key = 5; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table USING (key)) JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) JOIN local_table_join.distributed_table_pkey USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT COUNT(*) FROM distributed_table_pkey join distributed_table using(key) join postgres_table using(key) join local_partitioned_table using(key) where distributed_table_pkey.key = 5; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((local_table_join.distributed_table_pkey JOIN local_table_join.distributed_table USING (key)) JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed_table) as d1 JOIN postgres_table ON (postgres_table.key = d1.key AND d1.key < postgres_table.key) WHERE d1.key = 1 AND false; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE ((key OPERATOR(pg_catalog.<) key) AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table.key, distributed_table.value, distributed_table.value_2, random() AS random FROM local_table_join.distributed_table) d1 JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table ON (((postgres_table.key OPERATOR(pg_catalog.=) d1.key) AND (d1.key OPERATOR(pg_catalog.<) postgres_table.key)))) WHERE ((d1.key OPERATOR(pg_catalog.=) 1) AND false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed_table_pkey) as d1 JOIN postgres_table ON (postgres_table.key = d1.key AND d1.key < postgres_table.key) WHERE d1.key = 1 AND false; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE ((key OPERATOR(pg_catalog.<) key) AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2, random() AS random FROM local_table_join.distributed_table_pkey) d1 JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table ON (((postgres_table.key OPERATOR(pg_catalog.=) d1.key) AND (d1.key OPERATOR(pg_catalog.<) postgres_table.key)))) WHERE ((d1.key OPERATOR(pg_catalog.=) 1) AND false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed_partitioned_table) as d1 JOIN postgres_table ON (postgres_table.key = d1.key AND d1.key < postgres_table.key) WHERE d1.key = 1 AND false; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE ((key OPERATOR(pg_catalog.<) key) AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_partitioned_table.key, distributed_partitioned_table.value, random() AS random FROM local_table_join.distributed_partitioned_table) d1 JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table ON (((postgres_table.key OPERATOR(pg_catalog.=) d1.key) AND (d1.key OPERATOR(pg_catalog.<) postgres_table.key)))) WHERE ((d1.key OPERATOR(pg_catalog.=) 1) AND false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM (SELECT *, random() FROM distributed_partitioned_table) as d1 JOIN postgres_table ON (postgres_table.key::int = d1.key::int AND d1.key < postgres_table.key) WHERE d1.key::int = 1 AND false; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE ((key OPERATOR(pg_catalog.<) key) AND false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_partitioned_table.key, distributed_partitioned_table.value, random() AS random FROM local_table_join.distributed_partitioned_table) d1 JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table ON (((postgres_table.key OPERATOR(pg_catalog.=) d1.key) AND (d1.key OPERATOR(pg_catalog.<) postgres_table.key)))) WHERE ((d1.key OPERATOR(pg_catalog.=) 1) AND false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- different column names +SELECT a FROM postgres_table foo (a,b,c) JOIN distributed_table ON (distributed_table.key = foo.a) ORDER BY 1 LIMIT 1; +DEBUG: Wrapping relation "postgres_table" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a AS key FROM local_table_join.postgres_table foo(a, b, c) WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.a FROM ((SELECT foo_1.a AS key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) foo_1(a)) foo(a, b, c) JOIN local_table_join.distributed_table ON ((distributed_table.key OPERATOR(pg_catalog.=) foo.a))) ORDER BY foo.a LIMIT 1 +DEBUG: push down of limit count: 1 + a +--------------------------------------------------------------------- + 1 +(1 row) + +-- We will plan postgres table as the index is on key,value not just key +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key = 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table_composite JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (distributed_table_composite.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key = 10 OR distributed_table_composite.key = 20; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table_composite JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20)) + count +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key > 10 AND distributed_table_composite.value = 'text'; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table_composite JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.>) 10) AND (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key = 10 AND distributed_table_composite.value = 'text'; +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE ((key OPERATOR(pg_catalog.=) 10) AND (value OPERATOR(pg_catalog.=) 'text'::text)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite JOIN local_table_join.postgres_table USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 OR distributed_table_composite.key = 20) + AND (distributed_table_composite.value = 'text' OR distributed_table_composite.value = 'text'); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table_composite JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (((distributed_table_composite.key OPERATOR(pg_catalog.>) 10) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20)) AND ((distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 OR distributed_table_composite.value = 'text') + AND (distributed_table_composite.value = 'text' OR distributed_table_composite.key = 30); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE (((key OPERATOR(pg_catalog.>) 10) OR (value OPERATOR(pg_catalog.=) 'text'::text)) AND ((value OPERATOR(pg_catalog.=) 'text'::text) OR (key OPERATOR(pg_catalog.=) 30))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite JOIN local_table_join.postgres_table USING (key)) WHERE (((distributed_table_composite.key OPERATOR(pg_catalog.>) 10) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) AND ((distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 30))) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 AND distributed_table_composite.value = 'text') + OR (distributed_table_composite.value = 'text' AND distributed_table_composite.key = 30); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table_composite JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (((distributed_table_composite.key OPERATOR(pg_catalog.>) 10) AND (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) OR ((distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text) AND (distributed_table_composite.key OPERATOR(pg_catalog.=) 30))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 AND distributed_table_composite.key = 20) + OR (distributed_table_composite.value = 'text' AND distributed_table_composite.value = 'text'); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE (((key OPERATOR(pg_catalog.>) 10) AND (key OPERATOR(pg_catalog.=) 20)) OR ((value OPERATOR(pg_catalog.=) 'text'::text) AND (value OPERATOR(pg_catalog.=) 'text'::text))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite JOIN local_table_join.postgres_table USING (key)) WHERE (((distributed_table_composite.key OPERATOR(pg_catalog.>) 10) AND (distributed_table_composite.key OPERATOR(pg_catalog.=) 20)) OR ((distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text) AND (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text))) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- Known bug: #4269 +SELECT count(*) FROM distributed_table_composite foo(a,b,c) JOIN postgres_table ON(foo.a > 1) + WHERE foo.a IN (SELECT COUNT(*) FROM local_partitioned_table) AND (foo.a = 10 OR foo.b ='text'); +DEBUG: generating subplan XXX_1 for subquery SELECT count(*) AS count FROM local_table_join.local_partitioned_table +DEBUG: Wrapping relation "distributed_table_composite" "foo" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT a AS key, b AS value FROM local_table_join.distributed_table_composite foo(a, b, c) WHERE ((a OPERATOR(pg_catalog.>) 1) AND ((a OPERATOR(pg_catalog.=) 10) OR (b OPERATOR(pg_catalog.=) 'text'::text))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT foo_1.a AS key, foo_1.b AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) foo_1(a, b)) foo(a, b, c) JOIN local_table_join.postgres_table ON ((foo.a OPERATOR(pg_catalog.>) 1))) WHERE ((foo.a OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(count bigint))) AND ((foo.a OPERATOR(pg_catalog.=) 10) OR (foo.b OPERATOR(pg_catalog.=) 'text'::text))) +ERROR: column "a" does not exist +CONTEXT: while executing command on localhost:xxxxx +-- a unique index on key so dist table should be recursively planned +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey USING(value); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, postgres_table_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey USING (value)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON postgres_table.key = distributed_table_pkey.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey ON ((postgres_table.key OPERATOR(pg_catalog.=) distributed_table_pkey.key))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10; +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON ((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- it should favor distributed table only if it has equality on the unique column +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key > 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey ON ((distributed_table_pkey.key OPERATOR(pg_catalog.>) 10))) + count +--------------------------------------------------------------------- + 9000 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key < 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey ON ((distributed_table_pkey.key OPERATOR(pg_catalog.<) 10))) + count +--------------------------------------------------------------------- + 900 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10; +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON ((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 AND distributed_table_pkey.key > 10 ; +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE ((key OPERATOR(pg_catalog.>) 10) AND (key OPERATOR(pg_catalog.=) 10)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_pkey.key OPERATOR(pg_catalog.>) 10)))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 AND distributed_table_pkey.key > 10 ; +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE ((key OPERATOR(pg_catalog.>) 10) AND (key OPERATOR(pg_catalog.=) 10)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_pkey.key OPERATOR(pg_catalog.>) 10)))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 AND distributed_table_pkey.key > 10 AND postgres_table.key = 5; +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE ((key OPERATOR(pg_catalog.>) 10) AND (key OPERATOR(pg_catalog.=) 10)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_pkey.key OPERATOR(pg_catalog.>) 10) AND (postgres_table.key OPERATOR(pg_catalog.=) 5)))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key > 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR (distributed_table_pkey.key OPERATOR(pg_catalog.>) 10)))) + count +--------------------------------------------------------------------- + 9100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key = 20; +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE ((key OPERATOR(pg_catalog.=) 10) OR (key OPERATOR(pg_catalog.=) 20)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20)))) + count +--------------------------------------------------------------------- + 200 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key = 20 OR distributed_table_pkey.key = 30; +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE ((key OPERATOR(pg_catalog.=) 10) OR (key OPERATOR(pg_catalog.=) 20) OR (key OPERATOR(pg_catalog.=) 30)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20) OR (distributed_table_pkey.key OPERATOR(pg_catalog.=) 30)))) + count +--------------------------------------------------------------------- + 300 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key = ( + SELECT count(*) FROM distributed_table_pkey +); +DEBUG: generating subplan XXX_1 for subquery SELECT count(*) AS count FROM local_table_join.distributed_table_pkey +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT NULL::integer AS key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR (distributed_table_pkey.key OPERATOR(pg_catalog.=) (SELECT intermediate_result.count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)))))) + count +--------------------------------------------------------------------- + 200 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key = 5 and distributed_table_pkey.key > 15); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE ((key OPERATOR(pg_catalog.=) 10) OR ((key OPERATOR(pg_catalog.=) 5) AND (key OPERATOR(pg_catalog.>) 15))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR ((distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) AND (distributed_table_pkey.key OPERATOR(pg_catalog.>) 15))))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key > 10 and distributed_table_pkey.key > 15); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR ((distributed_table_pkey.key OPERATOR(pg_catalog.>) 10) AND (distributed_table_pkey.key OPERATOR(pg_catalog.>) 15))))) + count +--------------------------------------------------------------------- + 8600 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key > 10 and distributed_table_pkey.value = 'notext'); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_pkey WHERE ((key OPERATOR(pg_catalog.=) 10) OR ((key OPERATOR(pg_catalog.>) 10) AND (value OPERATOR(pg_catalog.=) 'notext'::text))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, distributed_table_pkey_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR ((distributed_table_pkey.key OPERATOR(pg_catalog.>) 10) AND (distributed_table_pkey.value OPERATOR(pg_catalog.=) 'notext'::text))))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key = 10 and distributed_table_pkey.value = 'notext'); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey ON (((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) OR ((distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_pkey.value OPERATOR(pg_catalog.=) 'notext'::text))))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON postgres_table.key = 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey ON ((postgres_table.key OPERATOR(pg_catalog.=) 10))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +select count(*) FROM postgres_table JOIN (SELECT a.key,random() FROM distributed_table a JOIN distributed_table b USING(key)) as foo USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN (SELECT a.key, random() AS random FROM (local_table_join.distributed_table a JOIN local_table_join.distributed_table b USING (key))) foo USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +select count(*) FROM (SELECT a.key, random() FROM distributed_table a JOIN distributed_table b USING(key)) as foo JOIN postgres_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT a.key, random() AS random FROM (local_table_join.distributed_table a JOIN local_table_join.distributed_table b USING (key))) foo JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM distributed_table) d1 USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN (SELECT distributed_table.key, distributed_table.value, distributed_table.value_2 FROM local_table_join.distributed_table) d1 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- since this is already router plannable, we don't recursively plan the postgres table +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM distributed_table LIMIT 1) d1 USING(key); +DEBUG: push down of limit count: 1 +DEBUG: generating subplan XXX_1 for subquery SELECT key, value, value_2 FROM local_table_join.distributed_table LIMIT 1 +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb)) d1 USING (key)) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- a unique index on key so dist table should be recursively planned +SELECT count(*) FROM postgres_table JOIN distributed_table_windex USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_windex USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_windex USING(value); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS key, postgres_table_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_windex USING (value)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_windex ON postgres_table.key = distributed_table_windex.key; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_windex ON ((postgres_table.key OPERATOR(pg_catalog.=) distributed_table_windex.key))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT count(*) FROM postgres_table JOIN distributed_table_windex ON distributed_table_windex.key = 10; +DEBUG: Wrapping relation "distributed_table_windex" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_windex WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT distributed_table_windex_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_windex_1) distributed_table_windex ON ((distributed_table_windex.key OPERATOR(pg_catalog.=) 10))) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- no unique index on value so local table should be recursively planned. +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.value = 'test'; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (distributed_table.value OPERATOR(pg_catalog.=) 'test'::text) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.key = 1; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (distributed_table.key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- if both local and distributed tables have a filter, we prefer local unless distributed table has unique indexes on any equality filter +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.value = 'test' AND postgres_table.value = 'test'; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.postgres_table WHERE (value OPERATOR(pg_catalog.=) 'test'::text) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT postgres_table_1.key, postgres_table_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) postgres_table_1) postgres_table USING (key)) WHERE ((distributed_table.value OPERATOR(pg_catalog.=) 'test'::text) AND (postgres_table.value OPERATOR(pg_catalog.=) 'test'::text)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.value = 'test' OR postgres_table.value = 'test'; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.distributed_table JOIN (SELECT postgres_table_1.key, postgres_table_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) postgres_table_1) postgres_table USING (key)) WHERE ((distributed_table.value OPERATOR(pg_catalog.=) 'test'::text) OR (postgres_table.value OPERATOR(pg_catalog.=) 'test'::text)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- multiple local/distributed tables +-- only local tables are recursively planned +SELECT count(*) FROM distributed_table d1 JOIN postgres_table p1 USING(key) JOIN distributed_table d2 USING(key) JOIN postgres_table p2 USING(key); +DEBUG: Wrapping relation "postgres_table" "p1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table p1 WHERE true +DEBUG: Wrapping relation "postgres_table" "p2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.postgres_table p2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((local_table_join.distributed_table d1 JOIN (SELECT p1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p1_1) p1 USING (key)) JOIN local_table_join.distributed_table d2 USING (key)) JOIN (SELECT p2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p2_1) p2 USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +SELECT + count(*) +FROM + distributed_table d1 JOIN postgres_table p1 USING(key) JOIN distributed_table d2 USING(key) JOIN postgres_table p2 USING(key) +WHERE + d1.value = '1'; +DEBUG: Wrapping relation "postgres_table" "p1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table p1 WHERE true +DEBUG: Wrapping relation "postgres_table" "p2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.postgres_table p2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((local_table_join.distributed_table d1 JOIN (SELECT p1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p1_1) p1 USING (key)) JOIN local_table_join.distributed_table d2 USING (key)) JOIN (SELECT p2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p2_1) p2 USING (key)) WHERE (d1.value OPERATOR(pg_catalog.=) '1'::text) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- if the filter is on the JOIN key, we can recursively plan the local +-- tables as filters are pushed down to the local tables +SELECT + count(*) +FROM + distributed_table d1 JOIN postgres_table p1 USING(key) JOIN distributed_table d2 USING(key) JOIN postgres_table p2 USING(key) +WHERE + d1.key = 1; +DEBUG: Wrapping relation "postgres_table" "p1" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table p1 WHERE (key OPERATOR(pg_catalog.=) 1) +DEBUG: Wrapping relation "postgres_table" "p2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.postgres_table p2 WHERE (key OPERATOR(pg_catalog.=) 1) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (((local_table_join.distributed_table d1 JOIN (SELECT p1_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p1_1) p1 USING (key)) JOIN local_table_join.distributed_table d2 USING (key)) JOIN (SELECT p2_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) p2_1) p2 USING (key)) WHERE (d1.key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + +CREATE view loc_view AS SELECT * FROM postgres_table WHERE key > 0; +UPDATE loc_view SET key = (SELECT COUNT(*) FROM distributed_table); +DEBUG: generating subplan XXX_1 for subquery SELECT count(*) AS count FROM local_table_join.distributed_table +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.postgres_table SET key = (SELECT intermediate_result.count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)) FROM local_table_join.postgres_table WHERE (postgres_table.key OPERATOR(pg_catalog.>) 0) +ERROR: cannot modify views when the query contains citus tables +SELECT count(*) +FROM + (SELECT * FROM (SELECT * FROM distributed_table) d1) d2 +JOIN postgres_table +USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT d1.key, d1.value, d1.value_2 FROM (SELECT distributed_table.key, distributed_table.value, distributed_table.value_2 FROM local_table_join.distributed_table) d1) d2 JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +-- will error as we don't support complex joins +SELECT COUNT(*) FROM postgres_table, distributed_table d1, distributed_table d2 WHERE d1.value = d2.value; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT NULL::integer AS key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table, local_table_join.distributed_table d1, local_table_join.distributed_table d2 WHERE (d1.value OPERATOR(pg_catalog.=) d2.value) +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +-- This will error because router planner will think that since reference tables have a single +-- shard, it contains only a single task for modify. However, updating a reference tables +-- will require multiple tasks. So requires some rewrite in router planner. +UPDATE reference_table SET key = 1 FROM postgres_table WHERE postgres_table.key = 10; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.reference_table SET key = 1 FROM (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table WHERE (postgres_table.key OPERATOR(pg_catalog.=) 10) +ERROR: relation postgres_table is not distributed +UPDATE reference_table SET key = 1 FROM (SELECT * FROM postgres_table) l WHERE l.key = 10; +DEBUG: generating subplan XXX_1 for subquery SELECT key, value, value_2 FROM local_table_join.postgres_table +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.reference_table SET key = 1 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb)) l WHERE (l.key OPERATOR(pg_catalog.=) 10) +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key) WHERE FALSE; +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE false +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table USING (key)) WHERE false + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT count(*) FROM (SELECT * FROM distributed_table JOIN postgres_table USING(key) WHERE false) foo JOIN local_partitioned_table USING(key); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE false +DEBUG: Wrapping relation "local_partitioned_table" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT key FROM local_table_join.local_partitioned_table WHERE false +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table.key, distributed_table.value, distributed_table.value_2, postgres_table.value, postgres_table.value_2 FROM (local_table_join.distributed_table JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE false) foo(key, value, value_2, value_1, value_2_1) JOIN (SELECT local_partitioned_table_1.key, NULL::text AS value FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_partitioned_table_1) local_partitioned_table USING (key)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +WITH dist_cte AS (SELECT * FROM distributed_table_pkey WHERE key = 5) +SELECT COUNT(*) FROM dist_cte JOIN postgres_table USING(key) WHERE dist_cte.key = 5; +DEBUG: CTE dist_cte is going to be inlined via distributed planning +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2 FROM local_table_join.distributed_table_pkey WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5)) dist_cte JOIN (SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table USING (key)) WHERE (dist_cte.key OPERATOR(pg_catalog.=) 5) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT COUNT(*) FROM postgres_table JOIN distributed_table_pkey USING(key) + WHERE (distributed_table_pkey.key IN (SELECT COUNT(*) AS count FROM postgres_table JOIN distributed_table USING(key)) ); +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: generating subplan XXX_2 for subquery SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table USING (key)) +DEBUG: Wrapping relation "postgres_table" to a subquery +DEBUG: generating subplan XXX_3 for subquery SELECT key FROM local_table_join.postgres_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT postgres_table_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) postgres_table_1) postgres_table JOIN local_table_join.distributed_table_pkey USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.count FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(count bigint))) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- PREPARED statements +PREPARE local_dist_table_join_select(int) AS SELECT COUNT(*) FROM distributed_table_pkey JOIN postgres_table USING(key) WHERE distributed_table_pkey.key = $1; +EXECUTE local_dist_table_join_select(10); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.postgres_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +EXECUTE local_dist_table_join_select(10); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.postgres_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +EXECUTE local_dist_table_join_select(10); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.postgres_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +EXECUTE local_dist_table_join_select(10); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.postgres_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +EXECUTE local_dist_table_join_select(10); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.postgres_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +EXECUTE local_dist_table_join_select(10); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 10) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.postgres_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 10) + count +--------------------------------------------------------------------- + 1 +(1 row) + +PREPARE local_dist_table_join_update(int) AS UPDATE postgres_table SET key = 5 FROM distributed_table_pkey WHERE distributed_table_pkey.key = $1; +EXECUTE local_dist_table_join_update(20); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 20) +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.postgres_table SET key = 5 FROM (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20) +EXECUTE local_dist_table_join_update(20); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 20) +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.postgres_table SET key = 5 FROM (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20) +EXECUTE local_dist_table_join_update(20); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 20) +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.postgres_table SET key = 5 FROM (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20) +EXECUTE local_dist_table_join_update(20); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 20) +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.postgres_table SET key = 5 FROM (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20) +EXECUTE local_dist_table_join_update(20); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 20) +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.postgres_table SET key = 5 FROM (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20) +EXECUTE local_dist_table_join_update(20); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 20) +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE local_table_join.postgres_table SET key = 5 FROM (SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 20) +PREPARE local_dist_table_join_subquery(int) AS SELECT COUNT(*) FROM postgres_table JOIN (SELECT * FROM distributed_table_pkey JOIN local_partitioned_table USING(key) WHERE distributed_table_pkey.key = $1) foo USING(key); +EXECUTE local_dist_table_join_subquery(5); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: generating subplan XXX_2 for subquery SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2, local_partitioned_table.value FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.local_partitioned_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2, intermediate_result.value_1 AS value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb, value_1 text)) foo(key, value, value_2, value_1) USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +EXECUTE local_dist_table_join_subquery(5); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: generating subplan XXX_2 for subquery SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2, local_partitioned_table.value FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.local_partitioned_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2, intermediate_result.value_1 AS value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb, value_1 text)) foo(key, value, value_2, value_1) USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +EXECUTE local_dist_table_join_subquery(5); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: generating subplan XXX_2 for subquery SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2, local_partitioned_table.value FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.local_partitioned_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2, intermediate_result.value_1 AS value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb, value_1 text)) foo(key, value, value_2, value_1) USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +EXECUTE local_dist_table_join_subquery(5); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: generating subplan XXX_2 for subquery SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2, local_partitioned_table.value FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.local_partitioned_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2, intermediate_result.value_1 AS value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb, value_1 text)) foo(key, value, value_2, value_1) USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +EXECUTE local_dist_table_join_subquery(5); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: generating subplan XXX_2 for subquery SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2, local_partitioned_table.value FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.local_partitioned_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2, intermediate_result.value_1 AS value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb, value_1 text)) foo(key, value, value_2, value_1) USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +EXECUTE local_dist_table_join_subquery(5); +DEBUG: Wrapping relation "distributed_table_pkey" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM local_table_join.distributed_table_pkey WHERE (key OPERATOR(pg_catalog.=) 5) +DEBUG: generating subplan XXX_2 for subquery SELECT distributed_table_pkey.key, distributed_table_pkey.value, distributed_table_pkey.value_2, local_partitioned_table.value FROM ((SELECT distributed_table_pkey_1.key, NULL::text AS value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_table_pkey_1) distributed_table_pkey JOIN local_table_join.local_partitioned_table USING (key)) WHERE (distributed_table_pkey.key OPERATOR(pg_catalog.=) 5) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.postgres_table JOIN (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.value_2, intermediate_result.value_1 AS value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, value_2 jsonb, value_1 text)) foo(key, value, value_2, value_1) USING (key)) + count +--------------------------------------------------------------------- + 100 +(1 row) + +PREPARE local_dist_table_join_filters(int) AS SELECT COUNT(*) FROM local_partitioned_table JOIN distributed_table_composite USING(key) + WHERE( + distributed_table_composite.key = $1 OR + distributed_table_composite.key = 20 OR + (distributed_table_composite.key = 10 AND distributed_table_composite.key > 0) OR + distributed_table_composite.value = 'text' + ); +EXECUTE local_dist_table_join_filters(20); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE ((key OPERATOR(pg_catalog.=) 20) OR (key OPERATOR(pg_catalog.=) 20) OR ((key OPERATOR(pg_catalog.=) 10) AND (key OPERATOR(pg_catalog.>) 0)) OR (value OPERATOR(pg_catalog.=) 'text'::text)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.local_partitioned_table JOIN (SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_composite.key OPERATOR(pg_catalog.>) 0)) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 2 +(1 row) + +EXECUTE local_dist_table_join_filters(20); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE ((key OPERATOR(pg_catalog.=) 20) OR (key OPERATOR(pg_catalog.=) 20) OR ((key OPERATOR(pg_catalog.=) 10) AND (key OPERATOR(pg_catalog.>) 0)) OR (value OPERATOR(pg_catalog.=) 'text'::text)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.local_partitioned_table JOIN (SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_composite.key OPERATOR(pg_catalog.>) 0)) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 2 +(1 row) + +EXECUTE local_dist_table_join_filters(20); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE ((key OPERATOR(pg_catalog.=) 20) OR (key OPERATOR(pg_catalog.=) 20) OR ((key OPERATOR(pg_catalog.=) 10) AND (key OPERATOR(pg_catalog.>) 0)) OR (value OPERATOR(pg_catalog.=) 'text'::text)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.local_partitioned_table JOIN (SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_composite.key OPERATOR(pg_catalog.>) 0)) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 2 +(1 row) + +EXECUTE local_dist_table_join_filters(20); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE ((key OPERATOR(pg_catalog.=) 20) OR (key OPERATOR(pg_catalog.=) 20) OR ((key OPERATOR(pg_catalog.=) 10) AND (key OPERATOR(pg_catalog.>) 0)) OR (value OPERATOR(pg_catalog.=) 'text'::text)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.local_partitioned_table JOIN (SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_composite.key OPERATOR(pg_catalog.>) 0)) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 2 +(1 row) + +EXECUTE local_dist_table_join_filters(20); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE ((key OPERATOR(pg_catalog.=) 20) OR (key OPERATOR(pg_catalog.=) 20) OR ((key OPERATOR(pg_catalog.=) 10) AND (key OPERATOR(pg_catalog.>) 0)) OR (value OPERATOR(pg_catalog.=) 'text'::text)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.local_partitioned_table JOIN (SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_composite.key OPERATOR(pg_catalog.>) 0)) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 2 +(1 row) + +EXECUTE local_dist_table_join_filters(20); +DEBUG: Wrapping relation "distributed_table_composite" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM local_table_join.distributed_table_composite WHERE ((key OPERATOR(pg_catalog.=) 20) OR (key OPERATOR(pg_catalog.=) 20) OR ((key OPERATOR(pg_catalog.=) 10) AND (key OPERATOR(pg_catalog.>) 0)) OR (value OPERATOR(pg_catalog.=) 'text'::text)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (local_table_join.local_partitioned_table JOIN (SELECT distributed_table_composite_1.key, distributed_table_composite_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) distributed_table_composite_1) distributed_table_composite USING (key)) WHERE ((distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR (distributed_table_composite.key OPERATOR(pg_catalog.=) 20) OR ((distributed_table_composite.key OPERATOR(pg_catalog.=) 10) AND (distributed_table_composite.key OPERATOR(pg_catalog.>) 0)) OR (distributed_table_composite.value OPERATOR(pg_catalog.=) 'text'::text)) + count +--------------------------------------------------------------------- + 2 +(1 row) + +CREATE TABLE local (key1 int, key2 int, key3 int); +INSERT INTO local VALUES (1,2,3); +ALTER TABLE local DROP column key2; +-- verify we ignore dropped columns +SELECT COUNT(*) FROM local JOIN distributed_table ON(key1 = key); +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key1 FROM local_table_join.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT local_1.key1, NULL::integer AS key3 FROM (SELECT intermediate_result.key1 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key1 integer)) local_1) local JOIN local_table_join.distributed_table ON ((local.key1 OPERATOR(pg_catalog.=) distributed_table.key))) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT * FROM local JOIN distributed_table ON(key1 = key) ORDER BY 1 LIMIT 1; +DEBUG: Wrapping relation "local" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key1, key3 FROM local_table_join.local WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT local.key1, local.key3, distributed_table.key, distributed_table.value, distributed_table.value_2 FROM ((SELECT local_1.key1, local_1.key3 FROM (SELECT intermediate_result.key1, intermediate_result.key3 FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key1 integer, key3 integer)) local_1) local JOIN local_table_join.distributed_table ON ((local.key1 OPERATOR(pg_catalog.=) distributed_table.key))) ORDER BY local.key1 LIMIT 1 +DEBUG: push down of limit count: 1 + key1 | key3 | key | value | value_2 +--------------------------------------------------------------------- + 1 | 3 | 1 | 1 | +(1 row) + +RESET client_min_messages; +\set VERBOSITY terse +DROP SCHEMA local_table_join CASCADE; +NOTICE: drop cascades to 16 other objects diff --git a/src/test/regress/expected/materialized_view.out b/src/test/regress/expected/materialized_view.out index 4ef7818a2..711167c8c 100644 --- a/src/test/regress/expected/materialized_view.out +++ b/src/test/regress/expected/materialized_view.out @@ -49,7 +49,11 @@ SELECT * FROM mode_counts WHERE l_shipmode = 'AIR' ORDER BY 2 DESC, 1 LIMIT 10; -- materialized views are local, cannot join with distributed tables SELECT count(*) FROM mode_counts JOIN temp_lineitem USING (l_shipmode); -ERROR: relation mode_counts is not distributed + count +--------------------------------------------------------------------- + 1706 +(1 row) + -- new data is not immediately reflected in the view INSERT INTO temp_lineitem SELECT * FROM air_shipped_lineitems; SELECT * FROM mode_counts WHERE l_shipmode = 'AIR' ORDER BY 2 DESC, 1 LIMIT 10; @@ -240,26 +244,24 @@ UPDATE small_view SET id = 1; ERROR: cannot change materialized view "small_view" -- for now, using materialized views in modify statements' FROM / WHERE clauses is not supported UPDATE large SET id=20 FROM small_view WHERE small_view.id=large.id; -ERROR: materialized views in modify queries are not supported SELECT * FROM large ORDER BY 1, 2; id | tenant_id --------------------------------------------------------------------- - 1 | 2 2 | 3 5 | 4 - 6 | 5 + 20 | 2 + 20 | 5 (4 rows) -- test on a router executable update statement, this will also fail UPDATE large SET id=28 FROM small_view WHERE small_view.id=large.id and small_view.tenant_id=2 and large.tenant_id=2; -ERROR: materialized views in modify queries are not supported SELECT * FROM large ORDER BY 1, 2; id | tenant_id --------------------------------------------------------------------- - 1 | 2 2 | 3 5 | 4 - 6 | 5 + 20 | 2 + 20 | 5 (4 rows) -- delete statement on large with subquery, this should succeed @@ -267,7 +269,7 @@ DELETE FROM large WHERE tenant_id in (SELECT tenant_id FROM small_view); SELECT * FROM large ORDER BY 1, 2; id | tenant_id --------------------------------------------------------------------- - 6 | 5 + 20 | 5 (1 row) -- INSERT INTO views is already not supported by PostgreSQL @@ -304,14 +306,13 @@ DELETE FROM small_view; ERROR: cannot change materialized view "small_view" -- using mat. view in modify statements' FROM / WHERE clauses is not valid yet UPDATE large_partitioned SET id=20 FROM small_view WHERE small_view.id=large_partitioned.id; -ERROR: materialized views in modify queries are not supported SELECT * FROM large_partitioned ORDER BY 1, 2; id | tenant_id --------------------------------------------------------------------- - 1 | 2 2 | 3 5 | 4 - 6 | 5 + 20 | 2 + 20 | 5 26 | 32 29 | 15 60 | 51 @@ -324,10 +325,12 @@ SELECT * FROM large_partitioned ORDER BY 1, 2; --------------------------------------------------------------------- 2 | 3 5 | 4 + 20 | 2 + 20 | 5 26 | 32 29 | 15 60 | 51 -(5 rows) +(7 rows) -- we should still have identical rows for next test statement, then insert new rows to both tables INSERT INTO large_partitioned VALUES(14, 14); diff --git a/src/test/regress/expected/mixed_relkind_tests.out b/src/test/regress/expected/mixed_relkind_tests.out index 1185b0d00..6abf7e65d 100644 --- a/src/test/regress/expected/mixed_relkind_tests.out +++ b/src/test/regress/expected/mixed_relkind_tests.out @@ -319,28 +319,55 @@ $$); (1 row) SET client_min_messages TO DEBUG1; --- should fail -SELECT * FROM partitioned_postgres_local_table JOIN distributed_table ON (true); -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins -SELECT * FROM partitioned_postgres_local_table JOIN partitioned_distributed_table ON (true); -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins -SELECT * FROM distributed_table JOIN partitioned_postgres_local_table ON (true); -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins -SELECT * FROM reference_table LEFT JOIN partitioned_distributed_table ON true; -ERROR: cannot pushdown the subquery -DETAIL: There exist a reference table in the outer part of the outer join +SELECT COUNT(*) FROM partitioned_postgres_local_table JOIN distributed_table ON (true); +DEBUG: Wrapping relation "partitioned_postgres_local_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS a FROM mixed_relkind_tests.partitioned_postgres_local_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) partitioned_postgres_local_table_1) partitioned_postgres_local_table JOIN mixed_relkind_tests.distributed_table ON (true)) + count +--------------------------------------------------------------------- + 36 +(1 row) + +SELECT COUNT(*) FROM partitioned_postgres_local_table JOIN partitioned_distributed_table ON (true); +DEBUG: Wrapping relation "partitioned_postgres_local_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS a FROM mixed_relkind_tests.partitioned_postgres_local_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT NULL::integer AS a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) partitioned_postgres_local_table_1) partitioned_postgres_local_table JOIN mixed_relkind_tests.partitioned_distributed_table ON (true)) + count +--------------------------------------------------------------------- + 36 +(1 row) + +SELECT COUNT(*) FROM distributed_table JOIN partitioned_postgres_local_table ON (true); +DEBUG: Wrapping relation "partitioned_postgres_local_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS a FROM mixed_relkind_tests.partitioned_postgres_local_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (mixed_relkind_tests.distributed_table JOIN (SELECT NULL::integer AS a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) partitioned_postgres_local_table_1) partitioned_postgres_local_table ON (true)) + count +--------------------------------------------------------------------- + 36 +(1 row) + INSERT INTO partitioned_distributed_table SELECT foo.* FROM partitioned_distributed_table AS foo JOIN citus_local_table ON (true); DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time -ERROR: direct joins between distributed and local tables are not supported +DEBUG: Wrapping relation "citus_local_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS a FROM mixed_relkind_tests.citus_local_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.a, foo.b FROM (mixed_relkind_tests.partitioned_distributed_table foo JOIN (SELECT NULL::integer AS a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table ON (true)) +DEBUG: performing repartitioned INSERT ... SELECT INSERT INTO partitioned_distributed_table SELECT foo.* FROM distributed_table AS foo JOIN citus_local_table ON (true); DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time -ERROR: direct joins between distributed and local tables are not supported +DEBUG: Wrapping relation "citus_local_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS a FROM mixed_relkind_tests.citus_local_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.a FROM (mixed_relkind_tests.distributed_table foo JOIN (SELECT NULL::integer AS a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table ON (true)) +DEBUG: performing repartitioned INSERT ... SELECT INSERT INTO distributed_table SELECT foo.a FROM partitioned_distributed_table AS foo JOIN citus_local_table ON (true); DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time -ERROR: direct joins between distributed and local tables are not supported +DEBUG: Wrapping relation "citus_local_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS a FROM mixed_relkind_tests.citus_local_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT foo.a FROM (mixed_relkind_tests.partitioned_distributed_table foo JOIN (SELECT NULL::integer AS a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) citus_local_table_1) citus_local_table ON (true)) +DEBUG: performing repartitioned INSERT ... SELECT +-- should fail +SELECT COUNT(*) FROM reference_table LEFT JOIN partitioned_distributed_table ON true; +ERROR: cannot pushdown the subquery +DETAIL: There exist a reference table in the outer part of the outer join -- non-colocated subquery should work SELECT COUNT(*) FROM (SELECT *, random() FROM partitioned_distributed_table) AS foo, @@ -350,7 +377,7 @@ DEBUG: generating subplan XXX_1 for subquery SELECT a, b, random() AS random FR DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT partitioned_distributed_table.a, partitioned_distributed_table.b, random() AS random FROM mixed_relkind_tests.partitioned_distributed_table) foo, (SELECT intermediate_result.a, intermediate_result.b, intermediate_result.random FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer, random double precision)) bar WHERE (foo.a OPERATOR(pg_catalog.=) bar.b) count --------------------------------------------------------------------- - 5 + 455 (1 row) SELECT COUNT(*) FROM @@ -364,21 +391,27 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS c 0 (1 row) --- should fail UPDATE partitioned_distributed_table SET b = foo.a FROM citus_local_table AS foo; -ERROR: cannot plan modifications with citus local tables and distributed tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins +DEBUG: Wrapping relation "citus_local_table" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM mixed_relkind_tests.citus_local_table foo WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE mixed_relkind_tests.partitioned_distributed_table SET b = foo.a FROM (SELECT foo_1.a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) foo_1) foo UPDATE partitioned_distributed_table SET b = foo.a FROM postgres_local_table AS foo; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins +DEBUG: Wrapping relation "postgres_local_table" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM mixed_relkind_tests.postgres_local_table foo WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE mixed_relkind_tests.partitioned_distributed_table SET b = foo.a FROM (SELECT foo_1.a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) foo_1) foo UPDATE partitioned_distributed_table SET a = foo.a FROM postgres_local_table AS foo WHERE foo.a = partitioned_distributed_table.a; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins +DEBUG: Wrapping relation "postgres_local_table" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM mixed_relkind_tests.postgres_local_table foo WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE mixed_relkind_tests.partitioned_distributed_table SET a = foo.a FROM (SELECT foo_1.a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) foo_1) foo WHERE (foo.a OPERATOR(pg_catalog.=) partitioned_distributed_table.a) UPDATE partitioned_distributed_table SET a = foo.a FROM citus_local_table AS foo WHERE foo.a = partitioned_distributed_table.a; -ERROR: cannot plan modifications with citus local tables and distributed tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins +DEBUG: Wrapping relation "citus_local_table" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM mixed_relkind_tests.citus_local_table foo WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE mixed_relkind_tests.partitioned_distributed_table SET a = foo.a FROM (SELECT foo_1.a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) foo_1) foo WHERE (foo.a OPERATOR(pg_catalog.=) partitioned_distributed_table.a) +-- should fail UPDATE partitioned_distributed_table SET a = foo.a FROM mat_view_on_part_dist AS foo WHERE foo.a = partitioned_distributed_table.a; -ERROR: materialized views in modify queries are not supported +DEBUG: Wrapping relation "mat_view_on_part_dist" "foo" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM mixed_relkind_tests.mat_view_on_part_dist foo WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE mixed_relkind_tests.partitioned_distributed_table SET a = foo.a FROM (SELECT foo_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) foo_1) foo WHERE (foo.a OPERATOR(pg_catalog.=) partitioned_distributed_table.a) UPDATE partitioned_distributed_table SET a = foo.a FROM partitioned_distributed_table AS foo WHERE foo.a < partitioned_distributed_table.a; ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns UPDATE partitioned_distributed_table SET a = foo.a FROM distributed_table AS foo WHERE foo.a < partitioned_distributed_table.a; @@ -392,33 +425,33 @@ UPDATE partitioned_distributed_table SET a = foo.a FROM view_on_ref AS foo WHERE SELECT COUNT(*) FROM partitioned_distributed_table p1 JOIN partitioned_distributed_table p2 USING (a); count --------------------------------------------------------------------- - 6 + 1014 (1 row) SELECT COUNT(*) FROM unlogged_distributed_table u1 JOIN partitioned_distributed_table p2 USING (a); count --------------------------------------------------------------------- - 6 + 78 (1 row) SELECT COUNT(*) FROM partitioned_distributed_table p1 LEFT JOIN partitioned_distributed_table p2 USING (a); count --------------------------------------------------------------------- - 6 + 1014 (1 row) -- lateral JOIN SELECT COUNT(*) FROM partitioned_distributed_table p1 JOIN LATERAL (SELECT * FROM partitioned_distributed_table p2 WHERE p1.a = p2.a) AS foo ON (true); count --------------------------------------------------------------------- - 6 + 1014 (1 row) -- router query SELECT COUNT(*) FROM partitioned_distributed_table p1 JOIN partitioned_distributed_table p2 USING (a) WHERE a = 1; count --------------------------------------------------------------------- - 1 + 169 (1 row) -- repartition query @@ -426,13 +459,13 @@ SET citus.enable_repartition_joins TO ON; SELECT COUNT(*) FROM partitioned_distributed_table p1 JOIN partitioned_distributed_table p2 USING (b) WHERE b = 1; count --------------------------------------------------------------------- - 1 + 0 (1 row) SELECT COUNT(*) FROM unlogged_distributed_table u1 JOIN partitioned_distributed_table p2 USING (b) WHERE b = 1; count --------------------------------------------------------------------- - 1 + 0 (1 row) RESET citus.enable_repartition_joins; @@ -443,7 +476,7 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT a, b FROM mixed_relkind_t DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte_1 count --------------------------------------------------------------------- - 6 + 78 (1 row) WITH cte_1 AS (SELECT * FROM partitioned_distributed_table) @@ -452,7 +485,7 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT a, b FROM mixed_relkind_t DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte_1 JOIN mixed_relkind_tests.partitioned_distributed_table USING (a)) count --------------------------------------------------------------------- - 6 + 1014 (1 row) WITH cte_1 AS (SELECT * FROM foreign_distributed_table) @@ -470,7 +503,7 @@ DEBUG: generating subplan XXX_1 for CTE cte_1: SELECT a, b FROM mixed_relkind_t DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte_1 JOIN mixed_relkind_tests.partitioned_distributed_table USING (b)) count --------------------------------------------------------------------- - 6 + 6084 (1 row) -- multi shard colocated update @@ -599,13 +632,13 @@ SET client_min_messages TO DEBUG1; SELECT sum(a) FROM partitioned_distributed_table; sum --------------------------------------------------------------------- - 15 + 195 (1 row) SELECT ceil(regr_syy(a, b)) FROM partitioned_distributed_table; ceil --------------------------------------------------------------------- - 18 + 228 (1 row) SELECT ceil(regr_syy(a, b)) FROM unlogged_distributed_table; @@ -674,8 +707,20 @@ $Q$); SELECT * FROM partitioned_distributed_table WHERE a = 1 ORDER BY 1,2 FOR UPDATE; a | b --------------------------------------------------------------------- - 1 | 2 -(1 row) + 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 | 0 +(13 rows) SELECT * FROM unlogged_distributed_table WHERE a = 1 ORDER BY 1,2 FOR UPDATE; a | b diff --git a/src/test/regress/expected/multi_modifications.out b/src/test/regress/expected/multi_modifications.out index 05ba0a3d7..ee24d4c99 100644 --- a/src/test/regress/expected/multi_modifications.out +++ b/src/test/regress/expected/multi_modifications.out @@ -270,8 +270,6 @@ CREATE TABLE bidders ( name text, id bigint ); DELETE FROM limit_orders USING bidders WHERE limit_orders.id = 246 AND limit_orders.bidder_id = bidders.id AND bidders.name = 'Bernie Madoff'; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins -- commands containing a CTE are supported WITH new_orders AS (INSERT INTO limit_orders VALUES (411, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66)) DELETE FROM limit_orders WHERE id < 0; @@ -429,13 +427,11 @@ ERROR: modifying the partition value of rows is not allowed UPDATE limit_orders SET id = 246 WHERE id = 246; UPDATE limit_orders SET id = 246 WHERE id = 246 AND symbol = 'GM'; UPDATE limit_orders SET id = limit_orders.id WHERE id = 246; --- UPDATEs with a FROM clause are unsupported +-- UPDATEs with a FROM clause are supported even with local tables UPDATE limit_orders SET limit_price = 0.00 FROM bidders WHERE limit_orders.id = 246 AND limit_orders.bidder_id = bidders.id AND bidders.name = 'Bernie Madoff'; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins -- should succeed with a CTE WITH deleted_orders AS (INSERT INTO limit_orders VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)) UPDATE limit_orders SET symbol = 'GM'; @@ -1306,7 +1302,8 @@ DELETE FROM summary_table WHERE id < ( ); CREATE TABLE multi_modifications.local (a int default 1, b int); INSERT INTO multi_modifications.local VALUES (default, (SELECT min(id) FROM summary_table)); -ERROR: cannot plan modifications of local tables involving distributed tables +ERROR: subqueries are not supported within INSERT queries +HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax. DROP TABLE raw_table; DROP TABLE summary_table; DROP TABLE reference_raw_table; diff --git a/src/test/regress/expected/multi_mx_modifications.out b/src/test/regress/expected/multi_mx_modifications.out index c78b1d98f..14468b95a 100644 --- a/src/test/regress/expected/multi_mx_modifications.out +++ b/src/test/regress/expected/multi_mx_modifications.out @@ -158,7 +158,6 @@ CREATE TABLE bidders ( name text, id bigint ); DELETE FROM limit_orders_mx USING bidders WHERE limit_orders_mx.id = 246 AND limit_orders_mx.bidder_id = bidders.id AND bidders.name = 'Bernie Madoff'; -ERROR: cannot plan modifications with local tables involving citus tables -- commands containing a CTE are supported WITH new_orders AS (INSERT INTO limit_orders_mx VALUES (411, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66)) DELETE FROM limit_orders_mx WHERE id < 0; @@ -225,7 +224,6 @@ UPDATE limit_orders_mx SET limit_price = 0.00 FROM bidders WHERE limit_orders_mx.id = 246 AND limit_orders_mx.bidder_id = bidders.id AND bidders.name = 'Bernie Madoff'; -ERROR: cannot plan modifications with local tables involving citus tables -- commands containing a CTE are supported WITH deleted_orders AS (INSERT INTO limit_orders_mx VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)) UPDATE limit_orders_mx SET symbol = 'GM'; diff --git a/src/test/regress/expected/multi_partitioning.out b/src/test/regress/expected/multi_partitioning.out index 2e177214e..c11d800b2 100644 --- a/src/test/regress/expected/multi_partitioning.out +++ b/src/test/regress/expected/multi_partitioning.out @@ -1430,7 +1430,7 @@ COMMIT; BEGIN; TRUNCATE partitioning_locks; SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; - relation | locktype | mode + relation | locktype | mode --------------------------------------------------------------------- partitioning_locks | relation | AccessExclusiveLock partitioning_locks | relation | AccessShareLock diff --git a/src/test/regress/expected/multi_shard_modify.out b/src/test/regress/expected/multi_shard_modify.out index 0b1fe4ce7..70a12381d 100644 --- a/src/test/regress/expected/multi_shard_modify.out +++ b/src/test/regress/expected/multi_shard_modify.out @@ -72,8 +72,11 @@ CREATE TABLE temp_nations(name text, key integer); SELECT master_modify_multiple_shards('DELETE FROM multi_shard_modify_test USING temp_nations WHERE multi_shard_modify_test.t_value = temp_nations.key AND temp_nations.name = ''foobar'' '); WARNING: master_modify_multiple_shards is deprecated and will be removed in a future release. HINT: Run the command directly -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins + master_modify_multiple_shards +--------------------------------------------------------------------- + 0 +(1 row) + -- commands with a USING clause are unsupported SELECT create_distributed_table('temp_nations', 'name', 'hash'); create_distributed_table diff --git a/src/test/regress/expected/multi_shard_update_delete.out b/src/test/regress/expected/multi_shard_update_delete.out index f5d845597..8fe0a5aec 100644 --- a/src/test/regress/expected/multi_shard_update_delete.out +++ b/src/test/regress/expected/multi_shard_update_delete.out @@ -730,19 +730,15 @@ SET value_2 = subquery.random FROM (SELECT user_id, random() WHERE users_test_table.user_id = subquery.user_id; -- Make following tests consistent UPDATE users_test_table SET value_2 = 0; --- Local tables are not supported +-- Joins with tables not supported UPDATE users_test_table SET value_2 = 5 FROM events_test_table_local WHERE users_test_table.user_id = events_test_table_local.user_id; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins UPDATE events_test_table_local SET value_2 = 5 FROM users_test_table WHERE events_test_table_local.user_id = users_test_table.user_id; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins -- Local tables in a subquery are supported through recursive planning UPDATE users_test_table SET value_2 = 5 diff --git a/src/test/regress/expected/multi_simple_queries.out b/src/test/regress/expected/multi_simple_queries.out index a5d95abd3..d518bb408 100644 --- a/src/test/regress/expected/multi_simple_queries.out +++ b/src/test/regress/expected/multi_simple_queries.out @@ -281,14 +281,18 @@ ORDER BY articles.id; SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard a2 WHERE a.id = a2.id LIMIT 1) AS special_price FROM articles a; ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns --- joins are not supported between local and distributed tables +-- joins are supported between local and distributed tables SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id; -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins --- inner joins are not supported (I think) + title | name +--------------------------------------------------------------------- +(0 rows) + +-- inner joins are supported SELECT * FROM (articles INNER JOIN authors ON articles.id = authors.id); -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins + id | author_id | title | word_count | name | id +--------------------------------------------------------------------- +(0 rows) + -- test use of EXECUTE statements within plpgsql DO $sharded_execute$ BEGIN diff --git a/src/test/regress/expected/multi_simple_queries_0.out b/src/test/regress/expected/multi_simple_queries_0.out index 1f64bba84..461d7db0a 100644 --- a/src/test/regress/expected/multi_simple_queries_0.out +++ b/src/test/regress/expected/multi_simple_queries_0.out @@ -226,12 +226,12 @@ SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard a2 WHERE a.id = AS special_price FROM articles a; ERROR: could not run distributed query with subquery outside the FROM, WHERE and HAVING clauses HINT: Consider using an equality filter on the distributed table's partition column. --- joins are not supported between local and distributed tables +-- joins are supported between local and distributed tables SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id; -ERROR: relation authors is not distributed --- inner joins are not supported (I think) +ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker' +-- inner joins are supported SELECT * FROM (articles INNER JOIN authors ON articles.id = authors.id); -ERROR: relation authors is not distributed +ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker' -- test use of EXECUTE statements within plpgsql DO $sharded_execute$ BEGIN diff --git a/src/test/regress/expected/multi_view.out b/src/test/regress/expected/multi_view.out index a7ab6a71d..926bc5fcf 100644 --- a/src/test/regress/expected/multi_view.out +++ b/src/test/regress/expected/multi_view.out @@ -920,11 +920,11 @@ CREATE VIEW small_view AS SELECT * from small where id < 100; \copy large FROM STDIN DELIMITER ',' -- running modify statements "on" views is still not supported, hence below two statements will fail UPDATE small_view SET id = 1; -ERROR: cannot modify views over distributed tables +ERROR: cannot modify views when the query contains citus tables DELETE FROM small_view; -ERROR: cannot modify views over distributed tables +ERROR: cannot modify views when the query contains citus tables INSERT INTO small_view VALUES(8, 5) ON CONFLICT(tenant_id) DO UPDATE SET tenant_id=99; -ERROR: cannot modify views over distributed tables +ERROR: cannot modify views when the query contains citus tables -- using views in modify statements' FROM / WHERE clauses is still valid UPDATE large SET id=20 FROM small_view WHERE small_view.id=large.id; SELECT * FROM large order by 1, 2; @@ -1054,7 +1054,7 @@ SELECT * FROM large ORDER BY 1, 2; -- INSERT INTO views is still not supported INSERT INTO small_view VALUES(3, 3); -ERROR: cannot modify views over distributed tables +ERROR: cannot modify views when the query contains citus tables DROP TABLE large; DROP TABLE small CASCADE; NOTICE: drop cascades to view small_view @@ -1081,9 +1081,9 @@ CREATE VIEW small_view AS SELECT * from small where id < 100; \copy large_partitioned FROM STDIN DELIMITER ',' -- running modify statements "on" views is still not supported, hence below two statements will fail UPDATE small_view SET id = 1; -ERROR: cannot modify views over distributed tables +ERROR: cannot modify views when the query contains citus tables DELETE FROM small_view; -ERROR: cannot modify views over distributed tables +ERROR: cannot modify views when the query contains citus tables UPDATE large_partitioned SET id=27 FROM small_view WHERE small_view.tenant_id=large_partitioned.tenant_id; SELECT * FROM large_partitioned ORDER BY 1, 2; id | tenant_id diff --git a/src/test/regress/expected/recursive_relation_planning_restriction_pushdown.out b/src/test/regress/expected/recursive_relation_planning_restriction_pushdown.out new file mode 100644 index 000000000..9f9309563 --- /dev/null +++ b/src/test/regress/expected/recursive_relation_planning_restriction_pushdown.out @@ -0,0 +1,476 @@ +--------------------------------------------------------------------- +-- recursive_relation_planning_restirction_pushdown +-- In this test file, we mosly test whether Citus +-- can successfully pushdown filters to the subquery +-- that is being recursively planned. This is done +-- for all types of JOINs +--------------------------------------------------------------------- +-- all the queries in this file have the +-- same tables/subqueries combination as below +-- because this test aims to hold the query planning +-- steady, but mostly ensure that filters are handled +-- properly. Note that local is the relation that is +-- recursively planned throughout the file +CREATE SCHEMA push_down_filters; +SET search_path TO push_down_filters; +CREATE TABLE local_table (key int, value int, time timestamptz); +CREATE TABLE distributed_table (key int, value int, metadata jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TYPE new_type AS (n int, m text); +CREATE TABLE local_table_type (key int, value new_type, value_2 jsonb); +CREATE TABLE distributed_table_type (key int, value new_type, value_2 jsonb); +SELECT create_distributed_table('distributed_table_type', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Setting the debug level so that filters can be observed +SET client_min_messages TO DEBUG1; +-- for the purposes of these tests, we always want to recursively +-- plan local tables. +SET citus.local_table_join_policy TO "prefer-local"; +-- there are no filters, hence cannot pushdown any filters +SELECT count(*) +FROM distributed_table u1 +JOIN distributed_table u2 USING(key) +JOIN local_table USING (key); +DEBUG: Wrapping relation "local_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM push_down_filters.local_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((push_down_filters.distributed_table u1 JOIN push_down_filters.distributed_table u2 USING (key)) JOIN (SELECT local_table_1.key, NULL::integer AS value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) local_table_1) local_table USING (key)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- composite types can be pushed down +SELECT count(*) +FROM distributed_table d1 +JOIN local_table_type d2 using(key) +WHERE d2.value = (83, 'citus8.3')::new_type; +DEBUG: Wrapping relation "local_table_type" "d2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table_type d2 WHERE (value OPERATOR(pg_catalog.=) '(83,citus8.3)'::push_down_filters.new_type) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table d1 JOIN (SELECT d2_1.key, d2_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'text'::citus_copy_format) intermediate_result(key integer, value push_down_filters.new_type)) d2_1) d2 USING (key)) WHERE (d2.value OPERATOR(pg_catalog.=) ROW(83, 'citus8.3'::text)::push_down_filters.new_type) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- composite types can be pushed down +SELECT count(*) +FROM distributed_table d1 +JOIN local_table_type d2 using(key) +WHERE d2.value = (83, 'citus8.3')::new_type +AND d2.key = 10; +DEBUG: Wrapping relation "local_table_type" "d2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table_type d2 WHERE ((key OPERATOR(pg_catalog.=) 10) AND (value OPERATOR(pg_catalog.=) '(83,citus8.3)'::push_down_filters.new_type)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table d1 JOIN (SELECT d2_1.key, d2_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'text'::citus_copy_format) intermediate_result(key integer, value push_down_filters.new_type)) d2_1) d2 USING (key)) WHERE ((d2.value OPERATOR(pg_catalog.=) ROW(83, 'citus8.3'::text)::push_down_filters.new_type) AND (d2.key OPERATOR(pg_catalog.=) 10)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- join on a composite type works +SELECT count(*) +FROM distributed_table_type d1 +JOIN local_table_type d2 USING(value); +DEBUG: Wrapping relation "local_table_type" "d2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table_type d2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table_type d1 JOIN (SELECT NULL::integer AS key, d2_1.value, NULL::jsonb AS value_2 FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'text'::citus_copy_format) intermediate_result(value push_down_filters.new_type)) d2_1) d2 USING (value)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- scalar array expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING (key) +WHERE u2.key > ANY(ARRAY[2, 1, 6]); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key FROM push_down_filters.local_table u2 WHERE (key OPERATOR(pg_catalog.>) ANY ('{2,1,6}'::integer[])) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, NULL::integer AS value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) u2_1) u2 USING (key)) WHERE (u2.key OPERATOR(pg_catalog.>) ANY (ARRAY[2, 1, 6])) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- array operators on the table can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(key) +WHERE ARRAY[u2.key, u2.value] @> (ARRAY[2, 3]); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table u2 WHERE (ARRAY[key, value] OPERATOR(pg_catalog.@>) '{2,3}'::integer[]) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer)) u2_1) u2 USING (key)) WHERE (ARRAY[u2.key, u2.value] OPERATOR(pg_catalog.@>) ARRAY[2, 3]) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- array operators on different tables cannot be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE ARRAY[u2.value, u1.value] @> (ARRAY[2, 3]); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (ARRAY[u2.value, u1.value] OPERATOR(pg_catalog.@>) ARRAY[2, 3]) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- coerced expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value/2.0 > 2)::int::bool::text::bool; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE (((((((value)::numeric OPERATOR(pg_catalog./) 2.0) OPERATOR(pg_catalog.>) '2'::numeric))::integer)::boolean)::text)::boolean +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (((((((u2.value)::numeric OPERATOR(pg_catalog./) 2.0) OPERATOR(pg_catalog.>) (2)::numeric))::integer)::boolean)::text)::boolean + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- case expression on a single table can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (CASE WHEN u2.value > 3 THEN u2.value > 2 ELSE false END); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE CASE WHEN (value OPERATOR(pg_catalog.>) 3) THEN (value OPERATOR(pg_catalog.>) 2) ELSE false END +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE CASE WHEN (u2.value OPERATOR(pg_catalog.>) 3) THEN (u2.value OPERATOR(pg_catalog.>) 2) ELSE false END + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- case expression multiple tables cannot be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (CASE WHEN u1.value > 4000 THEN u2.value / 100 > 1 ELSE false END); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE CASE WHEN (u1.value OPERATOR(pg_catalog.>) 4000) THEN ((u2.value OPERATOR(pg_catalog./) 100) OPERATOR(pg_catalog.>) 1) ELSE false END + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- coalesce expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE COALESCE((u2.key/5.0)::int::bool, false); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table u2 WHERE COALESCE(((((key)::numeric OPERATOR(pg_catalog./) 5.0))::integer)::boolean, false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer)) u2_1) u2 USING (value)) WHERE COALESCE(((((u2.key)::numeric OPERATOR(pg_catalog./) 5.0))::integer)::boolean, false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- nullif expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE NULLIF((u2.value/5.0)::int::bool, false); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE NULLIF(((((value)::numeric OPERATOR(pg_catalog./) 5.0))::integer)::boolean, false) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE NULLIF(((((u2.value)::numeric OPERATOR(pg_catalog./) 5.0))::integer)::boolean, false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- null test can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value IS NOT NULL; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE (value IS NOT NULL) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (u2.value IS NOT NULL) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- functions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE isfinite(u2.time); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value, "time" FROM push_down_filters.local_table u2 WHERE isfinite("time") +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, u2_1."time" FROM (SELECT intermediate_result.value, intermediate_result."time" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer, "time" timestamp with time zone)) u2_1) u2 USING (value)) WHERE isfinite(u2."time") + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- functions with multiple tables cannot be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE int4smaller(u2.value, u1.value) = 55; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (int4smaller(u2.value, u1.value) OPERATOR(pg_catalog.=) 55) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- functions with multiple columns from the same tables can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE int4smaller(u2.key, u2.value) = u2.key; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table u2 WHERE (int4smaller(key, value) OPERATOR(pg_catalog.=) key) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer)) u2_1) u2 USING (value)) WHERE (int4smaller(u2.key, u2.value) OPERATOR(pg_catalog.=) u2.key) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- row expressions can be pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE row(u2.value, 2, 3) > row(u2.value, 2, 3); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE (ROW(value, 2, 3) OPERATOR(pg_catalog.>) ROW(value, 2, 3)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (ROW(u2.value, 2, 3) OPERATOR(pg_catalog.>) ROW(u2.value, 2, 3)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- multiple expression from the same table can be pushed down together +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) + WHERE + (u2.key/1.0)::int::bool::text::bool AND + CASE WHEN u2.key > 4000 THEN u2.value / 100 > 1 ELSE false END AND + COALESCE((u2.key/50000)::bool, false) AND + NULLIF((u2.value/50000)::int::bool, false) AND + isfinite(u2.time) AND + u2.value IS DISTINCT FROM 50040 AND + row(u2.value, 2, 3) > row(2000, 2, 3); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value, "time" FROM push_down_filters.local_table u2 WHERE (((((((key)::numeric OPERATOR(pg_catalog./) 1.0))::integer)::boolean)::text)::boolean AND CASE WHEN (key OPERATOR(pg_catalog.>) 4000) THEN ((value OPERATOR(pg_catalog./) 100) OPERATOR(pg_catalog.>) 1) ELSE false END AND COALESCE(((key OPERATOR(pg_catalog./) 50000))::boolean, false) AND NULLIF(((value OPERATOR(pg_catalog./) 50000))::boolean, false) AND isfinite("time") AND (value IS DISTINCT FROM 50040) AND (ROW(value, 2, 3) OPERATOR(pg_catalog.>) ROW(2000, 2, 3))) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, u2_1."time" FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result."time" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer, "time" timestamp with time zone)) u2_1) u2 USING (value)) WHERE (((((((u2.key)::numeric OPERATOR(pg_catalog./) 1.0))::integer)::boolean)::text)::boolean AND CASE WHEN (u2.key OPERATOR(pg_catalog.>) 4000) THEN ((u2.value OPERATOR(pg_catalog./) 100) OPERATOR(pg_catalog.>) 1) ELSE false END AND COALESCE(((u2.key OPERATOR(pg_catalog./) 50000))::boolean, false) AND NULLIF(((u2.value OPERATOR(pg_catalog./) 50000))::boolean, false) AND isfinite(u2."time") AND (u2.value IS DISTINCT FROM 50040) AND (ROW(u2.value, 2, 3) OPERATOR(pg_catalog.>) ROW(2000, 2, 3))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- subqueries filters are not pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > + (SELECT avg(key) + FROM distributed_table); +DEBUG: generating subplan XXX_1 for subquery SELECT avg(key) AS avg FROM push_down_filters.distributed_table +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_2 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE ((u2.value)::numeric OPERATOR(pg_catalog.>) (SELECT intermediate_result.avg FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(avg numeric))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- even subqueries with constant values are not pushdowned +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > (SELECT 5); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (u2.value OPERATOR(pg_catalog.>) (SELECT 5)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- filters involving multiple tables aren't pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value * u1.key > 25; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE ((u2.value OPERATOR(pg_catalog.*) u1.key) OPERATOR(pg_catalog.>) 25) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- filter on other tables can only be pushdown +-- as long as they are equality filters on the +-- joining column +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u1.value = 3; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE (value OPERATOR(pg_catalog.=) 3) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (u1.value OPERATOR(pg_catalog.=) 3) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- but not when the filter is gt, lt or any other thing +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u1.value > 3; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (u1.value OPERATOR(pg_catalog.>) 3) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- when the filter is on another column than the +-- join column, that's obviously not pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u1.key = 3; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE (u1.key OPERATOR(pg_catalog.=) 3) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- or filters on the same table is pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > 4 OR u2.value = 4; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE ((value OPERATOR(pg_catalog.>) 4) OR (value OPERATOR(pg_catalog.=) 4)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE ((u2.value OPERATOR(pg_catalog.>) 4) OR (u2.value OPERATOR(pg_catalog.=) 4)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- and filters on the same table is pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > 2 and u2.time IS NULL; +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value, "time" FROM push_down_filters.local_table u2 WHERE ((value OPERATOR(pg_catalog.>) 2) AND ("time" IS NULL)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, u2_1."time" FROM (SELECT intermediate_result.value, intermediate_result."time" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer, "time" timestamp with time zone)) u2_1) u2 USING (value)) WHERE ((u2.value OPERATOR(pg_catalog.>) 2) AND (u2."time" IS NULL)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- filters on different tables are pushdown +-- only the ones that are not ANDed +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u2.value IS NULL) AND (u2.key > 4 OR u1.key > 3); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table u2 WHERE ((value OPERATOR(pg_catalog.>) 2) OR (value IS NULL)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer)) u2_1) u2 USING (value)) WHERE (((u2.value OPERATOR(pg_catalog.>) 2) OR (u2.value IS NULL)) AND ((u2.key OPERATOR(pg_catalog.>) 4) OR (u1.key OPERATOR(pg_catalog.>) 3))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- filters on different tables are pushdown +-- only the ones that are not ANDed +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u2.value IS NULL) OR (u2.key > 4 OR u1.key > 3); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer)) u2_1) u2 USING (value)) WHERE ((u2.value OPERATOR(pg_catalog.>) 2) OR (u2.value IS NULL) OR ((u2.key OPERATOR(pg_catalog.>) 4) OR (u1.key OPERATOR(pg_catalog.>) 3))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- filters on different tables are pushdown +-- only the ones that are not ANDed +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u2.value IS NULL) AND (u2.key > 4 OR u1.key > 3); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table u2 WHERE ((value OPERATOR(pg_catalog.>) 2) OR (value IS NULL)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer)) u2_1) u2 USING (value)) WHERE (((u2.value OPERATOR(pg_catalog.>) 2) OR (u2.value IS NULL)) AND ((u2.key OPERATOR(pg_catalog.>) 4) OR (u1.key OPERATOR(pg_catalog.>) 3))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- but volatile functions are not pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u1.value IS NULL) AND (u2.key = 10000 * random() OR u1.key > 3); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT key, value FROM push_down_filters.local_table u2 WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT u2_1.key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value integer)) u2_1) u2 USING (value)) WHERE (((u2.value OPERATOR(pg_catalog.>) 2) OR (u1.value IS NULL)) AND (((u2.key)::double precision OPERATOR(pg_catalog.=) ((10000)::double precision OPERATOR(pg_catalog.*) random())) OR (u1.key OPERATOR(pg_catalog.>) 3))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- constant results should be pushed down, but not supported yet +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 AND false); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE false +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) WHERE ((u2.value OPERATOR(pg_catalog.>) 2) AND false) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- we can still pushdown WHERE false +-- even if it is a LATERAL join +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +JOIN LATERAL + (SELECT value, + random() + FROM distributed_table + WHERE u2.value = 15) AS u3 USING (value) +WHERE (u2.value > 2 + AND FALSE); +DEBUG: Wrapping relation "local_table" "u2" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT value FROM push_down_filters.local_table u2 WHERE false +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((push_down_filters.distributed_table u1 JOIN (SELECT NULL::integer AS key, u2_1.value, NULL::timestamp with time zone AS "time" FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value integer)) u2_1) u2 USING (value)) JOIN LATERAL (SELECT distributed_table.value, random() AS random FROM push_down_filters.distributed_table WHERE (u2.value OPERATOR(pg_catalog.=) 15)) u3 USING (value)) WHERE ((u2.value OPERATOR(pg_catalog.>) 2) AND false) +ERROR: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +\set VERBOSITY terse +RESET client_min_messages; +DROP SCHEMA push_down_filters CASCADE; +NOTICE: drop cascades to 5 other objects diff --git a/src/test/regress/expected/replicate_reference_tables_to_coordinator.out b/src/test/regress/expected/replicate_reference_tables_to_coordinator.out index d3a50e814..72082810a 100644 --- a/src/test/regress/expected/replicate_reference_tables_to_coordinator.out +++ b/src/test/regress/expected/replicate_reference_tables_to_coordinator.out @@ -281,11 +281,15 @@ SELECT test_reference_local_join_func(); -- CTEs are allowed WITH ins AS (INSERT INTO numbers VALUES (1) RETURNING *) -SELECT * FROM numbers, local_table; +SELECT * FROM numbers, local_table ORDER BY 1,2; NOTICE: executing the command locally: INSERT INTO replicate_ref_to_coordinator.numbers_8000001 (a) VALUES (1) RETURNING a -NOTICE: executing the command locally: SELECT numbers.a, local_table.a FROM replicate_ref_to_coordinator.numbers_8000001 numbers, replicate_ref_to_coordinator.local_table +NOTICE: executing the command locally: SELECT numbers.a, local_table.a FROM replicate_ref_to_coordinator.numbers_8000001 numbers, (SELECT local_table_1.a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) local_table_1) local_table ORDER BY numbers.a, local_table.a a | a --------------------------------------------------------------------- + 2 | 2 + 2 | 4 + 2 | 7 + 2 | 20 20 | 2 20 | 4 20 | 7 @@ -294,10 +298,6 @@ NOTICE: executing the command locally: SELECT numbers.a, local_table.a FROM rep 21 | 4 21 | 7 21 | 20 - 2 | 2 - 2 | 4 - 2 | 7 - 2 | 20 (12 rows) WITH t AS (SELECT *, my_volatile_fn() x FROM numbers FOR UPDATE) diff --git a/src/test/regress/expected/subqueries_not_supported.out b/src/test/regress/expected/subqueries_not_supported.out index 13e064d59..0bb5da386 100644 --- a/src/test/regress/expected/subqueries_not_supported.out +++ b/src/test/regress/expected/subqueries_not_supported.out @@ -6,22 +6,9 @@ SET search_path TO not_supported, public; SET citus.coordinator_aggregation_strategy TO 'disabled'; SET client_min_messages TO DEBUG1; CREATE TABLE users_table_local AS SELECT * FROM users_table; --- we don't support subqueries with local tables when they are not leaf queries -SELECT - * -FROM - ( - SELECT - users_table_local.user_id - FROM - users_table_local, (SELECT user_id FROM events_table) as evs - WHERE users_table_local.user_id = evs.user_id - ) as foo; -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins RESET client_min_messages; -- we don't support subqueries with local tables when they are not leaf queries -SELECT user_id FROM users_table WHERE user_id IN +SELECT COUNT(user_id) FROM users_table WHERE user_id IN (SELECT user_id FROM diff --git a/src/test/regress/expected/with_executors.out b/src/test/regress/expected/with_executors.out index 519ec9ddd..df1c80625 100644 --- a/src/test/regress/expected/with_executors.out +++ b/src/test/regress/expected/with_executors.out @@ -302,13 +302,16 @@ SELECT min(user_id) FROM cte JOIN local_table ON (user_id = id); 1 (1 row) --- not if there are no distributed tables +-- even if there are no distributed tables WITH cte AS ( SELECT user_id FROM users_table ) SELECT min(user_id) FROM cte JOIN local_table ON (user_id = id) JOIN events_table USING (user_id); -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins + min +--------------------------------------------------------------------- + 1 +(1 row) + -- unless the distributed table is part of a recursively planned subquery WITH cte AS ( SELECT user_id FROM users_table @@ -319,15 +322,18 @@ SELECT min(user_id) FROM cte JOIN local_table ON (user_id = id) JOIN (SELECT * F 1 (1 row) --- joins between local and reference tables not allowed --- since the coordinator is not in the metadata at this stage +-- joins between local and reference tables are allowed +-- even when the coordinator is not in the metadata at this stage WITH cte AS ( SELECT user_id FROM users_table ) SELECT count(*) FROM local_table JOIN ref_table USING (id) WHERE id IN (SELECT * FROM cte); -ERROR: direct joins between distributed and local tables are not supported -HINT: Use CTE's or subqueries to select from local tables and use them in joins + count +--------------------------------------------------------------------- + 6 +(1 row) + -- CTEs should be able to terminate a router query WITH cte AS ( WITH cte_1 AS ( diff --git a/src/test/regress/expected/with_modifying.out b/src/test/regress/expected/with_modifying.out index 4704c31a3..e7e81a08a 100644 --- a/src/test/regress/expected/with_modifying.out +++ b/src/test/regress/expected/with_modifying.out @@ -668,19 +668,20 @@ SELECT * FROM raw_data ORDER BY val; --------------------------------------------------------------------- (0 rows) --- Test that local tables are barred +-- Test that local tables are can be updated +-- selecting from distributed tables UPDATE local_table lt SET val = mt.val FROM modify_table mt WHERE mt.id = lt.id; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins -- Including inside CTEs WITH cte AS ( UPDATE local_table lt SET val = mt.val FROM modify_table mt WHERE mt.id = lt.id RETURNING lt.id, lt.val ) SELECT * FROM cte JOIN modify_table mt ON mt.id = cte.id ORDER BY 1,2; -ERROR: cannot plan modifications with local tables involving citus tables -HINT: Use CTE's or subqueries to select from local tables and use them in joins + id | val | id | val +--------------------------------------------------------------------- +(0 rows) + -- Make sure checks for volatile functions apply to CTEs too WITH cte AS (UPDATE modify_table SET val = random() WHERE id = 3 RETURNING *) SELECT * FROM cte JOIN modify_table mt ON mt.id = 3 AND mt.id = cte.id ORDER BY 1,2; diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index 95434bf38..e35822bbc 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -110,7 +110,7 @@ test: multi_average_expression multi_working_columns multi_having_pushdown havin test: multi_array_agg multi_limit_clause multi_orderby_limit_pushdown test: multi_jsonb_agg multi_jsonb_object_agg multi_json_agg multi_json_object_agg bool_agg ch_bench_having chbenchmark_all_queries expression_reference_join anonymous_columns test: ch_bench_subquery_repartition -test: multi_agg_type_conversion multi_count_type_conversion +test: multi_agg_type_conversion multi_count_type_conversion recursive_relation_planning_restriction_pushdown test: multi_partition_pruning single_hash_repartition_join test: multi_join_pruning multi_hash_pruning intermediate_result_pruning test: multi_null_minmax_value_pruning cursors @@ -222,6 +222,11 @@ test: multi_modifying_xacts test: multi_repartition_udt multi_repartitioned_subquery_udf multi_subtransactions test: multi_transaction_recovery +test: local_dist_join_modifications +test: local_table_join +test: local_dist_join_mixed +test: citus_local_dist_joins + # --------- # multi_copy creates hash and range-partitioned tables and performs COPY # multi_router_planner creates hash partitioned tables. diff --git a/src/test/regress/sql/citus_local_dist_joins.sql b/src/test/regress/sql/citus_local_dist_joins.sql new file mode 100644 index 000000000..ceaff4218 --- /dev/null +++ b/src/test/regress/sql/citus_local_dist_joins.sql @@ -0,0 +1,258 @@ +CREATE SCHEMA citus_local_dist_joins; +SET search_path TO citus_local_dist_joins; + +SET client_min_messages to ERROR; +SELECT master_add_node('localhost', :master_port, groupId => 0) AS coordinator_nodeid \gset + + +CREATE TABLE citus_local(key int, value text); +SELECT create_citus_local_table('citus_local'); + +CREATE TABLE postgres_table (key int, value text, value_2 jsonb); +CREATE TABLE reference_table (key int, value text, value_2 jsonb); +SELECT create_reference_table('reference_table'); +CREATE TABLE distributed_table (key int, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + +CREATE TABLE distributed_table_pkey (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_pkey', 'key'); + +CREATE TABLE distributed_table_windex (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_windex', 'key'); +CREATE UNIQUE INDEX key_index ON distributed_table_windex (key); + +CREATE TABLE distributed_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE distributed_partitioned_table_1 PARTITION OF distributed_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE distributed_partitioned_table_2 PARTITION OF distributed_partitioned_table FOR VALUES FROM (50) TO (200); +SELECT create_distributed_table('distributed_partitioned_table', 'key'); + +CREATE TABLE local_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE local_partitioned_table_1 PARTITION OF local_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE local_partitioned_table_2 PARTITION OF local_partitioned_table FOR VALUES FROM (50) TO (200); + +CREATE TABLE distributed_table_composite (key int, value text, value_2 jsonb, primary key (key, value)); +SELECT create_distributed_table('distributed_table_composite', 'key'); + +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM postgres_table; +CREATE MATERIALIZED VIEW mv2 AS SELECT * FROM distributed_table; + +-- set log messages to debug1 so that we can see which tables are recursively planned. +SET client_min_messages TO DEBUG1; + +INSERT INTO postgres_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO reference_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_windex SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_pkey SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_composite SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO local_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO citus_local SELECT i, i::varchar(256) FROM generate_series(1, 100) i; + + +-- a unique index on key so dist table should be recursively planned +SELECT count(*) FROM citus_local JOIN distributed_table_windex USING(key); +SELECT count(*) FROM citus_local JOIN distributed_table_windex USING(value); +SELECT count(*) FROM citus_local JOIN distributed_table_windex ON citus_local.key = distributed_table_windex.key; +SELECT count(*) FROM citus_local JOIN distributed_table_windex ON distributed_table_windex.key = 10; + +-- no unique index, citus local table should be recursively planned +SELECT count(*) FROM citus_local JOIN distributed_table USING(key); +SELECT count(*) FROM citus_local JOIN distributed_table USING(value); +SELECT count(*) FROM citus_local JOIN distributed_table ON citus_local.key = distributed_table.key; +SELECT count(*) FROM citus_local JOIN distributed_table ON distributed_table.key = 10; + +SELECT count(*) FROM citus_local JOIN distributed_table USING(key) JOIN postgres_table USING (key) JOIN reference_table USING(key); + +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) JOIN reference_table USING (key) + JOIN citus_local USING(key) WHERE distributed_partitioned_table.key > 10 and distributed_partitioned_table.key = 10; + +-- update +BEGIN; +SELECT COUNT(DISTINCT value) FROM citus_local; +UPDATE + citus_local +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM citus_local; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; +UPDATE + distributed_table +SET + value = 'test' +FROM + citus_local +WHERE + distributed_table.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM distributed_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + citus_local +WHERE + distributed_table_pkey.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + citus_local +WHERE + distributed_table_windex.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +ROLLBACK; + +BEGIN; +UPDATE + mv1 +SET + value = 'test' +FROM + citus_local +WHERE + mv1.key = citus_local.key; +ROLLBACK; + +BEGIN; +UPDATE + citus_local +SET + value = 'test' +FROM + mv1 +WHERE + mv1.key = citus_local.key; +ROLLBACK; + +BEGIN; +UPDATE + citus_local +SET + value = 'test' +FROM + mv2 +WHERE + mv2.key = citus_local.key; +ROLLBACK; + +-- DELETE operations + +BEGIN; +SELECT COUNT(DISTINCT value) FROM citus_local; +DELETE FROM + citus_local +USING + distributed_table +WHERE + distributed_table.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM citus_local; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; +DELETE FROM + distributed_table +USING + citus_local +WHERE + distributed_table.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM distributed_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +DELETE FROM + distributed_table_pkey +USING + citus_local +WHERE + distributed_table_pkey.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +DELETE FROM + distributed_table_windex +USING + citus_local +WHERE + distributed_table_windex.key = citus_local.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +ROLLBACK; + +DELETE FROM + mv1 +USING + citus_local +WHERE + mv1.key = citus_local.key; + +DELETE FROM + citus_local +USING + mv1 +WHERE + mv1.key = citus_local.key; + +DELETE FROM + citus_local +USING + mv2 +WHERE + mv2.key = citus_local.key; + +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM (SELECT * FROM distributed_table LIMIT 1) d1) d2 using (key) JOIN reference_table USING(key) JOIN citus_local USING (key) JOIN (SELECT * FROM citus_local) c1 USING (key) WHERE d2.key > 10 AND d2.key = 10; +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM (SELECT * FROM distributed_table LIMIT 1) d1) d2 using (key) JOIN reference_table USING(key) JOIN citus_local USING (key) JOIN (SELECT * FROM citus_local) c1 USING (key) WHERE d2.key > 10 AND d2.key = 10; + + +SELECT + COUNT(*) +FROM + postgres_table p1 +JOIN + distributed_partitioned_table dp1 +USING (key) +JOIN + distributed_table d1 +USING (key) +JOIN + citus_local c1 +USING (key) +JOIN + postgres_table p2 +USING (key) +JOIN + reference_table r1 +USING (key) +JOIN + distributed_table d2 +USING (key) +JOIN + citus_local c2 +USING (key); + + +SET client_min_messages to ERROR; +DROP TABLE citus_local; +SELECT master_remove_node('localhost', :master_port); +\set VERBOSITY terse +DROP SCHEMA citus_local_dist_joins CASCADE; diff --git a/src/test/regress/sql/citus_local_tables_queries.sql b/src/test/regress/sql/citus_local_tables_queries.sql index 564b7c6c8..f98ead754 100644 --- a/src/test/regress/sql/citus_local_tables_queries.sql +++ b/src/test/regress/sql/citus_local_tables_queries.sql @@ -413,13 +413,14 @@ SET b = 6 FROM distributed_table_cte WHERE citus_local_table.a = distributed_table_cte.a; +SET citus.log_local_commands to off; -- just works WITH reference_table_cte AS (SELECT * FROM reference_table) UPDATE citus_local_table SET b = 6 FROM reference_table_cte WHERE citus_local_table.a = reference_table_cte.a; - +set citus.log_local_commands to on; ------------------------ ----- VIEW QUERIES ----- ------------------------ diff --git a/src/test/regress/sql/coordinator_shouldhaveshards.sql b/src/test/regress/sql/coordinator_shouldhaveshards.sql index 2c2e6231f..7c1157610 100644 --- a/src/test/regress/sql/coordinator_shouldhaveshards.sql +++ b/src/test/regress/sql/coordinator_shouldhaveshards.sql @@ -169,8 +169,8 @@ CREATE TABLE dist_table(a int); SELECT create_distributed_table('dist_table', 'a'); INSERT INTO dist_table VALUES(1); -SELECT * FROM local JOIN dist_table ON (a = x); -SELECT * FROM local JOIN dist_table ON (a = x) WHERE a = 1;; +SELECT * FROM local JOIN dist_table ON (a = x) ORDER BY 1,2,3; +SELECT * FROM local JOIN dist_table ON (a = x) WHERE a = 1 ORDER BY 1,2,3; -- intermediate results are allowed WITH cte_1 AS (SELECT * FROM dist_table ORDER BY 1 LIMIT 1) diff --git a/src/test/regress/sql/dml_recursive.sql b/src/test/regress/sql/dml_recursive.sql index 33c1b057e..cf456410b 100644 --- a/src/test/regress/sql/dml_recursive.sql +++ b/src/test/regress/sql/dml_recursive.sql @@ -281,8 +281,10 @@ SET dept = 5 FROM cte_1 WHERE distributed_table.tenant_id < cte_1.tenant_id; --- we don't support updating local table with a join with --- distributed tables +-- we support updating local table with a join with +-- distributed tables, though as the local table +-- is target here, distributed table is recursively +-- planned UPDATE local_table SET diff --git a/src/test/regress/sql/local_dist_join_mixed.sql b/src/test/regress/sql/local_dist_join_mixed.sql new file mode 100644 index 000000000..65ab89941 --- /dev/null +++ b/src/test/regress/sql/local_dist_join_mixed.sql @@ -0,0 +1,404 @@ +CREATE SCHEMA local_dist_join_mixed; +SET search_path TO local_dist_join_mixed; + + + +CREATE TABLE distributed (id bigserial PRIMARY KEY, + name text, + created_at timestamptz DEFAULT now()); +CREATE TABLE reference (id bigserial PRIMARY KEY, + title text); + +CREATE TABLE local (id bigserial PRIMARY KEY, + title text); + +-- these above restrictions brought us to the following schema +SELECT create_reference_table('reference'); +SELECT create_distributed_table('distributed', 'id'); + +INSERT INTO distributed SELECT i, i::text, now() FROM generate_series(0,100)i; +INSERT INTO reference SELECT i, i::text FROM generate_series(0,100)i; +INSERT INTO local SELECT i, i::text FROM generate_series(0,100)i; + +SET client_min_messages to DEBUG1; + +-- very simple 1-1 Joins +SELECT count(*) FROM distributed JOIN local USING (id); +SELECT count(*) FROM distributed JOIN local ON (name = title); +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text); +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int); +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +SELECT count(*) FROM distributed JOIN local USING (id) WHERE false; +SELECT count(*) FROM distributed d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 OR True; +SELECT count(*) FROM distributed d1 JOIN local ON (name::int + local.id > d1.id AND d1.id < local.title::int) WHERE d1.id = 1; +SELECT count(*) FROM distributed JOIN local ON (hashtext(name) = hashtext(title)); +SELECT hashtext(local.id::text) FROM distributed JOIN local ON (hashtext(name) = hashtext(title)) ORDER BY 1 LIMIT 4; +SELECT '' as "xxx", local.*, 'xxx' as "test" FROM distributed JOIN local ON (hashtext(name) = hashtext(title)) ORDER BY 1,2,3 LIMIT 4; +SELECT local.title, count(*) FROM distributed JOIN local USING (id) GROUP BY 1 ORDER BY 1, 2 DESC LIMIT 5; +SELECT distributed.id as id1, local.id as id2 FROM distributed JOIN local USING(id) ORDER BY distributed.id + local.id LIMIT 5; +SELECT distributed.id as id1, local.id as id2, count(*) FROM distributed JOIN local USING(id) GROUP BY distributed.id, local.id ORDER BY 1,2 LIMIT 5; + + +-- basic subqueries that cannot be pulled up +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local USING (id); +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = title); +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text); +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int); +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 AND false; +SELECT count(*) FROM (SELECT *, random() FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 OR true; + +-- pull up subqueries as they are pretty simple, local table should be recursively planned +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local USING (id); +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = title); +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text); +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int); +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 AND false; +SELECT count(*) FROM (SELECT * FROM distributed) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1 OR true; +SELECT count(*) FROM (SELECT * FROM distributed WHERE id = 2) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; +SELECT count(*) FROM (SELECT * FROM distributed WHERE false) as d1 JOIN local ON (name = d1.id::text AND d1.id < local.title::int) WHERE d1.id = 1; + +-- TEMPORARY table +CREATE TEMPORARY TABLE temp_local AS SELECT * FROM local; +SELECT count(*) FROM distributed JOIN temp_local USING (id); + +-- UNLOGGED table +CREATE UNLOGGED TABLE unlogged_local AS SELECT * FROM local; +SELECT count(*) FROM distributed JOIN unlogged_local USING (id); + +-- mat view +CREATE MATERIALIZED VIEW mat_view AS SELECT * FROM local; +SELECT count(*) FROM distributed JOIN mat_view USING (id); + +CREATE VIEW local_regular_view AS SELECT * FROM local; +CREATE VIEW dist_regular_view AS SELECT * FROM distributed; + +SELECT count(*) FROM distributed JOIN local_regular_view USING (id); +SELECT count(*) FROM local JOIN dist_regular_view USING (id); +SELECT count(*) FROM dist_regular_view JOIN local_regular_view USING (id); + + +-- join alias/table alias +SELECT COUNT(*) FROM (distributed JOIN local USING (id)) AS t(a,b,c,d) ORDER BY d,c,a,b LIMIT 3; +SELECT COUNT(*) FROM (distributed d1(x,y,y1) JOIN local l1(x,t) USING (x)) AS t(a,b,c,d) ORDER BY d,c,a,b LIMIT 3; + +-- final queries are pushdown queries +SELECT sum(d1.id + local.id) FROM distributed d1 JOIN local USING (id); +SELECT sum(d1.id + local.id) OVER (PARTITION BY d1.id) FROM distributed d1 JOIN local USING (id) ORDER BY 1 DESC LIMIT 4; +SELECT count(*) FROM distributed d1 JOIN local USING (id) LEFT JOIN distributed d2 USING (id) ORDER BY 1 DESC LIMIT 4; +SELECT count(DISTINCT d1.name::int * local.id) FROM distributed d1 JOIN local USING (id); + +-- final queries are router queries +SELECT sum(d1.id + local.id) FROM distributed d1 JOIN local USING (id) WHERE d1.id = 1; +SELECT sum(d1.id + local.id) OVER (PARTITION BY d1.id) FROM distributed d1 JOIN local USING (id) WHERE d1.id = 1 ORDER BY 1 DESC LIMIT 4; +SELECT count(*) FROM distributed d1 JOIN local USING (id) LEFT JOIN distributed d2 USING (id) WHERE d2.id = 1 ORDER BY 1 DESC LIMIT 4; + +-- final queries are pull to coordinator queries +SELECT sum(d1.id + local.id) OVER (PARTITION BY d1.id + local.id) FROM distributed d1 JOIN local USING (id) ORDER BY 1 DESC LIMIT 4; + + + +-- nested subqueries +SELECT + count(*) +FROM + (SELECT * FROM (SELECT * FROM distributed) as foo) as bar + JOIN + local + USING(id); + + +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + local + USING(id); + +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + local + USING(id); +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id); + +-- TODO: Unnecessary recursive planning for local +SELECT + count(*) +FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed LIMIT 1) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id); + +-- subqueries in WHERE clause +-- is not colocated, and the JOIN inside as well. +-- so should be recursively planned twice +SELECT + count(*) +FROM + distributed +WHERE + id > (SELECT + count(*) + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id) + ); + +-- two distributed tables are co-located and JOINed on distribution +-- key, so should be fine to pushdown +SELECT + count(*) +FROM + distributed d_upper +WHERE + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local) as foo2) as bar2 + USING(id) + ) IS NOT NULL; + +SELECT + count(*) +FROM + distributed d_upper +WHERE + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + local as foo + USING(id) + ) IS NOT NULL; + +SELECT + count(*) +FROM + distributed d_upper +WHERE d_upper.id > + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + local as foo + USING(id) + ); + +SELECT + count(*) +FROM + distributed d_upper +WHERE + (SELECT + bar.id + FROM + (SELECT *, random() FROM (SELECT *, random() FROM distributed WHERE distributed.id = d_upper.id) as foo) as bar + JOIN + (SELECT *, random() FROM (SELECT *,random() FROM local WHERE d_upper.id = id) as foo2) as bar2 + USING(id) + ) IS NOT NULL; + + + + +-- subqueries in the target list + +-- router, should work +select (SELECT local.id) FROM local, distributed WHERE distributed.id = 1 LIMIT 1; + +-- should fail +select (SELECT local.id) FROM local, distributed WHERE distributed.id != 1 LIMIT 1; + +-- currently not supported, but should work with https://github.com/citusdata/citus/pull/4360/files +SELECT + name, (SELECT id FROM local WHERE id = e.id) +FROM + distributed e +ORDER BY 1,2 LIMIT 1; + + +-- set operations + +SELECT local.* FROM distributed JOIN local USING (id) + EXCEPT +SELECT local.* FROM distributed JOIN local USING (id); + +SELECT distributed.* FROM distributed JOIN local USING (id) + EXCEPT +SELECT distributed.* FROM distributed JOIN local USING (id); + + +SELECT count(*) FROM +( + (SELECT * FROM (SELECT * FROM local) as f JOIN distributed USING (id)) + UNION ALL + (SELECT * FROM (SELECT * FROM local) as f2 JOIN distributed USING (id)) +) bar; + +SELECT count(*) FROM +( + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as fo) + UNION ALL + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as ba) +) bar; + +select count(DISTINCT id) +FROM +( + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as fo) + UNION ALL + (SELECT * FROM (SELECT distributed.* FROM local JOIN distributed USING (id)) as ba) +) bar; + +-- 25 Joins +select ' select count(*) from distributed ' || string_Agg('INNER +JOIN local u'|| x::text || ' USING (id)',' ') from +generate_Series(1,25)x; +\gexec + +select ' select count(*) from distributed ' || string_Agg('INNER +JOIN local u'|| x::text || ' ON (false)',' ') from +generate_Series(1,25)x; +\gexec + +select ' select count(*) from local ' || string_Agg('INNER +JOIN distributed u'|| x::text || ' USING (id)',' ') from +generate_Series(1,25)x; +\gexec + +select ' select count(*) from local ' || string_Agg('INNER +JOIN distributed u'|| x::text || ' ON (false)',' ') from +generate_Series(1,25)x; +\gexec + +-- lateral joins + +SELECT COUNT(*) FROM (VALUES (1), (2), (3)) as f(x) LATERAL JOIN (SELECT * FROM local WHERE id = x) as bar; + +SELECT COUNT(*) FROM local JOIN LATERAL (SELECT * FROM distributed WHERE local.id = distributed.id) as foo ON (true); +SELECT COUNT(*) FROM local JOIN LATERAL (SELECT * FROM distributed WHERE local.id > distributed.id) as foo ON (true); + +SELECT COUNT(*) FROM distributed JOIN LATERAL (SELECT * FROM local WHERE local.id = distributed.id) as foo ON (true); +SELECT COUNT(*) FROM distributed JOIN LATERAL (SELECT * FROM local WHERE local.id > distributed.id) as foo ON (true); + + + + +SELECT count(*) FROM distributed CROSS JOIN local; +SELECT count(*) FROM distributed CROSS JOIN local WHERE distributed.id = 1; + +-- w count(*) it works fine as PG ignores the inner tables +SELECT count(*) FROM distributed LEFT JOIN local USING (id); +SELECT count(*) FROM local LEFT JOIN distributed USING (id); + +SELECT id, name FROM distributed LEFT JOIN local USING (id) ORDER BY 1 LIMIT 1; +SELECT id, name FROM local LEFT JOIN distributed USING (id) ORDER BY 1 LIMIT 1; + + SELECT + foo1.id + FROM + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo9, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo8, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo7, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo6, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo5, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo4, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo3, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo2, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo10, + (SELECT local.id, local.title FROM local, distributed WHERE local.id = distributed.id ) as foo1 + WHERE + foo1.id = foo9.id AND + foo1.id = foo8.id AND + foo1.id = foo7.id AND + foo1.id = foo6.id AND + foo1.id = foo5.id AND + foo1.id = foo4.id AND + foo1.id = foo3.id AND + foo1.id = foo2.id AND + foo1.id = foo10.id AND + foo1.id = foo1.id +ORDER BY 1; + +SELECT + foo1.id +FROM + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo1, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo2, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo3, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo4, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id ) as foo5 +WHERE + foo1.id = foo4.id AND + foo1.id = foo2.id AND + foo1.id = foo3.id AND + foo1.id = foo4.id AND + foo1.id = foo5.id +ORDER BY 1; + +SELECT + foo1.id +FROM + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 1) as foo1, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 2) as foo2, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 3) as foo3, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 4) as foo4, + (SELECT local.id FROM distributed, local WHERE local.id = distributed.id AND distributed.id = 5) as foo5 +WHERE + foo1.id = foo4.id AND + foo1.id = foo2.id AND + foo1.id = foo3.id AND + foo1.id = foo4.id AND + foo1.id = foo5.id +ORDER BY 1; + +SELECT + count(*) +FROM + distributed +JOIN LATERAL + (SELECT + * + FROM + local + JOIN + distributed d2 + ON(true) + WHERE local.id = distributed.id AND d2.id = local.id) as foo +ON (true); + +SELECT local.title, local.title FROM local JOIN distributed USING(id) ORDER BY 1,2 LIMIt 1; +SELECT NULL FROM local JOIN distributed USING(id) ORDER BY 1 LIMIt 1; +SELECT distributed.name, distributed.name, local.title, local.title FROM local JOIN distributed USING(id) ORDER BY 1,2,3,4 LIMIT 1; +SELECT + COUNT(*) +FROM + local +JOIN + distributed +USING + (id) +JOIN + (SELECT id, NULL, NULL FROM distributed) foo +USING + (id); + +DROP SCHEMA local_dist_join_mixed CASCADE; diff --git a/src/test/regress/sql/local_dist_join_modifications.sql b/src/test/regress/sql/local_dist_join_modifications.sql new file mode 100644 index 000000000..843f04dbe --- /dev/null +++ b/src/test/regress/sql/local_dist_join_modifications.sql @@ -0,0 +1,372 @@ +CREATE SCHEMA local_dist_join_modifications; +SET search_path TO local_dist_join_modifications; + +CREATE TABLE postgres_table (key int, value text, value_2 jsonb); +CREATE TABLE reference_table (key int, value text, value_2 jsonb); +SELECT create_reference_table('reference_table'); +CREATE TABLE distributed_table (key int, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + +CREATE TABLE distributed_table_pkey (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_pkey', 'key'); + +CREATE TABLE distributed_table_windex (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_windex', 'key'); +CREATE UNIQUE INDEX key_index ON distributed_table_windex (key); + +CREATE TABLE distributed_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE distributed_partitioned_table_1 PARTITION OF distributed_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE distributed_partitioned_table_2 PARTITION OF distributed_partitioned_table FOR VALUES FROM (50) TO (200); +SELECT create_distributed_table('distributed_partitioned_table', 'key'); + +CREATE TABLE local_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE local_partitioned_table_1 PARTITION OF local_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE local_partitioned_table_2 PARTITION OF local_partitioned_table FOR VALUES FROM (50) TO (200); + +CREATE TABLE distributed_table_composite (key int, value text, value_2 jsonb, primary key (key, value)); +SELECT create_distributed_table('distributed_table_composite', 'key'); + +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM postgres_table; +CREATE MATERIALIZED VIEW mv2 AS SELECT * FROM distributed_table; + +-- set log messages to debug1 so that we can see which tables are recursively planned. +SET client_min_messages TO DEBUG1; + +INSERT INTO postgres_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO reference_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_windex SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_pkey SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_composite SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO local_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; + +SET citus.local_table_join_policy to 'auto'; + +-- we can support modification queries as well +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM postgres_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +ROLLBACK; + +BEGIN; +UPDATE + mv1 +SET + value = 'test' +FROM + postgres_table +WHERE + mv1.key = postgres_table.key; +ROLLBACK; + +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + mv1 +WHERE + mv1.key = postgres_table.key; +ROLLBACK; + +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + mv2 +WHERE + mv2.key = postgres_table.key; +ROLLBACK; + +-- in case of update/delete we always recursively plan +-- the tables other than target table no matter what the policy is + +SET citus.local_table_join_policy TO 'prefer-local'; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM postgres_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +ROLLBACK; + +SET citus.local_table_join_policy TO 'prefer-distributed'; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM postgres_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +UPDATE + distributed_table_pkey +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +ROLLBACK; + + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +UPDATE + distributed_table_windex +SET + value = 'test' +FROM + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +ROLLBACK; + +SET citus.local_table_join_policy TO 'auto'; + +-- modifications with multiple tables +BEGIN; +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table p1, postgres_table p2 +WHERE + distributed_table.key = p1.key AND p1.key = p2.key; +ROLLBACK; + +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + (SELECT * FROM distributed_table) d1 +WHERE + d1.key = postgres_table.key; +ROLLBACK; + +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + (SELECT * FROM distributed_table LIMIT 1) d1 +WHERE + d1.key = postgres_table.key; +ROLLBACK; + +BEGIN; +UPDATE + distributed_table +SET + value = 'test' +FROM + postgres_table p1, distributed_table d2 +WHERE + distributed_table.key = p1.key AND p1.key = d2.key; +ROLLBACK; + +-- pretty inefficient plan as it requires +-- recursive planninng of 2 distributed tables +BEGIN; +UPDATE + postgres_table +SET + value = 'test' +FROM + distributed_table d1, distributed_table d2 +WHERE + postgres_table.key = d1.key AND d1.key = d2.key; +ROLLBACK; +-- DELETE operations + +BEGIN; +SELECT COUNT(DISTINCT value) FROM postgres_table; +DELETE FROM + postgres_table +USING + distributed_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM postgres_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table; +DELETE FROM + distributed_table +USING + postgres_table +WHERE + distributed_table.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +DELETE FROM + distributed_table_pkey +USING + postgres_table +WHERE + distributed_table_pkey.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_pkey; +ROLLBACK; + +BEGIN; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +DELETE FROM + distributed_table_windex +USING + postgres_table +WHERE + distributed_table_windex.key = postgres_table.key; +SELECT COUNT(DISTINCT value) FROM distributed_table_windex; +ROLLBACK; + +DELETE FROM + mv1 +USING + postgres_table +WHERE + mv1.key = postgres_table.key; + +DELETE FROM + postgres_table +USING + mv1 +WHERE + mv1.key = postgres_table.key; + +DELETE FROM + postgres_table +USING + mv2 +WHERE + mv2.key = postgres_table.key; + + +SET client_min_messages to ERROR; +DROP SCHEMA local_dist_join_modifications CASCADE; diff --git a/src/test/regress/sql/local_table_join.sql b/src/test/regress/sql/local_table_join.sql new file mode 100644 index 000000000..49d71a551 --- /dev/null +++ b/src/test/regress/sql/local_table_join.sql @@ -0,0 +1,328 @@ +CREATE SCHEMA local_table_join; +SET search_path TO local_table_join; + +CREATE TABLE postgres_table (key int, value text, value_2 jsonb); +CREATE TABLE reference_table (key int, value text, value_2 jsonb); +SELECT create_reference_table('reference_table'); +CREATE TABLE distributed_table (key int, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + +CREATE TABLE distributed_table_pkey (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_pkey', 'key'); + +CREATE TABLE distributed_table_windex (key int primary key, value text, value_2 jsonb); +SELECT create_distributed_table('distributed_table_windex', 'key'); +CREATE UNIQUE INDEX key_index ON distributed_table_windex (key); + +CREATE TABLE distributed_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE distributed_partitioned_table_1 PARTITION OF distributed_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE distributed_partitioned_table_2 PARTITION OF distributed_partitioned_table FOR VALUES FROM (50) TO (200); +SELECT create_distributed_table('distributed_partitioned_table', 'key'); + +CREATE TABLE local_partitioned_table(key int, value text) PARTITION BY RANGE (key); +CREATE TABLE local_partitioned_table_1 PARTITION OF local_partitioned_table FOR VALUES FROM (0) TO (50); +CREATE TABLE local_partitioned_table_2 PARTITION OF local_partitioned_table FOR VALUES FROM (50) TO (200); + +CREATE TABLE distributed_table_composite (key int, value text, value_2 jsonb, primary key (key, value)); +SELECT create_distributed_table('distributed_table_composite', 'key'); + +INSERT INTO postgres_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO reference_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_windex SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_pkey SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO distributed_table_composite SELECT i, i::varchar(256) FROM generate_series(1, 100) i; +INSERT INTO local_partitioned_table SELECT i, i::varchar(256) FROM generate_series(1, 100) i; + +CREATE FUNCTION fake_fdw_handler() +RETURNS fdw_handler +AS 'citus' +LANGUAGE C STRICT; +CREATE FOREIGN DATA WRAPPER fake_fdw_1 HANDLER fake_fdw_handler; +CREATE SERVER fake_fdw_server_1 FOREIGN DATA WRAPPER fake_fdw_1; + +CREATE FOREIGN TABLE foreign_table ( + key int, + value text +) SERVER fake_fdw_server_1 OPTIONS (encoding 'utf-8', compression 'true'); + +CREATE MATERIALIZED VIEW mv1 AS SELECT * FROM postgres_table; +CREATE MATERIALIZED VIEW mv2 AS SELECT * FROM distributed_table; + + +SET client_min_messages TO DEBUG1; + + +-- the user doesn't allow local / distributed table joinn +SET citus.local_table_join_policy TO 'never'; +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key); +SELECT count(*) FROM postgres_table JOIN reference_table USING(key); + +-- the user prefers local table recursively planned +SET citus.local_table_join_policy TO 'prefer-local'; +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key); +SELECT count(*) FROM postgres_table JOIN reference_table USING(key); + + +-- the user prefers distributed table recursively planned +SET citus.local_table_join_policy TO 'prefer-distributed'; +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key); +SELECT count(*) FROM postgres_table JOIN reference_table USING(key); + +-- auto tests + +-- switch back to the default policy, which is auto +SET citus.local_table_join_policy to 'auto'; + +-- on the auto mode, the local tables should be recursively planned +-- unless a unique index exists in a column for distributed table +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key); +SELECT count(*) FROM reference_table JOIN postgres_table USING(key); +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) JOIN reference_table USING (key); + +-- partititoned local tables should work as well +SELECT count(*) FROM distributed_table JOIN local_partitioned_table USING(key); +SELECT count(*) FROM reference_table JOIN local_partitioned_table USING(key); +SELECT count(*) FROM distributed_table JOIN local_partitioned_table USING(key) JOIN reference_table USING (key); + +-- materialized views should work too +SELECT count(*) FROM distributed_table JOIN mv1 USING(key); +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv1 USING(key); +SELECT count(*) FROM reference_table JOIN mv1 USING(key); +SELECT count(*) FROM distributed_table JOIN mv1 USING(key) JOIN reference_table USING (key); +SELECT count(*) FROM distributed_table JOIN mv2 USING(key); +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv2 USING(key); +SELECT count(*) FROM reference_table JOIN mv2 USING(key); +SELECT count(*) FROM distributed_table JOIN mv2 USING(key) JOIN reference_table USING (key); + +-- foreign tables should work too +SELECT count(*) FROM foreign_table JOIN distributed_table USING(key); + +-- partitioned tables should work as well +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key); +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) WHERE distributed_partitioned_table.key = 10; +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) JOIN reference_table USING (key); + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key); +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) WHERE distributed_partitioned_table.key = 10; +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) JOIN reference_table USING (key); + +-- similar tests in transaction block should work fine + +BEGIN; +-- materialized views should work too +SELECT count(*) FROM distributed_table JOIN mv1 USING(key); +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv1 USING(key); +SELECT count(*) FROM reference_table JOIN mv1 USING(key); +SELECT count(*) FROM distributed_table JOIN mv1 USING(key) JOIN reference_table USING (key); +SELECT count(*) FROM distributed_table JOIN mv2 USING(key); +SELECT count(*) FROM (SELECT * FROM distributed_table) d1 JOIN mv2 USING(key); +SELECT count(*) FROM reference_table JOIN mv2 USING(key); +SELECT count(*) FROM distributed_table JOIN mv2 USING(key) JOIN reference_table USING (key); + +-- foreign tables should work too +SELECT count(*) FROM foreign_table JOIN distributed_table USING(key); + +-- partitioned tables should work as well +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key); +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) WHERE distributed_partitioned_table.key = 10; +SELECT count(*) FROM distributed_partitioned_table JOIN postgres_table USING(key) JOIN reference_table USING (key); + +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key); +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) WHERE distributed_partitioned_table.key = 10; +SELECT count(*) FROM distributed_partitioned_table JOIN local_partitioned_table USING(key) JOIN reference_table USING (key); +ROLLBACK; + +-- the conversions should be independent from the order of table entries in the query +SELECT COUNT(*) FROM postgres_table join distributed_table_pkey using(key) join local_partitioned_table using(key) join distributed_table using(key) where distributed_table_pkey.key = 5; +SELECT COUNT(*) FROM postgres_table join local_partitioned_table using(key) join distributed_table_pkey using(key) join distributed_table using(key) where distributed_table_pkey.key = 5; +SELECT COUNT(*) FROM postgres_table join distributed_table using(key) join local_partitioned_table using(key) join distributed_table_pkey using(key) where distributed_table_pkey.key = 5; +SELECT COUNT(*) FROM distributed_table_pkey join distributed_table using(key) join postgres_table using(key) join local_partitioned_table using(key) where distributed_table_pkey.key = 5; + +SELECT count(*) FROM (SELECT *, random() FROM distributed_table) as d1 JOIN postgres_table ON (postgres_table.key = d1.key AND d1.key < postgres_table.key) WHERE d1.key = 1 AND false; +SELECT count(*) FROM (SELECT *, random() FROM distributed_table_pkey) as d1 JOIN postgres_table ON (postgres_table.key = d1.key AND d1.key < postgres_table.key) WHERE d1.key = 1 AND false; +SELECT count(*) FROM (SELECT *, random() FROM distributed_partitioned_table) as d1 JOIN postgres_table ON (postgres_table.key = d1.key AND d1.key < postgres_table.key) WHERE d1.key = 1 AND false; +SELECT count(*) FROM (SELECT *, random() FROM distributed_partitioned_table) as d1 JOIN postgres_table ON (postgres_table.key::int = d1.key::int AND d1.key < postgres_table.key) WHERE d1.key::int = 1 AND false; + +-- different column names +SELECT a FROM postgres_table foo (a,b,c) JOIN distributed_table ON (distributed_table.key = foo.a) ORDER BY 1 LIMIT 1; + + +-- We will plan postgres table as the index is on key,value not just key +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key = 10; +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key = 10 OR distributed_table_composite.key = 20; +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key > 10 AND distributed_table_composite.value = 'text'; +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) WHERE distributed_table_composite.key = 10 AND distributed_table_composite.value = 'text'; +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 OR distributed_table_composite.key = 20) + AND (distributed_table_composite.value = 'text' OR distributed_table_composite.value = 'text'); +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 OR distributed_table_composite.value = 'text') + AND (distributed_table_composite.value = 'text' OR distributed_table_composite.key = 30); +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 AND distributed_table_composite.value = 'text') + OR (distributed_table_composite.value = 'text' AND distributed_table_composite.key = 30); +SELECT count(*) FROM distributed_table_composite JOIN postgres_table USING(key) + WHERE (distributed_table_composite.key > 10 AND distributed_table_composite.key = 20) + OR (distributed_table_composite.value = 'text' AND distributed_table_composite.value = 'text'); + +-- Known bug: #4269 +SELECT count(*) FROM distributed_table_composite foo(a,b,c) JOIN postgres_table ON(foo.a > 1) + WHERE foo.a IN (SELECT COUNT(*) FROM local_partitioned_table) AND (foo.a = 10 OR foo.b ='text'); + +-- a unique index on key so dist table should be recursively planned +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey USING(key); +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey USING(value); +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON postgres_table.key = distributed_table_pkey.key; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10; +-- it should favor distributed table only if it has equality on the unique column +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key > 10; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key < 10; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 AND distributed_table_pkey.key > 10 ; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 AND distributed_table_pkey.key > 10 ; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 AND distributed_table_pkey.key > 10 AND postgres_table.key = 5; + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key > 10; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key = 20; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key = 20 OR distributed_table_pkey.key = 30; +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR distributed_table_pkey.key = ( + SELECT count(*) FROM distributed_table_pkey +); +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key = 5 and distributed_table_pkey.key > 15); +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key > 10 and distributed_table_pkey.key > 15); +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key > 10 and distributed_table_pkey.value = 'notext'); +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON distributed_table_pkey.key = 10 OR (distributed_table_pkey.key = 10 and distributed_table_pkey.value = 'notext'); + +SELECT count(*) FROM postgres_table JOIN distributed_table_pkey ON postgres_table.key = 10; + + +select count(*) FROM postgres_table JOIN (SELECT a.key,random() FROM distributed_table a JOIN distributed_table b USING(key)) as foo USING(key); +select count(*) FROM (SELECT a.key, random() FROM distributed_table a JOIN distributed_table b USING(key)) as foo JOIN postgres_table USING(key); + +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM distributed_table) d1 USING(key); +-- since this is already router plannable, we don't recursively plan the postgres table +SELECT count(*) FROM postgres_table JOIN (SELECT * FROM distributed_table LIMIT 1) d1 USING(key); + +-- a unique index on key so dist table should be recursively planned +SELECT count(*) FROM postgres_table JOIN distributed_table_windex USING(key); +SELECT count(*) FROM postgres_table JOIN distributed_table_windex USING(value); +SELECT count(*) FROM postgres_table JOIN distributed_table_windex ON postgres_table.key = distributed_table_windex.key; +SELECT count(*) FROM postgres_table JOIN distributed_table_windex ON distributed_table_windex.key = 10; + +-- no unique index on value so local table should be recursively planned. +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.value = 'test'; + +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.key = 1; + + +-- if both local and distributed tables have a filter, we prefer local unless distributed table has unique indexes on any equality filter +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.value = 'test' AND postgres_table.value = 'test'; +SELECT count(*) FROM distributed_table JOIN postgres_table USING(key) WHERE distributed_table.value = 'test' OR postgres_table.value = 'test'; + + +-- multiple local/distributed tables +-- only local tables are recursively planned +SELECT count(*) FROM distributed_table d1 JOIN postgres_table p1 USING(key) JOIN distributed_table d2 USING(key) JOIN postgres_table p2 USING(key); + + +SELECT + count(*) +FROM + distributed_table d1 JOIN postgres_table p1 USING(key) JOIN distributed_table d2 USING(key) JOIN postgres_table p2 USING(key) +WHERE + d1.value = '1'; + +-- if the filter is on the JOIN key, we can recursively plan the local +-- tables as filters are pushed down to the local tables +SELECT + count(*) +FROM + distributed_table d1 JOIN postgres_table p1 USING(key) JOIN distributed_table d2 USING(key) JOIN postgres_table p2 USING(key) +WHERE + d1.key = 1; + +CREATE view loc_view AS SELECT * FROM postgres_table WHERE key > 0; +UPDATE loc_view SET key = (SELECT COUNT(*) FROM distributed_table); + +SELECT count(*) +FROM + (SELECT * FROM (SELECT * FROM distributed_table) d1) d2 +JOIN postgres_table +USING(key); + +-- will error as we don't support complex joins +SELECT COUNT(*) FROM postgres_table, distributed_table d1, distributed_table d2 WHERE d1.value = d2.value; + +-- This will error because router planner will think that since reference tables have a single +-- shard, it contains only a single task for modify. However, updating a reference tables +-- will require multiple tasks. So requires some rewrite in router planner. +UPDATE reference_table SET key = 1 FROM postgres_table WHERE postgres_table.key = 10; +UPDATE reference_table SET key = 1 FROM (SELECT * FROM postgres_table) l WHERE l.key = 10; + + +SELECT count(*) FROM postgres_table JOIN distributed_table USING(key) WHERE FALSE; + +SELECT count(*) FROM (SELECT * FROM distributed_table JOIN postgres_table USING(key) WHERE false) foo JOIN local_partitioned_table USING(key); + +WITH dist_cte AS (SELECT * FROM distributed_table_pkey WHERE key = 5) +SELECT COUNT(*) FROM dist_cte JOIN postgres_table USING(key) WHERE dist_cte.key = 5; + +SELECT COUNT(*) FROM postgres_table JOIN distributed_table_pkey USING(key) + WHERE (distributed_table_pkey.key IN (SELECT COUNT(*) AS count FROM postgres_table JOIN distributed_table USING(key)) ); + +-- PREPARED statements +PREPARE local_dist_table_join_select(int) AS SELECT COUNT(*) FROM distributed_table_pkey JOIN postgres_table USING(key) WHERE distributed_table_pkey.key = $1; + +EXECUTE local_dist_table_join_select(10); +EXECUTE local_dist_table_join_select(10); +EXECUTE local_dist_table_join_select(10); +EXECUTE local_dist_table_join_select(10); +EXECUTE local_dist_table_join_select(10); +EXECUTE local_dist_table_join_select(10); + +PREPARE local_dist_table_join_update(int) AS UPDATE postgres_table SET key = 5 FROM distributed_table_pkey WHERE distributed_table_pkey.key = $1; + +EXECUTE local_dist_table_join_update(20); +EXECUTE local_dist_table_join_update(20); +EXECUTE local_dist_table_join_update(20); +EXECUTE local_dist_table_join_update(20); +EXECUTE local_dist_table_join_update(20); +EXECUTE local_dist_table_join_update(20); + +PREPARE local_dist_table_join_subquery(int) AS SELECT COUNT(*) FROM postgres_table JOIN (SELECT * FROM distributed_table_pkey JOIN local_partitioned_table USING(key) WHERE distributed_table_pkey.key = $1) foo USING(key); + +EXECUTE local_dist_table_join_subquery(5); +EXECUTE local_dist_table_join_subquery(5); +EXECUTE local_dist_table_join_subquery(5); +EXECUTE local_dist_table_join_subquery(5); +EXECUTE local_dist_table_join_subquery(5); +EXECUTE local_dist_table_join_subquery(5); + +PREPARE local_dist_table_join_filters(int) AS SELECT COUNT(*) FROM local_partitioned_table JOIN distributed_table_composite USING(key) + WHERE( + distributed_table_composite.key = $1 OR + distributed_table_composite.key = 20 OR + (distributed_table_composite.key = 10 AND distributed_table_composite.key > 0) OR + distributed_table_composite.value = 'text' + ); + +EXECUTE local_dist_table_join_filters(20); +EXECUTE local_dist_table_join_filters(20); +EXECUTE local_dist_table_join_filters(20); +EXECUTE local_dist_table_join_filters(20); +EXECUTE local_dist_table_join_filters(20); +EXECUTE local_dist_table_join_filters(20); + +CREATE TABLE local (key1 int, key2 int, key3 int); +INSERT INTO local VALUES (1,2,3); +ALTER TABLE local DROP column key2; +-- verify we ignore dropped columns +SELECT COUNT(*) FROM local JOIN distributed_table ON(key1 = key); +SELECT * FROM local JOIN distributed_table ON(key1 = key) ORDER BY 1 LIMIT 1; + + +RESET client_min_messages; +\set VERBOSITY terse +DROP SCHEMA local_table_join CASCADE; diff --git a/src/test/regress/sql/mixed_relkind_tests.sql b/src/test/regress/sql/mixed_relkind_tests.sql index 795307d65..6f935c79d 100644 --- a/src/test/regress/sql/mixed_relkind_tests.sql +++ b/src/test/regress/sql/mixed_relkind_tests.sql @@ -101,15 +101,17 @@ INSERT INTO partitioned_distributed_table SELECT * FROM partitioned_distributed_ $$); SET client_min_messages TO DEBUG1; --- should fail -SELECT * FROM partitioned_postgres_local_table JOIN distributed_table ON (true); -SELECT * FROM partitioned_postgres_local_table JOIN partitioned_distributed_table ON (true); -SELECT * FROM distributed_table JOIN partitioned_postgres_local_table ON (true); -SELECT * FROM reference_table LEFT JOIN partitioned_distributed_table ON true; +SELECT COUNT(*) FROM partitioned_postgres_local_table JOIN distributed_table ON (true); +SELECT COUNT(*) FROM partitioned_postgres_local_table JOIN partitioned_distributed_table ON (true); +SELECT COUNT(*) FROM distributed_table JOIN partitioned_postgres_local_table ON (true); INSERT INTO partitioned_distributed_table SELECT foo.* FROM partitioned_distributed_table AS foo JOIN citus_local_table ON (true); INSERT INTO partitioned_distributed_table SELECT foo.* FROM distributed_table AS foo JOIN citus_local_table ON (true); INSERT INTO distributed_table SELECT foo.a FROM partitioned_distributed_table AS foo JOIN citus_local_table ON (true); +-- should fail +SELECT COUNT(*) FROM reference_table LEFT JOIN partitioned_distributed_table ON true; + + -- non-colocated subquery should work SELECT COUNT(*) FROM (SELECT *, random() FROM partitioned_distributed_table) AS foo, @@ -121,11 +123,11 @@ SELECT COUNT(*) FROM (SELECT *, random() FROM foreign_distributed_table) AS bar WHERE foo.a = bar.b; --- should fail UPDATE partitioned_distributed_table SET b = foo.a FROM citus_local_table AS foo; UPDATE partitioned_distributed_table SET b = foo.a FROM postgres_local_table AS foo; UPDATE partitioned_distributed_table SET a = foo.a FROM postgres_local_table AS foo WHERE foo.a = partitioned_distributed_table.a; UPDATE partitioned_distributed_table SET a = foo.a FROM citus_local_table AS foo WHERE foo.a = partitioned_distributed_table.a; +-- should fail UPDATE partitioned_distributed_table SET a = foo.a FROM mat_view_on_part_dist AS foo WHERE foo.a = partitioned_distributed_table.a; UPDATE partitioned_distributed_table SET a = foo.a FROM partitioned_distributed_table AS foo WHERE foo.a < partitioned_distributed_table.a; UPDATE partitioned_distributed_table SET a = foo.a FROM distributed_table AS foo WHERE foo.a < partitioned_distributed_table.a; diff --git a/src/test/regress/sql/multi_modifications.sql b/src/test/regress/sql/multi_modifications.sql index 3076aeeeb..0b272ca01 100644 --- a/src/test/regress/sql/multi_modifications.sql +++ b/src/test/regress/sql/multi_modifications.sql @@ -327,7 +327,7 @@ UPDATE limit_orders SET id = 246 WHERE id = 246; UPDATE limit_orders SET id = 246 WHERE id = 246 AND symbol = 'GM'; UPDATE limit_orders SET id = limit_orders.id WHERE id = 246; --- UPDATEs with a FROM clause are unsupported +-- UPDATEs with a FROM clause are supported even with local tables UPDATE limit_orders SET limit_price = 0.00 FROM bidders WHERE limit_orders.id = 246 AND limit_orders.bidder_id = bidders.id AND diff --git a/src/test/regress/sql/multi_shard_update_delete.sql b/src/test/regress/sql/multi_shard_update_delete.sql index c0cbdd791..41a102829 100644 --- a/src/test/regress/sql/multi_shard_update_delete.sql +++ b/src/test/regress/sql/multi_shard_update_delete.sql @@ -609,7 +609,7 @@ WHERE users_test_table.user_id = subquery.user_id; -- Make following tests consistent UPDATE users_test_table SET value_2 = 0; --- Local tables are not supported +-- Joins with tables not supported UPDATE users_test_table SET value_2 = 5 FROM events_test_table_local diff --git a/src/test/regress/sql/multi_simple_queries.sql b/src/test/regress/sql/multi_simple_queries.sql index a95f93f68..f987518e1 100644 --- a/src/test/regress/sql/multi_simple_queries.sql +++ b/src/test/regress/sql/multi_simple_queries.sql @@ -146,10 +146,10 @@ ORDER BY articles.id; SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard a2 WHERE a.id = a2.id LIMIT 1) AS special_price FROM articles a; --- joins are not supported between local and distributed tables +-- joins are supported between local and distributed tables SELECT title, authors.name FROM authors, articles WHERE authors.id = articles.author_id; --- inner joins are not supported (I think) +-- inner joins are supported SELECT * FROM (articles INNER JOIN authors ON articles.id = authors.id); -- test use of EXECUTE statements within plpgsql diff --git a/src/test/regress/sql/recursive_relation_planning_restriction_pushdown.sql b/src/test/regress/sql/recursive_relation_planning_restriction_pushdown.sql new file mode 100644 index 000000000..ebc1c5ccc --- /dev/null +++ b/src/test/regress/sql/recursive_relation_planning_restriction_pushdown.sql @@ -0,0 +1,266 @@ +---------------------------------------------------- +-- recursive_relation_planning_restirction_pushdown +-- In this test file, we mosly test whether Citus +-- can successfully pushdown filters to the subquery +-- that is being recursively planned. This is done +-- for all types of JOINs +---------------------------------------------------- + +-- all the queries in this file have the +-- same tables/subqueries combination as below +-- because this test aims to hold the query planning +-- steady, but mostly ensure that filters are handled +-- properly. Note that local is the relation that is +-- recursively planned throughout the file + +CREATE SCHEMA push_down_filters; +SET search_path TO push_down_filters; + +CREATE TABLE local_table (key int, value int, time timestamptz); + +CREATE TABLE distributed_table (key int, value int, metadata jsonb); +SELECT create_distributed_table('distributed_table', 'key'); + +CREATE TYPE new_type AS (n int, m text); +CREATE TABLE local_table_type (key int, value new_type, value_2 jsonb); + +CREATE TABLE distributed_table_type (key int, value new_type, value_2 jsonb); +SELECT create_distributed_table('distributed_table_type', 'key'); + +-- Setting the debug level so that filters can be observed +SET client_min_messages TO DEBUG1; + +-- for the purposes of these tests, we always want to recursively +-- plan local tables. +SET citus.local_table_join_policy TO "prefer-local"; + + +-- there are no filters, hence cannot pushdown any filters +SELECT count(*) +FROM distributed_table u1 +JOIN distributed_table u2 USING(key) +JOIN local_table USING (key); + +-- composite types can be pushed down +SELECT count(*) +FROM distributed_table d1 +JOIN local_table_type d2 using(key) +WHERE d2.value = (83, 'citus8.3')::new_type; + +-- composite types can be pushed down +SELECT count(*) +FROM distributed_table d1 +JOIN local_table_type d2 using(key) +WHERE d2.value = (83, 'citus8.3')::new_type +AND d2.key = 10; + +-- join on a composite type works +SELECT count(*) +FROM distributed_table_type d1 +JOIN local_table_type d2 USING(value); + +-- scalar array expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING (key) +WHERE u2.key > ANY(ARRAY[2, 1, 6]); + +-- array operators on the table can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(key) +WHERE ARRAY[u2.key, u2.value] @> (ARRAY[2, 3]); + + +-- array operators on different tables cannot be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE ARRAY[u2.value, u1.value] @> (ARRAY[2, 3]); + +-- coerced expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value/2.0 > 2)::int::bool::text::bool; + + +-- case expression on a single table can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (CASE WHEN u2.value > 3 THEN u2.value > 2 ELSE false END); + +-- case expression multiple tables cannot be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (CASE WHEN u1.value > 4000 THEN u2.value / 100 > 1 ELSE false END); + +-- coalesce expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE COALESCE((u2.key/5.0)::int::bool, false); + +-- nullif expressions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE NULLIF((u2.value/5.0)::int::bool, false); + +-- null test can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value IS NOT NULL; + +-- functions can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE isfinite(u2.time); + +-- functions with multiple tables cannot be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE int4smaller(u2.value, u1.value) = 55; + +-- functions with multiple columns from the same tables can be pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE int4smaller(u2.key, u2.value) = u2.key; + +-- row expressions can be pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE row(u2.value, 2, 3) > row(u2.value, 2, 3); + + + +-- multiple expression from the same table can be pushed down together +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) + WHERE + (u2.key/1.0)::int::bool::text::bool AND + CASE WHEN u2.key > 4000 THEN u2.value / 100 > 1 ELSE false END AND + COALESCE((u2.key/50000)::bool, false) AND + NULLIF((u2.value/50000)::int::bool, false) AND + isfinite(u2.time) AND + u2.value IS DISTINCT FROM 50040 AND + row(u2.value, 2, 3) > row(2000, 2, 3); + + +-- subqueries filters are not pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > + (SELECT avg(key) + FROM distributed_table); + +-- even subqueries with constant values are not pushdowned +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > (SELECT 5); + +-- filters involving multiple tables aren't pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value * u1.key > 25; + + +-- filter on other tables can only be pushdown +-- as long as they are equality filters on the +-- joining column +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u1.value = 3; + + +-- but not when the filter is gt, lt or any other thing +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u1.value > 3; + + +-- when the filter is on another column than the +-- join column, that's obviously not pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u1.key = 3; + + +-- or filters on the same table is pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > 4 OR u2.value = 4; + +-- and filters on the same table is pushdown +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE u2.value > 2 and u2.time IS NULL; + + +-- filters on different tables are pushdown +-- only the ones that are not ANDed +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u2.value IS NULL) AND (u2.key > 4 OR u1.key > 3); + +-- filters on different tables are pushdown +-- only the ones that are not ANDed +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u2.value IS NULL) OR (u2.key > 4 OR u1.key > 3); + + +-- filters on different tables are pushdown +-- only the ones that are not ANDed +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u2.value IS NULL) AND (u2.key > 4 OR u1.key > 3); + +-- but volatile functions are not pushed down +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 OR u1.value IS NULL) AND (u2.key = 10000 * random() OR u1.key > 3); + +-- constant results should be pushed down, but not supported yet +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +WHERE (u2.value > 2 AND false); + +-- we can still pushdown WHERE false +-- even if it is a LATERAL join +SELECT count(*) +FROM distributed_table u1 +JOIN local_table u2 USING(value) +JOIN LATERAL + (SELECT value, + random() + FROM distributed_table + WHERE u2.value = 15) AS u3 USING (value) +WHERE (u2.value > 2 + AND FALSE); + +\set VERBOSITY terse +RESET client_min_messages; +DROP SCHEMA push_down_filters CASCADE; + diff --git a/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql b/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql index 47f8df9a6..0587ebfaa 100644 --- a/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql +++ b/src/test/regress/sql/replicate_reference_tables_to_coordinator.sql @@ -145,7 +145,7 @@ SELECT test_reference_local_join_func(); -- CTEs are allowed WITH ins AS (INSERT INTO numbers VALUES (1) RETURNING *) -SELECT * FROM numbers, local_table; +SELECT * FROM numbers, local_table ORDER BY 1,2; WITH t AS (SELECT *, my_volatile_fn() x FROM numbers FOR UPDATE) SELECT * FROM numbers, local_table diff --git a/src/test/regress/sql/subqueries_not_supported.sql b/src/test/regress/sql/subqueries_not_supported.sql index a91c9a097..b08fe254c 100644 --- a/src/test/regress/sql/subqueries_not_supported.sql +++ b/src/test/regress/sql/subqueries_not_supported.sql @@ -9,21 +9,9 @@ SET client_min_messages TO DEBUG1; CREATE TABLE users_table_local AS SELECT * FROM users_table; --- we don't support subqueries with local tables when they are not leaf queries -SELECT - * -FROM - ( - SELECT - users_table_local.user_id - FROM - users_table_local, (SELECT user_id FROM events_table) as evs - WHERE users_table_local.user_id = evs.user_id - ) as foo; - RESET client_min_messages; -- we don't support subqueries with local tables when they are not leaf queries -SELECT user_id FROM users_table WHERE user_id IN +SELECT COUNT(user_id) FROM users_table WHERE user_id IN (SELECT user_id FROM diff --git a/src/test/regress/sql/with_executors.sql b/src/test/regress/sql/with_executors.sql index 231df8e0c..dec5fcd9b 100644 --- a/src/test/regress/sql/with_executors.sql +++ b/src/test/regress/sql/with_executors.sql @@ -231,7 +231,7 @@ WITH cte AS ( ) SELECT min(user_id) FROM cte JOIN local_table ON (user_id = id); --- not if there are no distributed tables +-- even if there are no distributed tables WITH cte AS ( SELECT user_id FROM users_table ) @@ -243,8 +243,8 @@ WITH cte AS ( ) SELECT min(user_id) FROM cte JOIN local_table ON (user_id = id) JOIN (SELECT * FROM events_table OFFSET 0) e USING (user_id); --- joins between local and reference tables not allowed --- since the coordinator is not in the metadata at this stage +-- joins between local and reference tables are allowed +-- even when the coordinator is not in the metadata at this stage WITH cte AS ( SELECT user_id FROM users_table ) diff --git a/src/test/regress/sql/with_modifying.sql b/src/test/regress/sql/with_modifying.sql index 5d6999264..058ff8e41 100644 --- a/src/test/regress/sql/with_modifying.sql +++ b/src/test/regress/sql/with_modifying.sql @@ -412,7 +412,8 @@ raw_data AS ( ) SELECT * FROM raw_data ORDER BY val; --- Test that local tables are barred +-- Test that local tables are can be updated +-- selecting from distributed tables UPDATE local_table lt SET val = mt.val FROM modify_table mt WHERE mt.id = lt.id;