mirror of https://github.com/citusdata/citus.git
Merge branch 'main' into ihalatci-extension-compat-test-report
commit
dbf0e647a9
|
|
@ -1,4 +1,4 @@
|
|||
black==23.11.0
|
||||
black==24.3.0
|
||||
click==8.1.7
|
||||
isort==5.12.0
|
||||
mypy-extensions==1.0.0
|
||||
|
|
|
|||
|
|
@ -346,12 +346,12 @@ CdcIsReferenceTableViaCatalog(Oid relationId)
|
|||
return false;
|
||||
}
|
||||
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock);
|
||||
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
|
||||
if (isNullArray[Anum_pg_dist_partition_partmethod - 1] ||
|
||||
|
|
@ -363,6 +363,8 @@ CdcIsReferenceTableViaCatalog(Oid relationId)
|
|||
*/
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
@ -374,6 +376,8 @@ CdcIsReferenceTableViaCatalog(Oid relationId)
|
|||
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
/*
|
||||
* A table is a reference table when its partition method is 'none'
|
||||
|
|
|
|||
|
|
@ -82,6 +82,7 @@ static void AppendStorageParametersToString(StringInfo stringBuffer,
|
|||
List *optionList);
|
||||
static const char * convert_aclright_to_string(int aclright);
|
||||
static void simple_quote_literal(StringInfo buf, const char *val);
|
||||
static SubscriptingRef * TargetEntryExprFindSubsRef(Expr *expr);
|
||||
static void AddVacuumParams(ReindexStmt *reindexStmt, StringInfo buffer);
|
||||
static void process_acl_items(Acl *acl, const char *relationName,
|
||||
const char *attributeName, List **defs);
|
||||
|
|
@ -1715,3 +1716,255 @@ RoleSpecString(RoleSpec *spec, bool withQuoteIdentifier)
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Recursively search an expression for a Param and return its paramid
|
||||
* Intended for indirection management: UPDATE SET () = (SELECT )
|
||||
* Does not cover all options but those supported by Citus.
|
||||
*/
|
||||
static int
|
||||
GetParamId(Node *expr)
|
||||
{
|
||||
int paramid = 0;
|
||||
|
||||
if (expr == NULL)
|
||||
{
|
||||
return paramid;
|
||||
}
|
||||
|
||||
/* If it's a Param, return its attnum */
|
||||
if (IsA(expr, Param))
|
||||
{
|
||||
Param *param = (Param *) expr;
|
||||
paramid = param->paramid;
|
||||
}
|
||||
/* If it's a FuncExpr, search in arguments */
|
||||
else if (IsA(expr, FuncExpr))
|
||||
{
|
||||
FuncExpr *func = (FuncExpr *) expr;
|
||||
ListCell *lc;
|
||||
|
||||
foreach(lc, func->args)
|
||||
{
|
||||
paramid = GetParamId((Node *) lfirst(lc));
|
||||
if (paramid != 0)
|
||||
{
|
||||
break; /* Stop at the first valid paramid */
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return paramid;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* list_sort comparator to sort target list by paramid (in MULTIEXPR)
|
||||
* Intended for indirection management: UPDATE SET () = (SELECT )
|
||||
*/
|
||||
static int
|
||||
target_list_cmp(const ListCell *a, const ListCell *b)
|
||||
{
|
||||
TargetEntry *tleA = lfirst(a);
|
||||
TargetEntry *tleB = lfirst(b);
|
||||
|
||||
/*
|
||||
* Deal with resjunk entries; sublinks are marked resjunk and
|
||||
* are placed at the end of the target list so this logic
|
||||
* ensures they stay grouped at the end of the target list:
|
||||
*/
|
||||
if (tleA->resjunk || tleB->resjunk)
|
||||
{
|
||||
return tleA->resjunk - tleB->resjunk;
|
||||
}
|
||||
|
||||
int la = GetParamId((Node *) tleA->expr);
|
||||
int lb = GetParamId((Node *) tleB->expr);
|
||||
|
||||
/*
|
||||
* Should be looking at legitimate param ids
|
||||
*/
|
||||
Assert(la > 0);
|
||||
Assert(lb > 0);
|
||||
|
||||
/*
|
||||
* Return -1, 0 or 1 depending on if la is less than,
|
||||
* equal to or greater than lb
|
||||
*/
|
||||
return (la > lb) - (la < lb);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Used by get_update_query_targetlist_def() (in ruleutils) to reorder the target
|
||||
* list on the left side of the update:
|
||||
* SET () = (SELECT )
|
||||
* Reordering the SELECT side only does not work, consider a case like:
|
||||
* SET (col_1, col3) = (SELECT 1, 3), (col_2) = (SELECT 2)
|
||||
* Without ensure_update_targetlist_in_param_order(), this will lead to an incorrect
|
||||
* deparsed query:
|
||||
* SET (col_1, col2) = (SELECT 1, 3), (col_3) = (SELECT 2)
|
||||
*/
|
||||
void
|
||||
ensure_update_targetlist_in_param_order(List *targetList)
|
||||
{
|
||||
bool need_to_sort_target_list = false;
|
||||
int previous_paramid = 0;
|
||||
ListCell *l;
|
||||
|
||||
foreach(l, targetList)
|
||||
{
|
||||
TargetEntry *tle = (TargetEntry *) lfirst(l);
|
||||
|
||||
if (!tle->resjunk)
|
||||
{
|
||||
int paramid = GetParamId((Node *) tle->expr);
|
||||
if (paramid < previous_paramid)
|
||||
{
|
||||
need_to_sort_target_list = true;
|
||||
break;
|
||||
}
|
||||
|
||||
previous_paramid = paramid;
|
||||
}
|
||||
}
|
||||
|
||||
if (need_to_sort_target_list)
|
||||
{
|
||||
list_sort(targetList, target_list_cmp);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ExpandMergedSubscriptingRefEntries takes a list of target entries and expands
|
||||
* each one that references a SubscriptingRef node that indicates multiple (field)
|
||||
* updates on the same attribute, which is applicable for array/json types atm.
|
||||
*/
|
||||
List *
|
||||
ExpandMergedSubscriptingRefEntries(List *targetEntryList)
|
||||
{
|
||||
List *newTargetEntryList = NIL;
|
||||
ListCell *tgtCell = NULL;
|
||||
|
||||
foreach(tgtCell, targetEntryList)
|
||||
{
|
||||
TargetEntry *targetEntry = (TargetEntry *) lfirst(tgtCell);
|
||||
List *expandedTargetEntries = NIL;
|
||||
|
||||
Expr *expr = targetEntry->expr;
|
||||
while (expr)
|
||||
{
|
||||
SubscriptingRef *subsRef = TargetEntryExprFindSubsRef(expr);
|
||||
if (!subsRef)
|
||||
{
|
||||
break;
|
||||
}
|
||||
|
||||
/*
|
||||
* Remove refexpr from the SubscriptingRef that we are about to
|
||||
* wrap in a new TargetEntry and save it for the next one.
|
||||
*/
|
||||
Expr *refexpr = subsRef->refexpr;
|
||||
subsRef->refexpr = NULL;
|
||||
|
||||
/*
|
||||
* Wrap the Expr that holds SubscriptingRef (directly or indirectly)
|
||||
* in a new TargetEntry; note that it doesn't have a refexpr anymore.
|
||||
*/
|
||||
TargetEntry *newTargetEntry = copyObject(targetEntry);
|
||||
newTargetEntry->expr = expr;
|
||||
expandedTargetEntries = lappend(expandedTargetEntries, newTargetEntry);
|
||||
|
||||
/* now inspect the refexpr that SubscriptingRef at hand were holding */
|
||||
expr = refexpr;
|
||||
}
|
||||
|
||||
if (expandedTargetEntries == NIL)
|
||||
{
|
||||
/* return original entry since it doesn't hold a SubscriptingRef node */
|
||||
newTargetEntryList = lappend(newTargetEntryList, targetEntry);
|
||||
}
|
||||
else
|
||||
{
|
||||
/*
|
||||
* Need to concat expanded target list entries in reverse order
|
||||
* to preserve ordering of the original target entry list.
|
||||
*/
|
||||
List *reversedTgtEntries = NIL;
|
||||
ListCell *revCell = NULL;
|
||||
foreach(revCell, expandedTargetEntries)
|
||||
{
|
||||
TargetEntry *tgtEntry = (TargetEntry *) lfirst(revCell);
|
||||
reversedTgtEntries = lcons(tgtEntry, reversedTgtEntries);
|
||||
}
|
||||
newTargetEntryList = list_concat(newTargetEntryList, reversedTgtEntries);
|
||||
}
|
||||
}
|
||||
|
||||
return newTargetEntryList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* TargetEntryExprFindSubsRef searches given Expr --assuming that it is part
|
||||
* of a target list entry-- to see if it directly (i.e.: itself) or indirectly
|
||||
* (e.g.: behind some level of coercions) holds a SubscriptingRef node.
|
||||
*
|
||||
* Returns the original SubscriptingRef node on success or NULL otherwise.
|
||||
*
|
||||
* Note that it wouldn't add much value to use expression_tree_walker here
|
||||
* since we are only interested in a subset of the fields of a few certain
|
||||
* node types.
|
||||
*/
|
||||
static SubscriptingRef *
|
||||
TargetEntryExprFindSubsRef(Expr *expr)
|
||||
{
|
||||
Node *node = (Node *) expr;
|
||||
while (node)
|
||||
{
|
||||
if (IsA(node, FieldStore))
|
||||
{
|
||||
/*
|
||||
* ModifyPartialQuerySupported doesn't allow INSERT/UPDATE via
|
||||
* FieldStore. If we decide supporting such commands, then we
|
||||
* should take the first element of "newvals" list into account
|
||||
* here. This is because, to support such commands, we will need
|
||||
* to expand merged FieldStore into separate target entries too.
|
||||
*
|
||||
* For this reason, this block is not reachable atm and need to
|
||||
* uncomment the following if we decide supporting such commands.
|
||||
*
|
||||
* """
|
||||
* FieldStore *fieldStore = (FieldStore *) node;
|
||||
* node = (Node *) linitial(fieldStore->newvals);
|
||||
* """
|
||||
*/
|
||||
ereport(ERROR, (errmsg("unexpectedly got FieldStore object when "
|
||||
"generating shard query")));
|
||||
}
|
||||
else if (IsA(node, CoerceToDomain))
|
||||
{
|
||||
CoerceToDomain *coerceToDomain = (CoerceToDomain *) node;
|
||||
if (coerceToDomain->coercionformat != COERCE_IMPLICIT_CAST)
|
||||
{
|
||||
/* not an implicit coercion, cannot reach to a SubscriptingRef */
|
||||
break;
|
||||
}
|
||||
|
||||
node = (Node *) coerceToDomain->arg;
|
||||
}
|
||||
else if (IsA(node, SubscriptingRef))
|
||||
{
|
||||
return (SubscriptingRef *) node;
|
||||
}
|
||||
else
|
||||
{
|
||||
/* got a node that we are not interested in */
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return NULL;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3509,6 +3509,8 @@ get_update_query_targetlist_def(Query *query, List *targetList,
|
|||
SubLink *cur_ma_sublink;
|
||||
List *ma_sublinks;
|
||||
|
||||
targetList = ExpandMergedSubscriptingRefEntries(targetList);
|
||||
|
||||
/*
|
||||
* Prepare to deal with MULTIEXPR assignments: collect the source SubLinks
|
||||
* into a list. We expect them to appear, in ID order, in resjunk tlist
|
||||
|
|
@ -3532,6 +3534,8 @@ get_update_query_targetlist_def(Query *query, List *targetList,
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
ensure_update_targetlist_in_param_order(targetList);
|
||||
}
|
||||
next_ma_cell = list_head(ma_sublinks);
|
||||
cur_ma_sublink = NULL;
|
||||
|
|
|
|||
|
|
@ -3525,6 +3525,8 @@ get_update_query_targetlist_def(Query *query, List *targetList,
|
|||
SubLink *cur_ma_sublink;
|
||||
List *ma_sublinks;
|
||||
|
||||
targetList = ExpandMergedSubscriptingRefEntries(targetList);
|
||||
|
||||
/*
|
||||
* Prepare to deal with MULTIEXPR assignments: collect the source SubLinks
|
||||
* into a list. We expect them to appear, in ID order, in resjunk tlist
|
||||
|
|
@ -3548,6 +3550,8 @@ get_update_query_targetlist_def(Query *query, List *targetList,
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
ensure_update_targetlist_in_param_order(targetList);
|
||||
}
|
||||
next_ma_cell = list_head(ma_sublinks);
|
||||
cur_ma_sublink = NULL;
|
||||
|
|
|
|||
|
|
@ -3542,6 +3542,8 @@ get_update_query_targetlist_def(Query *query, List *targetList,
|
|||
SubLink *cur_ma_sublink;
|
||||
List *ma_sublinks;
|
||||
|
||||
targetList = ExpandMergedSubscriptingRefEntries(targetList);
|
||||
|
||||
/*
|
||||
* Prepare to deal with MULTIEXPR assignments: collect the source SubLinks
|
||||
* into a list. We expect them to appear, in ID order, in resjunk tlist
|
||||
|
|
@ -3565,6 +3567,8 @@ get_update_query_targetlist_def(Query *query, List *targetList,
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
ensure_update_targetlist_in_param_order(targetList);
|
||||
}
|
||||
next_ma_cell = list_head(ma_sublinks);
|
||||
cur_ma_sublink = NULL;
|
||||
|
|
|
|||
|
|
@ -760,7 +760,7 @@ AdaptiveExecutorPreExecutorRun(CitusScanState *scanState)
|
|||
*/
|
||||
LockPartitionsForDistributedPlan(distributedPlan);
|
||||
|
||||
ExecuteSubPlans(distributedPlan);
|
||||
ExecuteSubPlans(distributedPlan, RequestedForExplainAnalyze(scanState));
|
||||
|
||||
scanState->finishedPreScan = true;
|
||||
}
|
||||
|
|
@ -3804,7 +3804,7 @@ PopAssignedPlacementExecution(WorkerSession *session)
|
|||
|
||||
|
||||
/*
|
||||
* PopAssignedPlacementExecution finds an executable task from the queue of assigned tasks.
|
||||
* PopUnAssignedPlacementExecution finds an executable task from the queue of unassigned tasks.
|
||||
*/
|
||||
static TaskPlacementExecution *
|
||||
PopUnassignedPlacementExecution(WorkerPool *workerPool)
|
||||
|
|
|
|||
|
|
@ -682,11 +682,13 @@ RegenerateTaskForFasthPathQuery(Job *workerJob)
|
|||
}
|
||||
|
||||
bool isLocalTableModification = false;
|
||||
bool delayedFastPath = false;
|
||||
GenerateSingleShardRouterTaskList(workerJob,
|
||||
relationShardList,
|
||||
placementList,
|
||||
shardId,
|
||||
isLocalTableModification);
|
||||
isLocalTableModification,
|
||||
delayedFastPath);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -42,6 +42,7 @@
|
|||
#include "distributed/merge_planner.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/multi_explain.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/multi_router_planner.h"
|
||||
|
|
@ -121,7 +122,7 @@ NonPushableInsertSelectExecScan(CustomScanState *node)
|
|||
bool binaryFormat =
|
||||
CanUseBinaryCopyFormatForTargetList(selectQuery->targetList);
|
||||
|
||||
ExecuteSubPlans(distSelectPlan);
|
||||
ExecuteSubPlans(distSelectPlan, RequestedForExplainAnalyze(scanState));
|
||||
|
||||
/*
|
||||
* We have a separate directory for each transaction, so choosing
|
||||
|
|
|
|||
|
|
@ -313,6 +313,7 @@ ExecuteLocalTaskListExtended(List *taskList,
|
|||
{
|
||||
int taskNumParams = numParams;
|
||||
Oid *taskParameterTypes = parameterTypes;
|
||||
int taskType = GetTaskQueryType(task);
|
||||
|
||||
if (task->parametersInQueryStringResolved)
|
||||
{
|
||||
|
|
@ -330,7 +331,7 @@ ExecuteLocalTaskListExtended(List *taskList,
|
|||
* for concatenated strings, we set queryStringList so that we can access
|
||||
* each query string.
|
||||
*/
|
||||
if (GetTaskQueryType(task) == TASK_QUERY_TEXT_LIST)
|
||||
if (taskType == TASK_QUERY_TEXT_LIST)
|
||||
{
|
||||
List *queryStringList = task->taskQuery.data.queryStringList;
|
||||
totalRowsProcessed +=
|
||||
|
|
@ -342,22 +343,31 @@ ExecuteLocalTaskListExtended(List *taskList,
|
|||
continue;
|
||||
}
|
||||
|
||||
Query *shardQuery = ParseQueryString(TaskQueryString(task),
|
||||
taskParameterTypes,
|
||||
taskNumParams);
|
||||
if (taskType != TASK_QUERY_LOCAL_PLAN)
|
||||
{
|
||||
Query *shardQuery = ParseQueryString(TaskQueryString(task),
|
||||
taskParameterTypes,
|
||||
taskNumParams);
|
||||
|
||||
int cursorOptions = CURSOR_OPT_PARALLEL_OK;
|
||||
|
||||
int cursorOptions = CURSOR_OPT_PARALLEL_OK;
|
||||
|
||||
/*
|
||||
* Altough the shardQuery is local to this node, we prefer planner()
|
||||
* over standard_planner(). The primary reason for that is Citus itself
|
||||
* is not very tolarent standard_planner() calls that doesn't go through
|
||||
* distributed_planner() because of the way that restriction hooks are
|
||||
* implemented. So, let planner to call distributed_planner() which
|
||||
* eventually calls standard_planner().
|
||||
*/
|
||||
localPlan = planner(shardQuery, NULL, cursorOptions, paramListInfo);
|
||||
/*
|
||||
* Altough the shardQuery is local to this node, we prefer planner()
|
||||
* over standard_planner(). The primary reason for that is Citus itself
|
||||
* is not very tolarent standard_planner() calls that doesn't go through
|
||||
* distributed_planner() because of the way that restriction hooks are
|
||||
* implemented. So, let planner to call distributed_planner() which
|
||||
* eventually calls standard_planner().
|
||||
*/
|
||||
localPlan = planner(shardQuery, NULL, cursorOptions, paramListInfo);
|
||||
}
|
||||
else
|
||||
{
|
||||
ereport(DEBUG2, (errmsg(
|
||||
"Local executor: Using task's cached local plan for task %u",
|
||||
task->taskId)));
|
||||
localPlan = TaskQueryLocalPlan(task);
|
||||
}
|
||||
}
|
||||
|
||||
char *shardQueryString = NULL;
|
||||
|
|
@ -754,24 +764,6 @@ ExecuteTaskPlan(PlannedStmt *taskPlan, char *queryString,
|
|||
localPlacementIndex) :
|
||||
CreateDestReceiver(DestNone);
|
||||
|
||||
/* Create a QueryDesc for the query */
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
|
||||
/* PG18+: nine‐arg CreateQueryDesc with a CachedPlan slot */
|
||||
QueryDesc *queryDesc = CreateQueryDesc(
|
||||
taskPlan, /* PlannedStmt *plannedstmt */
|
||||
NULL, /* CachedPlan *cplan (none) */
|
||||
queryString, /* const char *sourceText */
|
||||
GetActiveSnapshot(), /* Snapshot snapshot */
|
||||
InvalidSnapshot, /* Snapshot crosscheck_snapshot */
|
||||
destReceiver, /* DestReceiver *dest */
|
||||
paramListInfo, /* ParamListInfo params */
|
||||
queryEnv, /* QueryEnvironment *queryEnv */
|
||||
0 /* int instrument_options */
|
||||
);
|
||||
#else
|
||||
|
||||
/* PG15–17: eight‐arg CreateQueryDesc without CachedPlan */
|
||||
QueryDesc *queryDesc = CreateQueryDesc(
|
||||
taskPlan, /* PlannedStmt *plannedstmt */
|
||||
queryString, /* const char *sourceText */
|
||||
|
|
@ -782,8 +774,6 @@ ExecuteTaskPlan(PlannedStmt *taskPlan, char *queryString,
|
|||
queryEnv, /* QueryEnvironment *queryEnv */
|
||||
0 /* int instrument_options */
|
||||
);
|
||||
#endif
|
||||
|
||||
|
||||
ExecutorStart(queryDesc, eflags);
|
||||
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@
|
|||
#include "distributed/merge_executor.h"
|
||||
#include "distributed/merge_planner.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/multi_explain.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/multi_router_planner.h"
|
||||
#include "distributed/repartition_executor.h"
|
||||
|
|
@ -132,7 +133,7 @@ ExecuteSourceAtWorkerAndRepartition(CitusScanState *scanState)
|
|||
ereport(DEBUG1, (errmsg("Executing subplans of the source query and "
|
||||
"storing the results at the respective node(s)")));
|
||||
|
||||
ExecuteSubPlans(distSourcePlan);
|
||||
ExecuteSubPlans(distSourcePlan, RequestedForExplainAnalyze(scanState));
|
||||
|
||||
/*
|
||||
* We have a separate directory for each transaction, so choosing
|
||||
|
|
|
|||
|
|
@ -688,7 +688,7 @@ ExecuteQueryIntoDestReceiver(Query *query, ParamListInfo params, DestReceiver *d
|
|||
* ExecutePlanIntoDestReceiver executes a query plan and sends results to the given
|
||||
* DestReceiver.
|
||||
*/
|
||||
void
|
||||
uint64
|
||||
ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
|
||||
DestReceiver *dest)
|
||||
{
|
||||
|
|
@ -701,21 +701,6 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
|
|||
/* don't display the portal in pg_cursors, it is for internal use only */
|
||||
portal->visible = false;
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
|
||||
/* PostgreSQL 18+ adds a seventh “plansource” argument */
|
||||
PortalDefineQuery(
|
||||
portal,
|
||||
NULL, /* no prepared statement name */
|
||||
"", /* query text */
|
||||
CMDTAG_SELECT, /* command tag */
|
||||
list_make1(queryPlan),/* list of PlannedStmt* */
|
||||
NULL, /* no CachedPlan */
|
||||
NULL /* no CachedPlanSource */
|
||||
);
|
||||
#else
|
||||
|
||||
/* PostgreSQL 17-: six-arg signature */
|
||||
PortalDefineQuery(
|
||||
portal,
|
||||
NULL, /* no prepared statement name */
|
||||
|
|
@ -724,11 +709,12 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
|
|||
list_make1(queryPlan),/* list of PlannedStmt* */
|
||||
NULL /* no CachedPlan */
|
||||
);
|
||||
#endif
|
||||
|
||||
PortalStart(portal, params, eflags, GetActiveSnapshot());
|
||||
|
||||
|
||||
QueryCompletion qc = { 0 };
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
|
||||
/* PG 18+: six-arg signature (drop the run_once bool) */
|
||||
|
|
@ -737,7 +723,7 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
|
|||
false, /* isTopLevel */
|
||||
dest, /* DestReceiver *dest */
|
||||
dest, /* DestReceiver *altdest */
|
||||
NULL); /* QueryCompletion *qc */
|
||||
&qc); /* QueryCompletion *qc */
|
||||
#else
|
||||
|
||||
/* PG 17-: original seven-arg signature */
|
||||
|
|
@ -747,10 +733,12 @@ ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
|
|||
true, /* run_once */
|
||||
dest, /* DestReceiver *dest */
|
||||
dest, /* DestReceiver *altdest */
|
||||
NULL); /* QueryCompletion *qc */
|
||||
&qc); /* QueryCompletion *qc */
|
||||
#endif
|
||||
|
||||
PortalDrop(portal, false);
|
||||
|
||||
return qc.nprocessed;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -315,21 +315,6 @@ StartPortalForQueryExecution(const char *queryString)
|
|||
/* don't display the portal in pg_cursors, it is for internal use only */
|
||||
portal->visible = false;
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
|
||||
/* PG 18+: new CachedPlanSource slot */
|
||||
PortalDefineQuery(
|
||||
portal,
|
||||
NULL, /* no prepared‐stmt name */
|
||||
queryString, /* the SQL text */
|
||||
CMDTAG_SELECT, /* we’re running a SELECT */
|
||||
list_make1(queryPlan), /* plan trees */
|
||||
NULL, /* no CachedPlan */
|
||||
NULL /* no CachedPlanSource */
|
||||
);
|
||||
#else
|
||||
|
||||
/* PG 15–17: six‐arg signature */
|
||||
PortalDefineQuery(
|
||||
portal,
|
||||
NULL,
|
||||
|
|
@ -338,7 +323,6 @@ StartPortalForQueryExecution(const char *queryString)
|
|||
list_make1(queryPlan),
|
||||
NULL /* no CachedPlan */
|
||||
);
|
||||
#endif
|
||||
|
||||
int eflags = 0;
|
||||
PortalStart(portal, NULL, eflags, GetActiveSnapshot());
|
||||
|
|
|
|||
|
|
@ -30,13 +30,22 @@ int MaxIntermediateResult = 1048576; /* maximum size in KB the intermediate resu
|
|||
/* when this is true, we enforce intermediate result size limit in all executors */
|
||||
int SubPlanLevel = 0;
|
||||
|
||||
/*
|
||||
* SubPlanExplainAnalyzeContext is both a memory context for storing
|
||||
* subplans’ EXPLAIN ANALYZE output and a flag indicating that execution
|
||||
* is running under EXPLAIN ANALYZE for subplans.
|
||||
*/
|
||||
MemoryContext SubPlanExplainAnalyzeContext = NULL;
|
||||
SubPlanExplainOutputData *SubPlanExplainOutput;
|
||||
extern uint8 TotalExplainOutputCapacity;
|
||||
extern uint8 NumTasksOutput;
|
||||
|
||||
/*
|
||||
* ExecuteSubPlans executes a list of subplans from a distributed plan
|
||||
* by sequentially executing each plan from the top.
|
||||
*/
|
||||
void
|
||||
ExecuteSubPlans(DistributedPlan *distributedPlan)
|
||||
ExecuteSubPlans(DistributedPlan *distributedPlan, bool explainAnalyzeEnabled)
|
||||
{
|
||||
uint64 planId = distributedPlan->planId;
|
||||
List *subPlanList = distributedPlan->subPlanList;
|
||||
|
|
@ -47,6 +56,19 @@ ExecuteSubPlans(DistributedPlan *distributedPlan)
|
|||
return;
|
||||
}
|
||||
|
||||
/*
|
||||
* If the root DistributedPlan has EXPLAIN ANALYZE enabled,
|
||||
* its subplans should also have EXPLAIN ANALYZE enabled.
|
||||
*/
|
||||
if (explainAnalyzeEnabled)
|
||||
{
|
||||
SubPlanExplainAnalyzeContext = GetMemoryChunkContext(distributedPlan);
|
||||
}
|
||||
else
|
||||
{
|
||||
SubPlanExplainAnalyzeContext = NULL;
|
||||
}
|
||||
|
||||
HTAB *intermediateResultsHash = MakeIntermediateResultHTAB();
|
||||
RecordSubplanExecutionsOnNodes(intermediateResultsHash, distributedPlan);
|
||||
|
||||
|
|
@ -79,7 +101,23 @@ ExecuteSubPlans(DistributedPlan *distributedPlan)
|
|||
|
||||
TimestampTz startTimestamp = GetCurrentTimestamp();
|
||||
|
||||
ExecutePlanIntoDestReceiver(plannedStmt, params, copyDest);
|
||||
uint64 nprocessed;
|
||||
|
||||
PG_TRY();
|
||||
{
|
||||
nprocessed =
|
||||
ExecutePlanIntoDestReceiver(plannedStmt, params, copyDest);
|
||||
}
|
||||
PG_CATCH();
|
||||
{
|
||||
SubPlanExplainAnalyzeContext = NULL;
|
||||
SubPlanExplainOutput = NULL;
|
||||
TotalExplainOutputCapacity = 0;
|
||||
NumTasksOutput = 0;
|
||||
PG_RE_THROW();
|
||||
}
|
||||
PG_END_TRY();
|
||||
|
||||
|
||||
/*
|
||||
* EXPLAIN ANALYZE instrumentations. Calculating these are very light-weight,
|
||||
|
|
@ -94,10 +132,24 @@ ExecuteSubPlans(DistributedPlan *distributedPlan)
|
|||
subPlan->durationMillisecs += durationMicrosecs * MICRO_TO_MILLI_SECOND;
|
||||
|
||||
subPlan->bytesSentPerWorker = RemoteFileDestReceiverBytesSent(copyDest);
|
||||
subPlan->ntuples = nprocessed;
|
||||
subPlan->remoteWorkerCount = list_length(remoteWorkerNodeList);
|
||||
subPlan->writeLocalFile = entry->writeLocalFile;
|
||||
|
||||
SubPlanLevel--;
|
||||
|
||||
/*
|
||||
* Save the EXPLAIN ANALYZE output(s) for later extraction in ExplainSubPlans().
|
||||
* Because the SubPlan context isn’t available during distributed execution,
|
||||
* pass the pointer as a global variable in SubPlanExplainOutput.
|
||||
*/
|
||||
subPlan->totalExplainOutput = SubPlanExplainOutput;
|
||||
subPlan->numTasksOutput = NumTasksOutput;
|
||||
SubPlanExplainOutput = NULL;
|
||||
TotalExplainOutputCapacity = 0;
|
||||
NumTasksOutput = 0;
|
||||
FreeExecutorState(estate);
|
||||
}
|
||||
|
||||
SubPlanExplainAnalyzeContext = NULL;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -729,12 +729,13 @@ PartitionMethodViaCatalog(Oid relationId)
|
|||
return DISTRIBUTE_BY_INVALID;
|
||||
}
|
||||
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock);
|
||||
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
|
||||
if (isNullArray[Anum_pg_dist_partition_partmethod - 1])
|
||||
|
|
@ -742,6 +743,8 @@ PartitionMethodViaCatalog(Oid relationId)
|
|||
/* partition method cannot be NULL, still let's make sure */
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
return DISTRIBUTE_BY_INVALID;
|
||||
}
|
||||
|
||||
|
|
@ -750,6 +753,8 @@ PartitionMethodViaCatalog(Oid relationId)
|
|||
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
return partitionMethodChar;
|
||||
}
|
||||
|
|
@ -768,12 +773,12 @@ PartitionColumnViaCatalog(Oid relationId)
|
|||
return NULL;
|
||||
}
|
||||
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock);
|
||||
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
|
||||
if (isNullArray[Anum_pg_dist_partition_partkey - 1])
|
||||
|
|
@ -781,6 +786,8 @@ PartitionColumnViaCatalog(Oid relationId)
|
|||
/* partition key cannot be NULL, still let's make sure */
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
return NULL;
|
||||
}
|
||||
|
||||
|
|
@ -795,6 +802,8 @@ PartitionColumnViaCatalog(Oid relationId)
|
|||
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
return partitionColumn;
|
||||
}
|
||||
|
|
@ -813,12 +822,13 @@ ColocationIdViaCatalog(Oid relationId)
|
|||
return INVALID_COLOCATION_ID;
|
||||
}
|
||||
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), AccessShareLock);
|
||||
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
heap_deform_tuple(partitionTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
|
||||
if (isNullArray[Anum_pg_dist_partition_colocationid - 1])
|
||||
|
|
@ -826,6 +836,8 @@ ColocationIdViaCatalog(Oid relationId)
|
|||
/* colocation id cannot be NULL, still let's make sure */
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
return INVALID_COLOCATION_ID;
|
||||
}
|
||||
|
||||
|
|
@ -834,6 +846,8 @@ ColocationIdViaCatalog(Oid relationId)
|
|||
|
||||
heap_freetuple(partitionTuple);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
return colocationId;
|
||||
}
|
||||
|
|
@ -1741,10 +1755,11 @@ BuildCitusTableCacheEntry(Oid relationId)
|
|||
}
|
||||
|
||||
MemoryContext oldContext = NULL;
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
heap_deform_tuple(distPartitionTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
|
||||
CitusTableCacheEntry *cacheEntry =
|
||||
|
|
@ -1797,7 +1812,7 @@ BuildCitusTableCacheEntry(Oid relationId)
|
|||
cacheEntry->replicationModel = DatumGetChar(replicationModelDatum);
|
||||
}
|
||||
|
||||
if (isNullArray[Anum_pg_dist_partition_autoconverted - 1])
|
||||
if (isNullArray[GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor)])
|
||||
{
|
||||
/*
|
||||
* We don't expect this to happen, but set it to false (the default value)
|
||||
|
|
@ -1808,7 +1823,7 @@ BuildCitusTableCacheEntry(Oid relationId)
|
|||
else
|
||||
{
|
||||
cacheEntry->autoConverted = DatumGetBool(
|
||||
datumArray[Anum_pg_dist_partition_autoconverted - 1]);
|
||||
datumArray[GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor)]);
|
||||
}
|
||||
|
||||
heap_freetuple(distPartitionTuple);
|
||||
|
|
@ -1852,6 +1867,9 @@ BuildCitusTableCacheEntry(Oid relationId)
|
|||
|
||||
table_close(pgDistPartition, NoLock);
|
||||
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
cacheEntry->isValid = true;
|
||||
|
||||
return cacheEntry;
|
||||
|
|
@ -5011,10 +5029,13 @@ CitusTableTypeIdList(CitusTableType citusTableType)
|
|||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
while (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
memset(datumArray, 0, tupleDescriptor->natts * sizeof(Datum));
|
||||
memset(isNullArray, 0, tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
|
||||
Datum partMethodDatum = datumArray[Anum_pg_dist_partition_partmethod - 1];
|
||||
|
|
@ -5038,6 +5059,9 @@ CitusTableTypeIdList(CitusTableType citusTableType)
|
|||
heapTuple = systable_getnext(scanDescriptor);
|
||||
}
|
||||
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPartition, AccessShareLock);
|
||||
|
||||
|
|
|
|||
|
|
@ -573,13 +573,17 @@ FetchRelationIdFromPgPartitionHeapTuple(HeapTuple heapTuple, TupleDesc tupleDesc
|
|||
{
|
||||
Assert(heapTuple->t_tableOid == DistPartitionRelationId());
|
||||
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
Datum *datumArray = (Datum *) palloc(tupleDesc->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDesc->natts * sizeof(bool));
|
||||
|
||||
heap_deform_tuple(heapTuple, tupleDesc, datumArray, isNullArray);
|
||||
|
||||
Datum relationIdDatum = datumArray[Anum_pg_dist_partition_logicalrelid - 1];
|
||||
Oid relationId = DatumGetObjectId(relationIdDatum);
|
||||
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
return relationId;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -812,6 +812,7 @@ GenerateSizeQueryOnMultiplePlacements(List *shardIntervalList,
|
|||
{
|
||||
partitionedShardNames = lappend(partitionedShardNames, quotedShardName);
|
||||
}
|
||||
|
||||
/* for non-partitioned tables, we will use Postgres' size functions */
|
||||
else
|
||||
{
|
||||
|
|
@ -1919,23 +1920,22 @@ InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
|
|||
{
|
||||
char *distributionColumnString = NULL;
|
||||
|
||||
Datum newValues[Natts_pg_dist_partition];
|
||||
bool newNulls[Natts_pg_dist_partition];
|
||||
|
||||
/* open system catalog and insert new tuple */
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock);
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
|
||||
Datum *newValues = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *newNulls = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
/* form new tuple for pg_dist_partition */
|
||||
memset(newValues, 0, sizeof(newValues));
|
||||
memset(newNulls, false, sizeof(newNulls));
|
||||
|
||||
newValues[Anum_pg_dist_partition_logicalrelid - 1] =
|
||||
ObjectIdGetDatum(relationId);
|
||||
newValues[Anum_pg_dist_partition_partmethod - 1] =
|
||||
CharGetDatum(distributionMethod);
|
||||
newValues[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId);
|
||||
newValues[Anum_pg_dist_partition_repmodel - 1] = CharGetDatum(replicationModel);
|
||||
newValues[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(autoConverted);
|
||||
newValues[GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor)] =
|
||||
BoolGetDatum(autoConverted);
|
||||
|
||||
/* set partkey column to NULL for reference tables */
|
||||
if (distributionMethod != DISTRIBUTE_BY_NONE)
|
||||
|
|
@ -1951,7 +1951,7 @@ InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
|
|||
newNulls[Anum_pg_dist_partition_partkey - 1] = true;
|
||||
}
|
||||
|
||||
HeapTuple newTuple = heap_form_tuple(RelationGetDescr(pgDistPartition), newValues,
|
||||
HeapTuple newTuple = heap_form_tuple(tupleDescriptor, newValues,
|
||||
newNulls);
|
||||
|
||||
/* finally insert tuple, build index entries & register cache invalidation */
|
||||
|
|
@ -1963,6 +1963,9 @@ InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
|
|||
|
||||
CommandCounterIncrement();
|
||||
table_close(pgDistPartition, NoLock);
|
||||
|
||||
pfree(newValues);
|
||||
pfree(newNulls);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -2154,13 +2157,13 @@ UpdatePlacementGroupId(uint64 placementId, int groupId)
|
|||
ScanKeyData scanKey[1];
|
||||
int scanKeyCount = 1;
|
||||
bool indexOK = true;
|
||||
Datum values[Natts_pg_dist_placement];
|
||||
bool isnull[Natts_pg_dist_placement];
|
||||
bool replace[Natts_pg_dist_placement];
|
||||
bool colIsNull = false;
|
||||
|
||||
Relation pgDistPlacement = table_open(DistPlacementRelationId(), RowExclusiveLock);
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement);
|
||||
Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_placementid,
|
||||
BTEqualStrategyNumber, F_INT8EQ, Int64GetDatum(placementId));
|
||||
|
||||
|
|
@ -2177,8 +2180,6 @@ UpdatePlacementGroupId(uint64 placementId, int groupId)
|
|||
placementId)));
|
||||
}
|
||||
|
||||
memset(replace, 0, sizeof(replace));
|
||||
|
||||
values[Anum_pg_dist_placement_groupid - 1] = Int32GetDatum(groupId);
|
||||
isnull[Anum_pg_dist_placement_groupid - 1] = false;
|
||||
replace[Anum_pg_dist_placement_groupid - 1] = true;
|
||||
|
|
@ -2197,6 +2198,10 @@ UpdatePlacementGroupId(uint64 placementId, int groupId)
|
|||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPlacement, NoLock);
|
||||
|
||||
pfree(values);
|
||||
pfree(isnull);
|
||||
pfree(replace);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -2210,12 +2215,13 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted)
|
|||
ScanKeyData scanKey[1];
|
||||
int scanKeyCount = 1;
|
||||
bool indexOK = true;
|
||||
Datum values[Natts_pg_dist_partition];
|
||||
bool isnull[Natts_pg_dist_partition];
|
||||
bool replace[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock);
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
|
||||
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(citusTableId));
|
||||
|
||||
|
|
@ -2231,11 +2237,10 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted)
|
|||
citusTableId)));
|
||||
}
|
||||
|
||||
memset(replace, 0, sizeof(replace));
|
||||
|
||||
values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(autoConverted);
|
||||
isnull[Anum_pg_dist_partition_autoconverted - 1] = false;
|
||||
replace[Anum_pg_dist_partition_autoconverted - 1] = true;
|
||||
int autoconvertedindex = GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor);
|
||||
values[autoconvertedindex] = BoolGetDatum(autoConverted);
|
||||
isnull[autoconvertedindex] = false;
|
||||
replace[autoconvertedindex] = true;
|
||||
|
||||
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
|
||||
|
||||
|
|
@ -2247,6 +2252,10 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted)
|
|||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
|
||||
pfree(values);
|
||||
pfree(isnull);
|
||||
pfree(replace);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -2286,12 +2295,13 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut
|
|||
ScanKeyData scanKey[1];
|
||||
int scanKeyCount = 1;
|
||||
bool indexOK = true;
|
||||
Datum values[Natts_pg_dist_partition];
|
||||
bool isnull[Natts_pg_dist_partition];
|
||||
bool replace[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock);
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
|
||||
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(relationId));
|
||||
|
||||
|
|
@ -2307,8 +2317,6 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut
|
|||
relationId)));
|
||||
}
|
||||
|
||||
memset(replace, 0, sizeof(replace));
|
||||
|
||||
replace[Anum_pg_dist_partition_partmethod - 1] = true;
|
||||
values[Anum_pg_dist_partition_partmethod - 1] = CharGetDatum(distributionMethod);
|
||||
isnull[Anum_pg_dist_partition_partmethod - 1] = false;
|
||||
|
|
@ -2317,9 +2325,10 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut
|
|||
values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId);
|
||||
isnull[Anum_pg_dist_partition_colocationid - 1] = false;
|
||||
|
||||
replace[Anum_pg_dist_partition_autoconverted - 1] = true;
|
||||
values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(false);
|
||||
isnull[Anum_pg_dist_partition_autoconverted - 1] = false;
|
||||
int autoconvertedindex = GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor);
|
||||
replace[autoconvertedindex] = true;
|
||||
values[autoconvertedindex] = BoolGetDatum(false);
|
||||
isnull[autoconvertedindex] = false;
|
||||
|
||||
char *distributionColumnString = nodeToString((Node *) distributionColumn);
|
||||
|
||||
|
|
@ -2337,6 +2346,10 @@ UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distribut
|
|||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
|
||||
pfree(values);
|
||||
pfree(isnull);
|
||||
pfree(replace);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -2380,12 +2393,13 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca
|
|||
ScanKeyData scanKey[1];
|
||||
int scanKeyCount = 1;
|
||||
bool indexOK = true;
|
||||
Datum values[Natts_pg_dist_partition];
|
||||
bool isnull[Natts_pg_dist_partition];
|
||||
bool replace[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock);
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isnull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
|
||||
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(relationId));
|
||||
|
||||
|
|
@ -2401,8 +2415,6 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca
|
|||
relationId)));
|
||||
}
|
||||
|
||||
memset(replace, 0, sizeof(replace));
|
||||
|
||||
values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId);
|
||||
isnull[Anum_pg_dist_partition_colocationid - 1] = false;
|
||||
replace[Anum_pg_dist_partition_colocationid - 1] = true;
|
||||
|
|
@ -2411,9 +2423,10 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca
|
|||
isnull[Anum_pg_dist_partition_repmodel - 1] = false;
|
||||
replace[Anum_pg_dist_partition_repmodel - 1] = true;
|
||||
|
||||
values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(autoConverted);
|
||||
isnull[Anum_pg_dist_partition_autoconverted - 1] = false;
|
||||
replace[Anum_pg_dist_partition_autoconverted - 1] = true;
|
||||
int autoconvertedindex = GetAutoConvertedAttrIndexInPgDistPartition(tupleDescriptor);
|
||||
values[autoconvertedindex] = BoolGetDatum(autoConverted);
|
||||
isnull[autoconvertedindex] = false;
|
||||
replace[autoconvertedindex] = true;
|
||||
|
||||
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
|
||||
|
||||
|
|
@ -2424,6 +2437,10 @@ UpdateNoneDistTableMetadata(Oid relationId, char replicationModel, uint32 coloca
|
|||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
|
||||
pfree(values);
|
||||
pfree(isnull);
|
||||
pfree(replace);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -3149,8 +3166,8 @@ ScheduleBackgroundTask(int64 jobId, Oid owner, char *command, int dependingTaskC
|
|||
|
||||
values[Anum_pg_dist_background_task_nodes_involved - 1] =
|
||||
IntArrayToDatum(nodesInvolvedCount, nodesInvolved);
|
||||
nulls[Anum_pg_dist_background_task_nodes_involved - 1] = (nodesInvolvedCount ==
|
||||
0);
|
||||
nulls[Anum_pg_dist_background_task_nodes_involved - 1] =
|
||||
(nodesInvolvedCount == 0);
|
||||
|
||||
HeapTuple newTuple = heap_form_tuple(RelationGetDescr(pgDistBackgroundTask),
|
||||
values, nulls);
|
||||
|
|
@ -4420,3 +4437,23 @@ UnblockDependingBackgroundTasks(BackgroundTask *task)
|
|||
|
||||
table_close(pgDistBackgroundTasksDepend, NoLock);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GetAutoConvertedAttrIndexInPgDistPartition returns attrnum for autoconverted attr.
|
||||
*
|
||||
* autoconverted attr was added to table pg_dist_partition using alter operation after
|
||||
* the version where Citus started supporting downgrades, and it's only column that we've
|
||||
* introduced to pg_dist_partition since then.
|
||||
*
|
||||
* And in case of a downgrade + upgrade, tupleDesc->natts becomes greater than
|
||||
* Natts_pg_dist_partition and when this happens, then we know that attrnum autoconverted is
|
||||
* not Anum_pg_dist_partition_autoconverted anymore but tupleDesc->natts - 1.
|
||||
*/
|
||||
int
|
||||
GetAutoConvertedAttrIndexInPgDistPartition(TupleDesc tupleDesc)
|
||||
{
|
||||
return TupleDescSize(tupleDesc) == Natts_pg_dist_partition
|
||||
? (Anum_pg_dist_partition_autoconverted - 1)
|
||||
: tupleDesc->natts - 1;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -2930,7 +2930,7 @@ InsertNodeRow(int nodeid, char *nodeName, int32 nodePort, NodeMetadata *nodeMeta
|
|||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistNode);
|
||||
HeapTuple heapTuple = heap_form_tuple(tupleDescriptor, values, isNulls);
|
||||
|
||||
CatalogTupleInsert(pgDistNode, heapTuple);
|
||||
CATALOG_INSERT_WITH_SNAPSHOT(pgDistNode, heapTuple);
|
||||
|
||||
CitusInvalidateRelcacheByRelid(DistNodeRelationId());
|
||||
|
||||
|
|
|
|||
|
|
@ -439,6 +439,27 @@ SetTaskQueryStringList(Task *task, List *queryStringList)
|
|||
}
|
||||
|
||||
|
||||
void
|
||||
SetTaskQueryPlan(Task *task, Query *query, PlannedStmt *localPlan)
|
||||
{
|
||||
Assert(localPlan != NULL);
|
||||
task->taskQuery.queryType = TASK_QUERY_LOCAL_PLAN;
|
||||
task->taskQuery.data.localCompiled = (LocalCompilation *) palloc0(
|
||||
sizeof(LocalCompilation));
|
||||
task->taskQuery.data.localCompiled->query = query;
|
||||
task->taskQuery.data.localCompiled->plan = localPlan;
|
||||
task->queryCount = 1;
|
||||
}
|
||||
|
||||
|
||||
PlannedStmt *
|
||||
TaskQueryLocalPlan(Task *task)
|
||||
{
|
||||
Assert(task->taskQuery.queryType == TASK_QUERY_LOCAL_PLAN);
|
||||
return task->taskQuery.data.localCompiled->plan;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* DeparseTaskQuery is a general way of deparsing a query based on a task.
|
||||
*/
|
||||
|
|
@ -524,6 +545,26 @@ TaskQueryString(Task *task)
|
|||
{
|
||||
return task->taskQuery.data.queryStringLazy;
|
||||
}
|
||||
else if (taskQueryType == TASK_QUERY_LOCAL_PLAN)
|
||||
{
|
||||
Query *query = task->taskQuery.data.localCompiled->query;
|
||||
Assert(query != NULL);
|
||||
|
||||
/*
|
||||
* Use the query of the local compilation to generate the
|
||||
* query string. For local compiled tasks, the query is retained
|
||||
* for this purpose, which may be EXPLAIN ANALYZing the task, or
|
||||
* command logging. Generating the query string on the fly is
|
||||
* acceptable because the plan of the local compilation is used
|
||||
* for query execution.
|
||||
*/
|
||||
MemoryContext previousContext = MemoryContextSwitchTo(GetMemoryChunkContext(
|
||||
query));
|
||||
UpdateRelationToShardNames((Node *) query, task->relationShardList);
|
||||
MemoryContextSwitchTo(previousContext);
|
||||
return AnnotateQuery(DeparseTaskQuery(task, query),
|
||||
task->partitionKeyValue, task->colocationId);
|
||||
}
|
||||
|
||||
Query *jobQueryReferenceForLazyDeparsing =
|
||||
task->taskQuery.data.jobQueryReferenceForLazyDeparsing;
|
||||
|
|
|
|||
|
|
@ -135,13 +135,13 @@ static void AdjustReadIntermediateResultsCostInternal(RelOptInfo *relOptInfo,
|
|||
Const *resultFormatConst);
|
||||
static List * OuterPlanParamsList(PlannerInfo *root);
|
||||
static List * CopyPlanParamList(List *originalPlanParamList);
|
||||
static PlannerRestrictionContext * CreateAndPushPlannerRestrictionContext(void);
|
||||
static PlannerRestrictionContext * CreateAndPushPlannerRestrictionContext(
|
||||
FastPathRestrictionContext *fastPathContext);
|
||||
static PlannerRestrictionContext * CurrentPlannerRestrictionContext(void);
|
||||
static void PopPlannerRestrictionContext(void);
|
||||
static void ResetPlannerRestrictionContext(
|
||||
PlannerRestrictionContext *plannerRestrictionContext);
|
||||
static PlannedStmt * PlanFastPathDistributedStmt(DistributedPlanningContext *planContext,
|
||||
Node *distributionKeyValue);
|
||||
static PlannedStmt * PlanFastPathDistributedStmt(DistributedPlanningContext *planContext);
|
||||
static PlannedStmt * PlanDistributedStmt(DistributedPlanningContext *planContext,
|
||||
int rteIdCounter);
|
||||
static RTEListProperties * GetRTEListProperties(List *rangeTableList);
|
||||
|
|
@ -166,7 +166,7 @@ distributed_planner(Query *parse,
|
|||
{
|
||||
bool needsDistributedPlanning = false;
|
||||
bool fastPathRouterQuery = false;
|
||||
Node *distributionKeyValue = NULL;
|
||||
FastPathRestrictionContext fastPathContext = { 0 };
|
||||
|
||||
List *rangeTableList = ExtractRangeTableEntryList(parse);
|
||||
|
||||
|
|
@ -191,8 +191,7 @@ distributed_planner(Query *parse,
|
|||
&maybeHasForeignDistributedTable);
|
||||
if (needsDistributedPlanning)
|
||||
{
|
||||
fastPathRouterQuery = FastPathRouterQuery(parse, &distributionKeyValue);
|
||||
|
||||
fastPathRouterQuery = FastPathRouterQuery(parse, &fastPathContext);
|
||||
if (maybeHasForeignDistributedTable)
|
||||
{
|
||||
WarnIfListHasForeignDistributedTable(rangeTableList);
|
||||
|
|
@ -247,8 +246,9 @@ distributed_planner(Query *parse,
|
|||
*/
|
||||
HideCitusDependentObjectsOnQueriesOfPgMetaTables((Node *) parse, NULL);
|
||||
|
||||
/* create a restriction context and put it at the end if context list */
|
||||
planContext.plannerRestrictionContext = CreateAndPushPlannerRestrictionContext();
|
||||
/* create a restriction context and put it at the end of context list */
|
||||
planContext.plannerRestrictionContext = CreateAndPushPlannerRestrictionContext(
|
||||
&fastPathContext);
|
||||
|
||||
/*
|
||||
* We keep track of how many times we've recursed into the planner, primarily
|
||||
|
|
@ -264,7 +264,7 @@ distributed_planner(Query *parse,
|
|||
{
|
||||
if (fastPathRouterQuery)
|
||||
{
|
||||
result = PlanFastPathDistributedStmt(&planContext, distributionKeyValue);
|
||||
result = PlanFastPathDistributedStmt(&planContext);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
|
@ -649,30 +649,21 @@ IsMultiTaskPlan(DistributedPlan *distributedPlan)
|
|||
* the FastPathPlanner.
|
||||
*/
|
||||
static PlannedStmt *
|
||||
PlanFastPathDistributedStmt(DistributedPlanningContext *planContext,
|
||||
Node *distributionKeyValue)
|
||||
PlanFastPathDistributedStmt(DistributedPlanningContext *planContext)
|
||||
{
|
||||
FastPathRestrictionContext *fastPathContext =
|
||||
planContext->plannerRestrictionContext->fastPathRestrictionContext;
|
||||
Assert(fastPathContext != NULL);
|
||||
Assert(fastPathContext->fastPathRouterQuery);
|
||||
|
||||
planContext->plannerRestrictionContext->fastPathRestrictionContext->
|
||||
fastPathRouterQuery = true;
|
||||
FastPathPreprocessParseTree(planContext->query);
|
||||
|
||||
if (distributionKeyValue == NULL)
|
||||
if (!fastPathContext->delayFastPathPlanning)
|
||||
{
|
||||
/* nothing to record */
|
||||
planContext->plan = FastPathPlanner(planContext->originalQuery,
|
||||
planContext->query,
|
||||
planContext->boundParams);
|
||||
}
|
||||
else if (IsA(distributionKeyValue, Const))
|
||||
{
|
||||
fastPathContext->distributionKeyValue = (Const *) distributionKeyValue;
|
||||
}
|
||||
else if (IsA(distributionKeyValue, Param))
|
||||
{
|
||||
fastPathContext->distributionKeyHasParam = true;
|
||||
}
|
||||
|
||||
planContext->plan = FastPathPlanner(planContext->originalQuery, planContext->query,
|
||||
planContext->boundParams);
|
||||
|
||||
return CreateDistributedPlannedStmt(planContext);
|
||||
}
|
||||
|
|
@ -803,6 +794,8 @@ CreateDistributedPlannedStmt(DistributedPlanningContext *planContext)
|
|||
RaiseDeferredError(distributedPlan->planningError, ERROR);
|
||||
}
|
||||
|
||||
CheckAndBuildDelayedFastPathPlan(planContext, distributedPlan);
|
||||
|
||||
/* remember the plan's identifier for identifying subplans */
|
||||
distributedPlan->planId = planId;
|
||||
|
||||
|
|
@ -2407,13 +2400,15 @@ CopyPlanParamList(List *originalPlanParamList)
|
|||
|
||||
|
||||
/*
|
||||
* CreateAndPushPlannerRestrictionContext creates a new relation restriction context
|
||||
* and a new join context, inserts it to the beginning of the
|
||||
* plannerRestrictionContextList. Finally, the planner restriction context is
|
||||
* inserted to the beginning of the plannerRestrictionContextList and it is returned.
|
||||
* CreateAndPushPlannerRestrictionContext creates a new planner restriction
|
||||
* context with an empty relation restriction context and an empty join and
|
||||
* a copy of the given fast path restriction context (if present). Finally,
|
||||
* the planner restriction context is inserted to the beginning of the
|
||||
* global plannerRestrictionContextList and it is returned.
|
||||
*/
|
||||
static PlannerRestrictionContext *
|
||||
CreateAndPushPlannerRestrictionContext(void)
|
||||
CreateAndPushPlannerRestrictionContext(
|
||||
FastPathRestrictionContext *fastPathRestrictionContext)
|
||||
{
|
||||
PlannerRestrictionContext *plannerRestrictionContext =
|
||||
palloc0(sizeof(PlannerRestrictionContext));
|
||||
|
|
@ -2427,6 +2422,21 @@ CreateAndPushPlannerRestrictionContext(void)
|
|||
plannerRestrictionContext->fastPathRestrictionContext =
|
||||
palloc0(sizeof(FastPathRestrictionContext));
|
||||
|
||||
if (fastPathRestrictionContext != NULL)
|
||||
{
|
||||
/* copy the given fast path restriction context */
|
||||
FastPathRestrictionContext *plannersFastPathCtx =
|
||||
plannerRestrictionContext->fastPathRestrictionContext;
|
||||
plannersFastPathCtx->fastPathRouterQuery =
|
||||
fastPathRestrictionContext->fastPathRouterQuery;
|
||||
plannersFastPathCtx->distributionKeyValue =
|
||||
fastPathRestrictionContext->distributionKeyValue;
|
||||
plannersFastPathCtx->distributionKeyHasParam =
|
||||
fastPathRestrictionContext->distributionKeyHasParam;
|
||||
plannersFastPathCtx->delayFastPathPlanning =
|
||||
fastPathRestrictionContext->delayFastPathPlanning;
|
||||
}
|
||||
|
||||
plannerRestrictionContext->memoryContext = CurrentMemoryContext;
|
||||
|
||||
/* we'll apply logical AND as we add tables */
|
||||
|
|
|
|||
|
|
@ -43,8 +43,10 @@
|
|||
|
||||
#include "pg_version_constants.h"
|
||||
|
||||
#include "distributed/citus_clauses.h"
|
||||
#include "distributed/distributed_planner.h"
|
||||
#include "distributed/insert_select_planner.h"
|
||||
#include "distributed/local_executor.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/multi_physical_planner.h" /* only to use some utility functions */
|
||||
#include "distributed/multi_router_planner.h"
|
||||
|
|
@ -53,6 +55,7 @@
|
|||
#include "distributed/shardinterval_utils.h"
|
||||
|
||||
bool EnableFastPathRouterPlanner = true;
|
||||
bool EnableLocalFastPathQueryOptimization = true;
|
||||
|
||||
static bool ColumnAppearsMultipleTimes(Node *quals, Var *distributionKey);
|
||||
static bool DistKeyInSimpleOpExpression(Expr *clause, Var *distColumn,
|
||||
|
|
@ -61,6 +64,24 @@ static bool ConjunctionContainsColumnFilter(Node *node,
|
|||
Var *column,
|
||||
Node **distributionKeyValue);
|
||||
|
||||
/*
|
||||
* FastPathPreprocessParseTree is used to apply transformations on the parse tree
|
||||
* that are expected by the Postgres planner. This is called on both delayed FastPath
|
||||
* and non-delayed FastPath queries.
|
||||
*/
|
||||
void
|
||||
FastPathPreprocessParseTree(Query *parse)
|
||||
{
|
||||
/*
|
||||
* Citus planner relies on some of the transformations on constant
|
||||
* evaluation on the parse tree.
|
||||
*/
|
||||
parse->targetList =
|
||||
(List *) eval_const_expressions(NULL, (Node *) parse->targetList);
|
||||
parse->jointree->quals =
|
||||
(Node *) eval_const_expressions(NULL, (Node *) parse->jointree->quals);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* FastPathPlanner is intended to be used instead of standard_planner() for trivial
|
||||
|
|
@ -73,15 +94,6 @@ static bool ConjunctionContainsColumnFilter(Node *node,
|
|||
PlannedStmt *
|
||||
FastPathPlanner(Query *originalQuery, Query *parse, ParamListInfo boundParams)
|
||||
{
|
||||
/*
|
||||
* Citus planner relies on some of the transformations on constant
|
||||
* evaluation on the parse tree.
|
||||
*/
|
||||
parse->targetList =
|
||||
(List *) eval_const_expressions(NULL, (Node *) parse->targetList);
|
||||
parse->jointree->quals =
|
||||
(Node *) eval_const_expressions(NULL, (Node *) parse->jointree->quals);
|
||||
|
||||
PlannedStmt *result = GeneratePlaceHolderPlannedStmt(originalQuery);
|
||||
|
||||
return result;
|
||||
|
|
@ -112,9 +124,9 @@ GeneratePlaceHolderPlannedStmt(Query *parse)
|
|||
Plan *plan = &scanNode->plan;
|
||||
#endif
|
||||
|
||||
Node *distKey PG_USED_FOR_ASSERTS_ONLY = NULL;
|
||||
FastPathRestrictionContext fprCtxt PG_USED_FOR_ASSERTS_ONLY = { 0 };
|
||||
|
||||
Assert(FastPathRouterQuery(parse, &distKey));
|
||||
Assert(FastPathRouterQuery(parse, &fprCtxt));
|
||||
|
||||
/* there is only a single relation rte */
|
||||
#if PG_VERSION_NUM >= PG_VERSION_16
|
||||
|
|
@ -150,27 +162,83 @@ GeneratePlaceHolderPlannedStmt(Query *parse)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* InitializeFastPathContext - helper function to initialize a FastPath
|
||||
* restriction context with the details that the FastPath code path needs.
|
||||
*/
|
||||
static void
|
||||
InitializeFastPathContext(FastPathRestrictionContext *fastPathContext,
|
||||
Node *distributionKeyValue,
|
||||
bool canAvoidDeparse,
|
||||
Query *query)
|
||||
{
|
||||
Assert(fastPathContext != NULL);
|
||||
Assert(!fastPathContext->fastPathRouterQuery);
|
||||
Assert(!fastPathContext->delayFastPathPlanning);
|
||||
|
||||
/*
|
||||
* We're looking at a fast path query, so we can fill the
|
||||
* fastPathContext with relevant details.
|
||||
*/
|
||||
fastPathContext->fastPathRouterQuery = true;
|
||||
if (distributionKeyValue == NULL)
|
||||
{
|
||||
/* nothing to record */
|
||||
}
|
||||
else if (IsA(distributionKeyValue, Const))
|
||||
{
|
||||
fastPathContext->distributionKeyValue = (Const *) distributionKeyValue;
|
||||
}
|
||||
else if (IsA(distributionKeyValue, Param))
|
||||
{
|
||||
fastPathContext->distributionKeyHasParam = true;
|
||||
}
|
||||
|
||||
/*
|
||||
* If local execution and the fast path optimization to
|
||||
* avoid deparse are enabled, and it is safe to do local
|
||||
* execution..
|
||||
*/
|
||||
if (EnableLocalFastPathQueryOptimization &&
|
||||
EnableLocalExecution &&
|
||||
GetCurrentLocalExecutionStatus() != LOCAL_EXECUTION_DISABLED)
|
||||
{
|
||||
/*
|
||||
* .. we can delay fast path planning until we know whether
|
||||
* or not the shard is local. Make a final check for volatile
|
||||
* functions in the query tree to determine if we should delay
|
||||
* the fast path planning.
|
||||
*/
|
||||
fastPathContext->delayFastPathPlanning = canAvoidDeparse &&
|
||||
!FindNodeMatchingCheckFunction(
|
||||
(Node *) query,
|
||||
CitusIsVolatileFunction);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* FastPathRouterQuery gets a query and returns true if the query is eligible for
|
||||
* being a fast path router query.
|
||||
* being a fast path router query. It also fills the given fastPathContext with
|
||||
* details about the query such as the distribution key value (if available),
|
||||
* whether the distribution key is a parameter, and the range table entry for the
|
||||
* table being queried.
|
||||
* The requirements for the fast path query can be listed below:
|
||||
*
|
||||
* - SELECT/UPDATE/DELETE query without CTES, sublinks-subqueries, set operations
|
||||
* - The query should touch only a single hash distributed or reference table
|
||||
* - The distribution with equality operator should be in the WHERE clause
|
||||
* and it should be ANDed with any other filters. Also, the distribution
|
||||
* key should only exists once in the WHERE clause. So basically,
|
||||
* key should only exist once in the WHERE clause. So basically,
|
||||
* SELECT ... FROM dist_table WHERE dist_key = X
|
||||
* If the filter is a const, distributionKeyValue is set
|
||||
* - All INSERT statements (including multi-row INSERTs) as long as the commands
|
||||
* don't have any sublinks/CTEs etc
|
||||
* -
|
||||
*/
|
||||
bool
|
||||
FastPathRouterQuery(Query *query, Node **distributionKeyValue)
|
||||
FastPathRouterQuery(Query *query, FastPathRestrictionContext *fastPathContext)
|
||||
{
|
||||
FromExpr *joinTree = query->jointree;
|
||||
Node *quals = NULL;
|
||||
|
||||
if (!EnableFastPathRouterPlanner)
|
||||
{
|
||||
return false;
|
||||
|
|
@ -201,6 +269,7 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
|
|||
else if (query->commandType == CMD_INSERT)
|
||||
{
|
||||
/* we don't need to do any further checks, all INSERTs are fast-path */
|
||||
InitializeFastPathContext(fastPathContext, NULL, true, query);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
@ -225,6 +294,10 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
|
|||
return false;
|
||||
}
|
||||
|
||||
bool isFastPath = false;
|
||||
bool canAvoidDeparse = false;
|
||||
Node *distributionKeyValue = NULL;
|
||||
|
||||
/*
|
||||
* If the table doesn't have a distribution column, we don't need to
|
||||
* check anything further.
|
||||
|
|
@ -232,45 +305,62 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
|
|||
Var *distributionKey = PartitionColumn(distributedTableId, 1);
|
||||
if (!distributionKey)
|
||||
{
|
||||
return true;
|
||||
/*
|
||||
* Local execution may avoid a deparse on single shard distributed tables or
|
||||
* citus local tables. We don't yet support reference tables in this code-path
|
||||
* because modifications on reference tables are complicated to support here.
|
||||
*/
|
||||
canAvoidDeparse = IsCitusTableTypeCacheEntry(cacheEntry,
|
||||
SINGLE_SHARD_DISTRIBUTED) ||
|
||||
IsCitusTableTypeCacheEntry(cacheEntry, CITUS_LOCAL_TABLE);
|
||||
isFastPath = true;
|
||||
}
|
||||
|
||||
/* WHERE clause should not be empty for distributed tables */
|
||||
if (joinTree == NULL ||
|
||||
(IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && joinTree->quals ==
|
||||
NULL))
|
||||
else
|
||||
{
|
||||
return false;
|
||||
FromExpr *joinTree = query->jointree;
|
||||
Node *quals = NULL;
|
||||
|
||||
canAvoidDeparse = IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE);
|
||||
|
||||
if (joinTree == NULL ||
|
||||
(joinTree->quals == NULL && canAvoidDeparse))
|
||||
{
|
||||
/* no quals, not a fast path query */
|
||||
return false;
|
||||
}
|
||||
|
||||
quals = joinTree->quals;
|
||||
if (quals != NULL && IsA(quals, List))
|
||||
{
|
||||
quals = (Node *) make_ands_explicit((List *) quals);
|
||||
}
|
||||
|
||||
/*
|
||||
* Distribution column must be used in a simple equality match check and it must be
|
||||
* place at top level conjunction operator. In simple words, we should have
|
||||
* WHERE dist_key = VALUE [AND ....];
|
||||
*
|
||||
* We're also not allowing any other appearances of the distribution key in the quals.
|
||||
*
|
||||
* Overall the logic might sound fuzzy since it involves two individual checks:
|
||||
* (a) Check for top level AND operator with one side being "dist_key = const"
|
||||
* (b) Only allow single appearance of "dist_key" in the quals
|
||||
*
|
||||
* This is to simplify both of the individual checks and omit various edge cases
|
||||
* that might arise with multiple distribution keys in the quals.
|
||||
*/
|
||||
isFastPath = (ConjunctionContainsColumnFilter(quals, distributionKey,
|
||||
&distributionKeyValue) &&
|
||||
!ColumnAppearsMultipleTimes(quals, distributionKey));
|
||||
}
|
||||
|
||||
/* convert list of expressions into expression tree for further processing */
|
||||
quals = joinTree->quals;
|
||||
if (quals != NULL && IsA(quals, List))
|
||||
if (isFastPath)
|
||||
{
|
||||
quals = (Node *) make_ands_explicit((List *) quals);
|
||||
InitializeFastPathContext(fastPathContext, distributionKeyValue, canAvoidDeparse,
|
||||
query);
|
||||
}
|
||||
|
||||
/*
|
||||
* Distribution column must be used in a simple equality match check and it must be
|
||||
* place at top level conjunction operator. In simple words, we should have
|
||||
* WHERE dist_key = VALUE [AND ....];
|
||||
*
|
||||
* We're also not allowing any other appearances of the distribution key in the quals.
|
||||
*
|
||||
* Overall the logic might sound fuzzy since it involves two individual checks:
|
||||
* (a) Check for top level AND operator with one side being "dist_key = const"
|
||||
* (b) Only allow single appearance of "dist_key" in the quals
|
||||
*
|
||||
* This is to simplify both of the individual checks and omit various edge cases
|
||||
* that might arise with multiple distribution keys in the quals.
|
||||
*/
|
||||
if (ConjunctionContainsColumnFilter(quals, distributionKey, distributionKeyValue) &&
|
||||
!ColumnAppearsMultipleTimes(quals, distributionKey))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
return isFastPath;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -26,6 +26,7 @@
|
|||
#include "commands/tablecmds.h"
|
||||
#include "executor/tstoreReceiver.h"
|
||||
#include "lib/stringinfo.h"
|
||||
#include "nodes/nodeFuncs.h"
|
||||
#include "nodes/plannodes.h"
|
||||
#include "nodes/primnodes.h"
|
||||
#include "nodes/print.h"
|
||||
|
|
@ -73,6 +74,7 @@
|
|||
#include "distributed/placement_connection.h"
|
||||
#include "distributed/recursive_planning.h"
|
||||
#include "distributed/remote_commands.h"
|
||||
#include "distributed/subplan_execution.h"
|
||||
#include "distributed/tuple_destination.h"
|
||||
#include "distributed/tuplestore.h"
|
||||
#include "distributed/version_compat.h"
|
||||
|
|
@ -83,6 +85,7 @@
|
|||
bool ExplainDistributedQueries = true;
|
||||
bool ExplainAllTasks = false;
|
||||
int ExplainAnalyzeSortMethod = EXPLAIN_ANALYZE_SORT_BY_TIME;
|
||||
extern MemoryContext SubPlanExplainAnalyzeContext;
|
||||
|
||||
/*
|
||||
* If enabled, EXPLAIN ANALYZE output & other statistics of last worker task
|
||||
|
|
@ -90,6 +93,11 @@ int ExplainAnalyzeSortMethod = EXPLAIN_ANALYZE_SORT_BY_TIME;
|
|||
*/
|
||||
static char *SavedExplainPlan = NULL;
|
||||
static double SavedExecutionDurationMillisec = 0.0;
|
||||
static double SavedExplainPlanNtuples = 0;
|
||||
static double SavedExplainPlanNloops = 0;
|
||||
extern SubPlanExplainOutputData *SubPlanExplainOutput;
|
||||
uint8 TotalExplainOutputCapacity = 0;
|
||||
uint8 NumTasksOutput = 0;
|
||||
|
||||
/* struct to save explain flags */
|
||||
typedef struct
|
||||
|
|
@ -215,7 +223,8 @@ static const char * ExplainFormatStr(ExplainFormat format);
|
|||
#if PG_VERSION_NUM >= PG_VERSION_17
|
||||
static const char * ExplainSerializeStr(ExplainSerializeOption serializeOption);
|
||||
#endif
|
||||
static void ExplainWorkerPlan(PlannedStmt *plannedStmt, DestReceiver *dest,
|
||||
static void ExplainWorkerPlan(PlannedStmt *plannedStmt, DistributedSubPlan *subPlan,
|
||||
DestReceiver *dest,
|
||||
ExplainState *es,
|
||||
const char *queryString, ParamListInfo params,
|
||||
QueryEnvironment *queryEnv,
|
||||
|
|
@ -224,7 +233,9 @@ static void ExplainWorkerPlan(PlannedStmt *plannedStmt, DestReceiver *dest,
|
|||
const BufferUsage *bufusage,
|
||||
const MemoryContextCounters *mem_counters,
|
||||
#endif
|
||||
double *executionDurationMillisec);
|
||||
double *executionDurationMillisec,
|
||||
double *executionTuples,
|
||||
double *executionLoops);
|
||||
static ExplainFormat ExtractFieldExplainFormat(Datum jsonbDoc, const char *fieldName,
|
||||
ExplainFormat defaultValue);
|
||||
#if PG_VERSION_NUM >= PG_VERSION_17
|
||||
|
|
@ -256,7 +267,8 @@ static double elapsed_time(instr_time *starttime);
|
|||
static void ExplainPropertyBytes(const char *qlabel, int64 bytes, ExplainState *es);
|
||||
static uint64 TaskReceivedTupleData(Task *task);
|
||||
static bool ShowReceivedTupleData(CitusScanState *scanState, ExplainState *es);
|
||||
|
||||
static bool PlanStateAnalyzeWalker(PlanState *planState, void *ctx);
|
||||
static void ExtractAnalyzeStats(DistributedSubPlan *subPlan, PlanState *planState);
|
||||
|
||||
/* exports for SQL callable functions */
|
||||
PG_FUNCTION_INFO_V1(worker_last_saved_explain_analyze);
|
||||
|
|
@ -432,6 +444,84 @@ NonPushableMergeCommandExplainScan(CustomScanState *node, List *ancestors,
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* ExtractAnalyzeStats parses the EXPLAIN ANALYZE output of the pre-executed
|
||||
* subplans and injects the parsed statistics into queryDesc->planstate->instrument.
|
||||
*/
|
||||
static void
|
||||
ExtractAnalyzeStats(DistributedSubPlan *subPlan, PlanState *planState)
|
||||
{
|
||||
if (!planState)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
Instrumentation *instr = planState->instrument;
|
||||
if (!IsA(planState, CustomScanState))
|
||||
{
|
||||
instr->ntuples = subPlan->ntuples;
|
||||
instr->nloops = 1; /* subplan nodes are executed only once */
|
||||
return;
|
||||
}
|
||||
|
||||
Assert(IsA(planState, CustomScanState));
|
||||
|
||||
if (subPlan->numTasksOutput <= 0)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
ListCell *lc;
|
||||
int tasksOutput = 0;
|
||||
double tasksNtuples = 0;
|
||||
double tasksNloops = 0;
|
||||
memset(instr, 0, sizeof(Instrumentation));
|
||||
DistributedPlan *newdistributedPlan =
|
||||
((CitusScanState *) planState)->distributedPlan;
|
||||
|
||||
/*
|
||||
* Inject the earlier executed results—extracted from the workers' EXPLAIN output—
|
||||
* into the newly created tasks.
|
||||
*/
|
||||
foreach(lc, newdistributedPlan->workerJob->taskList)
|
||||
{
|
||||
Task *task = (Task *) lfirst(lc);
|
||||
uint32 taskId = task->taskId;
|
||||
|
||||
if (tasksOutput > subPlan->numTasksOutput)
|
||||
{
|
||||
break;
|
||||
}
|
||||
|
||||
if (!subPlan->totalExplainOutput[taskId].explainOutput)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
/*
|
||||
* Now feed the earlier saved output, which will be used
|
||||
* by RemoteExplain() when printing tasks
|
||||
*/
|
||||
MemoryContext taskContext = GetMemoryChunkContext(task);
|
||||
task->totalReceivedTupleData =
|
||||
subPlan->totalExplainOutput[taskId].totalReceivedTupleData;
|
||||
task->fetchedExplainAnalyzeExecutionDuration =
|
||||
subPlan->totalExplainOutput[taskId].executionDuration;
|
||||
task->fetchedExplainAnalyzePlan =
|
||||
MemoryContextStrdup(taskContext,
|
||||
subPlan->totalExplainOutput[taskId].explainOutput);
|
||||
tasksNtuples += subPlan->totalExplainOutput[taskId].executionNtuples;
|
||||
tasksNloops = subPlan->totalExplainOutput[taskId].executionNloops;
|
||||
|
||||
subPlan->totalExplainOutput[taskId].explainOutput = NULL;
|
||||
tasksOutput++;
|
||||
}
|
||||
|
||||
instr->ntuples = tasksNtuples;
|
||||
instr->nloops = tasksNloops;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ExplainSubPlans generates EXPLAIN output for subplans for CTEs
|
||||
* and complex subqueries. Because the planning for these queries
|
||||
|
|
@ -450,7 +540,6 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es)
|
|||
{
|
||||
DistributedSubPlan *subPlan = (DistributedSubPlan *) lfirst(subPlanCell);
|
||||
PlannedStmt *plan = subPlan->plan;
|
||||
IntoClause *into = NULL;
|
||||
ParamListInfo params = NULL;
|
||||
|
||||
/*
|
||||
|
|
@ -534,6 +623,11 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es)
|
|||
|
||||
ExplainOpenGroup("PlannedStmt", "PlannedStmt", false, es);
|
||||
|
||||
DestReceiver *dest = None_Receiver; /* No query execution */
|
||||
double executionDurationMillisec = 0.0;
|
||||
double executionTuples = 0;
|
||||
double executionLoops = 0;
|
||||
|
||||
/* Capture memory stats on PG17+ */
|
||||
#if PG_VERSION_NUM >= PG_VERSION_17
|
||||
if (es->memory)
|
||||
|
|
@ -541,46 +635,21 @@ ExplainSubPlans(DistributedPlan *distributedPlan, ExplainState *es)
|
|||
MemoryContextSwitchTo(saved_ctx);
|
||||
MemoryContextMemConsumed(planner_ctx, &mem_counters);
|
||||
}
|
||||
#endif
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
ExplainOnePlan(
|
||||
plan, /* PlannedStmt *plannedstmt */
|
||||
NULL, /* CachedPlan *cplan */
|
||||
NULL, /* CachedPlanSource *plansource */
|
||||
0, /* query_index */
|
||||
into, /* IntoClause *into */
|
||||
es, /* struct ExplainState *es */
|
||||
queryString, /* const char *queryString */
|
||||
params, /* ParamListInfo params */
|
||||
NULL, /* QueryEnvironment *queryEnv */
|
||||
&planduration, /* const instr_time *planduration */
|
||||
(es->buffers ? &bufusage : NULL),/* const BufferUsage *bufusage */
|
||||
(es->memory ? &mem_counters : NULL) /* const MemoryContextCounters *mem_counters */
|
||||
);
|
||||
#elif PG_VERSION_NUM >= PG_VERSION_17
|
||||
ExplainOnePlan(
|
||||
plan,
|
||||
into,
|
||||
es,
|
||||
queryString,
|
||||
params,
|
||||
NULL, /* QueryEnvironment *queryEnv */
|
||||
&planduration,
|
||||
(es->buffers ? &bufusage : NULL),
|
||||
(es->memory ? &mem_counters : NULL)
|
||||
);
|
||||
/* Execute EXPLAIN without ANALYZE */
|
||||
ExplainWorkerPlan(plan, subPlan, dest, es, queryString, params, NULL,
|
||||
&planduration,
|
||||
(es->buffers ? &bufusage : NULL),
|
||||
(es->memory ? &mem_counters : NULL),
|
||||
&executionDurationMillisec,
|
||||
&executionTuples,
|
||||
&executionLoops);
|
||||
#else
|
||||
ExplainOnePlan(
|
||||
plan,
|
||||
into,
|
||||
es,
|
||||
queryString,
|
||||
params,
|
||||
NULL, /* QueryEnvironment *queryEnv */
|
||||
&planduration,
|
||||
(es->buffers ? &bufusage : NULL)
|
||||
);
|
||||
|
||||
/* Execute EXPLAIN without ANALYZE */
|
||||
ExplainWorkerPlan(plan, subPlan, dest, es, queryString, params, NULL,
|
||||
&planduration, &executionDurationMillisec,
|
||||
&executionTuples, &executionLoops);
|
||||
#endif
|
||||
|
||||
ExplainCloseGroup("PlannedStmt", "PlannedStmt", false, es);
|
||||
|
|
@ -1251,17 +1320,19 @@ worker_last_saved_explain_analyze(PG_FUNCTION_ARGS)
|
|||
if (SavedExplainPlan != NULL)
|
||||
{
|
||||
int columnCount = tupleDescriptor->natts;
|
||||
if (columnCount != 2)
|
||||
if (columnCount != 4)
|
||||
{
|
||||
ereport(ERROR, (errmsg("expected 3 output columns in definition of "
|
||||
ereport(ERROR, (errmsg("expected 4 output columns in definition of "
|
||||
"worker_last_saved_explain_analyze, but got %d",
|
||||
columnCount)));
|
||||
}
|
||||
|
||||
bool columnNulls[2] = { false };
|
||||
Datum columnValues[2] = {
|
||||
bool columnNulls[4] = { false };
|
||||
Datum columnValues[4] = {
|
||||
CStringGetTextDatum(SavedExplainPlan),
|
||||
Float8GetDatum(SavedExecutionDurationMillisec)
|
||||
Float8GetDatum(SavedExecutionDurationMillisec),
|
||||
Float8GetDatum(SavedExplainPlanNtuples),
|
||||
Float8GetDatum(SavedExplainPlanNloops)
|
||||
};
|
||||
|
||||
tuplestore_putvalues(tupleStore, tupleDescriptor, columnValues, columnNulls);
|
||||
|
|
@ -1282,6 +1353,8 @@ worker_save_query_explain_analyze(PG_FUNCTION_ARGS)
|
|||
text *queryText = PG_GETARG_TEXT_P(0);
|
||||
char *queryString = text_to_cstring(queryText);
|
||||
double executionDurationMillisec = 0.0;
|
||||
double executionTuples = 0;
|
||||
double executionLoops = 0;
|
||||
|
||||
Datum explainOptions = PG_GETARG_DATUM(1);
|
||||
ExplainState *es = NewExplainState();
|
||||
|
|
@ -1398,16 +1471,19 @@ worker_save_query_explain_analyze(PG_FUNCTION_ARGS)
|
|||
}
|
||||
|
||||
/* do the actual EXPLAIN ANALYZE */
|
||||
ExplainWorkerPlan(plan, tupleStoreDest, es, queryString, boundParams, NULL,
|
||||
ExplainWorkerPlan(plan, NULL, tupleStoreDest, es, queryString, boundParams, NULL,
|
||||
&planDuration,
|
||||
(es->buffers ? &bufusage : NULL),
|
||||
(es->memory ? &mem_counters : NULL),
|
||||
&executionDurationMillisec);
|
||||
&executionDurationMillisec,
|
||||
&executionTuples,
|
||||
&executionLoops);
|
||||
#else
|
||||
|
||||
/* do the actual EXPLAIN ANALYZE */
|
||||
ExplainWorkerPlan(plan, tupleStoreDest, es, queryString, boundParams, NULL,
|
||||
&planDuration, &executionDurationMillisec);
|
||||
ExplainWorkerPlan(plan, NULL, tupleStoreDest, es, queryString, boundParams, NULL,
|
||||
&planDuration, &executionDurationMillisec,
|
||||
&executionTuples, &executionLoops);
|
||||
#endif
|
||||
|
||||
ExplainEndOutput(es);
|
||||
|
|
@ -1418,6 +1494,8 @@ worker_save_query_explain_analyze(PG_FUNCTION_ARGS)
|
|||
|
||||
SavedExplainPlan = pstrdup(es->str->data);
|
||||
SavedExecutionDurationMillisec = executionDurationMillisec;
|
||||
SavedExplainPlanNtuples = executionTuples;
|
||||
SavedExplainPlanNloops = executionLoops;
|
||||
|
||||
MemoryContextSwitchTo(oldContext);
|
||||
|
||||
|
|
@ -1606,22 +1684,7 @@ CitusExplainOneQuery(Query *query, int cursorOptions, IntoClause *into,
|
|||
}
|
||||
#endif
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
ExplainOnePlan(
|
||||
plan, /* PlannedStmt *plannedstmt */
|
||||
NULL, /* no CachedPlan */
|
||||
NULL, /* no CachedPlanSource */
|
||||
0, /* query_index */
|
||||
into, /* IntoClause *into */
|
||||
es, /* struct ExplainState *es */
|
||||
queryString, /* const char *queryString */
|
||||
params, /* ParamListInfo params */
|
||||
queryEnv, /* QueryEnvironment *queryEnv */
|
||||
&planduration, /* const instr_time *planduration */
|
||||
(es->buffers ? &bufusage : NULL), /* const BufferUsage *bufusage */
|
||||
(es->memory ? &mem_counters : NULL) /* const MemoryContextCounters *mem_counters */
|
||||
);
|
||||
#elif PG_VERSION_NUM >= PG_VERSION_17
|
||||
#if PG_VERSION_NUM >= PG_VERSION_17
|
||||
|
||||
/* PostgreSQL 17 signature (9 args: includes mem_counters) */
|
||||
ExplainOnePlan(
|
||||
|
|
@ -1662,11 +1725,13 @@ CreateExplainAnlyzeDestination(Task *task, TupleDestination *taskDest)
|
|||
tupleDestination->originalTask = task;
|
||||
tupleDestination->originalTaskDestination = taskDest;
|
||||
|
||||
TupleDesc lastSavedExplainAnalyzeTupDesc = CreateTemplateTupleDesc(2);
|
||||
TupleDesc lastSavedExplainAnalyzeTupDesc = CreateTemplateTupleDesc(4);
|
||||
|
||||
TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 1, "explain analyze", TEXTOID, 0,
|
||||
0);
|
||||
TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 2, "duration", FLOAT8OID, 0, 0);
|
||||
TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 3, "ntuples", FLOAT8OID, 0, 0);
|
||||
TupleDescInitEntry(lastSavedExplainAnalyzeTupDesc, 4, "nloops", FLOAT8OID, 0, 0);
|
||||
|
||||
tupleDestination->lastSavedExplainAnalyzeTupDesc = lastSavedExplainAnalyzeTupDesc;
|
||||
|
||||
|
|
@ -1677,6 +1742,51 @@ CreateExplainAnlyzeDestination(Task *task, TupleDestination *taskDest)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* EnsureExplainOutputCapacity is to ensure capacity for new entries. Input
|
||||
* parameter requiredSize is minimum number of elements needed.
|
||||
*/
|
||||
static void
|
||||
EnsureExplainOutputCapacity(int requiredSize)
|
||||
{
|
||||
if (requiredSize < TotalExplainOutputCapacity)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
int newCapacity =
|
||||
(TotalExplainOutputCapacity == 0) ? 32 : TotalExplainOutputCapacity * 2;
|
||||
|
||||
while (newCapacity <= requiredSize)
|
||||
{
|
||||
newCapacity *= 2;
|
||||
}
|
||||
|
||||
if (SubPlanExplainOutput == NULL)
|
||||
{
|
||||
SubPlanExplainOutput =
|
||||
(SubPlanExplainOutputData *) MemoryContextAllocZero(
|
||||
SubPlanExplainAnalyzeContext,
|
||||
newCapacity *
|
||||
sizeof(SubPlanExplainOutputData));
|
||||
}
|
||||
else
|
||||
{
|
||||
/* Use repalloc and manually zero the new memory */
|
||||
int oldSize = TotalExplainOutputCapacity * sizeof(SubPlanExplainOutputData);
|
||||
int newSize = newCapacity * sizeof(SubPlanExplainOutputData);
|
||||
|
||||
SubPlanExplainOutput =
|
||||
(SubPlanExplainOutputData *) repalloc(SubPlanExplainOutput, newSize);
|
||||
|
||||
/* Zero out the newly allocated memory */
|
||||
MemSet((char *) SubPlanExplainOutput + oldSize, 0, newSize - oldSize);
|
||||
}
|
||||
|
||||
TotalExplainOutputCapacity = newCapacity;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ExplainAnalyzeDestPutTuple implements TupleDestination->putTuple
|
||||
* for ExplainAnalyzeDestination.
|
||||
|
|
@ -1686,6 +1796,8 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task,
|
|||
int placementIndex, int queryNumber,
|
||||
HeapTuple heapTuple, uint64 tupleLibpqSize)
|
||||
{
|
||||
uint32 taskId = task->taskId;
|
||||
|
||||
ExplainAnalyzeDestination *tupleDestination = (ExplainAnalyzeDestination *) self;
|
||||
if (queryNumber == 0)
|
||||
{
|
||||
|
|
@ -1693,6 +1805,13 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task,
|
|||
originalTupDest->putTuple(originalTupDest, task, placementIndex, 0, heapTuple,
|
||||
tupleLibpqSize);
|
||||
tupleDestination->originalTask->totalReceivedTupleData += tupleLibpqSize;
|
||||
|
||||
if (SubPlanExplainAnalyzeContext)
|
||||
{
|
||||
EnsureExplainOutputCapacity(taskId + 1);
|
||||
SubPlanExplainOutput[taskId].totalReceivedTupleData =
|
||||
tupleDestination->originalTask->totalReceivedTupleData;
|
||||
}
|
||||
}
|
||||
else if (queryNumber == 1)
|
||||
{
|
||||
|
|
@ -1708,6 +1827,8 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task,
|
|||
}
|
||||
|
||||
Datum executionDuration = heap_getattr(heapTuple, 2, tupDesc, &isNull);
|
||||
Datum executionTuples = heap_getattr(heapTuple, 3, tupDesc, &isNull);
|
||||
Datum executionLoops = heap_getattr(heapTuple, 4, tupDesc, &isNull);
|
||||
|
||||
if (isNull)
|
||||
{
|
||||
|
|
@ -1717,6 +1838,8 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task,
|
|||
|
||||
char *fetchedExplainAnalyzePlan = TextDatumGetCString(explainAnalyze);
|
||||
double fetchedExplainAnalyzeExecutionDuration = DatumGetFloat8(executionDuration);
|
||||
double fetchedExplainAnalyzeTuples = DatumGetFloat8(executionTuples);
|
||||
double fetchedExplainAnalyzeLoops = DatumGetFloat8(executionLoops);
|
||||
|
||||
/*
|
||||
* Allocate fetchedExplainAnalyzePlan in the same context as the Task, since we are
|
||||
|
|
@ -1742,6 +1865,20 @@ ExplainAnalyzeDestPutTuple(TupleDestination *self, Task *task,
|
|||
placementIndex;
|
||||
tupleDestination->originalTask->fetchedExplainAnalyzeExecutionDuration =
|
||||
fetchedExplainAnalyzeExecutionDuration;
|
||||
|
||||
/* We should build tupleDestination in subPlan similar to the above */
|
||||
if (SubPlanExplainAnalyzeContext)
|
||||
{
|
||||
EnsureExplainOutputCapacity(taskId + 1);
|
||||
SubPlanExplainOutput[taskId].explainOutput =
|
||||
MemoryContextStrdup(SubPlanExplainAnalyzeContext,
|
||||
fetchedExplainAnalyzePlan);
|
||||
SubPlanExplainOutput[taskId].executionDuration =
|
||||
fetchedExplainAnalyzeExecutionDuration;
|
||||
SubPlanExplainOutput[taskId].executionNtuples = fetchedExplainAnalyzeTuples;
|
||||
SubPlanExplainOutput[taskId].executionNloops = fetchedExplainAnalyzeLoops;
|
||||
NumTasksOutput++;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
|
|
@ -1804,7 +1941,14 @@ ExplainAnalyzeDestTupleDescForQuery(TupleDestination *self, int queryNumber)
|
|||
bool
|
||||
RequestedForExplainAnalyze(CitusScanState *node)
|
||||
{
|
||||
return (node->customScanState.ss.ps.state->es_instrument != 0);
|
||||
/*
|
||||
* When running a distributed plan—either the root plan or a subplan’s
|
||||
* distributed fragment—we need to know if we’re under EXPLAIN ANALYZE.
|
||||
* Subplans can’t receive the EXPLAIN ANALYZE flag directly, so we use
|
||||
* SubPlanExplainAnalyzeContext as a flag to indicate that context.
|
||||
*/
|
||||
return (node->customScanState.ss.ps.state->es_instrument != 0) ||
|
||||
(SubPlanLevel > 0 && SubPlanExplainAnalyzeContext);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -1963,7 +2107,8 @@ FetchPlanQueryForExplainAnalyze(const char *queryString, ParamListInfo params)
|
|||
}
|
||||
|
||||
appendStringInfoString(fetchQuery,
|
||||
"SELECT explain_analyze_output, execution_duration "
|
||||
"SELECT explain_analyze_output, execution_duration, "
|
||||
"execution_ntuples, execution_nloops "
|
||||
"FROM worker_last_saved_explain_analyze()");
|
||||
|
||||
return fetchQuery->data;
|
||||
|
|
@ -2107,22 +2252,7 @@ ExplainOneQuery(Query *query, int cursorOptions,
|
|||
}
|
||||
#endif
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
ExplainOnePlan(
|
||||
plan, /* PlannedStmt *plannedstmt */
|
||||
NULL, /* CachedPlan *cplan */
|
||||
NULL, /* CachedPlanSource *plansource */
|
||||
0, /* query_index */
|
||||
into, /* IntoClause *into */
|
||||
es, /* struct ExplainState *es */
|
||||
queryString, /* const char *queryString */
|
||||
params, /* ParamListInfo params */
|
||||
queryEnv, /* QueryEnvironment *queryEnv */
|
||||
&planduration, /* const instr_time *planduration */
|
||||
(es->buffers ? &bufusage : NULL),
|
||||
(es->memory ? &mem_counters: NULL)
|
||||
);
|
||||
#elif PG_VERSION_NUM >= PG_VERSION_17
|
||||
#if PG_VERSION_NUM >= PG_VERSION_17
|
||||
ExplainOnePlan(
|
||||
plan,
|
||||
into,
|
||||
|
|
@ -2146,11 +2276,24 @@ ExplainOneQuery(Query *query, int cursorOptions,
|
|||
(es->buffers ? &bufusage : NULL)
|
||||
);
|
||||
#endif
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* PlanStateAnalyzeWalker Tree walker callback that visits each PlanState node in the
|
||||
* plan tree and extracts analyze statistics from CustomScanState tasks using
|
||||
* ExtractAnalyzeStats. Always returns false to recurse into all children.
|
||||
*/
|
||||
static bool
|
||||
PlanStateAnalyzeWalker(PlanState *planState, void *ctx)
|
||||
{
|
||||
DistributedSubPlan *subplan = (DistributedSubPlan *) ctx;
|
||||
ExtractAnalyzeStats(subplan, planState);
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ExplainWorkerPlan produces explain output into es. If es->analyze, it also executes
|
||||
* the given plannedStmt and sends the results to dest. It puts total time to execute in
|
||||
|
|
@ -2165,20 +2308,25 @@ ExplainOneQuery(Query *query, int cursorOptions,
|
|||
* destination.
|
||||
*/
|
||||
static void
|
||||
ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es,
|
||||
ExplainWorkerPlan(PlannedStmt *plannedstmt, DistributedSubPlan *subPlan, DestReceiver *dest, ExplainState *es,
|
||||
const char *queryString, ParamListInfo params, QueryEnvironment *queryEnv,
|
||||
const instr_time *planduration,
|
||||
#if PG_VERSION_NUM >= PG_VERSION_17
|
||||
const BufferUsage *bufusage,
|
||||
const MemoryContextCounters *mem_counters,
|
||||
#endif
|
||||
double *executionDurationMillisec)
|
||||
double *executionDurationMillisec,
|
||||
double *executionTuples,
|
||||
double *executionLoops)
|
||||
{
|
||||
QueryDesc *queryDesc;
|
||||
instr_time starttime;
|
||||
double totaltime = 0;
|
||||
int eflags;
|
||||
int instrument_option = 0;
|
||||
/* Sub-plan already executed; skipping execution */
|
||||
bool executeQuery = (es->analyze && !subPlan);
|
||||
bool executeSubplan = (es->analyze && subPlan);
|
||||
|
||||
Assert(plannedstmt->commandType != CMD_UTILITY);
|
||||
|
||||
|
|
@ -2208,19 +2356,6 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es
|
|||
UpdateActiveSnapshotCommandId();
|
||||
|
||||
/* Create a QueryDesc for the query */
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
queryDesc = CreateQueryDesc(
|
||||
plannedstmt, /* PlannedStmt *plannedstmt */
|
||||
NULL, /* CachedPlan *cplan (none) */
|
||||
queryString, /* const char *sourceText */
|
||||
GetActiveSnapshot(), /* Snapshot snapshot */
|
||||
InvalidSnapshot, /* Snapshot crosscheck_snapshot */
|
||||
dest, /* DestReceiver *dest */
|
||||
params, /* ParamListInfo params */
|
||||
queryEnv, /* QueryEnvironment *queryEnv */
|
||||
instrument_option /* int instrument_options */
|
||||
);
|
||||
#else
|
||||
queryDesc = CreateQueryDesc(
|
||||
plannedstmt, /* PlannedStmt *plannedstmt */
|
||||
queryString, /* const char *sourceText */
|
||||
|
|
@ -2231,10 +2366,9 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es
|
|||
queryEnv, /* QueryEnvironment *queryEnv */
|
||||
instrument_option /* int instrument_options */
|
||||
);
|
||||
#endif
|
||||
|
||||
/* Select execution options */
|
||||
if (es->analyze)
|
||||
if (executeQuery)
|
||||
eflags = 0; /* default run-to-completion flags */
|
||||
else
|
||||
eflags = EXEC_FLAG_EXPLAIN_ONLY;
|
||||
|
|
@ -2243,7 +2377,7 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es
|
|||
ExecutorStart(queryDesc, eflags);
|
||||
|
||||
/* Execute the plan for statistics if asked for */
|
||||
if (es->analyze)
|
||||
if (executeQuery)
|
||||
{
|
||||
ScanDirection dir = ForwardScanDirection;
|
||||
|
||||
|
|
@ -2266,6 +2400,12 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es
|
|||
|
||||
ExplainOpenGroup("Query", NULL, true, es);
|
||||
|
||||
if (executeSubplan)
|
||||
{
|
||||
ExtractAnalyzeStats(subPlan, queryDesc->planstate);
|
||||
planstate_tree_walker(queryDesc->planstate, PlanStateAnalyzeWalker, (void *) subPlan);
|
||||
}
|
||||
|
||||
/* Create textual dump of plan tree */
|
||||
ExplainPrintPlan(es, queryDesc);
|
||||
|
||||
|
|
@ -2338,6 +2478,13 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es
|
|||
*/
|
||||
INSTR_TIME_SET_CURRENT(starttime);
|
||||
|
||||
if (executeQuery)
|
||||
{
|
||||
Instrumentation *instr = queryDesc->planstate->instrument;
|
||||
*executionTuples = instr->ntuples;
|
||||
*executionLoops = instr->nloops;
|
||||
}
|
||||
|
||||
ExecutorEnd(queryDesc);
|
||||
|
||||
FreeQueryDesc(queryDesc);
|
||||
|
|
@ -2345,7 +2492,7 @@ ExplainWorkerPlan(PlannedStmt *plannedstmt, DestReceiver *dest, ExplainState *es
|
|||
PopActiveSnapshot();
|
||||
|
||||
/* We need a CCI just in case query expanded to multiple plans */
|
||||
if (es->analyze)
|
||||
if (executeQuery)
|
||||
CommandCounterIncrement();
|
||||
|
||||
totaltime += elapsed_time(&starttime);
|
||||
|
|
|
|||
|
|
@ -16,6 +16,8 @@
|
|||
#include "postgres.h"
|
||||
|
||||
#include "access/stratnum.h"
|
||||
#include "access/tupdesc.h"
|
||||
#include "access/tupdesc_details.h"
|
||||
#include "access/xact.h"
|
||||
#include "catalog/pg_opfamily.h"
|
||||
#include "catalog/pg_proc.h"
|
||||
|
|
@ -34,6 +36,7 @@
|
|||
#include "optimizer/pathnode.h"
|
||||
#include "optimizer/paths.h"
|
||||
#include "optimizer/planmain.h"
|
||||
#include "optimizer/planner.h"
|
||||
#include "optimizer/restrictinfo.h"
|
||||
#include "parser/parse_oper.h"
|
||||
#include "parser/parsetree.h"
|
||||
|
|
@ -81,6 +84,7 @@
|
|||
#include "distributed/relay_utility.h"
|
||||
#include "distributed/resource_lock.h"
|
||||
#include "distributed/shard_pruning.h"
|
||||
#include "distributed/shard_utils.h"
|
||||
#include "distributed/shardinterval_utils.h"
|
||||
|
||||
/* intermediate value for INSERT processing */
|
||||
|
|
@ -164,7 +168,7 @@ static List * SingleShardTaskList(Query *query, uint64 jobId,
|
|||
List *relationShardList, List *placementList,
|
||||
uint64 shardId, bool parametersInQueryResolved,
|
||||
bool isLocalTableModification, Const *partitionKeyValue,
|
||||
int colocationId);
|
||||
int colocationId, bool delayedFastPath);
|
||||
static bool RowLocksOnRelations(Node *node, List **rtiLockList);
|
||||
static void ReorderTaskPlacementsByTaskAssignmentPolicy(Job *job,
|
||||
TaskAssignmentPolicyType
|
||||
|
|
@ -173,7 +177,7 @@ static void ReorderTaskPlacementsByTaskAssignmentPolicy(Job *job,
|
|||
static bool ModifiesLocalTableWithRemoteCitusLocalTable(List *rangeTableList);
|
||||
static DeferredErrorMessage * DeferErrorIfUnsupportedLocalTableJoin(List *rangeTableList);
|
||||
static bool IsLocallyAccessibleCitusLocalTable(Oid relationId);
|
||||
|
||||
static bool ConvertToQueryOnShard(Query *query, Oid relationID, Oid shardRelationId);
|
||||
|
||||
/*
|
||||
* CreateRouterPlan attempts to create a router executor plan for the given
|
||||
|
|
@ -1940,7 +1944,9 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
|
|||
{
|
||||
GenerateSingleShardRouterTaskList(job, relationShardList,
|
||||
placementList, shardId,
|
||||
isLocalTableModification);
|
||||
isLocalTableModification,
|
||||
fastPathRestrictionContext->
|
||||
delayFastPathPlanning);
|
||||
}
|
||||
|
||||
job->requiresCoordinatorEvaluation = requiresCoordinatorEvaluation;
|
||||
|
|
@ -1948,6 +1954,262 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* CheckAttributesMatch checks if the attributes of the Citus table and the shard
|
||||
* table match.
|
||||
*
|
||||
* It is used to ensure that the shard table has the same schema as the Citus
|
||||
* table before replacing the Citus table OID with the shard table OID in the
|
||||
* parse tree we (Citus planner) recieved from Postgres.
|
||||
*/
|
||||
static
|
||||
bool
|
||||
CheckAttributesMatch(Oid citusTableId, Oid shardTableId)
|
||||
{
|
||||
bool same_schema = false;
|
||||
Relation citusRelation = RelationIdGetRelation(citusTableId);
|
||||
Relation shardRelation = RelationIdGetRelation(shardTableId);
|
||||
|
||||
if (RelationIsValid(citusRelation) && RelationIsValid(shardRelation))
|
||||
{
|
||||
TupleDesc citusTupDesc = citusRelation->rd_att;
|
||||
TupleDesc shardTupDesc = shardRelation->rd_att;
|
||||
|
||||
if (citusTupDesc->natts == shardTupDesc->natts)
|
||||
{
|
||||
/*
|
||||
* Do an attribute-by-attribute comparison. This is borrowed from
|
||||
* the Postgres function equalTupleDescs(), which we cannot use
|
||||
* because the citus table and shard table have different composite
|
||||
* types.
|
||||
*/
|
||||
same_schema = true;
|
||||
for (int i = 0; i < citusTupDesc->natts && same_schema; i++)
|
||||
{
|
||||
Form_pg_attribute attr1 = TupleDescAttr(citusTupDesc, i);
|
||||
Form_pg_attribute attr2 = TupleDescAttr(shardTupDesc, i);
|
||||
|
||||
if (strcmp(NameStr(attr1->attname), NameStr(attr2->attname)) != 0)
|
||||
{
|
||||
same_schema = false;
|
||||
}
|
||||
if (attr1->atttypid != attr2->atttypid)
|
||||
{
|
||||
same_schema = false;
|
||||
}
|
||||
if (attr1->atttypmod != attr2->atttypmod)
|
||||
{
|
||||
same_schema = false;
|
||||
}
|
||||
if (attr1->attcollation != attr2->attcollation)
|
||||
{
|
||||
same_schema = false;
|
||||
}
|
||||
|
||||
/* Record types derived from tables could have dropped fields. */
|
||||
if (attr1->attisdropped != attr2->attisdropped)
|
||||
{
|
||||
same_schema = false;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
RelationClose(citusRelation);
|
||||
RelationClose(shardRelation);
|
||||
return same_schema;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CheckAndBuildDelayedFastPathPlan() - if the query being planned is a fast
|
||||
* path query, not marked for deferred pruning and the placement for the task
|
||||
* is not a dummy placement then if the placement is local to this node we can
|
||||
* take a shortcut of replacing the OID of the citus table with the OID of the
|
||||
* shard in the query tree and plan that directly, instead of deparsing the
|
||||
* parse tree to a SQL query on the shard and parsing and planning that in
|
||||
* the local executor. Instead, the local executor can use the plan created
|
||||
* here.
|
||||
*/
|
||||
void
|
||||
CheckAndBuildDelayedFastPathPlan(DistributedPlanningContext *planContext,
|
||||
DistributedPlan *plan)
|
||||
{
|
||||
FastPathRestrictionContext *fastPathContext =
|
||||
planContext->plannerRestrictionContext->fastPathRestrictionContext;
|
||||
|
||||
if (!fastPathContext->delayFastPathPlanning)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
Job *job = plan->workerJob;
|
||||
Assert(job != NULL);
|
||||
|
||||
if (job->deferredPruning)
|
||||
{
|
||||
/* Execution time pruning => don't know which shard at this point */
|
||||
planContext->plan = FastPathPlanner(planContext->originalQuery,
|
||||
planContext->query,
|
||||
planContext->boundParams);
|
||||
return;
|
||||
}
|
||||
|
||||
List *tasks = job->taskList;
|
||||
Assert(list_length(tasks) == 1);
|
||||
Task *task = (Task *) linitial(tasks);
|
||||
List *placements = task->taskPlacementList;
|
||||
int32 localGroupId = GetLocalGroupId();
|
||||
ShardPlacement *primaryPlacement = (ShardPlacement *) linitial(placements);
|
||||
|
||||
bool isLocalExecution = !IsDummyPlacement(primaryPlacement) &&
|
||||
(primaryPlacement->groupId == localGroupId);
|
||||
bool canBuildLocalPlan = true;
|
||||
|
||||
if (isLocalExecution)
|
||||
{
|
||||
List *relationShards = task->relationShardList;
|
||||
Assert(list_length(relationShards) == 1);
|
||||
RelationShard *relationShard = (RelationShard *) linitial(relationShards);
|
||||
Assert(relationShard->shardId == primaryPlacement->shardId);
|
||||
|
||||
/*
|
||||
* Today FastPathRouterQuery() doesn't set delayFastPathPlanning to true for
|
||||
* reference tables. We should be looking at 1 placement, or their replication
|
||||
* factor.
|
||||
*/
|
||||
Assert(list_length(placements) == 1 || list_length(placements) ==
|
||||
TableShardReplicationFactor(relationShard->relationId));
|
||||
|
||||
canBuildLocalPlan = ConvertToQueryOnShard(planContext->query,
|
||||
relationShard->relationId,
|
||||
relationShard->shardId);
|
||||
if (canBuildLocalPlan)
|
||||
{
|
||||
/* Plan the query with the new shard relation id */
|
||||
planContext->plan = standard_planner(planContext->query, NULL,
|
||||
planContext->cursorOptions,
|
||||
planContext->boundParams);
|
||||
SetTaskQueryPlan(task, job->jobQuery, planContext->plan);
|
||||
|
||||
ereport(DEBUG2, (errmsg(
|
||||
"Fast-path router query: created local execution plan "
|
||||
"to avoid deparse and compile of shard query")));
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
/*
|
||||
* Either the shard is not local to this node, or it was not safe to replace
|
||||
* the OIDs in the parse tree; in any case we fall back to generating the shard
|
||||
* query and compiling that.
|
||||
*/
|
||||
Assert(!isLocalExecution || (isLocalExecution && !canBuildLocalPlan));
|
||||
|
||||
/* Fall back to fast path planner and generating SQL query on the shard */
|
||||
planContext->plan = FastPathPlanner(planContext->originalQuery,
|
||||
planContext->query,
|
||||
planContext->boundParams);
|
||||
UpdateRelationToShardNames((Node *) job->jobQuery, task->relationShardList);
|
||||
SetTaskQueryIfShouldLazyDeparse(task, job->jobQuery);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ConvertToQueryOnShard() converts the given query on a citus table (identified by
|
||||
* citusTableOid) to a query on a shard (identified by shardId).
|
||||
*
|
||||
* The function assumes that the query is a "fast path" query - it has only one
|
||||
* RangeTblEntry and one RTEPermissionInfo.
|
||||
*
|
||||
* It acquires the same lock on the shard that was acquired on the citus table
|
||||
* by the Postgres parser. It checks that the attribute numbers and metadata of
|
||||
* the shard table and citus table are identical - otherwise it is not safe
|
||||
* to proceed with this shortcut. Assuming the attributes do match, the actual
|
||||
* conversion involves changing the target list entries that reference the
|
||||
* citus table's oid to reference the shard's relation id instead. Finally,
|
||||
* it changes the RangeTblEntry's relid to the shard's relation id and (PG16+)
|
||||
* changes the RTEPermissionInfo's relid to the shard's relation id also.
|
||||
* At this point the Query is ready for the postgres planner.
|
||||
*/
|
||||
static bool
|
||||
ConvertToQueryOnShard(Query *query, Oid citusTableOid, Oid shardId)
|
||||
{
|
||||
Assert(list_length(query->rtable) == 1);
|
||||
RangeTblEntry *citusTableRte = (RangeTblEntry *) linitial(query->rtable);
|
||||
Assert(citusTableRte->relid == citusTableOid);
|
||||
|
||||
const char *citusTableName = get_rel_name(citusTableOid);
|
||||
Assert(citusTableName != NULL);
|
||||
|
||||
/* construct shard relation name */
|
||||
char *shardRelationName = pstrdup(citusTableName);
|
||||
AppendShardIdToName(&shardRelationName, shardId);
|
||||
|
||||
/* construct the schema name */
|
||||
char *schemaName = get_namespace_name(get_rel_namespace(citusTableOid));
|
||||
|
||||
/* now construct a range variable for the shard */
|
||||
RangeVar shardRangeVar = {
|
||||
.relname = shardRelationName,
|
||||
.schemaname = schemaName,
|
||||
.inh = citusTableRte->inh,
|
||||
.relpersistence = RELPERSISTENCE_PERMANENT,
|
||||
};
|
||||
|
||||
/* Must apply the same lock to the shard that was applied to the citus table */
|
||||
Oid shardRelationId = RangeVarGetRelidExtended(&shardRangeVar,
|
||||
citusTableRte->rellockmode,
|
||||
0, NULL, NULL);
|
||||
|
||||
/* Verify that the attributes of citus table and shard table match */
|
||||
if (!CheckAttributesMatch(citusTableOid, shardRelationId))
|
||||
{
|
||||
/* There is a difference between the attributes of the citus
|
||||
* table and the shard table. This can happen if there is a DROP
|
||||
* COLUMN on the citus table. In this case, we cannot
|
||||
* convert the query to a shard query, so clean up and return.
|
||||
*/
|
||||
UnlockRelationOid(shardRelationId, citusTableRte->rellockmode);
|
||||
ereport(DEBUG2, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||
errmsg(
|
||||
"Router planner fast path cannot modify parse tree for local execution: shard table \"%s.%s\" does not match the "
|
||||
"distributed table \"%s.%s\"",
|
||||
schemaName, shardRelationName, schemaName,
|
||||
citusTableName)));
|
||||
pfree(shardRelationName);
|
||||
pfree(schemaName);
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/* Change the target list entries that reference the original citus table's relation id */
|
||||
ListCell *lc = NULL;
|
||||
foreach(lc, query->targetList)
|
||||
{
|
||||
TargetEntry *targetEntry = (TargetEntry *) lfirst(lc);
|
||||
if (targetEntry->resorigtbl == citusTableOid)
|
||||
{
|
||||
targetEntry->resorigtbl = shardRelationId;
|
||||
}
|
||||
}
|
||||
|
||||
/* Change the range table entry's oid to that of the shard's */
|
||||
Assert(shardRelationId != InvalidOid);
|
||||
citusTableRte->relid = shardRelationId;
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_16
|
||||
|
||||
/* Change the range table permission oid to that of the shard's (PG16+) */
|
||||
Assert(list_length(query->rteperminfos) == 1);
|
||||
RTEPermissionInfo *rtePermInfo = (RTEPermissionInfo *) linitial(query->rteperminfos);
|
||||
rtePermInfo->relid = shardRelationId;
|
||||
#endif
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GenerateSingleShardRouterTaskList is a wrapper around other corresponding task
|
||||
* list generation functions specific to single shard selects and modifications.
|
||||
|
|
@ -1957,7 +2219,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
|
|||
void
|
||||
GenerateSingleShardRouterTaskList(Job *job, List *relationShardList,
|
||||
List *placementList, uint64 shardId, bool
|
||||
isLocalTableModification)
|
||||
isLocalTableModification, bool delayedFastPath)
|
||||
{
|
||||
Query *originalQuery = job->jobQuery;
|
||||
|
||||
|
|
@ -1970,7 +2232,8 @@ GenerateSingleShardRouterTaskList(Job *job, List *relationShardList,
|
|||
shardId,
|
||||
job->parametersInJobQueryResolved,
|
||||
isLocalTableModification,
|
||||
job->partitionKeyValue, job->colocationId);
|
||||
job->partitionKeyValue, job->colocationId,
|
||||
delayedFastPath);
|
||||
|
||||
/*
|
||||
* Queries to reference tables, or distributed tables with multiple replica's have
|
||||
|
|
@ -2001,7 +2264,8 @@ GenerateSingleShardRouterTaskList(Job *job, List *relationShardList,
|
|||
shardId,
|
||||
job->parametersInJobQueryResolved,
|
||||
isLocalTableModification,
|
||||
job->partitionKeyValue, job->colocationId);
|
||||
job->partitionKeyValue, job->colocationId,
|
||||
delayedFastPath);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -2096,7 +2360,7 @@ SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList,
|
|||
List *placementList, uint64 shardId,
|
||||
bool parametersInQueryResolved,
|
||||
bool isLocalTableModification, Const *partitionKeyValue,
|
||||
int colocationId)
|
||||
int colocationId, bool delayedFastPath)
|
||||
{
|
||||
TaskType taskType = READ_TASK;
|
||||
char replicationModel = 0;
|
||||
|
|
@ -2168,7 +2432,10 @@ SingleShardTaskList(Query *query, uint64 jobId, List *relationShardList,
|
|||
task->taskPlacementList = placementList;
|
||||
task->partitionKeyValue = partitionKeyValue;
|
||||
task->colocationId = colocationId;
|
||||
SetTaskQueryIfShouldLazyDeparse(task, query);
|
||||
if (!delayedFastPath)
|
||||
{
|
||||
SetTaskQueryIfShouldLazyDeparse(task, query);
|
||||
}
|
||||
task->anchorShardId = shardId;
|
||||
task->jobId = jobId;
|
||||
task->relationShardList = relationShardList;
|
||||
|
|
@ -2449,10 +2716,15 @@ PlanRouterQuery(Query *originalQuery,
|
|||
|
||||
/*
|
||||
* If this is an UPDATE or DELETE query which requires coordinator evaluation,
|
||||
* don't try update shard names, and postpone that to execution phase.
|
||||
* don't try update shard names, and postpone that to execution phase. Also, if
|
||||
* this is a delayed fast path query, we don't update the shard names
|
||||
* either, as the shard names will be updated in the fast path query planner.
|
||||
*/
|
||||
bool isUpdateOrDelete = UpdateOrDeleteOrMergeQuery(originalQuery);
|
||||
if (!(isUpdateOrDelete && RequiresCoordinatorEvaluation(originalQuery)))
|
||||
bool delayedFastPath =
|
||||
plannerRestrictionContext->fastPathRestrictionContext->delayFastPathPlanning;
|
||||
if (!(isUpdateOrDelete && RequiresCoordinatorEvaluation(originalQuery)) &&
|
||||
!delayedFastPath)
|
||||
{
|
||||
UpdateRelationToShardNames((Node *) originalQuery, *relationShardList);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -171,6 +171,14 @@ static bool FindQueryContainingRTEIdentityInternal(Node *node,
|
|||
|
||||
static int ParentCountPriorToAppendRel(List *appendRelList, AppendRelInfo *appendRelInfo);
|
||||
|
||||
static bool PartitionColumnSelectedForOuterJoin(Query *query,
|
||||
RelationRestrictionContext *
|
||||
restrictionContext,
|
||||
JoinRestrictionContext *
|
||||
joinRestrictionContext);
|
||||
|
||||
static bool PartitionColumnIsInTargetList(Query *query, JoinRestriction *joinRestriction,
|
||||
RelationRestrictionContext *restrictionContext);
|
||||
|
||||
/*
|
||||
* AllDistributionKeysInQueryAreEqual returns true if either
|
||||
|
|
@ -391,6 +399,80 @@ SafeToPushdownUnionSubquery(Query *originalQuery,
|
|||
return false;
|
||||
}
|
||||
|
||||
if (!PartitionColumnSelectedForOuterJoin(originalQuery,
|
||||
restrictionContext,
|
||||
joinRestrictionContext))
|
||||
{
|
||||
/* outer join does not select partition column of outer relation */
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* PartitionColumnSelectedForOuterJoin checks whether the partition column of
|
||||
* the outer relation is selected in the target list of the query.
|
||||
*
|
||||
* If there is no outer join, it returns true.
|
||||
*/
|
||||
static bool
|
||||
PartitionColumnSelectedForOuterJoin(Query *query,
|
||||
RelationRestrictionContext *restrictionContext,
|
||||
JoinRestrictionContext *joinRestrictionContext)
|
||||
{
|
||||
ListCell *joinRestrictionCell;
|
||||
foreach(joinRestrictionCell, joinRestrictionContext->joinRestrictionList)
|
||||
{
|
||||
JoinRestriction *joinRestriction = (JoinRestriction *) lfirst(
|
||||
joinRestrictionCell);
|
||||
|
||||
/* Restriction context includes alternative plans, sufficient to check for left joins.*/
|
||||
if (joinRestriction->joinType != JOIN_LEFT)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!PartitionColumnIsInTargetList(query, joinRestriction, restrictionContext))
|
||||
{
|
||||
/* outer join does not select partition column of outer relation */
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* PartitionColumnIsInTargetList checks whether the partition column of
|
||||
* the given relation is included in the target list of the query.
|
||||
*/
|
||||
static bool
|
||||
PartitionColumnIsInTargetList(Query *query, JoinRestriction *joinRestriction,
|
||||
RelationRestrictionContext *restrictionContext)
|
||||
{
|
||||
Relids relids = joinRestriction->outerrelRelids;
|
||||
int relationId = -1;
|
||||
Index partitionKeyIndex = InvalidAttrNumber;
|
||||
while ((relationId = bms_next_member(relids, relationId)) >= 0)
|
||||
{
|
||||
RangeTblEntry *rte = joinRestriction->plannerInfo->simple_rte_array[relationId];
|
||||
if (rte->rtekind != RTE_RELATION)
|
||||
{
|
||||
/* skip if it is not a relation */
|
||||
continue;
|
||||
}
|
||||
int targetRTEIndex = GetRTEIdentity(rte);
|
||||
PartitionKeyForRTEIdentityInQuery(query, targetRTEIndex,
|
||||
&partitionKeyIndex);
|
||||
if (partitionKeyIndex == 0)
|
||||
{
|
||||
/* partition key is not in the target list */
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -212,6 +212,7 @@ static const char * MaxSharedPoolSizeGucShowHook(void);
|
|||
static const char * LocalPoolSizeGucShowHook(void);
|
||||
static bool StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource
|
||||
source);
|
||||
static bool WarnIfLocalExecutionDisabled(bool *newval, void **extra, GucSource source);
|
||||
static void CitusAuthHook(Port *port, int status);
|
||||
static bool IsSuperuser(char *userName);
|
||||
static void AdjustDynamicLibraryPathForCdcDecoders(void);
|
||||
|
|
@ -385,51 +386,29 @@ static const struct config_enum_entry metadata_sync_mode_options[] = {
|
|||
|
||||
|
||||
/*----------------------------------------------------------------------*
|
||||
* On PG 18+ the hook signatures changed; we wrap the old Citus handlers
|
||||
* in fresh functions that match the new typedefs exactly.
|
||||
* On PG 18+ the hook signature changed; we wrap the old Citus handler
|
||||
* in a fresh function that matches the new typedef exactly.
|
||||
*----------------------------------------------------------------------*/
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
static bool
|
||||
citus_executor_start_adapter(QueryDesc *queryDesc, int eflags)
|
||||
{
|
||||
/* PG18+ expects a bool return */
|
||||
CitusExecutorStart(queryDesc, eflags);
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
citus_executor_run_adapter(QueryDesc *queryDesc,
|
||||
ScanDirection direction,
|
||||
uint64 count)
|
||||
{
|
||||
/* PG18+ has no run_once flag
|
||||
* call the original Citus hook (which still expects the old 4-arg form) */
|
||||
CitusExecutorRun(queryDesc, direction, count, true);
|
||||
}
|
||||
|
||||
|
||||
#else
|
||||
|
||||
/* PG15–17: adapter signatures must match the *old* typedefs */
|
||||
static void
|
||||
citus_executor_start_adapter(QueryDesc *queryDesc, int eflags)
|
||||
{
|
||||
CitusExecutorStart(queryDesc, eflags);
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
citus_executor_run_adapter(QueryDesc *queryDesc,
|
||||
ScanDirection direction,
|
||||
uint64 count,
|
||||
bool run_once)
|
||||
{
|
||||
CitusExecutorRun(queryDesc, direction, count, run_once);
|
||||
}
|
||||
|
||||
|
||||
uint64 count
|
||||
#if PG_VERSION_NUM < PG_VERSION_18
|
||||
, bool run_once
|
||||
#endif
|
||||
)
|
||||
{
|
||||
/* PG18+ has no run_once flag */
|
||||
CitusExecutorRun(queryDesc,
|
||||
direction,
|
||||
count,
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
true
|
||||
#else
|
||||
run_once
|
||||
#endif
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
/* shared library initialization function */
|
||||
|
|
@ -506,7 +485,7 @@ _PG_init(void)
|
|||
set_rel_pathlist_hook = multi_relation_restriction_hook;
|
||||
get_relation_info_hook = multi_get_relation_info_hook;
|
||||
set_join_pathlist_hook = multi_join_restriction_hook;
|
||||
ExecutorStart_hook = citus_executor_start_adapter;
|
||||
ExecutorStart_hook = CitusExecutorStart;
|
||||
ExecutorRun_hook = citus_executor_run_adapter;
|
||||
ExplainOneQuery_hook = CitusExplainOneQuery;
|
||||
prev_ExecutorEnd = ExecutorEnd_hook;
|
||||
|
|
@ -1426,6 +1405,17 @@ RegisterCitusConfigVariables(void)
|
|||
GUC_STANDARD,
|
||||
NULL, NULL, NULL);
|
||||
|
||||
DefineCustomBoolVariable(
|
||||
"citus.enable_local_fast_path_query_optimization",
|
||||
gettext_noop("Enables the planner to avoid a query deparse and planning if "
|
||||
"the shard is local to the current node."),
|
||||
NULL,
|
||||
&EnableLocalFastPathQueryOptimization,
|
||||
true,
|
||||
PGC_USERSET,
|
||||
GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE,
|
||||
WarnIfLocalExecutionDisabled, NULL, NULL);
|
||||
|
||||
DefineCustomBoolVariable(
|
||||
"citus.enable_local_reference_table_foreign_keys",
|
||||
gettext_noop("Enables foreign keys from/to local tables"),
|
||||
|
|
@ -2851,6 +2841,26 @@ WarnIfDeprecatedExecutorUsed(int *newval, void **extra, GucSource source)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* WarnIfLocalExecutionDisabled is used to emit a warning message when
|
||||
* enabling citus.enable_local_fast_path_query_optimization if
|
||||
* citus.enable_local_execution was disabled.
|
||||
*/
|
||||
static bool
|
||||
WarnIfLocalExecutionDisabled(bool *newval, void **extra, GucSource source)
|
||||
{
|
||||
if (*newval == true && EnableLocalExecution == false)
|
||||
{
|
||||
ereport(WARNING, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
|
||||
errmsg(
|
||||
"citus.enable_local_execution must be set in order for "
|
||||
"citus.enable_local_fast_path_query_optimization to be effective.")));
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* NoticeIfSubqueryPushdownEnabled prints a notice when a user sets
|
||||
* citus.subquery_pushdown to ON. It doesn't print the notice if the
|
||||
|
|
|
|||
|
|
@ -1,2 +1,3 @@
|
|||
-- citus--13.1-1--13.2-1
|
||||
-- bump version to 13.2-1
|
||||
#include "udfs/worker_last_saved_explain_analyze/13.2-1.sql"
|
||||
|
|
|
|||
|
|
@ -1,2 +1,5 @@
|
|||
-- citus--13.2-1--13.1-1
|
||||
-- downgrade version to 13.1-1
|
||||
|
||||
DROP FUNCTION IF EXISTS pg_catalog.worker_last_saved_explain_analyze();
|
||||
#include "../udfs/worker_last_saved_explain_analyze/9.4-1.sql"
|
||||
|
|
|
|||
10
src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/13.2-1.sql
generated
Normal file
10
src/backend/distributed/sql/udfs/worker_last_saved_explain_analyze/13.2-1.sql
generated
Normal file
|
|
@ -0,0 +1,10 @@
|
|||
|
||||
DROP FUNCTION pg_catalog.worker_last_saved_explain_analyze();
|
||||
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.worker_last_saved_explain_analyze()
|
||||
RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION,
|
||||
execution_ntuples DOUBLE PRECISION, execution_nloops DOUBLE PRECISION)
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus';
|
||||
COMMENT ON FUNCTION pg_catalog.worker_last_saved_explain_analyze() IS
|
||||
'Returns the saved explain analyze output for the last run query';
|
||||
|
|
@ -1,6 +1,9 @@
|
|||
|
||||
DROP FUNCTION pg_catalog.worker_last_saved_explain_analyze();
|
||||
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.worker_last_saved_explain_analyze()
|
||||
RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION)
|
||||
RETURNS TABLE(explain_analyze_output TEXT, execution_duration DOUBLE PRECISION,
|
||||
execution_ntuples DOUBLE PRECISION, execution_nloops DOUBLE PRECISION)
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus';
|
||||
COMMENT ON FUNCTION pg_catalog.worker_last_saved_explain_analyze() IS
|
||||
|
|
|
|||
|
|
@ -106,7 +106,7 @@ LogTransactionRecord(int32 groupId, char *transactionName, FullTransactionId out
|
|||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistTransaction);
|
||||
HeapTuple heapTuple = heap_form_tuple(tupleDescriptor, values, isNulls);
|
||||
|
||||
CatalogTupleInsert(pgDistTransaction, heapTuple);
|
||||
CATALOG_INSERT_WITH_SNAPSHOT(pgDistTransaction, heapTuple);
|
||||
|
||||
CommandCounterIncrement();
|
||||
|
||||
|
|
|
|||
|
|
@ -1907,20 +1907,6 @@ ExecuteSqlString(const char *sql)
|
|||
/* Don't display the portal in pg_cursors */
|
||||
portal->visible = false;
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
|
||||
/* PG18+ added a seventh “plansource” argument */
|
||||
PortalDefineQuery(
|
||||
portal,
|
||||
NULL, /* no prepared‐stmt name */
|
||||
sql, /* the query text */
|
||||
commandTag, /* the CommandTag */
|
||||
plantree_list, /* List of PlannedStmt* */
|
||||
NULL, /* no CachedPlan */
|
||||
NULL /* no CachedPlanSource */
|
||||
);
|
||||
#else
|
||||
|
||||
/* PG17-: six‐arg signature */
|
||||
PortalDefineQuery(
|
||||
portal,
|
||||
|
|
@ -1930,7 +1916,6 @@ ExecuteSqlString(const char *sql)
|
|||
plantree_list, /* List of PlannedStmt* */
|
||||
NULL /* no CachedPlan */
|
||||
);
|
||||
#endif
|
||||
|
||||
PortalStart(portal, NULL, 0, InvalidSnapshot);
|
||||
int16 format[] = { 1 };
|
||||
|
|
|
|||
|
|
@ -147,6 +147,31 @@ CopyNodeDistributedSubPlan(COPYFUNC_ARGS)
|
|||
|
||||
COPY_SCALAR_FIELD(subPlanId);
|
||||
COPY_NODE_FIELD(plan);
|
||||
COPY_SCALAR_FIELD(bytesSentPerWorker);
|
||||
COPY_SCALAR_FIELD(remoteWorkerCount);
|
||||
COPY_SCALAR_FIELD(durationMillisecs);
|
||||
COPY_SCALAR_FIELD(writeLocalFile);
|
||||
|
||||
if (newnode->totalExplainOutput)
|
||||
{
|
||||
MemSet(newnode->totalExplainOutput, 0, sizeof(newnode->totalExplainOutput));
|
||||
}
|
||||
|
||||
/* copy each SubPlanExplainOutput element */
|
||||
for (int i = 0; i < from->numTasksOutput; i++)
|
||||
{
|
||||
/* copy the explainOutput string pointer */
|
||||
COPY_STRING_FIELD(totalExplainOutput[i].explainOutput);
|
||||
|
||||
/* copy the executionDuration (double) */
|
||||
COPY_SCALAR_FIELD(totalExplainOutput[i].executionDuration);
|
||||
|
||||
/* copy the totalReceivedTupleData (uint64) */
|
||||
COPY_SCALAR_FIELD(totalExplainOutput[i].totalReceivedTupleData);
|
||||
}
|
||||
|
||||
COPY_SCALAR_FIELD(numTasksOutput);
|
||||
COPY_SCALAR_FIELD(ntuples);
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -287,6 +312,15 @@ CopyTaskQuery(Task *newnode, Task *from)
|
|||
break;
|
||||
}
|
||||
|
||||
case TASK_QUERY_LOCAL_PLAN:
|
||||
{
|
||||
newnode->taskQuery.data.localCompiled =
|
||||
(LocalCompilation *) palloc0(sizeof(LocalCompilation));
|
||||
COPY_NODE_FIELD(taskQuery.data.localCompiled->plan);
|
||||
COPY_NODE_FIELD(taskQuery.data.localCompiled->query);
|
||||
break;
|
||||
}
|
||||
|
||||
default:
|
||||
{
|
||||
break;
|
||||
|
|
|
|||
|
|
@ -215,6 +215,48 @@ OutDistributedSubPlan(OUTFUNC_ARGS)
|
|||
|
||||
WRITE_UINT_FIELD(subPlanId);
|
||||
WRITE_NODE_FIELD(plan);
|
||||
WRITE_UINT64_FIELD(bytesSentPerWorker);
|
||||
WRITE_INT_FIELD(remoteWorkerCount);
|
||||
WRITE_FLOAT_FIELD(durationMillisecs, "%.2f");
|
||||
WRITE_BOOL_FIELD(writeLocalFile);
|
||||
|
||||
appendStringInfoString(str, " totalExplainOutput [");
|
||||
for (int i = 0; i < node->numTasksOutput; i++)
|
||||
{
|
||||
const SubPlanExplainOutputData *e = &node->totalExplainOutput[i];
|
||||
|
||||
/* skip empty slots */
|
||||
if (e->explainOutput == NULL &&
|
||||
e->executionDuration == 0
|
||||
&& e->totalReceivedTupleData == 0)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
if (i > 0)
|
||||
{
|
||||
appendStringInfoChar(str, ' ');
|
||||
}
|
||||
|
||||
appendStringInfoChar(str, '(');
|
||||
|
||||
/* string pointer – prints quoted or NULL */
|
||||
WRITE_STRING_FIELD(totalExplainOutput[i].explainOutput);
|
||||
|
||||
/* double field */
|
||||
WRITE_FLOAT_FIELD(totalExplainOutput[i].executionDuration, "%.2f");
|
||||
|
||||
/* 64-bit unsigned – use the uint64 macro */
|
||||
WRITE_UINT64_FIELD(totalExplainOutput[i].totalReceivedTupleData);
|
||||
|
||||
appendStringInfoChar(str, ')');
|
||||
}
|
||||
|
||||
appendStringInfoChar(str, ']');
|
||||
|
||||
WRITE_INT_FIELD(numTasksOutput);
|
||||
WRITE_FLOAT_FIELD(ntuples, "%.2f");
|
||||
|
||||
}
|
||||
|
||||
void
|
||||
|
|
|
|||
|
|
@ -815,13 +815,14 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId,
|
|||
bool indexOK = true;
|
||||
int scanKeyCount = 1;
|
||||
ScanKeyData scanKey[1];
|
||||
Datum values[Natts_pg_dist_partition];
|
||||
bool isNull[Natts_pg_dist_partition];
|
||||
bool replace[Natts_pg_dist_partition];
|
||||
|
||||
Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock);
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||
|
||||
Datum *values = (Datum *) palloc0(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNull = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
bool *replace = (bool *) palloc0(tupleDescriptor->natts * sizeof(bool));
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
|
||||
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(distributedRelationId));
|
||||
|
||||
|
|
@ -838,10 +839,6 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId,
|
|||
distributedRelationName)));
|
||||
}
|
||||
|
||||
memset(values, 0, sizeof(values));
|
||||
memset(isNull, false, sizeof(isNull));
|
||||
memset(replace, false, sizeof(replace));
|
||||
|
||||
values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId);
|
||||
isNull[Anum_pg_dist_partition_colocationid - 1] = false;
|
||||
replace[Anum_pg_dist_partition_colocationid - 1] = true;
|
||||
|
|
@ -858,6 +855,10 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId,
|
|||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPartition, NoLock);
|
||||
|
||||
pfree(values);
|
||||
pfree(isNull);
|
||||
pfree(replace);
|
||||
|
||||
bool shouldSyncMetadata = ShouldSyncTableMetadata(distributedRelationId);
|
||||
if (shouldSyncMetadata && !localOnly)
|
||||
{
|
||||
|
|
@ -998,10 +999,12 @@ ColocationGroupTableList(uint32 colocationId, uint32 count)
|
|||
indexOK, NULL, scanKeyCount, scanKey);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
while (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
memset(datumArray, 0, tupleDescriptor->natts * sizeof(Datum));
|
||||
memset(isNullArray, 0, tupleDescriptor->natts * sizeof(bool));
|
||||
heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
Oid colocatedTableId = DatumGetObjectId(
|
||||
datumArray[Anum_pg_dist_partition_logicalrelid - 1]);
|
||||
|
|
@ -1020,6 +1023,8 @@ ColocationGroupTableList(uint32 colocationId, uint32 count)
|
|||
break;
|
||||
}
|
||||
}
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPartition, AccessShareLock);
|
||||
|
|
@ -1192,10 +1197,12 @@ ColocatedTableId(int32 colocationId)
|
|||
indexOK, NULL, scanKeyCount, scanKey);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
Datum *datumArray = (Datum *) palloc(tupleDescriptor->natts * sizeof(Datum));
|
||||
bool *isNullArray = (bool *) palloc(tupleDescriptor->natts * sizeof(bool));
|
||||
while (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
bool isNullArray[Natts_pg_dist_partition];
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
memset(datumArray, 0, tupleDescriptor->natts * sizeof(Datum));
|
||||
memset(isNullArray, 0, tupleDescriptor->natts * sizeof(bool));
|
||||
heap_deform_tuple(heapTuple, tupleDescriptor, datumArray, isNullArray);
|
||||
colocatedTableId = DatumGetObjectId(
|
||||
datumArray[Anum_pg_dist_partition_logicalrelid - 1]);
|
||||
|
|
@ -1223,6 +1230,8 @@ ColocatedTableId(int32 colocationId)
|
|||
|
||||
heapTuple = systable_getnext(scanDescriptor);
|
||||
}
|
||||
pfree(datumArray);
|
||||
pfree(isNullArray);
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistPartition, AccessShareLock);
|
||||
|
|
|
|||
|
|
@ -44,6 +44,7 @@ extern bool contain_nextval_expression_walker(Node *node, void *context);
|
|||
extern char * pg_get_replica_identity_command(Oid tableRelationId);
|
||||
extern List * pg_get_row_level_security_commands(Oid relationId);
|
||||
extern const char * RoleSpecString(RoleSpec *spec, bool withQuoteIdentifier);
|
||||
extern List * ExpandMergedSubscriptingRefEntries(List *targetEntryList);
|
||||
extern char * flatten_reloptions(Oid relid);
|
||||
|
||||
/* Function declarations for version dependent PostgreSQL ruleutils functions */
|
||||
|
|
@ -60,5 +61,6 @@ extern char * generate_operator_name(Oid operid, Oid arg1, Oid arg2);
|
|||
extern List * getOwnedSequences_internal(Oid relid, AttrNumber attnum, char deptype);
|
||||
extern void AppendOptionListToString(StringInfo stringData, List *options);
|
||||
|
||||
extern void ensure_update_targetlist_in_param_order(List *targetList);
|
||||
|
||||
#endif /* CITUS_RULEUTILS_H */
|
||||
|
|
|
|||
|
|
@ -27,7 +27,9 @@ extern bool UpdateRelationToShardNames(Node *node, List *relationShardList);
|
|||
extern void SetTaskQueryIfShouldLazyDeparse(Task *task, Query *query);
|
||||
extern void SetTaskQueryString(Task *task, char *queryString);
|
||||
extern void SetTaskQueryStringList(Task *task, List *queryStringList);
|
||||
extern void SetTaskQueryPlan(Task *task, Query *query, PlannedStmt *localPlan);
|
||||
extern char * TaskQueryString(Task *task);
|
||||
extern PlannedStmt * TaskQueryLocalPlan(Task *task);
|
||||
extern char * TaskQueryStringAtIndex(Task *task, int index);
|
||||
extern int GetTaskQueryType(Task *task);
|
||||
extern void AddInsertAliasIfNeeded(Query *query);
|
||||
|
|
|
|||
|
|
@ -99,6 +99,12 @@ typedef struct FastPathRestrictionContext
|
|||
* Set to true when distKey = Param; in the queryTree
|
||||
*/
|
||||
bool distributionKeyHasParam;
|
||||
|
||||
/*
|
||||
* Indicates to hold off calling the fast path planner until its
|
||||
* known if the shard is local or not.
|
||||
*/
|
||||
bool delayFastPathPlanning;
|
||||
} FastPathRestrictionContext;
|
||||
|
||||
typedef struct PlannerRestrictionContext
|
||||
|
|
|
|||
|
|
@ -466,4 +466,5 @@ extern bool IsBackgroundJobStatusTerminal(BackgroundJobStatus status);
|
|||
extern bool IsBackgroundTaskStatusTerminal(BackgroundTaskStatus status);
|
||||
extern Oid BackgroundJobStatusOid(BackgroundJobStatus status);
|
||||
extern Oid BackgroundTaskStatusOid(BackgroundTaskStatus status);
|
||||
extern int GetAutoConvertedAttrIndexInPgDistPartition(TupleDesc tupleDEsc);
|
||||
#endif /* METADATA_UTILITY_H */
|
||||
|
|
|
|||
|
|
@ -146,8 +146,8 @@ extern void ExecuteQueryStringIntoDestReceiver(const char *queryString, ParamLis
|
|||
DestReceiver *dest);
|
||||
extern void ExecuteQueryIntoDestReceiver(Query *query, ParamListInfo params,
|
||||
DestReceiver *dest);
|
||||
extern void ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
|
||||
DestReceiver *dest);
|
||||
extern uint64 ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
|
||||
DestReceiver *dest);
|
||||
extern void SetLocalMultiShardModifyModeToSequential(void);
|
||||
extern void EnsureSequentialMode(ObjectType objType);
|
||||
extern void SetLocalForceMaxQueryParallelization(void);
|
||||
|
|
|
|||
|
|
@ -174,9 +174,16 @@ typedef enum TaskQueryType
|
|||
TASK_QUERY_NULL,
|
||||
TASK_QUERY_TEXT,
|
||||
TASK_QUERY_OBJECT,
|
||||
TASK_QUERY_TEXT_LIST
|
||||
TASK_QUERY_TEXT_LIST,
|
||||
TASK_QUERY_LOCAL_PLAN,
|
||||
} TaskQueryType;
|
||||
|
||||
typedef struct LocalCompilation
|
||||
{
|
||||
PlannedStmt *plan; /* the local plan for this task */
|
||||
Query *query; /* query to deparse for EXPLAIN ANALYZE or local command logging */
|
||||
} LocalCompilation;
|
||||
|
||||
typedef struct TaskQuery
|
||||
{
|
||||
TaskQueryType queryType;
|
||||
|
|
@ -219,6 +226,15 @@ typedef struct TaskQuery
|
|||
* when we want to access each query string.
|
||||
*/
|
||||
List *queryStringList;
|
||||
|
||||
/*
|
||||
* For tasks that can be executed locally, this field contains the
|
||||
* local plan for the task. This is only set when the shard that the
|
||||
* task is assigned to is local to the node that executes the task.
|
||||
* The query field is used to deparse the query for EXPLAIN ANALYZE
|
||||
* or local command logging.
|
||||
*/
|
||||
LocalCompilation *localCompiled; /* only applies to local tasks */
|
||||
}data;
|
||||
}TaskQuery;
|
||||
|
||||
|
|
@ -474,6 +490,24 @@ typedef struct DistributedPlan
|
|||
} DistributedPlan;
|
||||
|
||||
|
||||
/*
|
||||
* SubPlanExplainOutputData Holds the EXPLAIN ANALYZE output and collected
|
||||
* statistics for a single task executed by a worker during distributed
|
||||
* query execution.
|
||||
* explainOutput — raw EXPLAIN ANALYZE output for the task
|
||||
* executionDuration — wall‑clock time taken to run the task
|
||||
* totalReceivedTupleData — total bytes of tuple data received from the worker
|
||||
*/
|
||||
typedef struct SubPlanExplainOutputData
|
||||
{
|
||||
char *explainOutput;
|
||||
double executionDuration;
|
||||
double executionNtuples;
|
||||
double executionNloops;
|
||||
uint64 totalReceivedTupleData;
|
||||
} SubPlanExplainOutputData;
|
||||
|
||||
|
||||
/*
|
||||
* DistributedSubPlan contains a subplan of a distributed plan. Subplans are
|
||||
* executed before the distributed query and their results are written to
|
||||
|
|
@ -492,6 +526,9 @@ typedef struct DistributedSubPlan
|
|||
uint32 remoteWorkerCount;
|
||||
double durationMillisecs;
|
||||
bool writeLocalFile;
|
||||
SubPlanExplainOutputData *totalExplainOutput;
|
||||
uint32 numTasksOutput; /* actual size of the above array */
|
||||
double ntuples; /* total tuples produced */
|
||||
} DistributedSubPlan;
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@
|
|||
|
||||
extern bool EnableRouterExecution;
|
||||
extern bool EnableFastPathRouterPlanner;
|
||||
extern bool EnableLocalFastPathQueryOptimization;
|
||||
|
||||
extern bool EnableNonColocatedRouterQueryPushdown;
|
||||
|
||||
|
|
@ -91,16 +92,19 @@ extern void GenerateSingleShardRouterTaskList(Job *job,
|
|||
List *relationShardList,
|
||||
List *placementList,
|
||||
uint64 shardId,
|
||||
bool isLocalTableModification);
|
||||
bool isLocalTableModification,
|
||||
bool delayedFastPath);
|
||||
|
||||
/*
|
||||
* FastPathPlanner is a subset of router planner, that's why we prefer to
|
||||
* keep the external function here.
|
||||
*/extern PlannedStmt * GeneratePlaceHolderPlannedStmt(Query *parse);
|
||||
|
||||
extern void FastPathPreprocessParseTree(Query *parse);
|
||||
extern PlannedStmt * FastPathPlanner(Query *originalQuery, Query *parse, ParamListInfo
|
||||
boundParams);
|
||||
extern bool FastPathRouterQuery(Query *query, Node **distributionKeyValue);
|
||||
extern bool FastPathRouterQuery(Query *query,
|
||||
FastPathRestrictionContext *fastPathContext);
|
||||
extern bool JoinConditionIsOnFalse(List *relOptInfo);
|
||||
extern Oid ResultRelationOidForQuery(Query *query);
|
||||
extern DeferredErrorMessage * TargetlistAndFunctionsSupported(Oid resultRelationId,
|
||||
|
|
@ -120,5 +124,7 @@ extern Job * RouterJob(Query *originalQuery,
|
|||
DeferredErrorMessage **planningError);
|
||||
extern bool ContainsOnlyLocalOrReferenceTables(RTEListProperties *rteProperties);
|
||||
extern RangeTblEntry * ExtractSourceResultRangeTableEntry(Query *query);
|
||||
extern void CheckAndBuildDelayedFastPathPlan(DistributedPlanningContext *planContext,
|
||||
DistributedPlan *plan);
|
||||
|
||||
#endif /* MULTI_ROUTER_PLANNER_H */
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@
|
|||
extern int MaxIntermediateResult;
|
||||
extern int SubPlanLevel;
|
||||
|
||||
extern void ExecuteSubPlans(DistributedPlan *distributedPlan);
|
||||
extern void ExecuteSubPlans(DistributedPlan *distributedPlan, bool explainAnalyzeEnabled);
|
||||
|
||||
/**
|
||||
* IntermediateResultsHashEntry is used to store which nodes need to receive
|
||||
|
|
|
|||
|
|
@ -13,6 +13,10 @@
|
|||
|
||||
#include "pg_version_constants.h"
|
||||
|
||||
/* we need these for PG-18’s PushActiveSnapshot/PopActiveSnapshot APIs */
|
||||
#include "access/xact.h"
|
||||
#include "utils/snapmgr.h"
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_18
|
||||
#define create_foreignscan_path_compat(a, b, c, d, e, f, g, h, i, j, k) \
|
||||
create_foreignscan_path( \
|
||||
|
|
@ -36,6 +40,14 @@
|
|||
/* PG-18 unified row-compare operator codes under COMPARE_* */
|
||||
#define ROWCOMPARE_NE COMPARE_NE
|
||||
|
||||
#define CATALOG_INSERT_WITH_SNAPSHOT(rel, tup) \
|
||||
do { \
|
||||
Snapshot __snap = GetTransactionSnapshot(); \
|
||||
PushActiveSnapshot(__snap); \
|
||||
CatalogTupleInsert((rel), (tup)); \
|
||||
PopActiveSnapshot(); \
|
||||
} while (0)
|
||||
|
||||
#elif PG_VERSION_NUM >= PG_VERSION_17
|
||||
#define create_foreignscan_path_compat(a, b, c, d, e, f, g, h, i, j, k) \
|
||||
create_foreignscan_path( \
|
||||
|
|
@ -43,6 +55,10 @@
|
|||
(e), (f), \
|
||||
(g), (h), (i), (j), (k) \
|
||||
)
|
||||
|
||||
/* no-op wrapper on older PGs */
|
||||
#define CATALOG_INSERT_WITH_SNAPSHOT(rel, tup) \
|
||||
CatalogTupleInsert((rel), (tup))
|
||||
#endif
|
||||
|
||||
#if PG_VERSION_NUM >= PG_VERSION_17
|
||||
|
|
@ -453,6 +469,10 @@ getStxstattarget_compat(HeapTuple tup)
|
|||
k) create_foreignscan_path(a, b, c, d, e, f, g, h, \
|
||||
i, k)
|
||||
|
||||
/* no-op wrapper on older PGs */
|
||||
#define CATALOG_INSERT_WITH_SNAPSHOT(rel, tup) \
|
||||
CatalogTupleInsert((rel), (tup))
|
||||
|
||||
#define getProcNo_compat(a) (a->pgprocno)
|
||||
#define getLxid_compat(a) (a->lxid)
|
||||
|
||||
|
|
|
|||
|
|
@ -119,11 +119,12 @@
|
|||
},
|
||||
"certifi": {
|
||||
"hashes": [
|
||||
"sha256:0569859f95fc761b18b45ef421b1290a0f65f147e92a1e5eb3e635f9a5e4e66f",
|
||||
"sha256:dc383c07b76109f368f6106eee2b593b04a011ea4d55f652c6ca24a754d1cdd1"
|
||||
"sha256:5a1e7645bc0ec61a09e26c36f6106dd4cf40c6db3a1fb6352b0244e7fb057c7b",
|
||||
"sha256:c198e21b1289c2ab85ee4e67bb4b4ef3ead0892059901a8d5b622f24a1101e90"
|
||||
],
|
||||
"index": "pypi",
|
||||
"markers": "python_version >= '3.6'",
|
||||
"version": "==2024.2.2"
|
||||
"version": "==2024.7.4"
|
||||
},
|
||||
"cffi": {
|
||||
"hashes": [
|
||||
|
|
|
|||
|
|
@ -147,6 +147,7 @@ DEPS = {
|
|||
"multi_mx_modifying_xacts": TestDeps(None, ["multi_mx_create_table"]),
|
||||
"multi_mx_router_planner": TestDeps(None, ["multi_mx_create_table"]),
|
||||
"multi_mx_copy_data": TestDeps(None, ["multi_mx_create_table"]),
|
||||
"multi_mx_modifications": TestDeps(None, ["multi_mx_create_table"]),
|
||||
"multi_mx_schema_support": TestDeps(None, ["multi_mx_copy_data"]),
|
||||
"multi_simple_queries": TestDeps("base_schedule"),
|
||||
"create_single_shard_table": TestDeps("minimal_schedule"),
|
||||
|
|
@ -225,6 +226,15 @@ DEPS = {
|
|||
repeatable=False,
|
||||
),
|
||||
"pg17": TestDeps("minimal_schedule", ["multi_behavioral_analytics_create_table"]),
|
||||
"multi_subquery_misc": TestDeps(
|
||||
"minimal_schedule", ["multi_behavioral_analytics_create_table"]
|
||||
),
|
||||
"multi_subquery_union": TestDeps(
|
||||
"minimal_schedule", ["multi_behavioral_analytics_create_table"]
|
||||
),
|
||||
"multi_subquery_in_where_clause": TestDeps(
|
||||
"minimal_schedule", ["multi_behavioral_analytics_create_table"]
|
||||
),
|
||||
}
|
||||
|
||||
|
||||
|
|
|
|||
|
|
@ -226,10 +226,12 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM insert_select_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM (insert_select_single_shard_table.nullkey_c1_t1 JOIN (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 USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
-- use a postgres local table
|
||||
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
|
||||
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
|
||||
|
|
@ -338,10 +340,12 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM insert_select_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM (insert_select_single_shard_table.nullkey_c1_t1 JOIN (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 USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
-- use a postgres local table
|
||||
INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
|
||||
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
|
||||
|
|
@ -400,10 +404,12 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM insert_select_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM (insert_select_single_shard_table.nullkey_c1_t1 JOIN (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 USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
-- use a postgres local table
|
||||
INSERT INTO citus_local_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
|
||||
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
|
||||
|
|
@ -416,6 +422,8 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator
|
|||
SELECT avg(a), avg(b) FROM citus_local_table ORDER BY 1, 2;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
avg | avg
|
||||
---------------------------------------------------------------------
|
||||
4.5270270270270270 | 4.5270270270270270
|
||||
|
|
@ -458,7 +466,9 @@ DEBUG: Creating router plan
|
|||
DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN reference_table USING (a) JOIN postgres_local_table USING (a) ORDER BY 1,2 OFFSET 7;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: distributed INSERT ... SELECT cannot select from a local table
|
||||
|
|
@ -470,10 +480,12 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM insert_select_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT citus_local_table.a, citus_local_table.b FROM ((SELECT citus_local_table_1.a, citus_local_table_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)) citus_local_table_1) citus_local_table JOIN insert_select_single_shard_table.nullkey_c1_t1 USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
-- use a distributed table
|
||||
INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2;
|
||||
DEBUG: INSERT target relation and all source relations of the SELECT must be colocated in distributed INSERT ... SELECT
|
||||
|
|
|
|||
|
|
@ -0,0 +1,377 @@
|
|||
-- Test local execution with local plan in a sharded environment.
|
||||
-- This is an enhancement to local execution where instead of deparsing
|
||||
-- and compiling the shard query, the planner replaces the OID of the
|
||||
-- distributed table with the OID of the local shard in the parse tree
|
||||
-- and plans that.
|
||||
--
|
||||
-- https://github.com/citusdata/citus/pull/8035
|
||||
CREATE SCHEMA local_shard_execution_local_plan;
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
SET citus.next_shard_id TO 86000000;
|
||||
-- Test row-based sharding
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
CREATE TABLE test_tbl (a int, b int, data_f double precision);
|
||||
SELECT create_distributed_table('test_tbl', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT setseed(0.42); -- make the random data inserted deterministic
|
||||
setseed
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO test_tbl
|
||||
SELECT (random()*20)::int AS a,
|
||||
(random()*20)::int AS b,
|
||||
random()*10000.0 AS data_f
|
||||
FROM generate_series(1, 10000);
|
||||
-- Put the shard on worker 1 to ensure consistent test output across different schedules
|
||||
SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 1
|
||||
SELECT citus_move_shard_placement(86000000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
|
||||
citus_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests
|
||||
wait_for_resource_cleanup
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
RESET client_min_messages;
|
||||
\c - - - :worker_1_port
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
-- This query resolves to a single shard (aka fast path)
|
||||
-- which is located on worker_1; with client_min_messages
|
||||
-- at DEBUG2 we see a message that the planner is avoiding
|
||||
-- query deparse and plan
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
BEGIN;
|
||||
-- force accessing local placements via remote connections first
|
||||
SET citus.enable_local_execution TO false;
|
||||
RESET client_min_messages;
|
||||
RESET citus.log_remote_commands;
|
||||
RESET citus.log_local_commands;
|
||||
SELECT count(*), b FROM test_tbl GROUP BY b ORDER BY b;
|
||||
count | b
|
||||
---------------------------------------------------------------------
|
||||
246 | 0
|
||||
530 | 1
|
||||
517 | 2
|
||||
530 | 3
|
||||
513 | 4
|
||||
487 | 5
|
||||
501 | 6
|
||||
490 | 7
|
||||
501 | 8
|
||||
480 | 9
|
||||
510 | 10
|
||||
472 | 11
|
||||
484 | 12
|
||||
505 | 13
|
||||
463 | 14
|
||||
484 | 15
|
||||
516 | 16
|
||||
523 | 17
|
||||
507 | 18
|
||||
481 | 19
|
||||
260 | 20
|
||||
(21 rows)
|
||||
|
||||
-- Now, even if we enable local execution back before the query that
|
||||
-- could normally benefit from fast path local query optimizations,
|
||||
-- this time it won't be the case because local execution was implicitly
|
||||
-- disabled by Citus as we accessed local shard placements via remote
|
||||
-- connections.
|
||||
SET citus.enable_local_execution TO true;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: issuing SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
SET client_min_messages TO ERROR; -- keep COMMIT output quiet
|
||||
COMMIT;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.enable_local_fast_path_query_optimization TO OFF;
|
||||
-- With local execution local plan disabled, the same query
|
||||
-- does query deparse and planning of the shard query and
|
||||
-- provides the same results
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
-- Run the same query on the other worker - the local
|
||||
-- execution path is not taken because the shard is not
|
||||
-- local to this worker
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: issuing SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.test_tbl_86000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
SET citus.next_shard_id TO 86001000;
|
||||
-- Test citus local and reference tables
|
||||
CREATE TABLE ref_tbl (a int, b int, data_f double precision);
|
||||
SELECT create_reference_table('ref_tbl');
|
||||
create_reference_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT setseed(0.42); -- make the random data inserted deterministic
|
||||
setseed
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO ref_tbl
|
||||
SELECT (random()*20)::int AS a,
|
||||
(random()*20)::int AS b,
|
||||
random()*10000.0 AS data_f
|
||||
FROM generate_series(1, 10000);
|
||||
SET citus.next_shard_id TO 86002000;
|
||||
CREATE TABLE citus_tbl (a int, b int, data_f double precision);
|
||||
SELECT citus_set_coordinator_host('localhost', :master_port);
|
||||
citus_set_coordinator_host
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT citus_add_local_table_to_metadata('citus_tbl');
|
||||
citus_add_local_table_to_metadata
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO citus_tbl SELECT a, b, data_f FROM ref_tbl;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
-- citus local table: can use the fast path optimization
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM citus_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.citus_tbl_86002000 citus_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
-- reference table: does not use the fast path optimization.
|
||||
-- It may be enabled in a future enhancement.
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM ref_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM local_shard_execution_local_plan.ref_tbl_86001000 ref_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
-- Now test local execution with local plan for a schema sharded table.
|
||||
SET citus.enable_schema_based_sharding to on;
|
||||
CREATE SCHEMA schema_sharding_test;
|
||||
SET search_path TO schema_sharding_test;
|
||||
SET citus.next_shard_id TO 87000000;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
CREATE TABLE test_tbl (a int, b int, data_f double precision);
|
||||
SELECT setseed(0.42); -- make the random data inserted deterministic
|
||||
setseed
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO test_tbl
|
||||
SELECT (random()*20)::int AS a,
|
||||
(random()*20)::int AS b,
|
||||
random()*10000.0 AS data_f
|
||||
FROM generate_series(1, 10000);
|
||||
-- Put the shard on worker 2 to ensure consistent test output across different schedules
|
||||
SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 2
|
||||
SELECT citus_move_shard_placement(87000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||
citus_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests
|
||||
wait_for_resource_cleanup
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
RESET client_min_messages;
|
||||
\c - - - :worker_1_port
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
-- Run the test query on worker_1; with schema based sharding
|
||||
-- the data is not local to this worker so local execution
|
||||
-- path is not taken.
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM schema_sharding_test.test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
NOTICE: issuing SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM schema_sharding_test.test_tbl_87000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
-- Run the test query on worker_2; with schema based sharding
|
||||
-- the data is local to this worker so local execution
|
||||
-- path is taken, and the planner avoids query deparse and
|
||||
-- planning of the shard query.
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM schema_sharding_test.test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM schema_sharding_test.test_tbl_87000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
SET citus.enable_local_fast_path_query_optimization TO OFF;
|
||||
-- Run the test query on worker_2 but with local execution
|
||||
-- local plan disabled; now the planner does query deparse
|
||||
-- and planning of the shard query.
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM schema_sharding_test.test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
NOTICE: executing the command locally: SELECT b, avg(data_f) AS avg, min(data_f) AS min, max(data_f) AS max, count(1) AS count FROM schema_sharding_test.test_tbl_87000000 test_tbl WHERE ((a OPERATOR(pg_catalog.=) 8) AND (b OPERATOR(pg_catalog.=) ANY (ARRAY[1, 3, 5, 8, 13, 21]))) GROUP BY b ORDER BY b
|
||||
b | avg | min | max | count
|
||||
---------------------------------------------------------------------
|
||||
1 | 4930.97455169836 | 130.09338419238 | 9961.37766951669 | 21
|
||||
3 | 5587.38637430731 | 1230.07668620184 | 9937.96225230491 | 23
|
||||
5 | 3987.47953221387 | 437.362539823312 | 9729.29912509372 | 25
|
||||
8 | 5028.45408903437 | 593.546207093687 | 9869.93823005882 | 27
|
||||
13 | 3900.7835426648 | 510.078935445757 | 7784.07104505068 | 18
|
||||
(5 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
SET client_min_messages to ERROR;
|
||||
DROP SCHEMA local_shard_execution_local_plan CASCADE;
|
||||
DROP SCHEMA schema_sharding_test CASCADE;
|
||||
RESET ALL;
|
||||
|
|
@ -2410,8 +2410,10 @@ NOTICE: executing the command locally: UPDATE local_shard_execution.event_respo
|
|||
SELECT count(*) FROM event_responses WHERE event_id = 16;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.event_responses_1480001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16)
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
|
@ -2420,8 +2422,10 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar
|
|||
SELECT count(*) FROM event_responses WHERE event_id = 16;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.event_responses_1480001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16)
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
|
@ -2430,8 +2434,10 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar
|
|||
UPDATE event_responses SET response = 'no' WHERE event_id = 16;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: query has a single distribution column value: 16
|
||||
NOTICE: executing the command locally: UPDATE local_shard_execution.event_responses_1480001 event_responses SET response = 'no'::local_shard_execution.invite_resp WHERE (event_id OPERATOR(pg_catalog.=) 16)
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
INSERT INTO event_responses VALUES (16, 666, 'maybe')
|
||||
ON CONFLICT (event_id, user_id)
|
||||
DO UPDATE SET response = EXCLUDED.response RETURNING *;
|
||||
|
|
@ -2529,8 +2535,10 @@ SET citus.log_remote_commands TO ON;
|
|||
SELECT * FROM event_responses_no_pkey WHERE event_id = 2;
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: query has a single distribution column value: 2
|
||||
NOTICE: executing the command locally: SELECT event_id, user_id, response FROM local_shard_execution.event_responses_no_pkey_1480007 event_responses_no_pkey WHERE (event_id OPERATOR(pg_catalog.=) 2)
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
event_id | user_id | response
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
|
|
|||
|
|
@ -193,8 +193,102 @@ execute p4(8);
|
|||
NOTICE: executing the command locally: UPDATE local_shard_execution_dropped_column.t1_2460000 t1 SET a = (a OPERATOR(pg_catalog.+) 1) WHERE (c OPERATOR(pg_catalog.=) 8)
|
||||
execute p4(8);
|
||||
NOTICE: executing the command locally: UPDATE local_shard_execution_dropped_column.t1_2460000 t1 SET a = (a OPERATOR(pg_catalog.+) 1) WHERE (c OPERATOR(pg_catalog.=) 8)
|
||||
-- Test that "Avoid deparse and planning of shard query for local execution" (*)
|
||||
-- does not take the fast path of modifying the parse tree with the shard OID, as
|
||||
-- the dropped column means the attribute check between the distributed table and
|
||||
-- shard fails. With client_min_messages at DEBUG2, we see "cannot modify parse tree
|
||||
-- for local execution", indicating that router planning has detected the difference.
|
||||
--
|
||||
-- (*) https://github.com/citusdata/citus/pull/8035
|
||||
SET client_min_messages to DEBUG2;
|
||||
prepare p5(int) as SELECT count(*) FROM t1 WHERE c = 8 and a = $1 GROUP BY c;
|
||||
execute p5(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
execute p5(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (a OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
RESET client_min_messages;
|
||||
\c - - - :master_port
|
||||
-- one another combination is that the shell table
|
||||
-- one other combination is that the shell table
|
||||
-- has a dropped column but not the shard, via rebalance operation
|
||||
SET search_path TO local_shard_execution_dropped_column;
|
||||
ALTER TABLE t1 DROP COLUMN a;
|
||||
|
|
@ -204,6 +298,12 @@ SELECT citus_move_shard_placement(2460000, 'localhost', :worker_1_port, 'localho
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests
|
||||
wait_for_resource_cleanup
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SET search_path TO local_shard_execution_dropped_column;
|
||||
-- show the dropped columns
|
||||
|
|
@ -331,6 +431,108 @@ execute p3(8);
|
|||
NOTICE: executing the command locally: INSERT INTO local_shard_execution_dropped_column.t1_2460000 AS citus_table_alias (c) VALUES (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8) ON CONFLICT DO NOTHING
|
||||
execute p3(8);
|
||||
NOTICE: executing the command locally: INSERT INTO local_shard_execution_dropped_column.t1_2460000 AS citus_table_alias (c) VALUES (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8), (8) ON CONFLICT DO NOTHING
|
||||
-- Test that "Avoid deparse and planning of shard query for local execution"
|
||||
-- does not take the fast path of modifying the parse tree with the shard OID
|
||||
-- for this scenario (rebalance) also.
|
||||
--
|
||||
-- (*) https://github.com/citusdata/citus/pull/8035
|
||||
SET client_min_messages to DEBUG2;
|
||||
prepare p4(int) as SELECT count(*) FROM t1 WHERE c = 8 and 5 = $1 GROUP BY c;
|
||||
execute p4(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Router planner fast path cannot modify parse tree for local execution: shard table "local_shard_execution_dropped_column.t1_2460000" does not match the distributed table "local_shard_execution_dropped_column.t1"
|
||||
DEBUG: query has a single distribution column value: 8
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
execute p4(5);
|
||||
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_dropped_column.t1_2460000 t1 WHERE ((c OPERATOR(pg_catalog.=) 8) AND (5 OPERATOR(pg_catalog.=) $1)) GROUP BY c
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
RESET client_min_messages;
|
||||
\c - - - :master_port
|
||||
DROP SCHEMA local_shard_execution_dropped_column CASCADE;
|
||||
NOTICE: drop cascades to table local_shard_execution_dropped_column.t1
|
||||
|
|
|
|||
|
|
@ -192,7 +192,9 @@ DEBUG: A mix of distributed and citus-local table, routable query is not possib
|
|||
DEBUG: Creating MERGE repartition plan
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: Collect source query results on coordinator
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
DEBUG: Create a MERGE task list that needs to be routed
|
||||
DEBUG: <Deparsed MERGE query: MERGE INTO schema_shard_table1.nullkey_c1_t1_4005006 citus_table_alias USING (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('merge_into_XXX_4005006'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) citus_local_table ON (citus_table_alias.a OPERATOR(pg_catalog.=) citus_local_table.a) WHEN MATCHED THEN UPDATE SET b = citus_local_table.b>
|
||||
DEBUG: distributed statement: MERGE INTO schema_shard_table1.nullkey_c1_t1_4005006 citus_table_alias USING (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('merge_into_XXX_4005006'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) citus_local_table ON (citus_table_alias.a OPERATOR(pg_catalog.=) citus_local_table.a) WHEN MATCHED THEN UPDATE SET b = citus_local_table.b
|
||||
|
|
|
|||
|
|
@ -2492,15 +2492,15 @@ Custom Scan (Citus Adaptive) (actual rows=1 loops=1)
|
|||
-> Distributed Subplan XXX_1
|
||||
Intermediate Data Size: 100 bytes
|
||||
Result destination: Write locally
|
||||
-> Custom Scan (Citus Adaptive) (actual rows=20 loops=1)
|
||||
-> Custom Scan (Citus Adaptive) (actual rows=10 loops=1)
|
||||
Task Count: 4
|
||||
Tuple data received from nodes: 160 bytes
|
||||
Tuple data received from nodes: 80 bytes
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Tuple data received from node: 64 bytes
|
||||
Tuple data received from node: 32 bytes
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Insert on dist_table_570017 citus_table_alias (actual rows=8 loops=1)
|
||||
-> Seq Scan on dist_table_570017 dist_table (actual rows=8 loops=1)
|
||||
-> Insert on dist_table_570017 citus_table_alias (actual rows=4 loops=1)
|
||||
-> Seq Scan on dist_table_570017 dist_table (actual rows=4 loops=1)
|
||||
Filter: (a IS NOT NULL)
|
||||
-> Distributed Subplan XXX_2
|
||||
Intermediate Data Size: 150 bytes
|
||||
|
|
@ -3228,6 +3228,159 @@ Custom Scan (Citus Adaptive) (actual rows=0 loops=1)
|
|||
-> Update on tbl_570036 tbl (actual rows=0 loops=1)
|
||||
-> Seq Scan on tbl_570036 tbl (actual rows=0 loops=1)
|
||||
Filter: (a = 1)
|
||||
-- EXPLAIN ANALYZE shouldn't execute SubPlans twice (bug #4212)
|
||||
SET search_path TO multi_explain;
|
||||
CREATE TABLE test_subplans (x int primary key, y int);
|
||||
SELECT create_distributed_table('test_subplans','x');
|
||||
|
||||
EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
Custom Scan (Citus Adaptive) (actual rows=1 loops=1)
|
||||
-> Distributed Subplan XXX_1
|
||||
Intermediate Data Size: 18 bytes
|
||||
Result destination: Write locally
|
||||
-> Custom Scan (Citus Adaptive) (actual rows=1 loops=1)
|
||||
Task Count: 1
|
||||
Tuple data received from nodes: 16 bytes
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Tuple data received from node: 16 bytes
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Insert on test_subplans_570038 (actual rows=1 loops=1)
|
||||
-> Result (actual rows=1 loops=1)
|
||||
Task Count: 1
|
||||
Tuple data received from nodes: 8 bytes
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Tuple data received from node: 8 bytes
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Function Scan on read_intermediate_result intermediate_result (actual rows=1 loops=1)
|
||||
-- Only one row must exist
|
||||
SELECT * FROM test_subplans;
|
||||
1|2
|
||||
-- Will fail with duplicate pk
|
||||
EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
ERROR: duplicate key value violates unique constraint "test_subplans_pkey_570038"
|
||||
DETAIL: Key (x)=(1) already exists.
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
-- Test JSON format
|
||||
TRUNCATE test_subplans;
|
||||
EXPLAIN (FORMAT JSON, COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Custom Scan",
|
||||
"Custom Plan Provider": "Citus Adaptive",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1,
|
||||
"Distributed Query": {
|
||||
"Subplans": [
|
||||
{
|
||||
"Intermediate Data Size": "18 bytes",
|
||||
"Result destination": "Write locally",
|
||||
"PlannedStmt": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Custom Scan",
|
||||
"Custom Plan Provider": "Citus Adaptive",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1,
|
||||
"Distributed Query": {
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tuple data received from nodes": "16 bytes",
|
||||
"Tasks Shown": "All",
|
||||
"Tasks": [
|
||||
{
|
||||
"Tuple data received from node": "16 bytes",
|
||||
"Node": "host=localhost port=xxxxx dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "ModifyTable",
|
||||
"Operation": "Insert",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Relation Name": "test_subplans_570038",
|
||||
"Alias": "test_subplans_570038",
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1,
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Result",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1
|
||||
}
|
||||
]
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tuple data received from nodes": "8 bytes",
|
||||
"Tasks Shown": "All",
|
||||
"Tasks": [
|
||||
{
|
||||
"Tuple data received from node": "8 bytes",
|
||||
"Node": "host=localhost port=xxxxx dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Function Scan",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Function Name": "read_intermediate_result",
|
||||
"Alias": "intermediate_result",
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
-- Only one row must exist
|
||||
SELECT * FROM test_subplans;
|
||||
1|2
|
||||
-- check when auto explain + analyze is enabled, we do not allow local execution.
|
||||
CREATE SCHEMA test_auto_explain;
|
||||
SET search_path TO 'test_auto_explain';
|
||||
|
|
|
|||
|
|
@ -2484,15 +2484,15 @@ Custom Scan (Citus Adaptive) (actual rows=1 loops=1)
|
|||
-> Distributed Subplan XXX_1
|
||||
Intermediate Data Size: 100 bytes
|
||||
Result destination: Write locally
|
||||
-> Custom Scan (Citus Adaptive) (actual rows=20 loops=1)
|
||||
-> Custom Scan (Citus Adaptive) (actual rows=10 loops=1)
|
||||
Task Count: 4
|
||||
Tuple data received from nodes: 160 bytes
|
||||
Tuple data received from nodes: 80 bytes
|
||||
Tasks Shown: One of 4
|
||||
-> Task
|
||||
Tuple data received from node: 64 bytes
|
||||
Tuple data received from node: 32 bytes
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Insert on dist_table_570017 citus_table_alias (actual rows=8 loops=1)
|
||||
-> Seq Scan on dist_table_570017 dist_table (actual rows=8 loops=1)
|
||||
-> Insert on dist_table_570017 citus_table_alias (actual rows=4 loops=1)
|
||||
-> Seq Scan on dist_table_570017 dist_table (actual rows=4 loops=1)
|
||||
Filter: (a IS NOT NULL)
|
||||
-> Distributed Subplan XXX_2
|
||||
Intermediate Data Size: 150 bytes
|
||||
|
|
@ -3217,6 +3217,159 @@ Custom Scan (Citus Adaptive) (actual rows=0 loops=1)
|
|||
-> Update on tbl_570036 tbl (actual rows=0 loops=1)
|
||||
-> Seq Scan on tbl_570036 tbl (actual rows=0 loops=1)
|
||||
Filter: (a = 1)
|
||||
-- EXPLAIN ANALYZE shouldn't execute SubPlans twice (bug #4212)
|
||||
SET search_path TO multi_explain;
|
||||
CREATE TABLE test_subplans (x int primary key, y int);
|
||||
SELECT create_distributed_table('test_subplans','x');
|
||||
|
||||
EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
Custom Scan (Citus Adaptive) (actual rows=1 loops=1)
|
||||
-> Distributed Subplan XXX_1
|
||||
Intermediate Data Size: 18 bytes
|
||||
Result destination: Write locally
|
||||
-> Custom Scan (Citus Adaptive) (actual rows=1 loops=1)
|
||||
Task Count: 1
|
||||
Tuple data received from nodes: 16 bytes
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Tuple data received from node: 16 bytes
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Insert on test_subplans_570038 (actual rows=1 loops=1)
|
||||
-> Result (actual rows=1 loops=1)
|
||||
Task Count: 1
|
||||
Tuple data received from nodes: 8 bytes
|
||||
Tasks Shown: All
|
||||
-> Task
|
||||
Tuple data received from node: 8 bytes
|
||||
Node: host=localhost port=xxxxx dbname=regression
|
||||
-> Function Scan on read_intermediate_result intermediate_result (actual rows=1 loops=1)
|
||||
-- Only one row must exist
|
||||
SELECT * FROM test_subplans;
|
||||
1|2
|
||||
-- Will fail with duplicate pk
|
||||
EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
ERROR: duplicate key value violates unique constraint "test_subplans_pkey_570038"
|
||||
DETAIL: Key (x)=(1) already exists.
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
-- Test JSON format
|
||||
TRUNCATE test_subplans;
|
||||
EXPLAIN (FORMAT JSON, COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Custom Scan",
|
||||
"Custom Plan Provider": "Citus Adaptive",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1,
|
||||
"Distributed Query": {
|
||||
"Subplans": [
|
||||
{
|
||||
"Intermediate Data Size": "18 bytes",
|
||||
"Result destination": "Write locally",
|
||||
"PlannedStmt": [
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Custom Scan",
|
||||
"Custom Plan Provider": "Citus Adaptive",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1,
|
||||
"Distributed Query": {
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tuple data received from nodes": "16 bytes",
|
||||
"Tasks Shown": "All",
|
||||
"Tasks": [
|
||||
{
|
||||
"Tuple data received from node": "16 bytes",
|
||||
"Node": "host=localhost port=xxxxx dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "ModifyTable",
|
||||
"Operation": "Insert",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Relation Name": "test_subplans_570038",
|
||||
"Alias": "test_subplans_570038",
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1,
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Result",
|
||||
"Parent Relationship": "Outer",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1
|
||||
}
|
||||
]
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
],
|
||||
"Job": {
|
||||
"Task Count": 1,
|
||||
"Tuple data received from nodes": "8 bytes",
|
||||
"Tasks Shown": "All",
|
||||
"Tasks": [
|
||||
{
|
||||
"Tuple data received from node": "8 bytes",
|
||||
"Node": "host=localhost port=xxxxx dbname=regression",
|
||||
"Remote Plan": [
|
||||
[
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Function Scan",
|
||||
"Parallel Aware": false,
|
||||
"Async Capable": false,
|
||||
"Function Name": "read_intermediate_result",
|
||||
"Alias": "intermediate_result",
|
||||
"Actual Rows": 1,
|
||||
"Actual Loops": 1
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"Triggers": [
|
||||
]
|
||||
}
|
||||
]
|
||||
-- Only one row must exist
|
||||
SELECT * FROM test_subplans;
|
||||
1|2
|
||||
-- check when auto explain + analyze is enabled, we do not allow local execution.
|
||||
CREATE SCHEMA test_auto_explain;
|
||||
SET search_path TO 'test_auto_explain';
|
||||
|
|
|
|||
|
|
@ -1503,7 +1503,9 @@ ALTER EXTENSION citus UPDATE TO '13.2-1';
|
|||
SELECT * FROM multi_extension.print_extension_changes();
|
||||
previous_object | current_object
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
function worker_last_saved_explain_analyze() TABLE(explain_analyze_output text, execution_duration double precision) |
|
||||
| function worker_last_saved_explain_analyze() TABLE(explain_analyze_output text, execution_duration double precision, execution_ntuples double precision, execution_nloops double precision)
|
||||
(2 rows)
|
||||
|
||||
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
|
||||
-- show running version
|
||||
|
|
|
|||
|
|
@ -330,6 +330,12 @@ UPDATE limit_orders SET (kind, limit_price) = ('buy', 999) WHERE id = 246 RETURN
|
|||
246 | GM | 30 | Mon Jul 02 16:32:15 2007 | buy | 999
|
||||
(1 row)
|
||||
|
||||
UPDATE limit_orders SET (kind, limit_price) = (SELECT 'buy'::order_side, 999) WHERE id = 246 RETURNING *;
|
||||
id | symbol | bidder_id | placed_at | kind | limit_price
|
||||
---------------------------------------------------------------------
|
||||
246 | GM | 30 | Mon Jul 02 16:32:15 2007 | buy | 999
|
||||
(1 row)
|
||||
|
||||
-- Test that on unique contraint violations, we fail fast
|
||||
\set VERBOSITY terse
|
||||
INSERT INTO limit_orders VALUES (275, 'ADR', 140, '2007-07-02 16:32:15', 'sell', 43.67);
|
||||
|
|
@ -435,6 +441,8 @@ UPDATE limit_orders SET limit_price = 0.00 FROM bidders
|
|||
-- 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';
|
||||
WITH deleted_orders AS (INSERT INTO limit_orders SELECT 400, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)
|
||||
UPDATE limit_orders SET symbol = 'GM';
|
||||
SELECT symbol, bidder_id FROM limit_orders WHERE id = 246;
|
||||
symbol | bidder_id
|
||||
---------------------------------------------------------------------
|
||||
|
|
@ -927,6 +935,17 @@ SELECT * FROM summary_table ORDER BY id;
|
|||
2 | 400 | 450.0000000000000000 | |
|
||||
(2 rows)
|
||||
|
||||
-- try different order of update targets
|
||||
UPDATE summary_table SET (average_value, min_value) =
|
||||
(SELECT avg(value), min(value) FROM raw_table WHERE id = 2)
|
||||
WHERE id = 2;
|
||||
SELECT * FROM summary_table ORDER BY id;
|
||||
id | min_value | average_value | count | uniques
|
||||
---------------------------------------------------------------------
|
||||
1 | | 200.0000000000000000 | |
|
||||
2 | 400 | 450.0000000000000000 | |
|
||||
(2 rows)
|
||||
|
||||
UPDATE summary_table SET min_value = 100
|
||||
WHERE id IN (SELECT id FROM raw_table WHERE id = 1 and value > 100) AND id = 1;
|
||||
SELECT * FROM summary_table ORDER BY id;
|
||||
|
|
@ -1103,6 +1122,16 @@ SELECT * FROM reference_summary_table ORDER BY id;
|
|||
2 | 400 | 450.0000000000000000 | |
|
||||
(2 rows)
|
||||
|
||||
UPDATE reference_summary_table SET (average_value, min_value) =
|
||||
(SELECT avg(value), min(value) FROM reference_raw_table WHERE id = 2)
|
||||
WHERE id = 2;
|
||||
SELECT * FROM reference_summary_table ORDER BY id;
|
||||
id | min_value | average_value | count | uniques
|
||||
---------------------------------------------------------------------
|
||||
1 | | 200.0000000000000000 | |
|
||||
2 | 400 | 450.0000000000000000 | |
|
||||
(2 rows)
|
||||
|
||||
-- no need partition colum equalities on reference tables
|
||||
UPDATE reference_summary_table SET (count) =
|
||||
(SELECT id AS inner_id FROM reference_raw_table WHERE value = 500)
|
||||
|
|
@ -1329,5 +1358,14 @@ DROP TABLE raw_table;
|
|||
DROP TABLE summary_table;
|
||||
DROP TABLE reference_raw_table;
|
||||
DROP TABLE reference_summary_table;
|
||||
DROP TABLE limit_orders;
|
||||
DROP TABLE multiple_hash;
|
||||
DROP TABLE range_partitioned;
|
||||
DROP TABLE append_partitioned;
|
||||
DROP TABLE bidders;
|
||||
DROP FUNCTION stable_append;
|
||||
DROP FUNCTION immutable_append;
|
||||
DROP FUNCTION temp_strict_func;
|
||||
DROP TYPE order_side;
|
||||
DROP SCHEMA multi_modifications CASCADE;
|
||||
NOTICE: drop cascades to table multi_modifications.local
|
||||
|
|
|
|||
|
|
@ -1,7 +1,7 @@
|
|||
--
|
||||
-- multi subquery in where queries aims to expand existing subquery pushdown
|
||||
-- regression tests to cover more cases specifically subqueries in WHERE clause
|
||||
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
|
||||
-- the tables that are used depends to multi_behavioral_analytics_create_table.sql
|
||||
--
|
||||
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
|
||||
-- subqueries in WHERE with greater operator
|
||||
|
|
|
|||
|
|
@ -2,7 +2,7 @@
|
|||
-- (i) Prepared statements
|
||||
-- (ii) PL/PGSQL functions
|
||||
-- (iii) SQL functions
|
||||
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
|
||||
-- the tables that are used depends to multi_behavioral_analytics_create_table.sql
|
||||
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
|
||||
SET citus.enable_router_execution TO false;
|
||||
PREPARE prepared_subquery_1 AS
|
||||
|
|
@ -352,6 +352,7 @@ ORDER BY 2 DESC;
|
|||
-- Similar to the above queries, but
|
||||
-- this time the joins are not removed because
|
||||
-- target list contains all the entries
|
||||
SET citus.enable_router_execution TO true;
|
||||
SELECT
|
||||
*
|
||||
FROM users_table t1
|
||||
|
|
|
|||
|
|
@ -1,7 +1,7 @@
|
|||
--
|
||||
-- multi subquery toplevel union queries aims to expand existing subquery pushdown
|
||||
-- regression tests to cover more cases
|
||||
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
|
||||
-- the tables that are used depends to multi_behavioral_analytics_create_table.sql
|
||||
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
|
||||
-- SET citus.next_shard_id TO 1400000;
|
||||
-- a very simple union query
|
||||
|
|
@ -1246,5 +1246,3 @@ SELECT user_id FROM users_table
|
|||
UNION SELECT u.user_id FROM users_table, users_udf() u;
|
||||
ERROR: cannot perform distributed planning on this query because parameterized queries for SQL functions referencing distributed tables are not supported
|
||||
HINT: Consider using PL/pgSQL functions instead.
|
||||
DROP TABLE events_reference_table;
|
||||
DROP TABLE users_reference_table;
|
||||
|
|
|
|||
|
|
@ -0,0 +1,717 @@
|
|||
CREATE SCHEMA multi_update_select;
|
||||
SET search_path TO multi_update_select;
|
||||
SET citus.next_shard_id TO 751000;
|
||||
-- specific tests related to get_update_query_targetlist_def
|
||||
-- we test only queries with sublinks, like:
|
||||
-- ( ... SET (...) = (SELECT ...))
|
||||
-- Reference tables
|
||||
CREATE TABLE test_ref_indirection (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_reference_table('test_ref_indirection');
|
||||
create_reference_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE test_ref_indirection_new (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_reference_table('test_ref_indirection_new');
|
||||
create_reference_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- Distributed tables
|
||||
CREATE TABLE test_dist_indirection (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_distributed_table('test_dist_indirection', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE test_dist_indirection_new (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_distributed_table('test_dist_indirection_new', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- those should work:
|
||||
INSERT INTO test_ref_indirection (id, col_bool, col_date, col_int, col_text)
|
||||
SELECT 1, true, '1970-01-01'::date, 1, 'one';
|
||||
INSERT INTO test_dist_indirection (id, col_bool, col_date, col_int, col_text)
|
||||
SELECT 1, true, '1970-01-01'::date, 1, 'one';
|
||||
INSERT INTO test_ref_indirection (id, col_text, col_bool, col_date, col_int)
|
||||
SELECT 2, 'two', false, '1970-01-01'::date, 2;
|
||||
INSERT INTO test_dist_indirection (id, col_text, col_bool, col_date, col_int)
|
||||
SELECT 2, 'two', false, '1970-01-01'::date, 2;
|
||||
INSERT INTO test_ref_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty';
|
||||
INSERT INTO test_dist_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty';
|
||||
INSERT INTO test_ref_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty';
|
||||
INSERT INTO test_dist_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty';
|
||||
INSERT INTO test_ref_indirection_new SELECT * FROM test_ref_indirection;
|
||||
INSERT INTO test_dist_indirection_new SELECT * FROM test_dist_indirection;
|
||||
SELECT * FROM test_ref_indirection ORDER BY id;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1970 | 1 | one
|
||||
2 | f | 01-01-1970 | 2 | two
|
||||
3 | f | 01-01-1970 | 0 | empty
|
||||
4 | f | 01-01-1970 | 0 | empty
|
||||
(4 rows)
|
||||
|
||||
SELECT * FROM test_dist_indirection ORDER BY id;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1970 | 1 | one
|
||||
2 | f | 01-01-1970 | 2 | two
|
||||
3 | f | 01-01-1970 | 0 | empty
|
||||
4 | f | 01-01-1970 | 0 | empty
|
||||
(4 rows)
|
||||
|
||||
SELECT * FROM test_ref_indirection_new ORDER BY id;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1970 | 1 | one
|
||||
2 | f | 01-01-1970 | 2 | two
|
||||
3 | f | 01-01-1970 | 0 | empty
|
||||
4 | f | 01-01-1970 | 0 | empty
|
||||
(4 rows)
|
||||
|
||||
SELECT * FROM test_dist_indirection_new ORDER BY id;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1970 | 1 | one
|
||||
2 | f | 01-01-1970 | 2 | two
|
||||
3 | f | 01-01-1970 | 0 | empty
|
||||
4 | f | 01-01-1970 | 0 | empty
|
||||
(4 rows)
|
||||
|
||||
-- now UPDATEs
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_date, col_int, col_text)
|
||||
= (SELECT true, '1970-01-01'::date, 1, 'ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1970 | 1 | ok
|
||||
2 | t | 01-01-1970 | 1 | ok
|
||||
3 | t | 01-01-1970 | 1 | ok
|
||||
4 | t | 01-01-1970 | 1 | ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_date, col_int, col_text)
|
||||
= (SELECT true, '1970-01-01'::date, 1, 'ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1970 | 1 | ok
|
||||
2 | t | 01-01-1970 | 1 | ok
|
||||
3 | t | 01-01-1970 | 1 | ok
|
||||
4 | t | 01-01-1970 | 1 | ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_date) = (select false, '1971-01-01'::date)
|
||||
, (col_int, col_text) = (select 2, '2 ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1971 | 2 | 2 ok
|
||||
2 | f | 01-01-1971 | 2 | 2 ok
|
||||
3 | f | 01-01-1971 | 2 | 2 ok
|
||||
4 | f | 01-01-1971 | 2 | 2 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_date) = (select false, '1971-01-01'::date)
|
||||
, (col_int, col_text) = (select 2, '2 ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1971 | 2 | 2 ok
|
||||
2 | f | 01-01-1971 | 2 | 2 ok
|
||||
3 | f | 01-01-1971 | 2 | 2 ok
|
||||
4 | f | 01-01-1971 | 2 | 2 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_int) = (select true, 3)
|
||||
, (col_text) = (select '3 ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1971 | 3 | 3 ok
|
||||
2 | t | 01-01-1971 | 3 | 3 ok
|
||||
3 | t | 01-01-1971 | 3 | 3 ok
|
||||
4 | t | 01-01-1971 | 3 | 3 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_int) = (select true, 3)
|
||||
, (col_text) = (select '3 ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1971 | 3 | 3 ok
|
||||
2 | t | 01-01-1971 | 3 | 3 ok
|
||||
3 | t | 01-01-1971 | 3 | 3 ok
|
||||
4 | t | 01-01-1971 | 3 | 3 ok
|
||||
(4 rows)
|
||||
|
||||
-- but those should work since 13.X
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_date, col_text, col_int, col_bool)
|
||||
= (SELECT '1972-01-01'::date, '4 ok', 4, false)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1972 | 4 | 4 ok
|
||||
2 | f | 01-01-1972 | 4 | 4 ok
|
||||
3 | f | 01-01-1972 | 4 | 4 ok
|
||||
4 | f | 01-01-1972 | 4 | 4 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_date, col_text, col_int, col_bool)
|
||||
= (SELECT '1972-01-01'::date, '4 ok', 4, false)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1972 | 4 | 4 ok
|
||||
2 | f | 01-01-1972 | 4 | 4 ok
|
||||
3 | f | 01-01-1972 | 4 | 4 ok
|
||||
4 | f | 01-01-1972 | 4 | 4 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_int, col_text) = (select 5, '5 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1972 | 5 | 5 ok
|
||||
2 | t | 01-01-1972 | 5 | 5 ok
|
||||
3 | t | 01-01-1972 | 5 | 5 ok
|
||||
4 | t | 01-01-1972 | 5 | 5 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_int, col_text) = (select 5, '5 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1972 | 5 | 5 ok
|
||||
2 | t | 01-01-1972 | 5 | 5 ok
|
||||
3 | t | 01-01-1972 | 5 | 5 ok
|
||||
4 | t | 01-01-1972 | 5 | 5 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_int, col_date) = (select 6, '1973-01-01'::date)
|
||||
, (col_text, col_bool) = (select '6 ok', false)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1973 | 6 | 6 ok
|
||||
2 | f | 01-01-1973 | 6 | 6 ok
|
||||
3 | f | 01-01-1973 | 6 | 6 ok
|
||||
4 | f | 01-01-1973 | 6 | 6 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_int, col_date) = (select 6, '1973-01-01'::date)
|
||||
, (col_text, col_bool) = (select '6 ok', false)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1973 | 6 | 6 ok
|
||||
2 | f | 01-01-1973 | 6 | 6 ok
|
||||
3 | f | 01-01-1973 | 6 | 6 ok
|
||||
4 | f | 01-01-1973 | 6 | 6 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1974 | 7 | 7 ok
|
||||
2 | t | 01-01-1974 | 7 | 7 ok
|
||||
3 | t | 01-01-1974 | 7 | 7 ok
|
||||
4 | t | 01-01-1974 | 7 | 7 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1974 | 7 | 7 ok
|
||||
2 | t | 01-01-1974 | 7 | 7 ok
|
||||
3 | t | 01-01-1974 | 7 | 7 ok
|
||||
4 | t | 01-01-1974 | 7 | 7 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok')
|
||||
, (col_int) = (select 8)
|
||||
, (col_bool) = (select false)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1975 | 8 | 8 ok
|
||||
2 | f | 01-01-1975 | 8 | 8 ok
|
||||
3 | f | 01-01-1975 | 8 | 8 ok
|
||||
4 | f | 01-01-1975 | 8 | 8 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok')
|
||||
, (col_int) = (select 8)
|
||||
, (col_bool) = (select false)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1975 | 8 | 8 ok
|
||||
2 | f | 01-01-1975 | 8 | 8 ok
|
||||
3 | f | 01-01-1975 | 8 | 8 ok
|
||||
4 | f | 01-01-1975 | 8 | 8 ok
|
||||
(4 rows)
|
||||
|
||||
--
|
||||
-- more restrictive ones, just in case we miss a wrong value
|
||||
--
|
||||
-- those should work
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_text) = (SELECT true, '9 ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1975 | 8 | 9 ok
|
||||
2 | t | 01-01-1975 | 8 | 9 ok
|
||||
3 | t | 01-01-1975 | 8 | 9 ok
|
||||
4 | t | 01-01-1975 | 8 | 9 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_text) = (SELECT true, '9 ok')
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1975 | 8 | 9 ok
|
||||
2 | t | 01-01-1975 | 8 | 9 ok
|
||||
3 | t | 01-01-1975 | 8 | 9 ok
|
||||
4 | t | 01-01-1975 | 8 | 9 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_text) = (SELECT false, '10 ok')
|
||||
WHERE id = 1
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1975 | 8 | 10 ok
|
||||
(1 row)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_text) = (SELECT false, '10 ok')
|
||||
WHERE id = 1
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | f | 01-01-1975 | 8 | 10 ok
|
||||
(1 row)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_text, col_bool) = (SELECT '11 ok', true)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1975 | 8 | 11 ok
|
||||
2 | t | 01-01-1975 | 8 | 11 ok
|
||||
3 | t | 01-01-1975 | 8 | 11 ok
|
||||
4 | t | 01-01-1975 | 8 | 11 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_text, col_bool) = (SELECT '11 ok', true)
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
1 | t | 01-01-1975 | 8 | 11 ok
|
||||
2 | t | 01-01-1975 | 8 | 11 ok
|
||||
3 | t | 01-01-1975 | 8 | 11 ok
|
||||
4 | t | 01-01-1975 | 8 | 11 ok
|
||||
(4 rows)
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_text, col_bool) = (SELECT '12 ok', false)
|
||||
WHERE id = 2
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
2 | f | 01-01-1975 | 8 | 12 ok
|
||||
(1 row)
|
||||
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_text, col_bool) = (SELECT '12 ok', false)
|
||||
WHERE id = 2
|
||||
RETURNING *;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
2 | f | 01-01-1975 | 8 | 12 ok
|
||||
(1 row)
|
||||
|
||||
-- several updates in CTE shoult not work
|
||||
with qq3 as (
|
||||
update test_ref_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '13', true)
|
||||
where id = 3
|
||||
returning *
|
||||
),
|
||||
qq4 as (
|
||||
update test_ref_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '14', false)
|
||||
where id = 4
|
||||
returning *
|
||||
)
|
||||
select * from qq3 union all select * from qq4;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
3 | t | 01-01-1975 | 8 | 13
|
||||
4 | f | 01-01-1975 | 8 | 14
|
||||
(2 rows)
|
||||
|
||||
with qq3 as (
|
||||
update test_dist_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '13', true)
|
||||
where id = 3
|
||||
returning *
|
||||
),
|
||||
qq4 as (
|
||||
update test_dist_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '14', false)
|
||||
where id = 4
|
||||
returning *
|
||||
)
|
||||
select * from qq3 union all select * from qq4;
|
||||
id | col_bool | col_date | col_int | col_text
|
||||
---------------------------------------------------------------------
|
||||
3 | t | 01-01-1975 | 8 | 13
|
||||
4 | f | 01-01-1975 | 8 | 14
|
||||
(2 rows)
|
||||
|
||||
DROP TABLE test_dist_indirection;
|
||||
DROP TABLE test_dist_indirection_new;
|
||||
DROP TABLE test_ref_indirection;
|
||||
DROP TABLE test_ref_indirection_new;
|
||||
-- https://github.com/citusdata/citus/issues/4092
|
||||
CREATE TABLE update_test (
|
||||
a INT DEFAULT 10,
|
||||
b INT,
|
||||
c TEXT
|
||||
);
|
||||
SELECT create_reference_table('update_test');
|
||||
create_reference_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO update_test VALUES (11, 41, 'car');
|
||||
INSERT INTO update_test VALUES (100, 20, 'bike');
|
||||
INSERT INTO update_test VALUES (100, 20, 'tractor');
|
||||
SELECT * FROM update_test;
|
||||
a | b | c
|
||||
---------------------------------------------------------------------
|
||||
11 | 41 | car
|
||||
100 | 20 | bike
|
||||
100 | 20 | tractor
|
||||
(3 rows)
|
||||
|
||||
UPDATE update_test
|
||||
SET (b,a) = (select a,b from update_test where b = 41 and c = 'car')
|
||||
WHERE a = 100 AND b = 20
|
||||
RETURNING *;
|
||||
a | b | c
|
||||
---------------------------------------------------------------------
|
||||
41 | 11 | bike
|
||||
41 | 11 | tractor
|
||||
(2 rows)
|
||||
|
||||
-- Test that multiple out of order columns and multiple sublinks are handled correctly.
|
||||
CREATE TABLE upd2_test (a1 int, b1 int, c1 int, d1 int, e1 int, f1 int, g1 int);
|
||||
SELECT create_reference_table('upd2_test');
|
||||
create_reference_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO upd2_test SELECT 1, 1, 1, 1, 1, 1, 1 FROM generate_series(1,5) c(i);
|
||||
UPDATE upd2_test set (b1, a1) = (SELECT 200, 100), (g1, f1, e1) = (SELECT 700, 600, 500), (d1, c1) = (SELECT 400, 300);
|
||||
SELECT * FROM upd2_test;
|
||||
a1 | b1 | c1 | d1 | e1 | f1 | g1
|
||||
---------------------------------------------------------------------
|
||||
100 | 200 | 300 | 400 | 500 | 600 | 700
|
||||
100 | 200 | 300 | 400 | 500 | 600 | 700
|
||||
100 | 200 | 300 | 400 | 500 | 600 | 700
|
||||
100 | 200 | 300 | 400 | 500 | 600 | 700
|
||||
100 | 200 | 300 | 400 | 500 | 600 | 700
|
||||
(5 rows)
|
||||
|
||||
UPDATE upd2_test set (g1, a1) = (SELECT 77, 11), (f1, b1) = (SELECT 66, 22), (e1, c1) = (SELECT 55, 33), (d1) = (SELECT 44);
|
||||
SELECT * FROM upd2_test;
|
||||
a1 | b1 | c1 | d1 | e1 | f1 | g1
|
||||
---------------------------------------------------------------------
|
||||
11 | 22 | 33 | 44 | 55 | 66 | 77
|
||||
11 | 22 | 33 | 44 | 55 | 66 | 77
|
||||
11 | 22 | 33 | 44 | 55 | 66 | 77
|
||||
11 | 22 | 33 | 44 | 55 | 66 | 77
|
||||
11 | 22 | 33 | 44 | 55 | 66 | 77
|
||||
(5 rows)
|
||||
|
||||
UPDATE upd2_test set (g1, a1) = (SELECT 7, 1), (f1) = (SELECT 6), (c1, e1) = (SELECT 3, 5), (b1) = (SELECT 2), (d1) = (SELECT 4);
|
||||
SELECT * FROM upd2_test;
|
||||
a1 | b1 | c1 | d1 | e1 | f1 | g1
|
||||
---------------------------------------------------------------------
|
||||
1 | 2 | 3 | 4 | 5 | 6 | 7
|
||||
1 | 2 | 3 | 4 | 5 | 6 | 7
|
||||
1 | 2 | 3 | 4 | 5 | 6 | 7
|
||||
1 | 2 | 3 | 4 | 5 | 6 | 7
|
||||
1 | 2 | 3 | 4 | 5 | 6 | 7
|
||||
(5 rows)
|
||||
|
||||
-- Test out of order updates on distributed table
|
||||
CREATE TABLE dist_test(a1 int, b1 numeric, c1 text, d1 int);
|
||||
SELECT create_distributed_table('dist_test', 'a1');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4);
|
||||
-- Router plan:
|
||||
UPDATE dist_test
|
||||
SET (d1, c1, b1) = (SELECT 7, 'tractor', 4.2)
|
||||
WHERE a1=1
|
||||
RETURNING *;
|
||||
a1 | b1 | c1 | d1
|
||||
---------------------------------------------------------------------
|
||||
1 | 4.2 | tractor | 7
|
||||
1 | 4.2 | tractor | 7
|
||||
(2 rows)
|
||||
|
||||
-- Pushdown plan:
|
||||
UPDATE dist_test
|
||||
SET (d1, c1, b1) = (SELECT X, 'car', Y)
|
||||
FROM (SELECT r.a * d1 as X, r.b * b1 as Y FROM update_test r, dist_test WHERE r.c=c1) upd
|
||||
WHERE dist_test.a1 > 2
|
||||
RETURNING *;
|
||||
a1 | b1 | c1 | d1 | x | y
|
||||
---------------------------------------------------------------------
|
||||
3 | 46.2 | car | 287 | 287 | 46.2
|
||||
3 | 46.2 | car | 287 | 287 | 46.2
|
||||
3 | 46.2 | car | 287 | 287 | 46.2
|
||||
(3 rows)
|
||||
|
||||
-- Test subscripting updates
|
||||
CREATE TABLE jsonb_subscript_update (id INT, data JSONB);
|
||||
SELECT create_distributed_table('jsonb_subscript_update', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}');
|
||||
UPDATE jsonb_subscript_update
|
||||
SET data['b'] = updated_vals.b::TEXT::jsonb,
|
||||
data['c'] = updated_vals.c::TEXT::jsonb,
|
||||
data['d'] = updated_vals.d::TEXT::jsonb
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'] AS a,
|
||||
data['a']::NUMERIC + 1 AS b,
|
||||
data['a']::NUMERIC + 2 AS c,
|
||||
data['a']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update
|
||||
) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id;
|
||||
SELECT * FROM jsonb_subscript_update ORDER BY 1,2;
|
||||
id | data
|
||||
---------------------------------------------------------------------
|
||||
1 | {"a": 1, "b": 2, "c": 3, "d": 4}
|
||||
2 | {"a": 2, "b": 3, "c": 4, "d": 5}
|
||||
(2 rows)
|
||||
|
||||
TRUNCATE jsonb_subscript_update;
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}');
|
||||
-- test router update with jsonb subscript
|
||||
UPDATE jsonb_subscript_update
|
||||
SET data['b'] = updated_vals.b::TEXT::jsonb,
|
||||
data['c'] = updated_vals.c::TEXT::jsonb,
|
||||
data['d'] = updated_vals.d::TEXT::jsonb
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'] AS a,
|
||||
data['a']::NUMERIC + 1 AS b,
|
||||
data['a']::NUMERIC + 2 AS c,
|
||||
data['a']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update
|
||||
) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id
|
||||
AND jsonb_subscript_update.id = 1;
|
||||
SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2;
|
||||
id | data
|
||||
---------------------------------------------------------------------
|
||||
1 | {"a": 1, "b": 2, "c": 3, "d": 4}
|
||||
(1 row)
|
||||
|
||||
TRUNCATE jsonb_subscript_update;
|
||||
-- Test updates on nested json objects
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": {"c":20, "d" : 200}}'), (2, '{"a": {"d":10, "c" : 100}}');
|
||||
BEGIN;
|
||||
UPDATE jsonb_subscript_update
|
||||
SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB
|
||||
FROM
|
||||
(SELECT id,
|
||||
DATA['a']['c'] AS a,
|
||||
DATA['a']['c']::NUMERIC + 1 AS b,
|
||||
DATA['a']['c']::NUMERIC + 2 AS c,
|
||||
DATA['a']['d']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id;
|
||||
SELECT * FROM jsonb_subscript_update ORDER BY 1,2;
|
||||
id | data
|
||||
---------------------------------------------------------------------
|
||||
1 | {"a": {"c": 223, "d": 200}}
|
||||
2 | {"a": {"c": 113, "d": 10}}
|
||||
(2 rows)
|
||||
|
||||
ROLLBACK;
|
||||
BEGIN;
|
||||
-- Router plan
|
||||
UPDATE jsonb_subscript_update
|
||||
SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB
|
||||
FROM
|
||||
(SELECT id,
|
||||
DATA['a']['c'] AS a,
|
||||
DATA['a']['c']::NUMERIC + 1 AS b,
|
||||
DATA['a']['c']::NUMERIC + 2 AS c,
|
||||
DATA['a']['d']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id
|
||||
AND jsonb_subscript_update.id = 1;
|
||||
SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2;
|
||||
id | data
|
||||
---------------------------------------------------------------------
|
||||
1 | {"a": {"c": 223, "d": 200}}
|
||||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
TRUNCATE jsonb_subscript_update;
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'), (4, '{"a": 4, "b": 10}');
|
||||
ALTER TABLE jsonb_subscript_update ADD CONSTRAINT pkey PRIMARY KEY (id, data);
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}')
|
||||
ON CONFLICT (id, data)
|
||||
DO UPDATE SET data['d']=(jsonb_subscript_update.data['a']::INT*100)::TEXT::JSONB,
|
||||
data['b']=(jsonb_subscript_update.data['a']::INT*-100)::TEXT::JSONB;
|
||||
SELECT * FROM jsonb_subscript_update ORDER BY 1,2;
|
||||
id | data
|
||||
---------------------------------------------------------------------
|
||||
1 | {"a": 1, "b": -100, "d": 100}
|
||||
2 | {"a": 2, "b": -200, "d": 200}
|
||||
4 | {"a": 4, "b": 10}
|
||||
(3 rows)
|
||||
|
||||
CREATE TABLE nested_obj_update(id INT, data JSONB, text_col TEXT);
|
||||
SELECT create_distributed_table('nested_obj_update', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO nested_obj_update VALUES
|
||||
(1, '{"a": [1,2,3], "b": [4,5,6], "c": [7,8,9], "d": [1,2,1,2]}', '4'),
|
||||
(2, '{"a": [10,20,30], "b": [41,51,61], "c": [72,82,92], "d": [11,21,11,21]}', '6');
|
||||
BEGIN;
|
||||
-- Pushdown plan
|
||||
UPDATE nested_obj_update
|
||||
SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB,
|
||||
data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB,
|
||||
data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB,
|
||||
text_col = (nested_obj_update.id*1000)::TEXT,
|
||||
data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB,
|
||||
data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB,
|
||||
data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'][0] AS a,
|
||||
data['b'][0]::NUMERIC + 1 AS b,
|
||||
data['c'][0]::NUMERIC + 2 AS c,
|
||||
data['c'][1]::NUMERIC + 3 AS d
|
||||
FROM nested_obj_update
|
||||
) updated_vals
|
||||
WHERE nested_obj_update.id = updated_vals.id;
|
||||
SELECT * FROM nested_obj_update ORDER BY 1,2,3;
|
||||
id | data | text_col
|
||||
---------------------------------------------------------------------
|
||||
1 | {"a": [4, 2, 3], "b": [4, 5, 18], "c": [33, 8, 9], "d": [1, 2, 1, 2, 2, null, 1]} | 1000
|
||||
2 | {"a": [60, 20, 30], "b": [41, 51, 148], "c": [255, 82, 92], "d": [11, 21, 11, 21, 4, null, 2]} | 2000
|
||||
(2 rows)
|
||||
|
||||
ROLLBACK;
|
||||
BEGIN;
|
||||
-- Router plan
|
||||
UPDATE nested_obj_update
|
||||
SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB,
|
||||
data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB,
|
||||
data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB,
|
||||
text_col = (nested_obj_update.id*1000)::TEXT,
|
||||
data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB,
|
||||
data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB,
|
||||
data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'][0] AS a,
|
||||
data['b'][0]::NUMERIC + 1 AS b,
|
||||
data['c'][0]::NUMERIC + 2 AS c,
|
||||
data['c'][1]::NUMERIC + 3 AS d
|
||||
FROM nested_obj_update
|
||||
) updated_vals
|
||||
WHERE nested_obj_update.id = updated_vals.id
|
||||
AND nested_obj_update.id = 2;
|
||||
SELECT * FROM nested_obj_update WHERE id = 2 ORDER BY 1,2,3;
|
||||
id | data | text_col
|
||||
---------------------------------------------------------------------
|
||||
2 | {"a": [60, 20, 30], "b": [41, 51, 148], "c": [255, 82, 92], "d": [11, 21, 11, 21, 4, null, 2]} | 2000
|
||||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
-- suppress cascade messages
|
||||
SET client_min_messages to ERROR;
|
||||
DROP SCHEMA multi_update_select CASCADE;
|
||||
RESET client_min_messages;
|
||||
|
|
@ -308,9 +308,11 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" "d1" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT NULL::integer AS "dummy-1" FROM query_single_shard_table.citus_local_table d1 WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT NULL::integer AS a, NULL::integer AS b FROM (SELECT intermediate_result."dummy-1" FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result("dummy-1" integer)) d1_1) d1, query_single_shard_table.nullkey_c1_t1
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
121
|
||||
|
|
@ -599,9 +601,11 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a, b FROM query_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT nullkey_c1_t1.a, nullkey_c1_t1.b, citus_local_table.b FROM (query_single_shard_table.nullkey_c1_t1 JOIN (SELECT citus_local_table_1.a, citus_local_table_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)) citus_local_table_1) citus_local_table USING (a)) ORDER BY nullkey_c1_t1.a, nullkey_c1_t1.b, citus_local_table.b
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
a | b | b
|
||||
---------------------------------------------------------------------
|
||||
1 | 1 | 1
|
||||
|
|
@ -850,9 +854,11 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (query_single_shard_table.nullkey_c1_t1 LEFT JOIN (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 USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
11
|
||||
|
|
@ -863,6 +869,7 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel
|
||||
DEBUG: recursively planning distributed relation "nullkey_c1_t1" since it is part of a distributed join node that is outer joined with a recurring rel
|
||||
|
|
@ -872,6 +879,7 @@ DEBUG: Creating router plan
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT a FROM query_single_shard_table.nullkey_c1_t1 WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count 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 LEFT JOIN (SELECT nullkey_c1_t1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) nullkey_c1_t1_1) nullkey_c1_t1 USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
14
|
||||
|
|
@ -910,6 +918,7 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: recursively planning left side of the full join since the other side is a recurring rel
|
||||
DEBUG: recursively planning distributed relation "nullkey_c1_t1" since it is part of a distributed join node that is outer joined with a recurring rel
|
||||
|
|
@ -919,6 +928,7 @@ DEBUG: Creating router plan
|
|||
DEBUG: generating subplan XXX_2 for subquery SELECT a FROM query_single_shard_table.nullkey_c1_t1 WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT nullkey_c1_t1_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) nullkey_c1_t1_1) nullkey_c1_t1 FULL JOIN (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 USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
14
|
||||
|
|
@ -1222,12 +1232,14 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" "t1" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table t1 WHERE true
|
||||
DEBUG: recursively planning right side of the left join since the outer side is a recurring rel
|
||||
DEBUG: recursively planning the distributed subquery since it is part of a distributed join node that is outer joined with a recurring rel
|
||||
DEBUG: skipping recursive planning for the subquery since it contains references to outer queries
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT t1_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)) t1_1) t1 LEFT JOIN LATERAL (SELECT t2.a, t2.b FROM query_single_shard_table.nullkey_c1_t1 t2 WHERE (t2.b OPERATOR(pg_catalog.>) t1.a)) q USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
11
|
||||
|
|
@ -1279,9 +1291,11 @@ DEBUG: Local tables cannot be used in distributed queries.
|
|||
DEBUG: Wrapping relation "citus_local_table" "t1" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.citus_local_table t1 WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM ((SELECT t1_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)) t1_1) t1 JOIN LATERAL (SELECT t2.a, t2.b FROM query_single_shard_table.nullkey_c1_t1 t2 WHERE (t2.b OPERATOR(pg_catalog.>) t1.a)) q USING (a))
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
|
@ -1428,7 +1442,9 @@ DEBUG: Creating router plan
|
|||
DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
INSERT INTO nullkey_c1_t1 SELECT * FROM postgres_local_table;
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
|
||||
|
|
@ -2112,9 +2128,11 @@ DEBUG: local table citus_local_table cannot be joined with these distributed ta
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT b FROM query_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: UPDATE query_single_shard_table.nullkey_c1_t1 SET b = 5 FROM (SELECT NULL::integer AS a, citus_local_table_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) citus_local_table_1) citus_local_table WHERE (nullkey_c1_t1.b OPERATOR(pg_catalog.=) citus_local_table.b)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
UPDATE nullkey_c1_t1 SET b = 5 FROM postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b;
|
||||
DEBUG: relation postgres_local_table is not distributed
|
||||
DEBUG: Wrapping relation "postgres_local_table" to a subquery
|
||||
|
|
@ -2164,9 +2182,11 @@ DEBUG: local table citus_local_table cannot be joined with these distributed ta
|
|||
DEBUG: Wrapping relation "citus_local_table" to a subquery
|
||||
DEBUG: Distributed planning for a fast-path router query
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Fast-path router query: created local execution plan to avoid deparse and compile of shard query
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT b FROM query_single_shard_table.citus_local_table WHERE true
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: DELETE FROM query_single_shard_table.nullkey_c1_t1 USING (SELECT NULL::integer AS a, citus_local_table_1.b FROM (SELECT intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(b integer)) citus_local_table_1) citus_local_table WHERE (nullkey_c1_t1.b OPERATOR(pg_catalog.=) citus_local_table.b)
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: Local executor: Using task's cached local plan for task 0
|
||||
DELETE FROM nullkey_c1_t1 USING postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b;
|
||||
DEBUG: relation postgres_local_table is not distributed
|
||||
DEBUG: Wrapping relation "postgres_local_table" to a subquery
|
||||
|
|
|
|||
|
|
@ -721,13 +721,11 @@ CALL exec_query_and_check_query_counters($$
|
|||
0, 0
|
||||
);
|
||||
-- same with explain analyze
|
||||
--
|
||||
-- this time, query_execution_multi_shard is incremented twice because of #4212
|
||||
CALL exec_query_and_check_query_counters($$
|
||||
EXPLAIN (ANALYZE)
|
||||
SELECT * FROM (SELECT * FROM dist_table OFFSET 0) q
|
||||
$$,
|
||||
1, 2
|
||||
1, 1
|
||||
);
|
||||
CALL exec_query_and_check_query_counters($$
|
||||
DELETE FROM dist_table WHERE a = 1
|
||||
|
|
@ -1041,9 +1039,6 @@ PL/pgSQL function exec_query_and_check_query_counters(text,bigint,bigint) line X
|
|||
-- A similar one but without the insert, so we would normally expect 2 increments
|
||||
-- for query_execution_single_shard and 2 for query_execution_multi_shard instead
|
||||
-- of 3 since the insert is not there anymore.
|
||||
--
|
||||
-- But this time we observe more counter increments because we execute the subplans
|
||||
-- twice because of #4212.
|
||||
CALL exec_query_and_check_query_counters($$
|
||||
EXPLAIN (ANALYZE)
|
||||
-- single-shard subplan (whole cte)
|
||||
|
|
@ -1057,7 +1052,7 @@ CALL exec_query_and_check_query_counters($$
|
|||
FROM (SELECT * FROM dist_table_1 ORDER BY a LIMIT 16) q -- multi-shard subplan (subquery q)
|
||||
JOIN cte ON q.a = cte.a
|
||||
$$,
|
||||
3, 4
|
||||
2, 2
|
||||
);
|
||||
-- safe to push-down
|
||||
CALL exec_query_and_check_query_counters($$
|
||||
|
|
|
|||
|
|
@ -0,0 +1,105 @@
|
|||
\set VERBOSITY terse
|
||||
SET citus.next_shard_id TO 1520000;
|
||||
CREATE SCHEMA subscripting_op;
|
||||
SET search_path TO subscripting_op;
|
||||
CREATE TABLE arr_subs_update(id INT, arr INT[], text_col TEXT, int_col_1 INT, int_col_2 INT);
|
||||
SELECT create_distributed_table('arr_subs_update', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO arr_subs_update
|
||||
VALUES (1, '{1,2,3}', 'foo', 50, 60),
|
||||
(2, '{4,5,6}', 'bar', 60, 70),
|
||||
(3, '{7,8,9}', 'baz', 70, 80);
|
||||
BEGIN;
|
||||
UPDATE arr_subs_update
|
||||
SET arr[1] = updated_vals.b,
|
||||
arr[3] = updated_vals.d,
|
||||
int_col_2 = 400,
|
||||
arr[2] = updated_vals.c
|
||||
FROM (
|
||||
SELECT id,
|
||||
arr[0] AS a,
|
||||
arr[1]::NUMERIC + 1 AS b,
|
||||
arr[2]::NUMERIC + 2 AS c,
|
||||
arr[3]::NUMERIC + 3 AS d
|
||||
FROM arr_subs_update
|
||||
) updated_vals
|
||||
WHERE arr_subs_update.id = updated_vals.id;
|
||||
SELECT * FROM arr_subs_update ORDER BY 1,2,3,4;
|
||||
id | arr | text_col | int_col_1 | int_col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | {2,4,6} | foo | 50 | 400
|
||||
2 | {5,7,9} | bar | 60 | 400
|
||||
3 | {8,10,12} | baz | 70 | 400
|
||||
(3 rows)
|
||||
|
||||
ROLLBACK;
|
||||
BEGIN;
|
||||
-- Test fast path router plan for subscripting update
|
||||
UPDATE arr_subs_update
|
||||
SET arr[1] = updated_vals.b,
|
||||
arr[3] = updated_vals.d,
|
||||
int_col_2 = 400,
|
||||
arr[2] = updated_vals.c
|
||||
FROM (
|
||||
SELECT id,
|
||||
arr[0] AS a,
|
||||
arr[1]::NUMERIC + 1 AS b,
|
||||
arr[2]::NUMERIC + 2 AS c,
|
||||
arr[3]::NUMERIC + 3 AS d
|
||||
FROM arr_subs_update
|
||||
) updated_vals
|
||||
WHERE arr_subs_update.id = updated_vals.id
|
||||
AND arr_subs_update.id = 1;
|
||||
SELECT * FROM arr_subs_update
|
||||
WHERE id=1 ORDER BY 1,2,3,4;
|
||||
id | arr | text_col | int_col_1 | int_col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | {2,4,6} | foo | 50 | 400
|
||||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
-- test if we can properly expand target list entries when there are dropped columns
|
||||
ALTER TABLE arr_subs_update DROP COLUMN int_col_1;
|
||||
BEGIN;
|
||||
UPDATE arr_subs_update
|
||||
SET arr[1] = updated_vals.b,
|
||||
arr[3] = updated_vals.d,
|
||||
int_col_2 = 400,
|
||||
arr[2] = updated_vals.c
|
||||
FROM (
|
||||
SELECT id,
|
||||
arr[0] AS a,
|
||||
arr[1]::NUMERIC + 1 AS b,
|
||||
arr[2]::NUMERIC + 2 AS c,
|
||||
arr[3]::NUMERIC + 3 AS d
|
||||
FROM arr_subs_update
|
||||
) updated_vals
|
||||
WHERE arr_subs_update.id = updated_vals.id;
|
||||
SELECT * FROM arr_subs_update ORDER BY 1,2,3,4;
|
||||
id | arr | text_col | int_col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | {2,4,6} | foo | 400
|
||||
2 | {5,7,9} | bar | 400
|
||||
3 | {8,10,12} | baz | 400
|
||||
(3 rows)
|
||||
|
||||
ROLLBACK;
|
||||
TRUNCATE arr_subs_update;
|
||||
INSERT INTO arr_subs_update VALUES (1, '{1,2,3}', 'foo', 60), (2, '{4,5,6}', 'bar', 70);
|
||||
ALTER TABLE arr_subs_update ADD CONSTRAINT pkey PRIMARY KEY (id, arr);
|
||||
INSERT INTO arr_subs_update VALUES (1, '{1,2,3}')
|
||||
ON CONFLICT (id, arr)
|
||||
DO UPDATE SET arr[0]=100, arr[1]=200, arr[5]=500;
|
||||
SELECT * FROM arr_subs_update ORDER BY 1,2,3,4;
|
||||
id | arr | text_col | int_col_2
|
||||
---------------------------------------------------------------------
|
||||
1 | [0:5]={100,200,2,3,NULL,500} | foo | 60
|
||||
2 | {4,5,6} | bar | 70
|
||||
(2 rows)
|
||||
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA subscripting_op CASCADE;
|
||||
|
|
@ -1469,5 +1469,140 @@ $$);
|
|||
f
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE dist1 (a int, b int);
|
||||
CREATE TABLE dist2 (a int, b int);
|
||||
SET citus.shard_count to 4;
|
||||
SELECT create_distributed_table('dist1', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT create_distributed_table('dist2', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO dist1 VALUES (1, 1), (2, 1), (3, 1), (4, 1), (5, 1);
|
||||
INSERT INTO dist2 VALUES (5, 2), (6, 2), (7, 2), (8, 2), (9, 2);
|
||||
-- safe to pushdown
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT * FROM dist1 JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT * FROM dist1 JOIN dist2 USING (a)
|
||||
) AS t1 ORDER BY 1;
|
||||
a | b | b
|
||||
---------------------------------------------------------------------
|
||||
5 | 1 | 2
|
||||
(1 row)
|
||||
|
||||
-- not safe to pushdown, the distribution key from the outer part of the outer join is not in the target list
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT dist2.a FROM dist1 LEFT JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT dist2.a FROM dist2
|
||||
) AS t1 ORDER BY 1;
|
||||
a
|
||||
---------------------------------------------------------------------
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
9
|
||||
|
||||
(6 rows)
|
||||
|
||||
set client_min_messages to DEBUG3;
|
||||
-- not safe to pushdown, as is, sub-plan is generated
|
||||
-- the distribution key from the outer part of the outer join is not in the target list
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT dist1.a FROM dist1 RIGHT JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT dist2.a FROM dist2
|
||||
) AS t1 ORDER BY 1;
|
||||
DEBUG: no shard pruning constraints on dist1 found
|
||||
DEBUG: shard count after pruning for dist1: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: no shard pruning constraints on dist1 found
|
||||
DEBUG: shard count after pruning for dist1: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: no shard pruning constraints on dist1 found
|
||||
DEBUG: shard count after pruning for dist1: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: generating subplan XXX_1 for subquery SELECT dist1.a FROM (union_pushdown.dist1 RIGHT JOIN union_pushdown.dist2 USING (a))
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: generating subplan XXX_2 for subquery SELECT a FROM union_pushdown.dist2
|
||||
DEBUG: Creating router plan
|
||||
DEBUG: generating subplan XXX_3 for subquery SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer) UNION SELECT intermediate_result.a FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(a integer)
|
||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_3'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) t1 ORDER BY a
|
||||
DEBUG: Creating router plan
|
||||
a
|
||||
---------------------------------------------------------------------
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
9
|
||||
|
||||
(6 rows)
|
||||
|
||||
-- safe to pushdown, the distribution key from the outer side of the RIGHT join is in the target list
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT dist2.a
|
||||
FROM dist1 RIGHT JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT dist2.a FROM dist2
|
||||
) AS t1
|
||||
ORDER BY 1;
|
||||
DEBUG: no shard pruning constraints on dist1 found
|
||||
DEBUG: shard count after pruning for dist1: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: Router planner cannot handle multi-shard select queries
|
||||
DEBUG: no shard pruning constraints on dist1 found
|
||||
DEBUG: shard count after pruning for dist1: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: no shard pruning constraints on dist2 found
|
||||
DEBUG: shard count after pruning for dist2: 4
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
DEBUG: assigned task to node localhost:xxxxx
|
||||
a
|
||||
---------------------------------------------------------------------
|
||||
5
|
||||
6
|
||||
7
|
||||
8
|
||||
9
|
||||
(5 rows)
|
||||
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA union_pushdown CASCADE;
|
||||
|
|
|
|||
|
|
@ -210,6 +210,7 @@ test: multi_outer_join
|
|||
# ---
|
||||
test: multi_complex_count_distinct multi_select_distinct
|
||||
test: multi_modifications
|
||||
test: multi_update_select
|
||||
test: multi_distribution_metadata
|
||||
test: multi_prune_shard_list
|
||||
test: multi_upsert multi_simple_queries multi_data_types
|
||||
|
|
|
|||
|
|
@ -70,6 +70,7 @@ test: metadata_sync_helpers
|
|||
|
||||
test: issue_6592
|
||||
test: executor_local_failure
|
||||
test: local_execution_local_plan
|
||||
|
||||
# test that no tests leaked intermediate results. This should always be last
|
||||
test: ensure_no_intermediate_data_leak
|
||||
|
|
|
|||
|
|
@ -93,6 +93,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: subscripting_op
|
||||
test: multi_agg_type_conversion multi_count_type_conversion recursive_relation_planning_restriction_pushdown
|
||||
test: multi_partition_pruning single_hash_repartition_join unsupported_lateral_subqueries
|
||||
test: multi_join_pruning multi_hash_pruning intermediate_result_pruning
|
||||
|
|
|
|||
|
|
@ -0,0 +1,214 @@
|
|||
-- Test local execution with local plan in a sharded environment.
|
||||
-- This is an enhancement to local execution where instead of deparsing
|
||||
-- and compiling the shard query, the planner replaces the OID of the
|
||||
-- distributed table with the OID of the local shard in the parse tree
|
||||
-- and plans that.
|
||||
--
|
||||
-- https://github.com/citusdata/citus/pull/8035
|
||||
|
||||
CREATE SCHEMA local_shard_execution_local_plan;
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
|
||||
SET citus.next_shard_id TO 86000000;
|
||||
|
||||
-- Test row-based sharding
|
||||
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
CREATE TABLE test_tbl (a int, b int, data_f double precision);
|
||||
SELECT create_distributed_table('test_tbl', 'a');
|
||||
SELECT setseed(0.42); -- make the random data inserted deterministic
|
||||
INSERT INTO test_tbl
|
||||
SELECT (random()*20)::int AS a,
|
||||
(random()*20)::int AS b,
|
||||
random()*10000.0 AS data_f
|
||||
FROM generate_series(1, 10000);
|
||||
|
||||
-- Put the shard on worker 1 to ensure consistent test output across different schedules
|
||||
SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 1
|
||||
SELECT citus_move_shard_placement(86000000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
|
||||
SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests
|
||||
RESET client_min_messages;
|
||||
|
||||
\c - - - :worker_1_port
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
|
||||
-- This query resolves to a single shard (aka fast path)
|
||||
-- which is located on worker_1; with client_min_messages
|
||||
-- at DEBUG2 we see a message that the planner is avoiding
|
||||
-- query deparse and plan
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
BEGIN;
|
||||
-- force accessing local placements via remote connections first
|
||||
SET citus.enable_local_execution TO false;
|
||||
RESET client_min_messages;
|
||||
RESET citus.log_remote_commands;
|
||||
RESET citus.log_local_commands;
|
||||
SELECT count(*), b FROM test_tbl GROUP BY b ORDER BY b;
|
||||
-- Now, even if we enable local execution back before the query that
|
||||
-- could normally benefit from fast path local query optimizations,
|
||||
-- this time it won't be the case because local execution was implicitly
|
||||
-- disabled by Citus as we accessed local shard placements via remote
|
||||
-- connections.
|
||||
SET citus.enable_local_execution TO true;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
SET client_min_messages TO ERROR; -- keep COMMIT output quiet
|
||||
COMMIT;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
|
||||
SET citus.enable_local_fast_path_query_optimization TO OFF;
|
||||
|
||||
-- With local execution local plan disabled, the same query
|
||||
-- does query deparse and planning of the shard query and
|
||||
-- provides the same results
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
|
||||
-- Run the same query on the other worker - the local
|
||||
-- execution path is not taken because the shard is not
|
||||
-- local to this worker
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
\c - - - :master_port
|
||||
|
||||
SET search_path TO local_shard_execution_local_plan;
|
||||
SET citus.next_shard_id TO 86001000;
|
||||
|
||||
-- Test citus local and reference tables
|
||||
CREATE TABLE ref_tbl (a int, b int, data_f double precision);
|
||||
SELECT create_reference_table('ref_tbl');
|
||||
SELECT setseed(0.42); -- make the random data inserted deterministic
|
||||
INSERT INTO ref_tbl
|
||||
SELECT (random()*20)::int AS a,
|
||||
(random()*20)::int AS b,
|
||||
random()*10000.0 AS data_f
|
||||
FROM generate_series(1, 10000);
|
||||
|
||||
SET citus.next_shard_id TO 86002000;
|
||||
CREATE TABLE citus_tbl (a int, b int, data_f double precision);
|
||||
SELECT citus_set_coordinator_host('localhost', :master_port);
|
||||
SELECT citus_add_local_table_to_metadata('citus_tbl');
|
||||
INSERT INTO citus_tbl SELECT a, b, data_f FROM ref_tbl;
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
|
||||
-- citus local table: can use the fast path optimization
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM citus_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
-- reference table: does not use the fast path optimization.
|
||||
-- It may be enabled in a future enhancement.
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM ref_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
\c - - - :master_port
|
||||
|
||||
-- Now test local execution with local plan for a schema sharded table.
|
||||
|
||||
SET citus.enable_schema_based_sharding to on;
|
||||
CREATE SCHEMA schema_sharding_test;
|
||||
SET search_path TO schema_sharding_test;
|
||||
|
||||
SET citus.next_shard_id TO 87000000;
|
||||
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
CREATE TABLE test_tbl (a int, b int, data_f double precision);
|
||||
SELECT setseed(0.42); -- make the random data inserted deterministic
|
||||
INSERT INTO test_tbl
|
||||
SELECT (random()*20)::int AS a,
|
||||
(random()*20)::int AS b,
|
||||
random()*10000.0 AS data_f
|
||||
FROM generate_series(1, 10000);
|
||||
|
||||
-- Put the shard on worker 2 to ensure consistent test output across different schedules
|
||||
SET client_min_messages to ERROR; -- suppress warning if shard is already on worker 2
|
||||
SELECT citus_move_shard_placement(87000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||
SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests
|
||||
RESET client_min_messages;
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
|
||||
-- Run the test query on worker_1; with schema based sharding
|
||||
-- the data is not local to this worker so local execution
|
||||
-- path is not taken.
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM schema_sharding_test.test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
\c - - - :worker_2_port
|
||||
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.log_remote_commands TO ON;
|
||||
SET citus.log_local_commands TO ON;
|
||||
|
||||
-- Run the test query on worker_2; with schema based sharding
|
||||
-- the data is local to this worker so local execution
|
||||
-- path is taken, and the planner avoids query deparse and
|
||||
-- planning of the shard query.
|
||||
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM schema_sharding_test.test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
SET citus.enable_local_fast_path_query_optimization TO OFF;
|
||||
|
||||
-- Run the test query on worker_2 but with local execution
|
||||
-- local plan disabled; now the planner does query deparse
|
||||
-- and planning of the shard query.
|
||||
SELECT b, AVG(data_f), MIN(data_f), MAX(data_f), COUNT(1)
|
||||
FROM schema_sharding_test.test_tbl
|
||||
WHERE a = 8 AND b IN (1,3,5,8,13,21)
|
||||
GROUP BY b
|
||||
ORDER BY b;
|
||||
|
||||
\c - - - :master_port
|
||||
|
||||
SET client_min_messages to ERROR;
|
||||
DROP SCHEMA local_shard_execution_local_plan CASCADE;
|
||||
DROP SCHEMA schema_sharding_test CASCADE;
|
||||
RESET ALL;
|
||||
|
|
@ -78,14 +78,37 @@ execute p4(8);
|
|||
execute p4(8);
|
||||
execute p4(8);
|
||||
|
||||
-- Test that "Avoid deparse and planning of shard query for local execution" (*)
|
||||
-- does not take the fast path of modifying the parse tree with the shard OID, as
|
||||
-- the dropped column means the attribute check between the distributed table and
|
||||
-- shard fails. With client_min_messages at DEBUG2, we see "cannot modify parse tree
|
||||
-- for local execution", indicating that router planning has detected the difference.
|
||||
--
|
||||
-- (*) https://github.com/citusdata/citus/pull/8035
|
||||
|
||||
SET client_min_messages to DEBUG2;
|
||||
prepare p5(int) as SELECT count(*) FROM t1 WHERE c = 8 and a = $1 GROUP BY c;
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
execute p5(5);
|
||||
RESET client_min_messages;
|
||||
|
||||
\c - - - :master_port
|
||||
|
||||
-- one another combination is that the shell table
|
||||
-- one other combination is that the shell table
|
||||
-- has a dropped column but not the shard, via rebalance operation
|
||||
SET search_path TO local_shard_execution_dropped_column;
|
||||
ALTER TABLE t1 DROP COLUMN a;
|
||||
|
||||
SELECT citus_move_shard_placement(2460000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||
SELECT public.wait_for_resource_cleanup(); -- otherwise fails flakiness tests
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SET search_path TO local_shard_execution_dropped_column;
|
||||
|
|
@ -132,5 +155,25 @@ execute p3(8);
|
|||
execute p3(8);
|
||||
execute p3(8);
|
||||
|
||||
-- Test that "Avoid deparse and planning of shard query for local execution"
|
||||
-- does not take the fast path of modifying the parse tree with the shard OID
|
||||
-- for this scenario (rebalance) also.
|
||||
--
|
||||
-- (*) https://github.com/citusdata/citus/pull/8035
|
||||
|
||||
SET client_min_messages to DEBUG2;
|
||||
prepare p4(int) as SELECT count(*) FROM t1 WHERE c = 8 and 5 = $1 GROUP BY c;
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
execute p4(5);
|
||||
RESET client_min_messages;
|
||||
|
||||
\c - - - :master_port
|
||||
DROP SCHEMA local_shard_execution_dropped_column CASCADE;
|
||||
|
|
|
|||
|
|
@ -1166,6 +1166,32 @@ PREPARE q2(int_wrapper_type) AS WITH a AS (UPDATE tbl SET b = $1 WHERE a = 1 RET
|
|||
EXPLAIN (COSTS false) EXECUTE q2('(1)');
|
||||
EXPLAIN :default_analyze_flags EXECUTE q2('(1)');
|
||||
|
||||
-- EXPLAIN ANALYZE shouldn't execute SubPlans twice (bug #4212)
|
||||
SET search_path TO multi_explain;
|
||||
CREATE TABLE test_subplans (x int primary key, y int);
|
||||
SELECT create_distributed_table('test_subplans','x');
|
||||
|
||||
EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
|
||||
-- Only one row must exist
|
||||
SELECT * FROM test_subplans;
|
||||
|
||||
-- Will fail with duplicate pk
|
||||
EXPLAIN (COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
|
||||
-- Test JSON format
|
||||
TRUNCATE test_subplans;
|
||||
EXPLAIN (FORMAT JSON, COSTS off, ANALYZE on, TIMING off, SUMMARY off)
|
||||
WITH a AS (INSERT INTO test_subplans VALUES (1,2) RETURNING *)
|
||||
SELECT * FROM a;
|
||||
|
||||
-- Only one row must exist
|
||||
SELECT * FROM test_subplans;
|
||||
|
||||
-- check when auto explain + analyze is enabled, we do not allow local execution.
|
||||
CREATE SCHEMA test_auto_explain;
|
||||
SET search_path TO 'test_auto_explain';
|
||||
|
|
|
|||
|
|
@ -234,6 +234,7 @@ SELECT kind, limit_price FROM limit_orders WHERE id = 246;
|
|||
|
||||
-- multi-column UPDATE with RETURNING
|
||||
UPDATE limit_orders SET (kind, limit_price) = ('buy', 999) WHERE id = 246 RETURNING *;
|
||||
UPDATE limit_orders SET (kind, limit_price) = (SELECT 'buy'::order_side, 999) WHERE id = 246 RETURNING *;
|
||||
|
||||
-- Test that on unique contraint violations, we fail fast
|
||||
\set VERBOSITY terse
|
||||
|
|
@ -337,6 +338,9 @@ UPDATE limit_orders SET limit_price = 0.00 FROM bidders
|
|||
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';
|
||||
|
||||
WITH deleted_orders AS (INSERT INTO limit_orders SELECT 400, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)
|
||||
UPDATE limit_orders SET symbol = 'GM';
|
||||
|
||||
SELECT symbol, bidder_id FROM limit_orders WHERE id = 246;
|
||||
|
||||
-- updates referencing just a var are supported
|
||||
|
|
@ -584,6 +588,13 @@ WHERE id = 2;
|
|||
|
||||
SELECT * FROM summary_table ORDER BY id;
|
||||
|
||||
-- try different order of update targets
|
||||
UPDATE summary_table SET (average_value, min_value) =
|
||||
(SELECT avg(value), min(value) FROM raw_table WHERE id = 2)
|
||||
WHERE id = 2;
|
||||
|
||||
SELECT * FROM summary_table ORDER BY id;
|
||||
|
||||
UPDATE summary_table SET min_value = 100
|
||||
WHERE id IN (SELECT id FROM raw_table WHERE id = 1 and value > 100) AND id = 1;
|
||||
|
||||
|
|
@ -712,6 +723,12 @@ WHERE id = 2;
|
|||
|
||||
SELECT * FROM reference_summary_table ORDER BY id;
|
||||
|
||||
UPDATE reference_summary_table SET (average_value, min_value) =
|
||||
(SELECT avg(value), min(value) FROM reference_raw_table WHERE id = 2)
|
||||
WHERE id = 2;
|
||||
|
||||
SELECT * FROM reference_summary_table ORDER BY id;
|
||||
|
||||
-- no need partition colum equalities on reference tables
|
||||
UPDATE reference_summary_table SET (count) =
|
||||
(SELECT id AS inner_id FROM reference_raw_table WHERE value = 500)
|
||||
|
|
@ -902,4 +919,15 @@ DROP TABLE raw_table;
|
|||
DROP TABLE summary_table;
|
||||
DROP TABLE reference_raw_table;
|
||||
DROP TABLE reference_summary_table;
|
||||
DROP TABLE limit_orders;
|
||||
DROP TABLE multiple_hash;
|
||||
DROP TABLE range_partitioned;
|
||||
DROP TABLE append_partitioned;
|
||||
DROP TABLE bidders;
|
||||
|
||||
DROP FUNCTION stable_append;
|
||||
DROP FUNCTION immutable_append;
|
||||
DROP FUNCTION temp_strict_func;
|
||||
DROP TYPE order_side;
|
||||
|
||||
DROP SCHEMA multi_modifications CASCADE;
|
||||
|
|
|
|||
|
|
@ -1,7 +1,7 @@
|
|||
--
|
||||
-- multi subquery in where queries aims to expand existing subquery pushdown
|
||||
-- regression tests to cover more cases specifically subqueries in WHERE clause
|
||||
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
|
||||
-- the tables that are used depends to multi_behavioral_analytics_create_table.sql
|
||||
--
|
||||
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
|
||||
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
-- (ii) PL/PGSQL functions
|
||||
-- (iii) SQL functions
|
||||
|
||||
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
|
||||
-- the tables that are used depends to multi_behavioral_analytics_create_table.sql
|
||||
|
||||
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
|
||||
SET citus.enable_router_execution TO false;
|
||||
|
|
@ -213,6 +213,7 @@ ORDER BY 2 DESC;
|
|||
-- Similar to the above queries, but
|
||||
-- this time the joins are not removed because
|
||||
-- target list contains all the entries
|
||||
SET citus.enable_router_execution TO true;
|
||||
SELECT
|
||||
*
|
||||
FROM users_table t1
|
||||
|
|
|
|||
|
|
@ -1,7 +1,7 @@
|
|||
--
|
||||
-- multi subquery toplevel union queries aims to expand existing subquery pushdown
|
||||
-- regression tests to cover more cases
|
||||
-- the tables that are used depends to multi_insert_select_behavioral_analytics_create_table.sql
|
||||
-- the tables that are used depends to multi_behavioral_analytics_create_table.sql
|
||||
|
||||
-- We don't need shard id sequence here, so commented out to prevent conflicts with concurrent tests
|
||||
-- SET citus.next_shard_id TO 1400000;
|
||||
|
|
@ -898,6 +898,3 @@ LANGUAGE sql stable;
|
|||
|
||||
SELECT user_id FROM users_table
|
||||
UNION SELECT u.user_id FROM users_table, users_udf() u;
|
||||
|
||||
DROP TABLE events_reference_table;
|
||||
DROP TABLE users_reference_table;
|
||||
|
|
|
|||
|
|
@ -0,0 +1,416 @@
|
|||
CREATE SCHEMA multi_update_select;
|
||||
SET search_path TO multi_update_select;
|
||||
|
||||
SET citus.next_shard_id TO 751000;
|
||||
|
||||
-- specific tests related to get_update_query_targetlist_def
|
||||
-- we test only queries with sublinks, like:
|
||||
-- ( ... SET (...) = (SELECT ...))
|
||||
|
||||
-- Reference tables
|
||||
CREATE TABLE test_ref_indirection (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_reference_table('test_ref_indirection');
|
||||
|
||||
CREATE TABLE test_ref_indirection_new (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_reference_table('test_ref_indirection_new');
|
||||
|
||||
-- Distributed tables
|
||||
CREATE TABLE test_dist_indirection (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_distributed_table('test_dist_indirection', 'id');
|
||||
|
||||
CREATE TABLE test_dist_indirection_new (
|
||||
id bigint primary key
|
||||
, col_bool bool , col_date date , col_int integer , col_text text
|
||||
);
|
||||
SELECT create_distributed_table('test_dist_indirection_new', 'id');
|
||||
|
||||
-- those should work:
|
||||
INSERT INTO test_ref_indirection (id, col_bool, col_date, col_int, col_text)
|
||||
SELECT 1, true, '1970-01-01'::date, 1, 'one';
|
||||
INSERT INTO test_dist_indirection (id, col_bool, col_date, col_int, col_text)
|
||||
SELECT 1, true, '1970-01-01'::date, 1, 'one';
|
||||
|
||||
INSERT INTO test_ref_indirection (id, col_text, col_bool, col_date, col_int)
|
||||
SELECT 2, 'two', false, '1970-01-01'::date, 2;
|
||||
INSERT INTO test_dist_indirection (id, col_text, col_bool, col_date, col_int)
|
||||
SELECT 2, 'two', false, '1970-01-01'::date, 2;
|
||||
|
||||
INSERT INTO test_ref_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty';
|
||||
INSERT INTO test_dist_indirection SELECT 3, false, '1970-01-01'::date, 0, 'empty';
|
||||
INSERT INTO test_ref_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty';
|
||||
INSERT INTO test_dist_indirection SELECT 4, false, '1970-01-01'::date, 0, 'empty';
|
||||
|
||||
INSERT INTO test_ref_indirection_new SELECT * FROM test_ref_indirection;
|
||||
INSERT INTO test_dist_indirection_new SELECT * FROM test_dist_indirection;
|
||||
|
||||
SELECT * FROM test_ref_indirection ORDER BY id;
|
||||
SELECT * FROM test_dist_indirection ORDER BY id;
|
||||
|
||||
SELECT * FROM test_ref_indirection_new ORDER BY id;
|
||||
SELECT * FROM test_dist_indirection_new ORDER BY id;
|
||||
|
||||
-- now UPDATEs
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_date, col_int, col_text)
|
||||
= (SELECT true, '1970-01-01'::date, 1, 'ok')
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_date, col_int, col_text)
|
||||
= (SELECT true, '1970-01-01'::date, 1, 'ok')
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_date) = (select false, '1971-01-01'::date)
|
||||
, (col_int, col_text) = (select 2, '2 ok')
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_date) = (select false, '1971-01-01'::date)
|
||||
, (col_int, col_text) = (select 2, '2 ok')
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_int) = (select true, 3)
|
||||
, (col_text) = (select '3 ok')
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_int) = (select true, 3)
|
||||
, (col_text) = (select '3 ok')
|
||||
RETURNING *;
|
||||
|
||||
-- but those should work since 13.X
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_date, col_text, col_int, col_bool)
|
||||
= (SELECT '1972-01-01'::date, '4 ok', 4, false)
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_date, col_text, col_int, col_bool)
|
||||
= (SELECT '1972-01-01'::date, '4 ok', 4, false)
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_int, col_text) = (select 5, '5 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_int, col_text) = (select 5, '5 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_int, col_date) = (select 6, '1973-01-01'::date)
|
||||
, (col_text, col_bool) = (select '6 ok', false)
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_int, col_date) = (select 6, '1973-01-01'::date)
|
||||
, (col_text, col_bool) = (select '6 ok', false)
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_int, col_date, col_text) = (select 7, '1974-01-01'::date, '7 ok')
|
||||
, (col_bool) = (select true)
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok')
|
||||
, (col_int) = (select 8)
|
||||
, (col_bool) = (select false)
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_date, col_text) = (select '1975-01-01'::date, '8 ok')
|
||||
, (col_int) = (select 8)
|
||||
, (col_bool) = (select false)
|
||||
RETURNING *;
|
||||
|
||||
--
|
||||
-- more restrictive ones, just in case we miss a wrong value
|
||||
--
|
||||
-- those should work
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_text) = (SELECT true, '9 ok')
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_text) = (SELECT true, '9 ok')
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_bool, col_text) = (SELECT false, '10 ok')
|
||||
WHERE id = 1
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_bool, col_text) = (SELECT false, '10 ok')
|
||||
WHERE id = 1
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_text, col_bool) = (SELECT '11 ok', true)
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_text, col_bool) = (SELECT '11 ok', true)
|
||||
RETURNING *;
|
||||
|
||||
UPDATE test_ref_indirection
|
||||
SET (col_text, col_bool) = (SELECT '12 ok', false)
|
||||
WHERE id = 2
|
||||
RETURNING *;
|
||||
UPDATE test_dist_indirection
|
||||
SET (col_text, col_bool) = (SELECT '12 ok', false)
|
||||
WHERE id = 2
|
||||
RETURNING *;
|
||||
|
||||
-- several updates in CTE shoult not work
|
||||
with qq3 as (
|
||||
update test_ref_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '13', true)
|
||||
where id = 3
|
||||
returning *
|
||||
),
|
||||
qq4 as (
|
||||
update test_ref_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '14', false)
|
||||
where id = 4
|
||||
returning *
|
||||
)
|
||||
select * from qq3 union all select * from qq4;
|
||||
with qq3 as (
|
||||
update test_dist_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '13', true)
|
||||
where id = 3
|
||||
returning *
|
||||
),
|
||||
qq4 as (
|
||||
update test_dist_indirection
|
||||
SET (col_text, col_bool)
|
||||
= (SELECT '14', false)
|
||||
where id = 4
|
||||
returning *
|
||||
)
|
||||
select * from qq3 union all select * from qq4;
|
||||
|
||||
DROP TABLE test_dist_indirection;
|
||||
DROP TABLE test_dist_indirection_new;
|
||||
DROP TABLE test_ref_indirection;
|
||||
DROP TABLE test_ref_indirection_new;
|
||||
|
||||
-- https://github.com/citusdata/citus/issues/4092
|
||||
CREATE TABLE update_test (
|
||||
a INT DEFAULT 10,
|
||||
b INT,
|
||||
c TEXT
|
||||
);
|
||||
|
||||
SELECT create_reference_table('update_test');
|
||||
INSERT INTO update_test VALUES (11, 41, 'car');
|
||||
INSERT INTO update_test VALUES (100, 20, 'bike');
|
||||
INSERT INTO update_test VALUES (100, 20, 'tractor');
|
||||
SELECT * FROM update_test;
|
||||
|
||||
UPDATE update_test
|
||||
SET (b,a) = (select a,b from update_test where b = 41 and c = 'car')
|
||||
WHERE a = 100 AND b = 20
|
||||
RETURNING *;
|
||||
|
||||
-- Test that multiple out of order columns and multiple sublinks are handled correctly.
|
||||
CREATE TABLE upd2_test (a1 int, b1 int, c1 int, d1 int, e1 int, f1 int, g1 int);
|
||||
SELECT create_reference_table('upd2_test');
|
||||
|
||||
INSERT INTO upd2_test SELECT 1, 1, 1, 1, 1, 1, 1 FROM generate_series(1,5) c(i);
|
||||
|
||||
UPDATE upd2_test set (b1, a1) = (SELECT 200, 100), (g1, f1, e1) = (SELECT 700, 600, 500), (d1, c1) = (SELECT 400, 300);
|
||||
SELECT * FROM upd2_test;
|
||||
|
||||
UPDATE upd2_test set (g1, a1) = (SELECT 77, 11), (f1, b1) = (SELECT 66, 22), (e1, c1) = (SELECT 55, 33), (d1) = (SELECT 44);
|
||||
SELECT * FROM upd2_test;
|
||||
|
||||
UPDATE upd2_test set (g1, a1) = (SELECT 7, 1), (f1) = (SELECT 6), (c1, e1) = (SELECT 3, 5), (b1) = (SELECT 2), (d1) = (SELECT 4);
|
||||
SELECT * FROM upd2_test;
|
||||
|
||||
-- Test out of order updates on distributed table
|
||||
CREATE TABLE dist_test(a1 int, b1 numeric, c1 text, d1 int);
|
||||
SELECT create_distributed_table('dist_test', 'a1');
|
||||
INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (1, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (2, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4);
|
||||
INSERT INTO dist_test VALUES (3, 10.0, 'xxx', 4);
|
||||
|
||||
-- Router plan:
|
||||
UPDATE dist_test
|
||||
SET (d1, c1, b1) = (SELECT 7, 'tractor', 4.2)
|
||||
WHERE a1=1
|
||||
RETURNING *;
|
||||
|
||||
-- Pushdown plan:
|
||||
UPDATE dist_test
|
||||
SET (d1, c1, b1) = (SELECT X, 'car', Y)
|
||||
FROM (SELECT r.a * d1 as X, r.b * b1 as Y FROM update_test r, dist_test WHERE r.c=c1) upd
|
||||
WHERE dist_test.a1 > 2
|
||||
RETURNING *;
|
||||
|
||||
-- Test subscripting updates
|
||||
CREATE TABLE jsonb_subscript_update (id INT, data JSONB);
|
||||
SELECT create_distributed_table('jsonb_subscript_update', 'id');
|
||||
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}');
|
||||
|
||||
UPDATE jsonb_subscript_update
|
||||
SET data['b'] = updated_vals.b::TEXT::jsonb,
|
||||
data['c'] = updated_vals.c::TEXT::jsonb,
|
||||
data['d'] = updated_vals.d::TEXT::jsonb
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'] AS a,
|
||||
data['a']::NUMERIC + 1 AS b,
|
||||
data['a']::NUMERIC + 2 AS c,
|
||||
data['a']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update
|
||||
) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id;
|
||||
|
||||
SELECT * FROM jsonb_subscript_update ORDER BY 1,2;
|
||||
|
||||
TRUNCATE jsonb_subscript_update;
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}');
|
||||
|
||||
-- test router update with jsonb subscript
|
||||
UPDATE jsonb_subscript_update
|
||||
SET data['b'] = updated_vals.b::TEXT::jsonb,
|
||||
data['c'] = updated_vals.c::TEXT::jsonb,
|
||||
data['d'] = updated_vals.d::TEXT::jsonb
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'] AS a,
|
||||
data['a']::NUMERIC + 1 AS b,
|
||||
data['a']::NUMERIC + 2 AS c,
|
||||
data['a']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update
|
||||
) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id
|
||||
AND jsonb_subscript_update.id = 1;
|
||||
|
||||
SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2;
|
||||
|
||||
TRUNCATE jsonb_subscript_update;
|
||||
|
||||
-- Test updates on nested json objects
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": {"c":20, "d" : 200}}'), (2, '{"a": {"d":10, "c" : 100}}');
|
||||
|
||||
BEGIN;
|
||||
UPDATE jsonb_subscript_update
|
||||
SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB
|
||||
FROM
|
||||
(SELECT id,
|
||||
DATA['a']['c'] AS a,
|
||||
DATA['a']['c']::NUMERIC + 1 AS b,
|
||||
DATA['a']['c']::NUMERIC + 2 AS c,
|
||||
DATA['a']['d']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id;
|
||||
|
||||
SELECT * FROM jsonb_subscript_update ORDER BY 1,2;
|
||||
ROLLBACK;
|
||||
|
||||
BEGIN;
|
||||
-- Router plan
|
||||
UPDATE jsonb_subscript_update
|
||||
SET DATA['a']['c'] = (updated_vals.d + updated_vals.a::NUMERIC)::TEXT::JSONB
|
||||
FROM
|
||||
(SELECT id,
|
||||
DATA['a']['c'] AS a,
|
||||
DATA['a']['c']::NUMERIC + 1 AS b,
|
||||
DATA['a']['c']::NUMERIC + 2 AS c,
|
||||
DATA['a']['d']::NUMERIC + 3 AS d
|
||||
FROM jsonb_subscript_update) updated_vals
|
||||
WHERE jsonb_subscript_update.id = updated_vals.id
|
||||
AND jsonb_subscript_update.id = 1;
|
||||
|
||||
SELECT * FROM jsonb_subscript_update WHERE id = 1 ORDER BY 1,2;
|
||||
ROLLBACK;
|
||||
|
||||
TRUNCATE jsonb_subscript_update;
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}'), (4, '{"a": 4, "b": 10}');
|
||||
|
||||
ALTER TABLE jsonb_subscript_update ADD CONSTRAINT pkey PRIMARY KEY (id, data);
|
||||
|
||||
INSERT INTO jsonb_subscript_update VALUES (1, '{"a": 1}'), (2, '{"a": 2}')
|
||||
ON CONFLICT (id, data)
|
||||
DO UPDATE SET data['d']=(jsonb_subscript_update.data['a']::INT*100)::TEXT::JSONB,
|
||||
data['b']=(jsonb_subscript_update.data['a']::INT*-100)::TEXT::JSONB;
|
||||
|
||||
SELECT * FROM jsonb_subscript_update ORDER BY 1,2;
|
||||
|
||||
CREATE TABLE nested_obj_update(id INT, data JSONB, text_col TEXT);
|
||||
SELECT create_distributed_table('nested_obj_update', 'id');
|
||||
INSERT INTO nested_obj_update VALUES
|
||||
(1, '{"a": [1,2,3], "b": [4,5,6], "c": [7,8,9], "d": [1,2,1,2]}', '4'),
|
||||
(2, '{"a": [10,20,30], "b": [41,51,61], "c": [72,82,92], "d": [11,21,11,21]}', '6');
|
||||
|
||||
BEGIN;
|
||||
-- Pushdown plan
|
||||
UPDATE nested_obj_update
|
||||
SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB,
|
||||
data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB,
|
||||
data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB,
|
||||
text_col = (nested_obj_update.id*1000)::TEXT,
|
||||
data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB,
|
||||
data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB,
|
||||
data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'][0] AS a,
|
||||
data['b'][0]::NUMERIC + 1 AS b,
|
||||
data['c'][0]::NUMERIC + 2 AS c,
|
||||
data['c'][1]::NUMERIC + 3 AS d
|
||||
FROM nested_obj_update
|
||||
) updated_vals
|
||||
WHERE nested_obj_update.id = updated_vals.id;
|
||||
|
||||
SELECT * FROM nested_obj_update ORDER BY 1,2,3;
|
||||
ROLLBACK;
|
||||
|
||||
BEGIN;
|
||||
-- Router plan
|
||||
UPDATE nested_obj_update
|
||||
SET data['a'][0] = (updated_vals.b * 1)::TEXT::JSONB,
|
||||
data['b'][2] = (updated_vals.c * 2)::TEXT::JSONB,
|
||||
data['c'][0] = (updated_vals.d * 3)::TEXT::JSONB,
|
||||
text_col = (nested_obj_update.id*1000)::TEXT,
|
||||
data['a'][0] = (text_col::INT * data['a'][0]::INT)::TEXT::JSONB,
|
||||
data['d'][6] = (nested_obj_update.id*1)::TEXT::JSONB,
|
||||
data['d'][4] = (nested_obj_update.id*2)::TEXT::JSONB
|
||||
FROM (
|
||||
SELECT id,
|
||||
data['a'][0] AS a,
|
||||
data['b'][0]::NUMERIC + 1 AS b,
|
||||
data['c'][0]::NUMERIC + 2 AS c,
|
||||
data['c'][1]::NUMERIC + 3 AS d
|
||||
FROM nested_obj_update
|
||||
) updated_vals
|
||||
WHERE nested_obj_update.id = updated_vals.id
|
||||
AND nested_obj_update.id = 2;
|
||||
|
||||
SELECT * FROM nested_obj_update WHERE id = 2 ORDER BY 1,2,3;
|
||||
ROLLBACK;
|
||||
|
||||
-- suppress cascade messages
|
||||
SET client_min_messages to ERROR;
|
||||
DROP SCHEMA multi_update_select CASCADE;
|
||||
RESET client_min_messages;
|
||||
|
||||
|
|
@ -476,13 +476,11 @@ CALL exec_query_and_check_query_counters($$
|
|||
);
|
||||
|
||||
-- same with explain analyze
|
||||
--
|
||||
-- this time, query_execution_multi_shard is incremented twice because of #4212
|
||||
CALL exec_query_and_check_query_counters($$
|
||||
EXPLAIN (ANALYZE)
|
||||
SELECT * FROM (SELECT * FROM dist_table OFFSET 0) q
|
||||
$$,
|
||||
1, 2
|
||||
1, 1
|
||||
);
|
||||
|
||||
CALL exec_query_and_check_query_counters($$
|
||||
|
|
@ -807,9 +805,6 @@ CALL exec_query_and_check_query_counters($$
|
|||
-- A similar one but without the insert, so we would normally expect 2 increments
|
||||
-- for query_execution_single_shard and 2 for query_execution_multi_shard instead
|
||||
-- of 3 since the insert is not there anymore.
|
||||
--
|
||||
-- But this time we observe more counter increments because we execute the subplans
|
||||
-- twice because of #4212.
|
||||
CALL exec_query_and_check_query_counters($$
|
||||
EXPLAIN (ANALYZE)
|
||||
-- single-shard subplan (whole cte)
|
||||
|
|
@ -823,7 +818,7 @@ CALL exec_query_and_check_query_counters($$
|
|||
FROM (SELECT * FROM dist_table_1 ORDER BY a LIMIT 16) q -- multi-shard subplan (subquery q)
|
||||
JOIN cte ON q.a = cte.a
|
||||
$$,
|
||||
3, 4
|
||||
2, 2
|
||||
);
|
||||
|
||||
-- safe to push-down
|
||||
|
|
|
|||
|
|
@ -0,0 +1,93 @@
|
|||
\set VERBOSITY terse
|
||||
|
||||
SET citus.next_shard_id TO 1520000;
|
||||
|
||||
CREATE SCHEMA subscripting_op;
|
||||
SET search_path TO subscripting_op;
|
||||
|
||||
CREATE TABLE arr_subs_update(id INT, arr INT[], text_col TEXT, int_col_1 INT, int_col_2 INT);
|
||||
SELECT create_distributed_table('arr_subs_update', 'id');
|
||||
INSERT INTO arr_subs_update
|
||||
VALUES (1, '{1,2,3}', 'foo', 50, 60),
|
||||
(2, '{4,5,6}', 'bar', 60, 70),
|
||||
(3, '{7,8,9}', 'baz', 70, 80);
|
||||
|
||||
BEGIN;
|
||||
UPDATE arr_subs_update
|
||||
SET arr[1] = updated_vals.b,
|
||||
arr[3] = updated_vals.d,
|
||||
int_col_2 = 400,
|
||||
arr[2] = updated_vals.c
|
||||
FROM (
|
||||
SELECT id,
|
||||
arr[0] AS a,
|
||||
arr[1]::NUMERIC + 1 AS b,
|
||||
arr[2]::NUMERIC + 2 AS c,
|
||||
arr[3]::NUMERIC + 3 AS d
|
||||
FROM arr_subs_update
|
||||
) updated_vals
|
||||
WHERE arr_subs_update.id = updated_vals.id;
|
||||
|
||||
SELECT * FROM arr_subs_update ORDER BY 1,2,3,4;
|
||||
|
||||
ROLLBACK;
|
||||
|
||||
BEGIN;
|
||||
-- Test fast path router plan for subscripting update
|
||||
UPDATE arr_subs_update
|
||||
SET arr[1] = updated_vals.b,
|
||||
arr[3] = updated_vals.d,
|
||||
int_col_2 = 400,
|
||||
arr[2] = updated_vals.c
|
||||
FROM (
|
||||
SELECT id,
|
||||
arr[0] AS a,
|
||||
arr[1]::NUMERIC + 1 AS b,
|
||||
arr[2]::NUMERIC + 2 AS c,
|
||||
arr[3]::NUMERIC + 3 AS d
|
||||
FROM arr_subs_update
|
||||
) updated_vals
|
||||
WHERE arr_subs_update.id = updated_vals.id
|
||||
AND arr_subs_update.id = 1;
|
||||
|
||||
SELECT * FROM arr_subs_update
|
||||
WHERE id=1 ORDER BY 1,2,3,4;
|
||||
|
||||
ROLLBACK;
|
||||
|
||||
-- test if we can properly expand target list entries when there are dropped columns
|
||||
ALTER TABLE arr_subs_update DROP COLUMN int_col_1;
|
||||
|
||||
BEGIN;
|
||||
UPDATE arr_subs_update
|
||||
SET arr[1] = updated_vals.b,
|
||||
arr[3] = updated_vals.d,
|
||||
int_col_2 = 400,
|
||||
arr[2] = updated_vals.c
|
||||
FROM (
|
||||
SELECT id,
|
||||
arr[0] AS a,
|
||||
arr[1]::NUMERIC + 1 AS b,
|
||||
arr[2]::NUMERIC + 2 AS c,
|
||||
arr[3]::NUMERIC + 3 AS d
|
||||
FROM arr_subs_update
|
||||
) updated_vals
|
||||
WHERE arr_subs_update.id = updated_vals.id;
|
||||
|
||||
SELECT * FROM arr_subs_update ORDER BY 1,2,3,4;
|
||||
|
||||
ROLLBACK;
|
||||
|
||||
TRUNCATE arr_subs_update;
|
||||
INSERT INTO arr_subs_update VALUES (1, '{1,2,3}', 'foo', 60), (2, '{4,5,6}', 'bar', 70);
|
||||
|
||||
ALTER TABLE arr_subs_update ADD CONSTRAINT pkey PRIMARY KEY (id, arr);
|
||||
|
||||
INSERT INTO arr_subs_update VALUES (1, '{1,2,3}')
|
||||
ON CONFLICT (id, arr)
|
||||
DO UPDATE SET arr[0]=100, arr[1]=200, arr[5]=500;
|
||||
|
||||
SELECT * FROM arr_subs_update ORDER BY 1,2,3,4;
|
||||
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA subscripting_op CASCADE;
|
||||
|
|
@ -1109,5 +1109,49 @@ SELECT k, COUNT(*) FROM v GROUP BY k ORDER BY k;
|
|||
$$);
|
||||
|
||||
|
||||
CREATE TABLE dist1 (a int, b int);
|
||||
CREATE TABLE dist2 (a int, b int);
|
||||
SET citus.shard_count to 4;
|
||||
SELECT create_distributed_table('dist1', 'a');
|
||||
SELECT create_distributed_table('dist2', 'a');
|
||||
INSERT INTO dist1 VALUES (1, 1), (2, 1), (3, 1), (4, 1), (5, 1);
|
||||
INSERT INTO dist2 VALUES (5, 2), (6, 2), (7, 2), (8, 2), (9, 2);
|
||||
|
||||
-- safe to pushdown
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT * FROM dist1 JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT * FROM dist1 JOIN dist2 USING (a)
|
||||
) AS t1 ORDER BY 1;
|
||||
|
||||
-- not safe to pushdown, the distribution key from the outer part of the outer join is not in the target list
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT dist2.a FROM dist1 LEFT JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT dist2.a FROM dist2
|
||||
) AS t1 ORDER BY 1;
|
||||
|
||||
set client_min_messages to DEBUG3;
|
||||
-- not safe to pushdown, as is, sub-plan is generated
|
||||
-- the distribution key from the outer part of the outer join is not in the target list
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT dist1.a FROM dist1 RIGHT JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT dist2.a FROM dist2
|
||||
) AS t1 ORDER BY 1;
|
||||
|
||||
-- safe to pushdown, the distribution key from the outer side of the RIGHT join is in the target list
|
||||
SELECT * FROM
|
||||
(
|
||||
SELECT dist2.a
|
||||
FROM dist1 RIGHT JOIN dist2 USING (a)
|
||||
UNION
|
||||
SELECT dist2.a FROM dist2
|
||||
) AS t1
|
||||
ORDER BY 1;
|
||||
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA union_pushdown CASCADE;
|
||||
|
|
|
|||
Loading…
Reference in New Issue