Merge pull request #4358 from citusdata/local_distributed_table_join

Support local/citus local distributed/subquery joins
cte_inline_improve
SaitTalhaNisanci 2020-12-15 18:29:39 +03:00 committed by GitHub
commit 26284bf2a1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
67 changed files with 8099 additions and 412 deletions

View File

@ -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);
}

View File

@ -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);

View File

@ -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;

View File

@ -789,7 +789,8 @@ RouterModifyTaskForShardInterval(Query *originalQuery,
&relationShardList,
&prunedShardIntervalListList,
replacePrunedQueryWithDummy,
&multiShardModifyQuery, NULL);
&multiShardModifyQuery, NULL,
false);
Assert(!multiShardModifyQuery);

View File

@ -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;
}

View File

@ -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);

View File

@ -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,

View File

@ -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 */

View File

@ -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;

View File

@ -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

View File

@ -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)

View File

@ -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)
{

View File

@ -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;

View File

@ -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."),

View File

@ -327,6 +327,7 @@ CopyNodeTask(COPYFUNC_ARGS)
COPY_SCALAR_FIELD(fetchedExplainAnalyzePlacementIndex);
COPY_STRING_FIELD(fetchedExplainAnalyzePlan);
COPY_SCALAR_FIELD(fetchedExplainAnalyzeExecutionDuration);
COPY_SCALAR_FIELD(isLocalTableModification);
}

View File

@ -540,6 +540,7 @@ OutTask(OUTFUNC_ARGS)
WRITE_INT_FIELD(fetchedExplainAnalyzePlacementIndex);
WRITE_STRING_FIELD(fetchedExplainAnalyzePlan);
WRITE_FLOAT_FIELD(fetchedExplainAnalyzeExecutionDuration, "%.2f");
WRITE_BOOL_FIELD(isLocalTableModification);
}

View File

@ -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);

View File

@ -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 */

View File

@ -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);

View File

@ -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,

View File

@ -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);

View File

@ -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 */

View File

@ -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

View File

@ -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 */

View File

@ -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 */

View File

@ -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 */

View File

@ -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_ */

View File

@ -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;

View File

@ -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 --
---------------------------------------------------------------------

View File

@ -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 --
---------------------------------------------------------------------

View File

@ -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
---------------------------------------------------------------------

View File

@ -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

File diff suppressed because one or more lines are too long

View File

@ -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;

File diff suppressed because it is too large Load Diff

View File

@ -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);

View File

@ -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

View File

@ -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;

View File

@ -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';

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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)

View File

@ -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

View File

@ -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 (

View File

@ -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;

View File

@ -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.

View File

@ -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;

View File

@ -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 -----
------------------------

View File

@ -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)

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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

View File

@ -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
)

View File

@ -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;