mirror of https://github.com/citusdata/citus.git
Merge pull request #4358 from citusdata/local_distributed_table_join
Support local/citus local distributed/subquery joinscte_inline_improve
commit
26284bf2a1
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -789,7 +789,8 @@ RouterModifyTaskForShardInterval(Query *originalQuery,
|
|||
&relationShardList,
|
||||
&prunedShardIntervalListList,
|
||||
replacePrunedQueryWithDummy,
|
||||
&multiShardModifyQuery, NULL);
|
||||
&multiShardModifyQuery, NULL,
|
||||
false);
|
||||
|
||||
Assert(!multiShardModifyQuery);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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."),
|
||||
|
|
|
@ -327,6 +327,7 @@ CopyNodeTask(COPYFUNC_ARGS)
|
|||
COPY_SCALAR_FIELD(fetchedExplainAnalyzePlacementIndex);
|
||||
COPY_STRING_FIELD(fetchedExplainAnalyzePlan);
|
||||
COPY_SCALAR_FIELD(fetchedExplainAnalyzeExecutionDuration);
|
||||
COPY_SCALAR_FIELD(isLocalTableModification);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -540,6 +540,7 @@ OutTask(OUTFUNC_ARGS)
|
|||
WRITE_INT_FIELD(fetchedExplainAnalyzePlacementIndex);
|
||||
WRITE_STRING_FIELD(fetchedExplainAnalyzePlan);
|
||||
WRITE_FLOAT_FIELD(fetchedExplainAnalyzeExecutionDuration, "%.2f");
|
||||
WRITE_BOOL_FIELD(isLocalTableModification);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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 */
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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_ */
|
||||
|
|
|
@ -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;
|
|
@ -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 --
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -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 --
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -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
|
@ -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
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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';
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 (
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
|
@ -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 -----
|
||||
------------------------
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
)
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
Loading…
Reference in New Issue