Merge pull request #3922 from citusdata/fix/coordinator-evaluation

pull/3977/head
Marco Slot 2020-07-07 10:59:54 +02:00 committed by GitHub
commit 0b32a80f58
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
34 changed files with 461 additions and 480 deletions

View File

@ -44,11 +44,3 @@ TODO: to be written by someone with enough knowledge to write how, when and why
Implemented in `multi_copy.c` Implemented in `multi_copy.c`
TODO: to be written by someone with enough knowledge to write how, when and why it is used. TODO: to be written by someone with enough knowledge to write how, when and why it is used.
## MASTER_HOST host
Implemented in `multi_copy.c`
Triggered by the `MASTER_HOST` option being set on the copy command. Also accepts `MASTER_PORT`
TODO: to be written by someone with enough knowledge to write how, when and why it is used.

View File

@ -102,7 +102,7 @@ IsIndexRenameStmt(RenameStmt *renameStmt)
* PreprocessIndexStmt determines whether a given CREATE INDEX statement involves * PreprocessIndexStmt determines whether a given CREATE INDEX statement involves
* a distributed table. If so (and if the statement does not use unsupported * a distributed table. If so (and if the statement does not use unsupported
* options), it modifies the input statement to ensure proper execution against * options), it modifies the input statement to ensure proper execution against
* the master node table and creates a DDLJob to encapsulate information needed * the coordinator node table and creates a DDLJob to encapsulate information needed
* during the worker node portion of DDL execution before returning that DDLJob * during the worker node portion of DDL execution before returning that DDLJob
* in a List. If no distributed table is involved, this function returns NIL. * in a List. If no distributed table is involved, this function returns NIL.
*/ */
@ -194,7 +194,7 @@ PreprocessIndexStmt(Node *node, const char *createIndexCommand)
* PreprocessReindexStmt determines whether a given REINDEX statement involves * PreprocessReindexStmt determines whether a given REINDEX statement involves
* a distributed table. If so (and if the statement does not use unsupported * a distributed table. If so (and if the statement does not use unsupported
* options), it modifies the input statement to ensure proper execution against * options), it modifies the input statement to ensure proper execution against
* the master node table and creates a DDLJob to encapsulate information needed * the coordinator node table and creates a DDLJob to encapsulate information needed
* during the worker node portion of DDL execution before returning that DDLJob * during the worker node portion of DDL execution before returning that DDLJob
* in a List. If no distributed table is involved, this function returns NIL. * in a List. If no distributed table is involved, this function returns NIL.
*/ */
@ -302,7 +302,7 @@ PreprocessReindexStmt(Node *node, const char *reindexCommand)
* PreprocessDropIndexStmt determines whether a given DROP INDEX statement involves * PreprocessDropIndexStmt determines whether a given DROP INDEX statement involves
* a distributed table. If so (and if the statement does not use unsupported * a distributed table. If so (and if the statement does not use unsupported
* options), it modifies the input statement to ensure proper execution against * options), it modifies the input statement to ensure proper execution against
* the master node table and creates a DDLJob to encapsulate information needed * the coordinator node table and creates a DDLJob to encapsulate information needed
* during the worker node portion of DDL execution before returning that DDLJob * during the worker node portion of DDL execution before returning that DDLJob
* in a List. If no distributed table is involved, this function returns NIL. * in a List. If no distributed table is involved, this function returns NIL.
*/ */

View File

@ -4498,7 +4498,7 @@ ExtractParametersFromParamList(ParamListInfo paramListInfo,
/* /*
* Use 0 for data types where the oid values can be different on * Use 0 for data types where the oid values can be different on
* the master and worker nodes. Therefore, the worker nodes can * the coordinator and worker nodes. Therefore, the worker nodes can
* infer the correct oid. * infer the correct oid.
*/ */
if (parameterData->ptype >= FirstNormalObjectId && !useOriginalCustomTypeOids) if (parameterData->ptype >= FirstNormalObjectId && !useOriginalCustomTypeOids)

View File

@ -297,7 +297,7 @@ CitusBeginReadOnlyScan(CustomScanState *node, EState *estate, int eflags)
* *
* TODO: evaluate stable functions * TODO: evaluate stable functions
*/ */
ExecuteMasterEvaluableExpressions(jobQuery, planState); ExecuteCoordinatorEvaluableExpressions(jobQuery, planState);
/* job query no longer has parameters, so we should not send any */ /* job query no longer has parameters, so we should not send any */
workerJob->parametersInJobQueryResolved = true; workerJob->parametersInJobQueryResolved = true;
@ -347,7 +347,7 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags)
if (ModifyJobNeedsEvaluation(workerJob)) if (ModifyJobNeedsEvaluation(workerJob))
{ {
ExecuteMasterEvaluableExpressions(jobQuery, planState); ExecuteCoordinatorEvaluableExpressions(jobQuery, planState);
/* job query no longer has parameters, so we should not send any */ /* job query no longer has parameters, so we should not send any */
workerJob->parametersInJobQueryResolved = true; workerJob->parametersInJobQueryResolved = true;
@ -375,7 +375,7 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags)
RegenerateTaskForFasthPathQuery(workerJob); RegenerateTaskForFasthPathQuery(workerJob);
} }
} }
else if (workerJob->requiresMasterEvaluation) else if (workerJob->requiresCoordinatorEvaluation)
{ {
/* /*
* When there is no deferred pruning, but we did evaluate functions, then * When there is no deferred pruning, but we did evaluate functions, then
@ -428,7 +428,7 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags)
static bool static bool
ModifyJobNeedsEvaluation(Job *workerJob) ModifyJobNeedsEvaluation(Job *workerJob)
{ {
if (workerJob->requiresMasterEvaluation) if (workerJob->requiresCoordinatorEvaluation)
{ {
/* query contains functions that need to be evaluated on the coordinator */ /* query contains functions that need to be evaluated on the coordinator */
return true; return true;

View File

@ -162,21 +162,6 @@ HasReplicatedDistributedTable(List *relationOids)
} }
/*
* MaxMasterConnectionCount returns the number of connections a master can open.
* A master cannot create more than a certain number of file descriptors (FDs).
* Every task requires 2 FDs, one file and one connection. Some FDs are taken by
* the VFD pool and there is currently no way to reclaim these before opening a
* connection. We therefore assume some FDs to be reserved for VFDs, based on
* observing a typical size of the pool on a Citus master.
*/
int
MaxMasterConnectionCount(void)
{
return Max((max_files_per_process - RESERVED_FD_COUNT) / 2, 1);
}
/* /*
* RemoveJobDirectory gets automatically called at portal drop (end of query) or * RemoveJobDirectory gets automatically called at portal drop (end of query) or
* at transaction abort. The function removes the job directory and releases the * at transaction abort. The function removes the job directory and releases the

View File

@ -50,7 +50,7 @@ static CustomPathMethods CitusCustomScanPathMethods = {
}; };
/* /*
* MasterNodeSelectPlan takes in a distributed plan and a custom scan node which * PlanCombineQuery takes in a distributed plan and a custom scan node which
* wraps remote part of the plan. This function finds the combine query structure * wraps remote part of the plan. This function finds the combine query structure
* in the multi plan, and builds the final select plan to execute on the tuples * in the multi plan, and builds the final select plan to execute on the tuples
* returned by remote scan on the coordinator node. Note that this select * returned by remote scan on the coordinator node. Note that this select
@ -58,7 +58,7 @@ static CustomPathMethods CitusCustomScanPathMethods = {
* filled into the tuple store inside provided custom scan. * filled into the tuple store inside provided custom scan.
*/ */
PlannedStmt * PlannedStmt *
MasterNodeSelectPlan(DistributedPlan *distributedPlan, CustomScan *remoteScan) PlanCombineQuery(DistributedPlan *distributedPlan, CustomScan *remoteScan)
{ {
Query *combineQuery = distributedPlan->combineQuery; Query *combineQuery = distributedPlan->combineQuery;

View File

@ -1368,7 +1368,7 @@ static PlannedStmt *
FinalizeNonRouterPlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan, FinalizeNonRouterPlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan,
CustomScan *customScan) CustomScan *customScan)
{ {
PlannedStmt *finalPlan = MasterNodeSelectPlan(distributedPlan, customScan); PlannedStmt *finalPlan = PlanCombineQuery(distributedPlan, customScan);
finalPlan->queryId = localPlan->queryId; finalPlan->queryId = localPlan->queryId;
finalPlan->utilityStmt = localPlan->utilityStmt; finalPlan->utilityStmt = localPlan->utilityStmt;

View File

@ -311,8 +311,8 @@ PartitionColumnInTableList(Var *column, List *tableNodeList)
/* /*
* ShouldPullDistinctColumn returns true if distinct aggregate should pull * ShouldPullDistinctColumn returns true if distinct aggregate should pull
* individual columns from worker to master and evaluate aggregate operation * individual columns from worker to coordinator and evaluate aggregate operation
* at master. * on the coordinator.
* *
* Pull cases are: * Pull cases are:
* - repartition subqueries * - repartition subqueries

View File

@ -348,7 +348,8 @@ CreateDistributedInsertSelectPlan(Query *originalQuery,
workerJob->dependentJobList = NIL; workerJob->dependentJobList = NIL;
workerJob->jobId = jobId; workerJob->jobId = jobId;
workerJob->jobQuery = originalQuery; workerJob->jobQuery = originalQuery;
workerJob->requiresMasterEvaluation = RequiresMasterEvaluation(originalQuery); workerJob->requiresCoordinatorEvaluation =
RequiresCoordinatorEvaluation(originalQuery);
/* and finally the multi plan */ /* and finally the multi plan */
distributedPlan->workerJob = workerJob; distributedPlan->workerJob = workerJob;

View File

@ -190,7 +190,7 @@ IsLocalPlanCachingSupported(Job *currentJob, DistributedPlan *originalDistribute
* We do not cache plans with volatile functions in the query. * We do not cache plans with volatile functions in the query.
* *
* The reason we care about volatile functions is primarily that we * The reason we care about volatile functions is primarily that we
* already executed them in ExecuteMasterEvaluableExpressions * already executed them in ExecuteCoordinatorEvaluableExpressions
* and since we're falling back to the original query tree here we would * and since we're falling back to the original query tree here we would
* execute them again if we execute the plan. * execute them again if we execute the plan.
*/ */

View File

@ -167,7 +167,7 @@ PG_FUNCTION_INFO_V1(worker_save_query_explain_analyze);
/* /*
* CitusExplainScan is a custom scan explain callback function which is used to * CitusExplainScan is a custom scan explain callback function which is used to
* print explain information of a Citus plan which includes both master and * print explain information of a Citus plan which includes both combine query and
* distributed plan. * distributed plan.
*/ */
void void

View File

@ -165,7 +165,7 @@ static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
RelationRestrictionContext *restrictionContext, RelationRestrictionContext *restrictionContext,
uint32 taskId, uint32 taskId,
TaskType taskType, TaskType taskType,
bool modifyRequiresMasterEvaluation); bool modifyRequiresCoordinatorEvaluation);
static bool ShardIntervalsEqual(FmgrInfo *comparisonFunction, static bool ShardIntervalsEqual(FmgrInfo *comparisonFunction,
Oid collation, Oid collation,
ShardInterval *firstInterval, ShardInterval *firstInterval,
@ -2015,7 +2015,7 @@ BuildJob(Query *jobQuery, List *dependentJobList)
job->jobId = UniqueJobId(); job->jobId = UniqueJobId();
job->jobQuery = jobQuery; job->jobQuery = jobQuery;
job->dependentJobList = dependentJobList; job->dependentJobList = dependentJobList;
job->requiresMasterEvaluation = false; job->requiresCoordinatorEvaluation = false;
return job; return job;
} }
@ -2289,7 +2289,7 @@ List *
QueryPushdownSqlTaskList(Query *query, uint64 jobId, QueryPushdownSqlTaskList(Query *query, uint64 jobId,
RelationRestrictionContext *relationRestrictionContext, RelationRestrictionContext *relationRestrictionContext,
List *prunedRelationShardList, TaskType taskType, bool List *prunedRelationShardList, TaskType taskType, bool
modifyRequiresMasterEvaluation) modifyRequiresCoordinatorEvaluation)
{ {
List *sqlTaskList = NIL; List *sqlTaskList = NIL;
ListCell *restrictionCell = NULL; ListCell *restrictionCell = NULL;
@ -2393,7 +2393,7 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId,
relationRestrictionContext, relationRestrictionContext,
taskIdIndex, taskIdIndex,
taskType, taskType,
modifyRequiresMasterEvaluation); modifyRequiresCoordinatorEvaluation);
subqueryTask->jobId = jobId; subqueryTask->jobId = jobId;
sqlTaskList = lappend(sqlTaskList, subqueryTask); sqlTaskList = lappend(sqlTaskList, subqueryTask);
@ -2570,7 +2570,7 @@ ErrorIfUnsupportedShardDistribution(Query *query)
static Task * static Task *
QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
RelationRestrictionContext *restrictionContext, uint32 taskId, RelationRestrictionContext *restrictionContext, uint32 taskId,
TaskType taskType, bool modifyRequiresMasterEvaluation) TaskType taskType, bool modifyRequiresCoordinatorEvaluation)
{ {
Query *taskQuery = copyObject(originalQuery); Query *taskQuery = copyObject(originalQuery);
@ -2672,7 +2672,7 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
Task *subqueryTask = CreateBasicTask(jobId, taskId, taskType, NULL); Task *subqueryTask = CreateBasicTask(jobId, taskId, taskType, NULL);
if ((taskType == MODIFY_TASK && !modifyRequiresMasterEvaluation) || if ((taskType == MODIFY_TASK && !modifyRequiresCoordinatorEvaluation) ||
taskType == READ_TASK) taskType == READ_TASK)
{ {
pg_get_query_def(taskQuery, queryString); pg_get_query_def(taskQuery, queryString);

View File

@ -1302,9 +1302,9 @@ MasterIrreducibleExpressionWalker(Node *expression, WalkerState *state)
/* /*
* In order for statement replication to give us consistent results it's important * In order for statement replication to give us consistent results it's important
* that we either disallow or evaluate on the master anything which has a volatility * that we either disallow or evaluate on the coordinator anything which has a
* category above IMMUTABLE. Newer versions of postgres might add node types which * volatility category above IMMUTABLE. Newer versions of postgres might add node
* should be checked in this function. * types which should be checked in this function.
* *
* Look through contain_mutable_functions_walker or future PG's equivalent for new * Look through contain_mutable_functions_walker or future PG's equivalent for new
* node types before bumping this version number to fix compilation; e.g. for any * node types before bumping this version number to fix compilation; e.g. for any
@ -1451,7 +1451,7 @@ RouterInsertJob(Query *originalQuery)
} }
Job *job = CreateJob(originalQuery); Job *job = CreateJob(originalQuery);
job->requiresMasterEvaluation = RequiresMasterEvaluation(originalQuery); job->requiresCoordinatorEvaluation = RequiresCoordinatorEvaluation(originalQuery);
job->deferredPruning = true; job->deferredPruning = true;
job->partitionKeyValue = ExtractInsertPartitionKeyValue(originalQuery); job->partitionKeyValue = ExtractInsertPartitionKeyValue(originalQuery);
@ -1471,7 +1471,7 @@ CreateJob(Query *query)
job->taskList = NIL; job->taskList = NIL;
job->dependentJobList = NIL; job->dependentJobList = NIL;
job->subqueryPushdown = false; job->subqueryPushdown = false;
job->requiresMasterEvaluation = false; job->requiresCoordinatorEvaluation = false;
job->deferredPruning = false; job->deferredPruning = false;
return job; return job;
@ -1625,8 +1625,8 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
/* router planner should create task even if it doesn't hit a shard at all */ /* router planner should create task even if it doesn't hit a shard at all */
bool replacePrunedQueryWithDummy = true; bool replacePrunedQueryWithDummy = true;
/* check if this query requires master evaluation */ /* check if this query requires coordinator evaluation */
bool requiresMasterEvaluation = RequiresMasterEvaluation(originalQuery); bool requiresCoordinatorEvaluation = RequiresCoordinatorEvaluation(originalQuery);
FastPathRestrictionContext *fastPathRestrictionContext = FastPathRestrictionContext *fastPathRestrictionContext =
plannerRestrictionContext->fastPathRestrictionContext; plannerRestrictionContext->fastPathRestrictionContext;
@ -1688,7 +1688,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
relationRestrictionContext, relationRestrictionContext,
prunedShardIntervalListList, prunedShardIntervalListList,
MODIFY_TASK, MODIFY_TASK,
requiresMasterEvaluation); requiresCoordinatorEvaluation);
} }
else else
{ {
@ -1696,7 +1696,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
placementList, shardId); placementList, shardId);
} }
job->requiresMasterEvaluation = requiresMasterEvaluation; job->requiresCoordinatorEvaluation = requiresCoordinatorEvaluation;
return job; return job;
} }
@ -2152,10 +2152,11 @@ PlanRouterQuery(Query *originalQuery,
} }
/* /*
* If this is an UPDATE or DELETE query which requires master evaluation, * 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.
*/ */
if (!(UpdateOrDeleteQuery(originalQuery) && RequiresMasterEvaluation(originalQuery))) bool isUpdateOrDelete = UpdateOrDeleteQuery(originalQuery);
if (!(isUpdateOrDelete && RequiresCoordinatorEvaluation(originalQuery)))
{ {
UpdateRelationToShardNames((Node *) originalQuery, *relationShardList); UpdateRelationToShardNames((Node *) originalQuery, *relationShardList);
} }

View File

@ -46,7 +46,7 @@ The join order planner is applied to the join tree in the original query and gen
The logical optimizer uses commutativity rules to push project and select operators down below the `MultiCollect` nodes. Everything above the `MultiCollect` operator will be is executed on the coordinator and everything below on the workers. Additionally, the optimizer uses distributivity rules to push down operators below the `MultiJoin` nodes, such that filters and projections are applied prior to joins. This is primarily relevant for re-partition joins which first try to reduce the data by applying selections and projections, and then re-partitioning the result. The logical optimizer uses commutativity rules to push project and select operators down below the `MultiCollect` nodes. Everything above the `MultiCollect` operator will be is executed on the coordinator and everything below on the workers. Additionally, the optimizer uses distributivity rules to push down operators below the `MultiJoin` nodes, such that filters and projections are applied prior to joins. This is primarily relevant for re-partition joins which first try to reduce the data by applying selections and projections, and then re-partitioning the result.
A number of SQL clauses like aggregates, GROUP BY, ORDER BY, LIMIT can only be pushed down below the `MultiCollect` under certain conditions. All these clauses are bundled together in a `MultiExtendedOpNode`. After the basic transformation, the `MultiExtendedOpNode`s are directly above the `MultiCollect` nodes. They are then split into a master and a worker part and the worker part is pushed down below the `MultiCollect`. A number of SQL clauses like aggregates, GROUP BY, ORDER BY, LIMIT can only be pushed down below the `MultiCollect` under certain conditions. All these clauses are bundled together in a `MultiExtendedOpNode`. After the basic transformation, the `MultiExtendedOpNode`s are directly above the `MultiCollect` nodes. They are then split into a coordinator and a worker part and the worker part is pushed down below the `MultiCollect`.
### Physical planner ### Physical planner

View File

@ -34,22 +34,23 @@
static bool IsVariableExpression(Node *node); static bool IsVariableExpression(Node *node);
static Expr * citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod, static Expr * citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod,
Oid result_collation, Oid result_collation,
MasterEvaluationContext *masterEvaluationContext); CoordinatorEvaluationContext *
coordinatorEvaluationContext);
static bool CitusIsVolatileFunctionIdChecker(Oid func_id, void *context); static bool CitusIsVolatileFunctionIdChecker(Oid func_id, void *context);
static bool CitusIsMutableFunctionIdChecker(Oid func_id, void *context); static bool CitusIsMutableFunctionIdChecker(Oid func_id, void *context);
static bool ShouldEvaluateExpression(Expr *expression); static bool ShouldEvaluateExpression(Expr *expression);
static bool ShouldEvaluateFunctionWithMasterContext(MasterEvaluationContext * static bool ShouldEvaluateFunctions(CoordinatorEvaluationContext *evaluationContext);
evaluationContext);
static void FixFunctionArguments(Node *expr); static void FixFunctionArguments(Node *expr);
static bool FixFunctionArgumentsWalker(Node *expr, void *context); static bool FixFunctionArgumentsWalker(Node *expr, void *context);
/* /*
* RequiresMasterEvaluation returns the executor needs to reparse and * RequiresCoordinatorEvaluation returns the executor needs to reparse and
* try to execute this query, which is the case if the query contains * try to execute this query, which is the case if the query contains
* any stable or volatile function. * any stable or volatile function.
*/ */
bool bool
RequiresMasterEvaluation(Query *query) RequiresCoordinatorEvaluation(Query *query)
{ {
if (query->commandType == CMD_SELECT && !query->hasModifyingCTE) if (query->commandType == CMD_SELECT && !query->hasModifyingCTE)
{ {
@ -61,25 +62,25 @@ RequiresMasterEvaluation(Query *query)
/* /*
* ExecuteMasterEvaluableExpressions evaluates expressions and parameters * ExecuteCoordinatorEvaluableExpressions evaluates expressions and parameters
* that can be resolved to a constant. * that can be resolved to a constant.
*/ */
void void
ExecuteMasterEvaluableExpressions(Query *query, PlanState *planState) ExecuteCoordinatorEvaluableExpressions(Query *query, PlanState *planState)
{ {
MasterEvaluationContext masterEvaluationContext; CoordinatorEvaluationContext coordinatorEvaluationContext;
masterEvaluationContext.planState = planState; coordinatorEvaluationContext.planState = planState;
if (query->commandType == CMD_SELECT) if (query->commandType == CMD_SELECT)
{ {
masterEvaluationContext.evaluationMode = EVALUATE_PARAMS; coordinatorEvaluationContext.evaluationMode = EVALUATE_PARAMS;
} }
else else
{ {
masterEvaluationContext.evaluationMode = EVALUATE_FUNCTIONS_PARAMS; coordinatorEvaluationContext.evaluationMode = EVALUATE_FUNCTIONS_PARAMS;
} }
PartiallyEvaluateExpression((Node *) query, &masterEvaluationContext); PartiallyEvaluateExpression((Node *) query, &coordinatorEvaluationContext);
} }
@ -91,7 +92,7 @@ ExecuteMasterEvaluableExpressions(Query *query, PlanState *planState)
*/ */
Node * Node *
PartiallyEvaluateExpression(Node *expression, PartiallyEvaluateExpression(Node *expression,
MasterEvaluationContext *masterEvaluationContext) CoordinatorEvaluationContext *coordinatorEvaluationContext)
{ {
if (expression == NULL || IsA(expression, Const)) if (expression == NULL || IsA(expression, Const))
{ {
@ -112,10 +113,10 @@ PartiallyEvaluateExpression(Node *expression,
exprType(expression), exprType(expression),
exprTypmod(expression), exprTypmod(expression),
exprCollation(expression), exprCollation(expression),
masterEvaluationContext); coordinatorEvaluationContext);
} }
else if (ShouldEvaluateExpression((Expr *) expression) && else if (ShouldEvaluateExpression((Expr *) expression) &&
ShouldEvaluateFunctionWithMasterContext(masterEvaluationContext)) ShouldEvaluateFunctions(coordinatorEvaluationContext))
{ {
if (FindNodeCheck(expression, IsVariableExpression)) if (FindNodeCheck(expression, IsVariableExpression))
{ {
@ -132,19 +133,19 @@ PartiallyEvaluateExpression(Node *expression,
*/ */
return (Node *) expression_tree_mutator(expression, return (Node *) expression_tree_mutator(expression,
PartiallyEvaluateExpression, PartiallyEvaluateExpression,
masterEvaluationContext); coordinatorEvaluationContext);
} }
return (Node *) citus_evaluate_expr((Expr *) expression, return (Node *) citus_evaluate_expr((Expr *) expression,
exprType(expression), exprType(expression),
exprTypmod(expression), exprTypmod(expression),
exprCollation(expression), exprCollation(expression),
masterEvaluationContext); coordinatorEvaluationContext);
} }
else if (nodeTag == T_Query) else if (nodeTag == T_Query)
{ {
Query *query = (Query *) expression; Query *query = (Query *) expression;
MasterEvaluationContext subContext = *masterEvaluationContext; CoordinatorEvaluationContext subContext = *coordinatorEvaluationContext;
if (query->commandType != CMD_SELECT) if (query->commandType != CMD_SELECT)
{ {
/* /*
@ -165,7 +166,7 @@ PartiallyEvaluateExpression(Node *expression,
{ {
return (Node *) expression_tree_mutator(expression, return (Node *) expression_tree_mutator(expression,
PartiallyEvaluateExpression, PartiallyEvaluateExpression,
masterEvaluationContext); coordinatorEvaluationContext);
} }
return expression; return expression;
@ -173,12 +174,12 @@ PartiallyEvaluateExpression(Node *expression,
/* /*
* ShouldEvaluateFunctionWithMasterContext is a helper function which is used to * ShouldEvaluateFunctions is a helper function which is used to
* decide whether the function/expression should be evaluated with the input * decide whether the function/expression should be evaluated with the input
* masterEvaluationContext. * coordinatorEvaluationContext.
*/ */
static bool static bool
ShouldEvaluateFunctionWithMasterContext(MasterEvaluationContext *evaluationContext) ShouldEvaluateFunctions(CoordinatorEvaluationContext *evaluationContext)
{ {
if (evaluationContext == NULL) if (evaluationContext == NULL)
{ {
@ -269,7 +270,7 @@ IsVariableExpression(Node *node)
static Expr * static Expr *
citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod, citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod,
Oid result_collation, Oid result_collation,
MasterEvaluationContext *masterEvaluationContext) CoordinatorEvaluationContext *coordinatorEvaluationContext)
{ {
PlanState *planState = NULL; PlanState *planState = NULL;
EState *estate; EState *estate;
@ -280,19 +281,19 @@ citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod,
int16 resultTypLen; int16 resultTypLen;
bool resultTypByVal; bool resultTypByVal;
if (masterEvaluationContext) if (coordinatorEvaluationContext)
{ {
planState = masterEvaluationContext->planState; planState = coordinatorEvaluationContext->planState;
if (IsA(expr, Param)) if (IsA(expr, Param))
{ {
if (masterEvaluationContext->evaluationMode == EVALUATE_NONE) if (coordinatorEvaluationContext->evaluationMode == EVALUATE_NONE)
{ {
/* bail out, the caller doesn't want params to be evaluated */ /* bail out, the caller doesn't want params to be evaluated */
return expr; return expr;
} }
} }
else if (masterEvaluationContext->evaluationMode != EVALUATE_FUNCTIONS_PARAMS) else if (coordinatorEvaluationContext->evaluationMode != EVALUATE_FUNCTIONS_PARAMS)
{ {
/* should only get here for node types we should evaluate */ /* should only get here for node types we should evaluate */
Assert(ShouldEvaluateExpression(expr)); Assert(ShouldEvaluateExpression(expr));

View File

@ -96,7 +96,7 @@ copyJobInfo(Job *newnode, Job *from)
COPY_NODE_FIELD(taskList); COPY_NODE_FIELD(taskList);
COPY_NODE_FIELD(dependentJobList); COPY_NODE_FIELD(dependentJobList);
COPY_SCALAR_FIELD(subqueryPushdown); COPY_SCALAR_FIELD(subqueryPushdown);
COPY_SCALAR_FIELD(requiresMasterEvaluation); COPY_SCALAR_FIELD(requiresCoordinatorEvaluation);
COPY_SCALAR_FIELD(deferredPruning); COPY_SCALAR_FIELD(deferredPruning);
COPY_NODE_FIELD(partitionKeyValue); COPY_NODE_FIELD(partitionKeyValue);
COPY_NODE_FIELD(localPlannedStatements); COPY_NODE_FIELD(localPlannedStatements);

View File

@ -340,7 +340,7 @@ OutJobFields(StringInfo str, const Job *node)
WRITE_NODE_FIELD(taskList); WRITE_NODE_FIELD(taskList);
WRITE_NODE_FIELD(dependentJobList); WRITE_NODE_FIELD(dependentJobList);
WRITE_BOOL_FIELD(subqueryPushdown); WRITE_BOOL_FIELD(subqueryPushdown);
WRITE_BOOL_FIELD(requiresMasterEvaluation); WRITE_BOOL_FIELD(requiresCoordinatorEvaluation);
WRITE_BOOL_FIELD(deferredPruning); WRITE_BOOL_FIELD(deferredPruning);
WRITE_NODE_FIELD(partitionKeyValue); WRITE_NODE_FIELD(partitionKeyValue);
WRITE_NODE_FIELD(localPlannedStatements); WRITE_NODE_FIELD(localPlannedStatements);

View File

@ -17,10 +17,10 @@
/* /*
* MasterEvaluationMode is used to signal what expressions in the query * CoordinatorEvaluationMode is used to signal what expressions in the query
* should be evaluated on the coordinator. * should be evaluated on the coordinator.
*/ */
typedef enum MasterEvaluationMode typedef enum CoordinatorEvaluationMode
{ {
/* evaluate nothing */ /* evaluate nothing */
EVALUATE_NONE = 0, EVALUATE_NONE = 0,
@ -30,23 +30,24 @@ typedef enum MasterEvaluationMode
/* evaluate both the functions/expressions and the external paramaters */ /* evaluate both the functions/expressions and the external paramaters */
EVALUATE_FUNCTIONS_PARAMS EVALUATE_FUNCTIONS_PARAMS
} MasterEvaluationMode; } CoordinatorEvaluationMode;
/* /*
* This struct is used to pass information to master * This struct is used to pass information to master
* evaluation logic. * evaluation logic.
*/ */
typedef struct MasterEvaluationContext typedef struct CoordinatorEvaluationContext
{ {
PlanState *planState; PlanState *planState;
MasterEvaluationMode evaluationMode; CoordinatorEvaluationMode evaluationMode;
} MasterEvaluationContext; } CoordinatorEvaluationContext;
extern bool RequiresMasterEvaluation(Query *query); extern bool RequiresCoordinatorEvaluation(Query *query);
extern void ExecuteMasterEvaluableExpressions(Query *query, PlanState *planState); extern void ExecuteCoordinatorEvaluableExpressions(Query *query, PlanState *planState);
extern Node * PartiallyEvaluateExpression(Node *expression, extern Node * PartiallyEvaluateExpression(Node *expression,
MasterEvaluationContext *masterEvaluationContext); CoordinatorEvaluationContext *
coordinatorEvaluationContext);
extern bool CitusIsVolatileFunction(Node *node); extern bool CitusIsVolatileFunction(Node *node);
extern bool CitusIsMutableFunction(Node *node); extern bool CitusIsMutableFunction(Node *node);

View File

@ -1,6 +1,6 @@
/*------------------------------------------------------------------------- /*-------------------------------------------------------------------------
* *
* merge_planner.h * combine_query_planner.h
* Function declarations for building planned statements; these statements * Function declarations for building planned statements; these statements
* are then executed on the coordinator node. * are then executed on the coordinator node.
* *
@ -9,8 +9,8 @@
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
#ifndef MERGE_PLANNER_H #ifndef COMBINE_QUERY_PLANNER_H
#define MERGE_PLANNER_H #define COMBINE_QUERY_PLANNER_H
#include "lib/stringinfo.h" #include "lib/stringinfo.h"
#include "nodes/parsenodes.h" #include "nodes/parsenodes.h"
@ -23,16 +23,16 @@
#endif #endif
/* Function declarations for building local plans on the master node */ /* Function declarations for building local plans on the coordinator node */
struct DistributedPlan; struct DistributedPlan;
struct CustomScan; struct CustomScan;
extern Path * CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInfo, extern Path * CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInfo,
Index restrictionIndex, RangeTblEntry *rte, Index restrictionIndex, RangeTblEntry *rte,
CustomScan *remoteScan); CustomScan *remoteScan);
extern PlannedStmt * MasterNodeSelectPlan(struct DistributedPlan *distributedPlan, extern PlannedStmt * PlanCombineQuery(struct DistributedPlan *distributedPlan,
struct CustomScan *dataScan); struct CustomScan *dataScan);
extern Unique * make_unique_from_sortclauses(Plan *lefttree, List *distinctList); extern Unique * make_unique_from_sortclauses(Plan *lefttree, List *distinctList);
extern bool ReplaceCitusExtraDataContainer; extern bool ReplaceCitusExtraDataContainer;
extern CustomScan *ReplaceCitusExtraDataContainerWithCustomScan; extern CustomScan *ReplaceCitusExtraDataContainerWithCustomScan;
#endif /* MERGE_PLANNER_H */ #endif /* COMBINE_QUERY_PLANNER_H */

View File

@ -24,7 +24,7 @@
/* /*
* In our distributed database, we need a mechanism to make remote procedure * In our distributed database, we need a mechanism to make remote procedure
* calls between clients, the master node, and worker nodes. These remote calls * calls between clients, the coordinator node, and worker nodes. These remote calls
* require serializing and deserializing values and function signatures between * require serializing and deserializing values and function signatures between
* nodes; and for these, we currently use PostgreSQL's built-in type and * nodes; and for these, we currently use PostgreSQL's built-in type and
* function definition system. This approach is by no means ideal however; and * function definition system. This approach is by no means ideal however; and
@ -41,7 +41,7 @@
*/ */
/* Number of tuple fields that master node functions return */ /* Number of tuple fields that coordinator node functions return */
#define TABLE_METADATA_FIELDS 7 #define TABLE_METADATA_FIELDS 7
#define CANDIDATE_NODE_FIELDS 2 #define CANDIDATE_NODE_FIELDS 2
#define WORKER_NODE_FIELDS 2 #define WORKER_NODE_FIELDS 2

View File

@ -163,12 +163,13 @@ typedef struct DistributedPlanningContext
/* /*
* CitusCustomScanPath is injected into the planner during the master query planning phase * CitusCustomScanPath is injected into the planner during the combine query planning
* of the logical planner. * phase of the logical planner.
* We call out to the standard planner to plan the master query part for the output of the *
* logical planner. This makes it easier to implement new sql features into the logical * We call out to the standard planner to plan the combine query part for the output of
* planner by not having to manually implement the plan creation for the query on the * the logical planner. This makes it easier to implement new sql features into the
* master. * logical planner by not having to manually implement the plan creation for the combine
* query on the coordinator..
*/ */
typedef struct CitusCustomScanPath typedef struct CitusCustomScanPath
{ {
@ -176,7 +177,7 @@ typedef struct CitusCustomScanPath
/* /*
* Custom scan node computed by the citus planner that will produce the tuples for the * Custom scan node computed by the citus planner that will produce the tuples for the
* path we are injecting during the planning of the master query * path we are injecting during the planning of the combine query
*/ */
CustomScan *remoteScan; CustomScan *remoteScan;
} CitusCustomScanPath; } CitusCustomScanPath;

View File

@ -15,7 +15,7 @@
/* /*
* ExtendedOpNodeProperties is a helper structure that is used to * ExtendedOpNodeProperties is a helper structure that is used to
* share the common information among the worker and master extended * share the common information among the worker and coordinator extended
* op nodes. * op nodes.
* *
* It is designed to be a read-only singleton object per extended op node * It is designed to be a read-only singleton object per extended op node

View File

@ -1,8 +1,8 @@
/*------------------------------------------------------------------------- /*-------------------------------------------------------------------------
* *
* metadata_utility.h * metadata_utility.h
* Type and function declarations used for reading and modifying master * Type and function declarations used for reading and modifying
* node's metadata. * coordinator node's metadata.
* *
* Copyright (c) Citus Data, Inc. * Copyright (c) Citus Data, Inc.
* *

View File

@ -156,7 +156,7 @@ typedef struct MultiCartesianProduct
/* /*
* MultiExtendedOp defines a set of extended operators that operate on columns * MultiExtendedOp defines a set of extended operators that operate on columns
* in relational algebra. This node allows us to distinguish between operations * in relational algebra. This node allows us to distinguish between operations
* in the master and worker nodes, and also captures the following: * in the coordinator and worker nodes, and also captures the following:
* *
* (1) Aggregate functions such as sums or averages; * (1) Aggregate functions such as sums or averages;
* (2) Grouping of attributes; these groupings may also be tied to aggregates; * (2) Grouping of attributes; these groupings may also be tied to aggregates;

View File

@ -151,7 +151,7 @@ typedef struct Job
List *taskList; List *taskList;
List *dependentJobList; List *dependentJobList;
bool subqueryPushdown; bool subqueryPushdown;
bool requiresMasterEvaluation; /* only applies to modify jobs */ bool requiresCoordinatorEvaluation; /* only applies to modify jobs */
bool deferredPruning; bool deferredPruning;
Const *partitionKeyValue; Const *partitionKeyValue;
@ -599,7 +599,7 @@ extern List * QueryPushdownSqlTaskList(Query *query, uint64 jobId,
RelationRestrictionContext * RelationRestrictionContext *
relationRestrictionContext, relationRestrictionContext,
List *prunedRelationShardList, TaskType taskType, List *prunedRelationShardList, TaskType taskType,
bool modifyRequiresMasterEvaluation); bool modifyRequiresCoordinatorEvaluation);
/* function declarations for managing jobs */ /* function declarations for managing jobs */
extern uint64 UniqueJobId(void); extern uint64 UniqueJobId(void);

View File

@ -58,7 +58,7 @@ typedef enum
} TaskExecStatus; } TaskExecStatus;
/* Enumeration to track file transmits to the master node */ /* Enumeration to track file transmits to the coordinator node */
typedef enum typedef enum
{ {
EXEC_TRANSMIT_INVALID_FIRST = 0, EXEC_TRANSMIT_INVALID_FIRST = 0,
@ -186,7 +186,6 @@ extern void CleanupTaskExecution(TaskExecution *taskExecution);
extern void ErrorSizeLimitIsExceeded(void); extern void ErrorSizeLimitIsExceeded(void);
extern bool TaskExecutionFailed(TaskExecution *taskExecution); extern bool TaskExecutionFailed(TaskExecution *taskExecution);
extern void AdjustStateForFailure(TaskExecution *taskExecution); extern void AdjustStateForFailure(TaskExecution *taskExecution);
extern int MaxMasterConnectionCount(void);
extern TupleTableSlot * TaskTrackerExecScan(CustomScanState *node); extern TupleTableSlot * TaskTrackerExecScan(CustomScanState *node);

View File

@ -4,9 +4,9 @@
* definition of the "server" relation (pg_dist_placement). * definition of the "server" relation (pg_dist_placement).
* *
* This table keeps information on remote shards and their whereabouts on the * This table keeps information on remote shards and their whereabouts on the
* master node. The table's contents are updated and used as follows: (i) the * coordinator node. The table's contents are updated and used as follows: (i) the
* worker nodes send periodic reports about the shards they contain, and (ii) * worker nodes send periodic reports about the shards they contain, and (ii)
* the master reconciles these shard reports, and determines outdated, under- * the coordinator reconciles these shard reports, and determines outdated, under-
* and over-replicated shards. * and over-replicated shards.
* *
* Copyright (c) Citus Data, Inc. * Copyright (c) Citus Data, Inc.

View File

@ -1,7 +1,7 @@
-- This test relies on metadata being synced -- This test relies on metadata being synced
-- that's why is should be executed on MX schedule -- that's why is should be executed on MX schedule
CREATE SCHEMA master_evaluation; CREATE SCHEMA coordinator_evaluation;
SET search_path TO master_evaluation; SET search_path TO coordinator_evaluation;
-- create a volatile function that returns the local node id -- create a volatile function that returns the local node id
CREATE OR REPLACE FUNCTION get_local_node_id_volatile() CREATE OR REPLACE FUNCTION get_local_node_id_volatile()
RETURNS INT AS $$ RETURNS INT AS $$
@ -29,8 +29,8 @@ SELECT create_distributed_function('get_local_node_id_volatile_sum_with_param(in
(1 row) (1 row)
CREATE TABLE master_evaluation_table (key int, value int); CREATE TABLE coordinator_evaluation_table (key int, value int);
SELECT create_distributed_table('master_evaluation_table', 'key'); SELECT create_distributed_table('coordinator_evaluation_table', 'key');
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -44,16 +44,16 @@ SELECT get_local_node_id_volatile();
(1 row) (1 row)
-- load data -- load data
INSERT INTO master_evaluation_table SELECT i, i FROM generate_series(0,100)i; INSERT INTO coordinator_evaluation_table SELECT i, i FROM generate_series(0,100)i;
-- we expect that the function is evaluated on the worker node, so we should get a row -- we expect that the function is evaluated on the worker node, so we should get a row
SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table WHERE key = 1; SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table WHERE key = 1;
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
-- make sure that it is also true for fast-path router queries with paramaters -- make sure that it is also true for fast-path router queries with paramaters
PREPARE fast_path_router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table WHERE key = $1; PREPARE fast_path_router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table WHERE key = $1;
execute fast_path_router_with_param(1); execute fast_path_router_with_param(1);
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -103,13 +103,13 @@ execute fast_path_router_with_param(8);
(1 row) (1 row)
-- same query as fast_path_router_with_param, but with consts -- same query as fast_path_router_with_param, but with consts
SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table WHERE key = 1; SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table WHERE key = 1;
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
PREPARE router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table m1 JOIN master_evaluation_table m2 USING(key) WHERE key = $1; PREPARE router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table m1 JOIN coordinator_evaluation_table m2 USING(key) WHERE key = $1;
execute router_with_param(1); execute router_with_param(1);
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -159,21 +159,21 @@ execute router_with_param(8);
(1 row) (1 row)
-- same query as router_with_param, but with consts -- same query as router_with_param, but with consts
SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table m1 JOIN master_evaluation_table m2 USING(key) WHERE key = 1; SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table m1 JOIN coordinator_evaluation_table m2 USING(key) WHERE key = 1;
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
-- for multi-shard queries, we still expect the evaluation to happen on the workers -- for multi-shard queries, we still expect the evaluation to happen on the workers
SELECT count(*), max(get_local_node_id_volatile()) != 0, min(get_local_node_id_volatile()) != 0 FROM master_evaluation_table; SELECT count(*), max(get_local_node_id_volatile()) != 0, min(get_local_node_id_volatile()) != 0 FROM coordinator_evaluation_table;
count | ?column? | ?column? count | ?column? | ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
101 | t | t 101 | t | t
(1 row) (1 row)
-- when executed locally, we expect to get the result from the coordinator -- when executed locally, we expect to get the result from the coordinator
SELECT (SELECT count(*) FROM master_evaluation_table), get_local_node_id_volatile() = 0; SELECT (SELECT count(*) FROM coordinator_evaluation_table), get_local_node_id_volatile() = 0;
count | ?column? count | ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
101 | t 101 | t
@ -181,7 +181,7 @@ SELECT (SELECT count(*) FROM master_evaluation_table), get_local_node_id_volatil
-- make sure that we get the results from the workers when the query is sent to workers -- make sure that we get the results from the workers when the query is sent to workers
SET citus.task_assignment_policy TO "round-robin"; SET citus.task_assignment_policy TO "round-robin";
SELECT (SELECT count(*) FROM master_evaluation_table), get_local_node_id_volatile() = 0; SELECT (SELECT count(*) FROM coordinator_evaluation_table), get_local_node_id_volatile() = 0;
count | ?column? count | ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
101 | f 101 | f
@ -189,13 +189,13 @@ SELECT (SELECT count(*) FROM master_evaluation_table), get_local_node_id_volatil
RESET citus.task_assignment_policy; RESET citus.task_assignment_policy;
-- for multi-shard SELECTs, we don't try to evaluate on the coordinator -- for multi-shard SELECTs, we don't try to evaluate on the coordinator
SELECT min(get_local_node_id_volatile()) > 0 FROM master_evaluation_table; SELECT min(get_local_node_id_volatile()) > 0 FROM coordinator_evaluation_table;
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
SELECT count(*) > 0 FROM master_evaluation_table WHERE value >= get_local_node_id_volatile(); SELECT count(*) > 0 FROM coordinator_evaluation_table WHERE value >= get_local_node_id_volatile();
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -204,7 +204,7 @@ SELECT count(*) > 0 FROM master_evaluation_table WHERE value >= get_local_node_i
-- let's have some tests around expressions -- let's have some tests around expressions
-- for modifications, we expect the evaluation to happen on the coordinator -- for modifications, we expect the evaluation to happen on the coordinator
-- thus the results should be 0 -- thus the results should be 0
PREPARE insert_with_param_expression(int) AS INSERT INTO master_evaluation_table (key, value) VALUES ($1 + get_local_node_id_volatile(), $1 + get_local_node_id_volatile()) RETURNING key, value; PREPARE insert_with_param_expression(int) AS INSERT INTO coordinator_evaluation_table (key, value) VALUES ($1 + get_local_node_id_volatile(), $1 + get_local_node_id_volatile()) RETURNING key, value;
EXECUTE insert_with_param_expression(0); EXECUTE insert_with_param_expression(0);
key | value key | value
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -249,7 +249,7 @@ EXECUTE insert_with_param_expression(0);
-- for modifications, we expect the evaluation to happen on the coordinator -- for modifications, we expect the evaluation to happen on the coordinator
-- thus the results should be 0 -- thus the results should be 0
PREPARE insert_with_param(int) AS INSERT INTO master_evaluation_table (key, value) VALUES ($1, $1) RETURNING key, value; PREPARE insert_with_param(int) AS INSERT INTO coordinator_evaluation_table (key, value) VALUES ($1, $1) RETURNING key, value;
EXECUTE insert_with_param(0 + get_local_node_id_volatile()); EXECUTE insert_with_param(0 + get_local_node_id_volatile());
key | value key | value
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -292,7 +292,7 @@ EXECUTE insert_with_param(0 + get_local_node_id_volatile());
0 | 0 0 | 0
(1 row) (1 row)
PREPARE router_select_with_param_expression(int) AS SELECT value > 0 FROM master_evaluation_table WHERE key = $1 + get_local_node_id_volatile(); PREPARE router_select_with_param_expression(int) AS SELECT value > 0 FROM coordinator_evaluation_table WHERE key = $1 + get_local_node_id_volatile();
-- for selects, we expect the evaluation to happen on the workers -- for selects, we expect the evaluation to happen on the workers
-- this means that the query should be hitting multiple workers -- this means that the query should be hitting multiple workers
SET client_min_messages TO DEBUG2; SET client_min_messages TO DEBUG2;
@ -353,7 +353,7 @@ DEBUG: Router planner cannot handle multi-shard select queries
t t
(1 row) (1 row)
PREPARE router_select_with_param(int) AS SELECT DISTINCT value FROM master_evaluation_table WHERE key = $1; PREPARE router_select_with_param(int) AS SELECT DISTINCT value FROM coordinator_evaluation_table WHERE key = $1;
-- this time the parameter itself is a function, so should be evaluated -- this time the parameter itself is a function, so should be evaluated
-- on the coordinator -- on the coordinator
EXECUTE router_select_with_param(0 + get_local_node_id_volatile()); EXECUTE router_select_with_param(0 + get_local_node_id_volatile());
@ -460,7 +460,7 @@ EXECUTE router_select_with_param(get_local_node_id_volatile());
(1 row) (1 row)
-- this time use the parameter inside the function -- this time use the parameter inside the function
PREPARE router_select_with_parameter_in_function(int) AS SELECT bool_and(get_local_node_id_volatile_sum_with_param($1) > 1) FROM master_evaluation_table WHERE key = get_local_node_id_volatile_sum_with_param($1); PREPARE router_select_with_parameter_in_function(int) AS SELECT bool_and(get_local_node_id_volatile_sum_with_param($1) > 1) FROM coordinator_evaluation_table WHERE key = get_local_node_id_volatile_sum_with_param($1);
EXECUTE router_select_with_parameter_in_function(0); EXECUTE router_select_with_parameter_in_function(0);
DEBUG: Router planner cannot handle multi-shard select queries DEBUG: Router planner cannot handle multi-shard select queries
bool_and bool_and
@ -514,8 +514,8 @@ DEBUG: Router planner cannot handle multi-shard select queries
RESET client_min_messages; RESET client_min_messages;
RESET citus.log_remote_commands; RESET citus.log_remote_commands;
-- numeric has different casting affects, so some tests on that -- numeric has different casting affects, so some tests on that
CREATE TABLE master_evaluation_table_2 (key numeric, value numeric); CREATE TABLE coordinator_evaluation_table_2 (key numeric, value numeric);
SELECT create_distributed_table('master_evaluation_table_2', 'key'); SELECT create_distributed_table('coordinator_evaluation_table_2', 'key');
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -529,13 +529,13 @@ BEGIN
RETURN trunc(random() * (end_int-start_int) + start_int); RETURN trunc(random() * (end_int-start_int) + start_int);
END; END;
$$ LANGUAGE 'plpgsql' STRICT; $$ LANGUAGE 'plpgsql' STRICT;
CREATE OR REPLACE PROCEDURE master_evaluation.test_procedure(int) CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int)
LANGUAGE plpgsql LANGUAGE plpgsql
AS $procedure$ AS $procedure$
DECLARE filterKey INTEGER; DECLARE filterKey INTEGER;
BEGIN BEGIN
filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1;
PERFORM DISTINCT value FROM master_evaluation_table_2 WHERE key = filterKey; PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey;
END; END;
$procedure$; $procedure$;
-- we couldn't find a meaningful query to write for this -- we couldn't find a meaningful query to write for this
@ -567,13 +567,13 @@ DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan DEBUG: Creating router plan
DEBUG: Plan is router executable DEBUG: Plan is router executable
CALL test_procedure(100); CALL test_procedure(100);
CREATE OR REPLACE PROCEDURE master_evaluation.test_procedure_2(int) CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure_2(int)
LANGUAGE plpgsql LANGUAGE plpgsql
AS $procedure$ AS $procedure$
DECLARE filterKey INTEGER; DECLARE filterKey INTEGER;
BEGIN BEGIN
filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1;
INSERT INTO master_evaluation_table_2 VALUES (filterKey, filterKey); INSERT INTO coordinator_evaluation_table_2 VALUES (filterKey, filterKey);
END; END;
$procedure$; $procedure$;
RESET citus.log_remote_commands ; RESET citus.log_remote_commands ;
@ -586,11 +586,11 @@ CALL test_procedure_2(100);
CALL test_procedure_2(100); CALL test_procedure_2(100);
CALL test_procedure_2(100); CALL test_procedure_2(100);
CALL test_procedure_2(100); CALL test_procedure_2(100);
SELECT count(*) FROM master_evaluation_table_2 WHERE key = 101; SELECT count(*) FROM coordinator_evaluation_table_2 WHERE key = 101;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
7 7
(1 row) (1 row)
SET client_min_messages TO ERROR; SET client_min_messages TO ERROR;
DROP SCHEMA master_evaluation CASCADE; DROP SCHEMA coordinator_evaluation CASCADE;

View File

@ -7,10 +7,10 @@
-- (b) Local Execution vs Remote Execution -- (b) Local Execution vs Remote Execution
-- (c) Parameters on distribution key vs Parameters on non-dist key -- (c) Parameters on distribution key vs Parameters on non-dist key
-- vs Non-parametrized queries -- vs Non-parametrized queries
-- (d) Master Function Evaluation Required vs -- (d) Coordinator Function Evaluation Required vs
-- Master Function Evaluation Not Required -- Coordinator Function Evaluation Not Required
CREATE SCHEMA master_evaluation_combinations; CREATE SCHEMA coordinator_evaluation_combinations;
SET search_path TO master_evaluation_combinations; SET search_path TO coordinator_evaluation_combinations;
SET citus.next_shard_id TO 1170000; SET citus.next_shard_id TO 1170000;
-- create a volatile function that returns the local node id -- create a volatile function that returns the local node id
CREATE OR REPLACE FUNCTION get_local_node_id_volatile() CREATE OR REPLACE FUNCTION get_local_node_id_volatile()
@ -824,10 +824,10 @@ EXECUTE router_with_only_function;
\c - - - :worker_2_port \c - - - :worker_2_port
SET citus.log_local_commands TO ON; SET citus.log_local_commands TO ON;
SET search_path TO master_evaluation_combinations; SET search_path TO coordinator_evaluation_combinations;
-- show that the data with user_id = 3 is local -- show that the data with user_id = 3 is local
SELECT count(*) FROM user_info_data WHERE user_id = 3; SELECT count(*) FROM user_info_data WHERE user_id = 3;
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -836,63 +836,63 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM master_eva
-- make sure that it is also true for fast-path router queries with paramaters -- make sure that it is also true for fast-path router queries with paramaters
PREPARE fast_path_router_with_param(int) AS SELECT count(*) FROM user_info_data WHERE user_id = $1; PREPARE fast_path_router_with_param(int) AS SELECT count(*) FROM user_info_data WHERE user_id = $1;
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute fast_path_router_with_param(3); execute fast_path_router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3; SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -901,49 +901,49 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
-- make sure that it is also true for fast-path router queries with paramaters -- make sure that it is also true for fast-path router queries with paramaters
PREPARE fast_path_router_with_param_and_func(int) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = $1; PREPARE fast_path_router_with_param_and_func(int) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = $1;
execute fast_path_router_with_param_and_func(3); execute fast_path_router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute fast_path_router_with_param_and_func(3); execute fast_path_router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute fast_path_router_with_param_and_func(3); execute fast_path_router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute fast_path_router_with_param_and_func(3); execute fast_path_router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute fast_path_router_with_param_and_func(3); execute fast_path_router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute fast_path_router_with_param_and_func(3); execute fast_path_router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute fast_path_router_with_param_and_func(3); execute fast_path_router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -958,56 +958,56 @@ execute fast_path_router_with_param_and_func(8);
PREPARE fast_path_router_with_param_and_func_on_non_dist_key(int) AS PREPARE fast_path_router_with_param_and_func_on_non_dist_key(int) AS
SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND user_index = $1; SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND user_index = $1;
EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND u_data = ('name3', 23)::user_data; SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND u_data = ('name3', 23)::user_data;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -1015,77 +1015,77 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
PREPARE fast_path_router_with_param_on_non_dist_key_and_func(user_data) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND u_data = $1; PREPARE fast_path_router_with_param_on_non_dist_key_and_func(user_data) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND u_data = $1;
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
SELECT count(*) FROM user_info_data WHERE user_id = 3 AND u_data = ('name3', 23)::user_data; SELECT count(*) FROM user_info_data WHERE user_id = 3 AND u_data = ('name3', 23)::user_data;
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -1093,63 +1093,63 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM master_eva
PREPARE fast_path_router_with_param_on_non_dist_key(user_data) AS SELECT count(*) FROM user_info_data WHERE user_id = 3 AND u_data = $1; PREPARE fast_path_router_with_param_on_non_dist_key(user_data) AS SELECT count(*) FROM user_info_data WHERE user_id = 3 AND u_data = $1;
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE fast_path_router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -1157,63 +1157,63 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM master_eva
PREPARE fast_path_router_with_only_function AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3; PREPARE fast_path_router_with_only_function AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3;
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE fast_path_router_with_only_function; EXECUTE fast_path_router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = 3; SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = 3;
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -1222,63 +1222,63 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_ev
-- make sure that it is also true for fast-path router queries with paramaters -- make sure that it is also true for fast-path router queries with paramaters
PREPARE router_with_param(int) AS SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = $1; PREPARE router_with_param(int) AS SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = $1;
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
execute router_with_param(3); execute router_with_param(3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) $1)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
SELECT get_local_node_id_volatile() > 0 FROM user_info_data m1 JOIN user_info_data m2 USING(user_id) WHERE m1.user_id = 3; SELECT get_local_node_id_volatile() > 0 FROM user_info_data m1 JOIN user_info_data m2 USING(user_id) WHERE m1.user_id = 3;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -1286,56 +1286,56 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
PREPARE router_with_param_and_func(int) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data m1 JOIN user_info_data m2 USING(user_id) WHERE m1.user_id = $1; PREPARE router_with_param_and_func(int) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data m1 JOIN user_info_data m2 USING(user_id) WHERE m1.user_id = $1;
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
execute router_with_param_and_func(3); execute router_with_param_and_func(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) $1)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -1344,49 +1344,49 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
PREPARE router_with_param_and_func_on_non_dist_key(int) AS PREPARE router_with_param_and_func_on_non_dist_key(int) AS
SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND user_id = 3 AND user_index = $1; SELECT get_local_node_id_volatile() > 0 FROM user_info_data WHERE user_id = 3 AND user_id = 3 AND user_index = $1;
EXECUTE router_with_param_and_func_on_non_dist_key(3); EXECUTE router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_and_func_on_non_dist_key(3); EXECUTE router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_and_func_on_non_dist_key(3); EXECUTE router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_and_func_on_non_dist_key(3); EXECUTE router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_and_func_on_non_dist_key(3); EXECUTE router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_and_func_on_non_dist_key(3); EXECUTE router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_and_func_on_non_dist_key(3); EXECUTE router_with_param_and_func_on_non_dist_key(3);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM master_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM coordinator_evaluation_combinations.user_info_data_1170001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) $1))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -1394,21 +1394,21 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
-- same query as router_with_param, but with consts -- same query as router_with_param, but with consts
SELECT get_local_node_id_volatile() > 0 FROM user_info_data m1 JOIN user_info_data m2 USING(user_id) WHERE m1.user_id = 3; SELECT get_local_node_id_volatile() > 0 FROM user_info_data m1 JOIN user_info_data m2 USING(user_id) WHERE m1.user_id = 3;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 m1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 m2(user_id, u_data, user_index) USING (user_id)) WHERE (m1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = ('name3', 23)::user_data; SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = ('name3', 23)::user_data;
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = ('name3', 23)::user_data; SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = ('name3', 23)::user_data;
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -1416,70 +1416,70 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_ev
PREPARE router_with_param_on_non_dist_key(user_data) AS SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = $1; PREPARE router_with_param_on_non_dist_key(user_data) AS SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = $1;
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = ('name3', 23)::user_data; SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = ('name3', 23)::user_data;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -1487,70 +1487,70 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
PREPARE router_with_param_on_non_dist_key_and_func(user_data) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = $1; PREPARE router_with_param_on_non_dist_key_and_func(user_data) AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE u1.user_id = 3 AND u1.u_data = $1;
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data); EXECUTE router_with_param_on_non_dist_key_and_func(('name3', 23)::user_data);
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = 3 AND u1.u_data = ('name3', 23)::user_data; SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = 3 AND u1.u_data = ('name3', 23)::user_data;
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) 3) AND (u1.u_data OPERATOR(pg_catalog.=) ROW('name3'::text, 23)::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -1558,70 +1558,70 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_ev
PREPARE router_with_two_params(user_data, int) AS SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = $2 AND u1.u_data = $1; PREPARE router_with_two_params(user_data, int) AS SELECT count(*) FROM user_info_data u1 JOIN user_info_data u2 USING (user_id) WHERE user_id = $2 AND u1.u_data = $1;
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
EXECUTE router_with_two_params(('name3', 23)::user_data, 3); EXECUTE router_with_two_params(('name3', 23)::user_data, 3);
NOTICE: executing the command locally: SELECT count(*) AS count FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations.user_data)) NOTICE: executing the command locally: SELECT count(*) AS count FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE ((u1.user_id OPERATOR(pg_catalog.=) $2) AND (u1.u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations.user_data))
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING(user_id) WHERE user_id = 3; SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING(user_id) WHERE user_id = 3;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -1629,56 +1629,56 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
PREPARE router_with_only_function AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING(user_id) WHERE user_id = 3; PREPARE router_with_only_function AS SELECT get_local_node_id_volatile() > 0 FROM user_info_data u1 JOIN user_info_data u2 USING(user_id) WHERE user_id = 3;
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
(1 row) (1 row)
EXECUTE router_with_only_function; EXECUTE router_with_only_function;
NOTICE: executing the command locally: SELECT (master_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (master_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN master_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3) NOTICE: executing the command locally: SELECT (coordinator_evaluation_combinations.get_local_node_id_volatile() OPERATOR(pg_catalog.>) 0) FROM (coordinator_evaluation_combinations.user_info_data_1170001 u1(user_id, u_data, user_index) JOIN coordinator_evaluation_combinations.user_info_data_1170001 u2(user_id, u_data, user_index) USING (user_id)) WHERE (u1.user_id OPERATOR(pg_catalog.=) 3)
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
t t
@ -1687,4 +1687,4 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g
-- suppress notices -- suppress notices
\c - - - :master_port \c - - - :master_port
SET client_min_messages TO ERROR; SET client_min_messages TO ERROR;
DROP SCHEMA master_evaluation_combinations CASCADE; DROP SCHEMA coordinator_evaluation_combinations CASCADE;

View File

@ -36,7 +36,7 @@ test: recursive_dml_queries_mx multi_mx_truncate_from_worker
test: multi_mx_repartition_udt_prepare mx_foreign_key_to_reference_table test: multi_mx_repartition_udt_prepare mx_foreign_key_to_reference_table
test: multi_mx_repartition_join_w1 multi_mx_repartition_join_w2 multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2 test: multi_mx_repartition_join_w1 multi_mx_repartition_join_w2 multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2
test: multi_mx_metadata test: multi_mx_metadata
test: master_evaluation master_evaluation_modify master_evaluation_select test: coordinator_evaluation coordinator_evaluation_modify coordinator_evaluation_select
test: multi_mx_call test: multi_mx_call
test: multi_mx_function_call_delegation test: multi_mx_function_call_delegation
test: multi_mx_modifications local_shard_execution test: multi_mx_modifications local_shard_execution

View File

@ -1,7 +1,7 @@
-- This test relies on metadata being synced -- This test relies on metadata being synced
-- that's why is should be executed on MX schedule -- that's why is should be executed on MX schedule
CREATE SCHEMA master_evaluation; CREATE SCHEMA coordinator_evaluation;
SET search_path TO master_evaluation; SET search_path TO coordinator_evaluation;
-- create a volatile function that returns the local node id -- create a volatile function that returns the local node id
CREATE OR REPLACE FUNCTION get_local_node_id_volatile() CREATE OR REPLACE FUNCTION get_local_node_id_volatile()
@ -23,20 +23,20 @@ END; $$ language plpgsql VOLATILE;
SELECT create_distributed_function('get_local_node_id_volatile_sum_with_param(int)'); SELECT create_distributed_function('get_local_node_id_volatile_sum_with_param(int)');
CREATE TABLE master_evaluation_table (key int, value int); CREATE TABLE coordinator_evaluation_table (key int, value int);
SELECT create_distributed_table('master_evaluation_table', 'key'); SELECT create_distributed_table('coordinator_evaluation_table', 'key');
-- show that local id is 0, we'll use this information -- show that local id is 0, we'll use this information
SELECT get_local_node_id_volatile(); SELECT get_local_node_id_volatile();
-- load data -- load data
INSERT INTO master_evaluation_table SELECT i, i FROM generate_series(0,100)i; INSERT INTO coordinator_evaluation_table SELECT i, i FROM generate_series(0,100)i;
-- we expect that the function is evaluated on the worker node, so we should get a row -- we expect that the function is evaluated on the worker node, so we should get a row
SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table WHERE key = 1; SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table WHERE key = 1;
-- make sure that it is also true for fast-path router queries with paramaters -- make sure that it is also true for fast-path router queries with paramaters
PREPARE fast_path_router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table WHERE key = $1; PREPARE fast_path_router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table WHERE key = $1;
execute fast_path_router_with_param(1); execute fast_path_router_with_param(1);
execute fast_path_router_with_param(2); execute fast_path_router_with_param(2);
@ -48,9 +48,9 @@ execute fast_path_router_with_param(7);
execute fast_path_router_with_param(8); execute fast_path_router_with_param(8);
-- same query as fast_path_router_with_param, but with consts -- same query as fast_path_router_with_param, but with consts
SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table WHERE key = 1; SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table WHERE key = 1;
PREPARE router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table m1 JOIN master_evaluation_table m2 USING(key) WHERE key = $1; PREPARE router_with_param(int) AS SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table m1 JOIN coordinator_evaluation_table m2 USING(key) WHERE key = $1;
execute router_with_param(1); execute router_with_param(1);
execute router_with_param(2); execute router_with_param(2);
@ -62,29 +62,29 @@ execute router_with_param(7);
execute router_with_param(8); execute router_with_param(8);
-- same query as router_with_param, but with consts -- same query as router_with_param, but with consts
SELECT get_local_node_id_volatile() > 0 FROM master_evaluation_table m1 JOIN master_evaluation_table m2 USING(key) WHERE key = 1; SELECT get_local_node_id_volatile() > 0 FROM coordinator_evaluation_table m1 JOIN coordinator_evaluation_table m2 USING(key) WHERE key = 1;
-- for multi-shard queries, we still expect the evaluation to happen on the workers -- for multi-shard queries, we still expect the evaluation to happen on the workers
SELECT count(*), max(get_local_node_id_volatile()) != 0, min(get_local_node_id_volatile()) != 0 FROM master_evaluation_table; SELECT count(*), max(get_local_node_id_volatile()) != 0, min(get_local_node_id_volatile()) != 0 FROM coordinator_evaluation_table;
-- when executed locally, we expect to get the result from the coordinator -- when executed locally, we expect to get the result from the coordinator
SELECT (SELECT count(*) FROM master_evaluation_table), get_local_node_id_volatile() = 0; SELECT (SELECT count(*) FROM coordinator_evaluation_table), get_local_node_id_volatile() = 0;
-- make sure that we get the results from the workers when the query is sent to workers -- make sure that we get the results from the workers when the query is sent to workers
SET citus.task_assignment_policy TO "round-robin"; SET citus.task_assignment_policy TO "round-robin";
SELECT (SELECT count(*) FROM master_evaluation_table), get_local_node_id_volatile() = 0; SELECT (SELECT count(*) FROM coordinator_evaluation_table), get_local_node_id_volatile() = 0;
RESET citus.task_assignment_policy; RESET citus.task_assignment_policy;
-- for multi-shard SELECTs, we don't try to evaluate on the coordinator -- for multi-shard SELECTs, we don't try to evaluate on the coordinator
SELECT min(get_local_node_id_volatile()) > 0 FROM master_evaluation_table; SELECT min(get_local_node_id_volatile()) > 0 FROM coordinator_evaluation_table;
SELECT count(*) > 0 FROM master_evaluation_table WHERE value >= get_local_node_id_volatile(); SELECT count(*) > 0 FROM coordinator_evaluation_table WHERE value >= get_local_node_id_volatile();
-- let's have some tests around expressions -- let's have some tests around expressions
-- for modifications, we expect the evaluation to happen on the coordinator -- for modifications, we expect the evaluation to happen on the coordinator
-- thus the results should be 0 -- thus the results should be 0
PREPARE insert_with_param_expression(int) AS INSERT INTO master_evaluation_table (key, value) VALUES ($1 + get_local_node_id_volatile(), $1 + get_local_node_id_volatile()) RETURNING key, value; PREPARE insert_with_param_expression(int) AS INSERT INTO coordinator_evaluation_table (key, value) VALUES ($1 + get_local_node_id_volatile(), $1 + get_local_node_id_volatile()) RETURNING key, value;
EXECUTE insert_with_param_expression(0); EXECUTE insert_with_param_expression(0);
EXECUTE insert_with_param_expression(0); EXECUTE insert_with_param_expression(0);
EXECUTE insert_with_param_expression(0); EXECUTE insert_with_param_expression(0);
@ -95,7 +95,7 @@ EXECUTE insert_with_param_expression(0);
-- for modifications, we expect the evaluation to happen on the coordinator -- for modifications, we expect the evaluation to happen on the coordinator
-- thus the results should be 0 -- thus the results should be 0
PREPARE insert_with_param(int) AS INSERT INTO master_evaluation_table (key, value) VALUES ($1, $1) RETURNING key, value; PREPARE insert_with_param(int) AS INSERT INTO coordinator_evaluation_table (key, value) VALUES ($1, $1) RETURNING key, value;
EXECUTE insert_with_param(0 + get_local_node_id_volatile()); EXECUTE insert_with_param(0 + get_local_node_id_volatile());
EXECUTE insert_with_param(0 + get_local_node_id_volatile()); EXECUTE insert_with_param(0 + get_local_node_id_volatile());
EXECUTE insert_with_param(0 + get_local_node_id_volatile()); EXECUTE insert_with_param(0 + get_local_node_id_volatile());
@ -104,7 +104,7 @@ EXECUTE insert_with_param(0 + get_local_node_id_volatile());
EXECUTE insert_with_param(0 + get_local_node_id_volatile()); EXECUTE insert_with_param(0 + get_local_node_id_volatile());
EXECUTE insert_with_param(0 + get_local_node_id_volatile()); EXECUTE insert_with_param(0 + get_local_node_id_volatile());
PREPARE router_select_with_param_expression(int) AS SELECT value > 0 FROM master_evaluation_table WHERE key = $1 + get_local_node_id_volatile(); PREPARE router_select_with_param_expression(int) AS SELECT value > 0 FROM coordinator_evaluation_table WHERE key = $1 + get_local_node_id_volatile();
-- for selects, we expect the evaluation to happen on the workers -- for selects, we expect the evaluation to happen on the workers
-- this means that the query should be hitting multiple workers -- this means that the query should be hitting multiple workers
@ -118,7 +118,7 @@ EXECUTE router_select_with_param_expression(0);
EXECUTE router_select_with_param_expression(0); EXECUTE router_select_with_param_expression(0);
EXECUTE router_select_with_param_expression(0); EXECUTE router_select_with_param_expression(0);
PREPARE router_select_with_param(int) AS SELECT DISTINCT value FROM master_evaluation_table WHERE key = $1; PREPARE router_select_with_param(int) AS SELECT DISTINCT value FROM coordinator_evaluation_table WHERE key = $1;
-- this time the parameter itself is a function, so should be evaluated -- this time the parameter itself is a function, so should be evaluated
-- on the coordinator -- on the coordinator
@ -140,7 +140,7 @@ EXECUTE router_select_with_param(get_local_node_id_volatile());
EXECUTE router_select_with_param(get_local_node_id_volatile()); EXECUTE router_select_with_param(get_local_node_id_volatile());
-- this time use the parameter inside the function -- this time use the parameter inside the function
PREPARE router_select_with_parameter_in_function(int) AS SELECT bool_and(get_local_node_id_volatile_sum_with_param($1) > 1) FROM master_evaluation_table WHERE key = get_local_node_id_volatile_sum_with_param($1); PREPARE router_select_with_parameter_in_function(int) AS SELECT bool_and(get_local_node_id_volatile_sum_with_param($1) > 1) FROM coordinator_evaluation_table WHERE key = get_local_node_id_volatile_sum_with_param($1);
EXECUTE router_select_with_parameter_in_function(0); EXECUTE router_select_with_parameter_in_function(0);
EXECUTE router_select_with_parameter_in_function(0); EXECUTE router_select_with_parameter_in_function(0);
EXECUTE router_select_with_parameter_in_function(0); EXECUTE router_select_with_parameter_in_function(0);
@ -153,8 +153,8 @@ RESET client_min_messages;
RESET citus.log_remote_commands; RESET citus.log_remote_commands;
-- numeric has different casting affects, so some tests on that -- numeric has different casting affects, so some tests on that
CREATE TABLE master_evaluation_table_2 (key numeric, value numeric); CREATE TABLE coordinator_evaluation_table_2 (key numeric, value numeric);
SELECT create_distributed_table('master_evaluation_table_2', 'key'); SELECT create_distributed_table('coordinator_evaluation_table_2', 'key');
CREATE OR REPLACE FUNCTION TEST_RANDOM (INTEGER, INTEGER) RETURNS INTEGER AS $$ CREATE OR REPLACE FUNCTION TEST_RANDOM (INTEGER, INTEGER) RETURNS INTEGER AS $$
DECLARE DECLARE
@ -165,13 +165,13 @@ BEGIN
END; END;
$$ LANGUAGE 'plpgsql' STRICT; $$ LANGUAGE 'plpgsql' STRICT;
CREATE OR REPLACE PROCEDURE master_evaluation.test_procedure(int) CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int)
LANGUAGE plpgsql LANGUAGE plpgsql
AS $procedure$ AS $procedure$
DECLARE filterKey INTEGER; DECLARE filterKey INTEGER;
BEGIN BEGIN
filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1;
PERFORM DISTINCT value FROM master_evaluation_table_2 WHERE key = filterKey; PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey;
END; END;
$procedure$; $procedure$;
@ -187,13 +187,13 @@ CALL test_procedure(100);
CALL test_procedure(100); CALL test_procedure(100);
CALL test_procedure(100); CALL test_procedure(100);
CREATE OR REPLACE PROCEDURE master_evaluation.test_procedure_2(int) CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure_2(int)
LANGUAGE plpgsql LANGUAGE plpgsql
AS $procedure$ AS $procedure$
DECLARE filterKey INTEGER; DECLARE filterKey INTEGER;
BEGIN BEGIN
filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1;
INSERT INTO master_evaluation_table_2 VALUES (filterKey, filterKey); INSERT INTO coordinator_evaluation_table_2 VALUES (filterKey, filterKey);
END; END;
$procedure$; $procedure$;
@ -208,7 +208,7 @@ CALL test_procedure_2(100);
CALL test_procedure_2(100); CALL test_procedure_2(100);
CALL test_procedure_2(100); CALL test_procedure_2(100);
CALL test_procedure_2(100); CALL test_procedure_2(100);
SELECT count(*) FROM master_evaluation_table_2 WHERE key = 101; SELECT count(*) FROM coordinator_evaluation_table_2 WHERE key = 101;
SET client_min_messages TO ERROR; SET client_min_messages TO ERROR;
DROP SCHEMA master_evaluation CASCADE; DROP SCHEMA coordinator_evaluation CASCADE;

View File

@ -1,8 +1,8 @@
-- This test relies on metadata being synced -- This test relies on metadata being synced
-- that's why is should be executed on MX schedule -- that's why is should be executed on MX schedule
CREATE SCHEMA master_evaluation_combinations_modify; CREATE SCHEMA coordinator_evaluation_combinations_modify;
SET search_path TO master_evaluation_combinations_modify; SET search_path TO coordinator_evaluation_combinations_modify;
-- in this test, we are considering combinations of -- in this test, we are considering combinations of
-- several Citus features, and there is one prepared -- several Citus features, and there is one prepared
@ -11,8 +11,8 @@ SET search_path TO master_evaluation_combinations_modify;
-- (b) Local Execution vs Remote Execution -- (b) Local Execution vs Remote Execution
-- (c) Parameters on distribution key vs Parameters on non-dist key -- (c) Parameters on distribution key vs Parameters on non-dist key
-- vs Non-parametrized queries -- vs Non-parametrized queries
-- (d) Master Function Evaluation Required vs -- (d) Coordinator Function Evaluation Required vs
-- Master Function Evaluation Not Required -- Coordinator Function Evaluation Not Required
SET citus.next_shard_id TO 1180000; SET citus.next_shard_id TO 1180000;
@ -270,7 +270,7 @@ EXECUTE router_with_only_function;
\c - - - :worker_2_port \c - - - :worker_2_port
SET citus.log_local_commands TO ON; SET citus.log_local_commands TO ON;
SET search_path TO master_evaluation_combinations_modify; SET search_path TO coordinator_evaluation_combinations_modify;
-- returns 2 on the worker -- returns 2 on the worker
CREATE OR REPLACE FUNCTION get_constant_stable() CREATE OR REPLACE FUNCTION get_constant_stable()
@ -538,4 +538,4 @@ EXECUTE router_with_only_function;
-- suppress notices -- suppress notices
\c - - - :master_port \c - - - :master_port
SET client_min_messages TO ERROR; SET client_min_messages TO ERROR;
DROP SCHEMA master_evaluation_combinations_modify CASCADE; DROP SCHEMA coordinator_evaluation_combinations_modify CASCADE;

View File

@ -9,11 +9,11 @@
-- (b) Local Execution vs Remote Execution -- (b) Local Execution vs Remote Execution
-- (c) Parameters on distribution key vs Parameters on non-dist key -- (c) Parameters on distribution key vs Parameters on non-dist key
-- vs Non-parametrized queries -- vs Non-parametrized queries
-- (d) Master Function Evaluation Required vs -- (d) Coordinator Function Evaluation Required vs
-- Master Function Evaluation Not Required -- Coordinator Function Evaluation Not Required
CREATE SCHEMA master_evaluation_combinations; CREATE SCHEMA coordinator_evaluation_combinations;
SET search_path TO master_evaluation_combinations; SET search_path TO coordinator_evaluation_combinations;
SET citus.next_shard_id TO 1170000; SET citus.next_shard_id TO 1170000;
@ -232,7 +232,7 @@ EXECUTE router_with_only_function;
\c - - - :worker_2_port \c - - - :worker_2_port
SET citus.log_local_commands TO ON; SET citus.log_local_commands TO ON;
SET search_path TO master_evaluation_combinations; SET search_path TO coordinator_evaluation_combinations;
-- show that the data with user_id = 3 is local -- show that the data with user_id = 3 is local
SELECT count(*) FROM user_info_data WHERE user_id = 3; SELECT count(*) FROM user_info_data WHERE user_id = 3;
@ -416,4 +416,4 @@ EXECUTE router_with_only_function;
-- suppress notices -- suppress notices
\c - - - :master_port \c - - - :master_port
SET client_min_messages TO ERROR; SET client_min_messages TO ERROR;
DROP SCHEMA master_evaluation_combinations CASCADE; DROP SCHEMA coordinator_evaluation_combinations CASCADE;