Rename master evaluation to coordinator evaluation

pull/3922/head
Marco Slot 2020-06-17 16:20:16 +02:00 committed by citus bot
parent 23ffaabe52
commit b4fec63bc0
20 changed files with 435 additions and 431 deletions

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

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

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

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

@ -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"
@ -29,10 +29,10 @@ 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

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

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