diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index 81d6cef87..e3d2e5636 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -297,7 +297,7 @@ CitusBeginReadOnlyScan(CustomScanState *node, EState *estate, int eflags) * * TODO: evaluate stable functions */ - ExecuteMasterEvaluableExpressions(jobQuery, planState); + ExecuteCoordinatorEvaluableExpressions(jobQuery, planState); /* job query no longer has parameters, so we should not send any */ workerJob->parametersInJobQueryResolved = true; @@ -347,7 +347,7 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags) if (ModifyJobNeedsEvaluation(workerJob)) { - ExecuteMasterEvaluableExpressions(jobQuery, planState); + ExecuteCoordinatorEvaluableExpressions(jobQuery, planState); /* job query no longer has parameters, so we should not send any */ workerJob->parametersInJobQueryResolved = true; @@ -375,7 +375,7 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags) RegenerateTaskForFasthPathQuery(workerJob); } } - else if (workerJob->requiresMasterEvaluation) + else if (workerJob->requiresCoordinatorEvaluation) { /* * 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 ModifyJobNeedsEvaluation(Job *workerJob) { - if (workerJob->requiresMasterEvaluation) + if (workerJob->requiresCoordinatorEvaluation) { /* query contains functions that need to be evaluated on the coordinator */ return true; diff --git a/src/backend/distributed/planner/combine_query_planner.c b/src/backend/distributed/planner/combine_query_planner.c index 9fa651960..9202a5d93 100644 --- a/src/backend/distributed/planner/combine_query_planner.c +++ b/src/backend/distributed/planner/combine_query_planner.c @@ -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 * 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 @@ -58,7 +58,7 @@ static CustomPathMethods CitusCustomScanPathMethods = { * filled into the tuple store inside provided custom scan. */ PlannedStmt * -MasterNodeSelectPlan(DistributedPlan *distributedPlan, CustomScan *remoteScan) +PlanCombineQuery(DistributedPlan *distributedPlan, CustomScan *remoteScan) { Query *combineQuery = distributedPlan->combineQuery; diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 25a4b95f4..21daf030c 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -1368,7 +1368,7 @@ static PlannedStmt * FinalizeNonRouterPlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan, CustomScan *customScan) { - PlannedStmt *finalPlan = MasterNodeSelectPlan(distributedPlan, customScan); + PlannedStmt *finalPlan = PlanCombineQuery(distributedPlan, customScan); finalPlan->queryId = localPlan->queryId; finalPlan->utilityStmt = localPlan->utilityStmt; diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 22f882aeb..8256d8eb2 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -348,7 +348,8 @@ CreateDistributedInsertSelectPlan(Query *originalQuery, workerJob->dependentJobList = NIL; workerJob->jobId = jobId; workerJob->jobQuery = originalQuery; - workerJob->requiresMasterEvaluation = RequiresMasterEvaluation(originalQuery); + workerJob->requiresCoordinatorEvaluation = + RequiresCoordinatorEvaluation(originalQuery); /* and finally the multi plan */ distributedPlan->workerJob = workerJob; diff --git a/src/backend/distributed/planner/local_plan_cache.c b/src/backend/distributed/planner/local_plan_cache.c index d9299f500..881693e64 100644 --- a/src/backend/distributed/planner/local_plan_cache.c +++ b/src/backend/distributed/planner/local_plan_cache.c @@ -190,7 +190,7 @@ IsLocalPlanCachingSupported(Job *currentJob, DistributedPlan *originalDistribute * We do not cache plans with volatile functions in the query. * * 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 * execute them again if we execute the plan. */ diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 5bfaece89..955fca0b0 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -165,7 +165,7 @@ static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, RelationRestrictionContext *restrictionContext, uint32 taskId, TaskType taskType, - bool modifyRequiresMasterEvaluation); + bool modifyRequiresCoordinatorEvaluation); static bool ShardIntervalsEqual(FmgrInfo *comparisonFunction, Oid collation, ShardInterval *firstInterval, @@ -2015,7 +2015,7 @@ BuildJob(Query *jobQuery, List *dependentJobList) job->jobId = UniqueJobId(); job->jobQuery = jobQuery; job->dependentJobList = dependentJobList; - job->requiresMasterEvaluation = false; + job->requiresCoordinatorEvaluation = false; return job; } @@ -2289,7 +2289,7 @@ List * QueryPushdownSqlTaskList(Query *query, uint64 jobId, RelationRestrictionContext *relationRestrictionContext, List *prunedRelationShardList, TaskType taskType, bool - modifyRequiresMasterEvaluation) + modifyRequiresCoordinatorEvaluation) { List *sqlTaskList = NIL; ListCell *restrictionCell = NULL; @@ -2393,7 +2393,7 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId, relationRestrictionContext, taskIdIndex, taskType, - modifyRequiresMasterEvaluation); + modifyRequiresCoordinatorEvaluation); subqueryTask->jobId = jobId; sqlTaskList = lappend(sqlTaskList, subqueryTask); @@ -2570,7 +2570,7 @@ ErrorIfUnsupportedShardDistribution(Query *query) static Task * QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, RelationRestrictionContext *restrictionContext, uint32 taskId, - TaskType taskType, bool modifyRequiresMasterEvaluation) + TaskType taskType, bool modifyRequiresCoordinatorEvaluation) { Query *taskQuery = copyObject(originalQuery); @@ -2672,7 +2672,7 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, Task *subqueryTask = CreateBasicTask(jobId, taskId, taskType, NULL); - if ((taskType == MODIFY_TASK && !modifyRequiresMasterEvaluation) || + if ((taskType == MODIFY_TASK && !modifyRequiresCoordinatorEvaluation) || taskType == READ_TASK) { pg_get_query_def(taskQuery, queryString); diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 0eaa00a6d..bdf634d0e 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -1302,9 +1302,9 @@ MasterIrreducibleExpressionWalker(Node *expression, WalkerState *state) /* * 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 - * category above IMMUTABLE. Newer versions of postgres might add node types which - * should be checked in this function. + * that we either disallow or evaluate on the coordinator anything which has a + * volatility category above IMMUTABLE. Newer versions of postgres might add node + * types which should be checked in this function. * * 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 @@ -1451,7 +1451,7 @@ RouterInsertJob(Query *originalQuery) } Job *job = CreateJob(originalQuery); - job->requiresMasterEvaluation = RequiresMasterEvaluation(originalQuery); + job->requiresCoordinatorEvaluation = RequiresCoordinatorEvaluation(originalQuery); job->deferredPruning = true; job->partitionKeyValue = ExtractInsertPartitionKeyValue(originalQuery); @@ -1471,7 +1471,7 @@ CreateJob(Query *query) job->taskList = NIL; job->dependentJobList = NIL; job->subqueryPushdown = false; - job->requiresMasterEvaluation = false; + job->requiresCoordinatorEvaluation = false; job->deferredPruning = false; 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 */ bool replacePrunedQueryWithDummy = true; - /* check if this query requires master evaluation */ - bool requiresMasterEvaluation = RequiresMasterEvaluation(originalQuery); + /* check if this query requires coordinator evaluation */ + bool requiresCoordinatorEvaluation = RequiresCoordinatorEvaluation(originalQuery); FastPathRestrictionContext *fastPathRestrictionContext = plannerRestrictionContext->fastPathRestrictionContext; @@ -1688,7 +1688,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon relationRestrictionContext, prunedShardIntervalListList, MODIFY_TASK, - requiresMasterEvaluation); + requiresCoordinatorEvaluation); } else { @@ -1696,7 +1696,7 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon placementList, shardId); } - job->requiresMasterEvaluation = requiresMasterEvaluation; + job->requiresCoordinatorEvaluation = requiresCoordinatorEvaluation; 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. */ - if (!(UpdateOrDeleteQuery(originalQuery) && RequiresMasterEvaluation(originalQuery))) + bool isUpdateOrDelete = UpdateOrDeleteQuery(originalQuery); + if (!(isUpdateOrDelete && RequiresCoordinatorEvaluation(originalQuery))) { UpdateRelationToShardNames((Node *) originalQuery, *relationShardList); } diff --git a/src/backend/distributed/utils/citus_clauses.c b/src/backend/distributed/utils/citus_clauses.c index 9e7e7c517..442412651 100644 --- a/src/backend/distributed/utils/citus_clauses.c +++ b/src/backend/distributed/utils/citus_clauses.c @@ -34,22 +34,23 @@ static bool IsVariableExpression(Node *node); static Expr * citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod, Oid result_collation, - MasterEvaluationContext *masterEvaluationContext); + CoordinatorEvaluationContext * + coordinatorEvaluationContext); static bool CitusIsVolatileFunctionIdChecker(Oid func_id, void *context); static bool CitusIsMutableFunctionIdChecker(Oid func_id, void *context); static bool ShouldEvaluateExpression(Expr *expression); -static bool ShouldEvaluateFunctionWithMasterContext(MasterEvaluationContext * - evaluationContext); +static bool ShouldEvaluateFunctions(CoordinatorEvaluationContext *evaluationContext); static void FixFunctionArguments(Node *expr); 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 * any stable or volatile function. */ bool -RequiresMasterEvaluation(Query *query) +RequiresCoordinatorEvaluation(Query *query) { 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. */ 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) { - masterEvaluationContext.evaluationMode = EVALUATE_PARAMS; + coordinatorEvaluationContext.evaluationMode = EVALUATE_PARAMS; } 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 * PartiallyEvaluateExpression(Node *expression, - MasterEvaluationContext *masterEvaluationContext) + CoordinatorEvaluationContext *coordinatorEvaluationContext) { if (expression == NULL || IsA(expression, Const)) { @@ -112,10 +113,10 @@ PartiallyEvaluateExpression(Node *expression, exprType(expression), exprTypmod(expression), exprCollation(expression), - masterEvaluationContext); + coordinatorEvaluationContext); } else if (ShouldEvaluateExpression((Expr *) expression) && - ShouldEvaluateFunctionWithMasterContext(masterEvaluationContext)) + ShouldEvaluateFunctions(coordinatorEvaluationContext)) { if (FindNodeCheck(expression, IsVariableExpression)) { @@ -132,19 +133,19 @@ PartiallyEvaluateExpression(Node *expression, */ return (Node *) expression_tree_mutator(expression, PartiallyEvaluateExpression, - masterEvaluationContext); + coordinatorEvaluationContext); } return (Node *) citus_evaluate_expr((Expr *) expression, exprType(expression), exprTypmod(expression), exprCollation(expression), - masterEvaluationContext); + coordinatorEvaluationContext); } else if (nodeTag == T_Query) { Query *query = (Query *) expression; - MasterEvaluationContext subContext = *masterEvaluationContext; + CoordinatorEvaluationContext subContext = *coordinatorEvaluationContext; if (query->commandType != CMD_SELECT) { /* @@ -165,7 +166,7 @@ PartiallyEvaluateExpression(Node *expression, { return (Node *) expression_tree_mutator(expression, PartiallyEvaluateExpression, - masterEvaluationContext); + coordinatorEvaluationContext); } 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 - * masterEvaluationContext. + * coordinatorEvaluationContext. */ static bool -ShouldEvaluateFunctionWithMasterContext(MasterEvaluationContext *evaluationContext) +ShouldEvaluateFunctions(CoordinatorEvaluationContext *evaluationContext) { if (evaluationContext == NULL) { @@ -269,7 +270,7 @@ IsVariableExpression(Node *node) static Expr * citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod, Oid result_collation, - MasterEvaluationContext *masterEvaluationContext) + CoordinatorEvaluationContext *coordinatorEvaluationContext) { PlanState *planState = NULL; EState *estate; @@ -280,19 +281,19 @@ citus_evaluate_expr(Expr *expr, Oid result_type, int32 result_typmod, int16 resultTypLen; bool resultTypByVal; - if (masterEvaluationContext) + if (coordinatorEvaluationContext) { - planState = masterEvaluationContext->planState; + planState = coordinatorEvaluationContext->planState; 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 */ 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 */ Assert(ShouldEvaluateExpression(expr)); diff --git a/src/backend/distributed/utils/citus_copyfuncs.c b/src/backend/distributed/utils/citus_copyfuncs.c index d3f64e792..3314fd8a0 100644 --- a/src/backend/distributed/utils/citus_copyfuncs.c +++ b/src/backend/distributed/utils/citus_copyfuncs.c @@ -96,7 +96,7 @@ copyJobInfo(Job *newnode, Job *from) COPY_NODE_FIELD(taskList); COPY_NODE_FIELD(dependentJobList); COPY_SCALAR_FIELD(subqueryPushdown); - COPY_SCALAR_FIELD(requiresMasterEvaluation); + COPY_SCALAR_FIELD(requiresCoordinatorEvaluation); COPY_SCALAR_FIELD(deferredPruning); COPY_NODE_FIELD(partitionKeyValue); COPY_NODE_FIELD(localPlannedStatements); diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index c5586b84b..1dd20a825 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -340,7 +340,7 @@ OutJobFields(StringInfo str, const Job *node) WRITE_NODE_FIELD(taskList); WRITE_NODE_FIELD(dependentJobList); WRITE_BOOL_FIELD(subqueryPushdown); - WRITE_BOOL_FIELD(requiresMasterEvaluation); + WRITE_BOOL_FIELD(requiresCoordinatorEvaluation); WRITE_BOOL_FIELD(deferredPruning); WRITE_NODE_FIELD(partitionKeyValue); WRITE_NODE_FIELD(localPlannedStatements); diff --git a/src/include/distributed/citus_clauses.h b/src/include/distributed/citus_clauses.h index 7ebdfca31..8cfbe18ee 100644 --- a/src/include/distributed/citus_clauses.h +++ b/src/include/distributed/citus_clauses.h @@ -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. */ -typedef enum MasterEvaluationMode +typedef enum CoordinatorEvaluationMode { /* evaluate nothing */ EVALUATE_NONE = 0, @@ -30,23 +30,24 @@ typedef enum MasterEvaluationMode /* evaluate both the functions/expressions and the external paramaters */ EVALUATE_FUNCTIONS_PARAMS -} MasterEvaluationMode; +} CoordinatorEvaluationMode; /* * This struct is used to pass information to master * evaluation logic. */ -typedef struct MasterEvaluationContext +typedef struct CoordinatorEvaluationContext { PlanState *planState; - MasterEvaluationMode evaluationMode; -} MasterEvaluationContext; + CoordinatorEvaluationMode evaluationMode; +} CoordinatorEvaluationContext; -extern bool RequiresMasterEvaluation(Query *query); -extern void ExecuteMasterEvaluableExpressions(Query *query, PlanState *planState); +extern bool RequiresCoordinatorEvaluation(Query *query); +extern void ExecuteCoordinatorEvaluableExpressions(Query *query, PlanState *planState); extern Node * PartiallyEvaluateExpression(Node *expression, - MasterEvaluationContext *masterEvaluationContext); + CoordinatorEvaluationContext * + coordinatorEvaluationContext); extern bool CitusIsVolatileFunction(Node *node); extern bool CitusIsMutableFunction(Node *node); diff --git a/src/include/distributed/combine_query_planner.h b/src/include/distributed/combine_query_planner.h index 926d3200b..ca17c3560 100644 --- a/src/include/distributed/combine_query_planner.h +++ b/src/include/distributed/combine_query_planner.h @@ -1,6 +1,6 @@ /*------------------------------------------------------------------------- * - * merge_planner.h + * combine_query_planner.h * Function declarations for building planned statements; these statements * are then executed on the coordinator node. * @@ -9,8 +9,8 @@ *------------------------------------------------------------------------- */ -#ifndef MERGE_PLANNER_H -#define MERGE_PLANNER_H +#ifndef COMBINE_QUERY_PLANNER_H +#define COMBINE_QUERY_PLANNER_H #include "lib/stringinfo.h" #include "nodes/parsenodes.h" @@ -29,10 +29,10 @@ struct CustomScan; extern Path * CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInfo, Index restrictionIndex, RangeTblEntry *rte, CustomScan *remoteScan); -extern PlannedStmt * MasterNodeSelectPlan(struct DistributedPlan *distributedPlan, - struct CustomScan *dataScan); +extern PlannedStmt * PlanCombineQuery(struct DistributedPlan *distributedPlan, + struct CustomScan *dataScan); extern Unique * make_unique_from_sortclauses(Plan *lefttree, List *distinctList); extern bool ReplaceCitusExtraDataContainer; extern CustomScan *ReplaceCitusExtraDataContainerWithCustomScan; -#endif /* MERGE_PLANNER_H */ +#endif /* COMBINE_QUERY_PLANNER_H */ diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 2d6d9ef8f..c21659088 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -151,7 +151,7 @@ typedef struct Job List *taskList; List *dependentJobList; bool subqueryPushdown; - bool requiresMasterEvaluation; /* only applies to modify jobs */ + bool requiresCoordinatorEvaluation; /* only applies to modify jobs */ bool deferredPruning; Const *partitionKeyValue; @@ -599,7 +599,7 @@ extern List * QueryPushdownSqlTaskList(Query *query, uint64 jobId, RelationRestrictionContext * relationRestrictionContext, List *prunedRelationShardList, TaskType taskType, - bool modifyRequiresMasterEvaluation); + bool modifyRequiresCoordinatorEvaluation); /* function declarations for managing jobs */ extern uint64 UniqueJobId(void); diff --git a/src/test/regress/expected/master_evaluation.out b/src/test/regress/expected/coordinator_evaluation.out similarity index 86% rename from src/test/regress/expected/master_evaluation.out rename to src/test/regress/expected/coordinator_evaluation.out index 75d1c403c..2d676cb0f 100644 --- a/src/test/regress/expected/master_evaluation.out +++ b/src/test/regress/expected/coordinator_evaluation.out @@ -1,7 +1,7 @@ -- This test relies on metadata being synced -- that's why is should be executed on MX schedule -CREATE SCHEMA master_evaluation; -SET search_path TO master_evaluation; +CREATE SCHEMA coordinator_evaluation; +SET search_path TO coordinator_evaluation; -- create a volatile function that returns the local node id CREATE OR REPLACE FUNCTION get_local_node_id_volatile() RETURNS INT AS $$ @@ -29,8 +29,8 @@ SELECT create_distributed_function('get_local_node_id_volatile_sum_with_param(in (1 row) -CREATE TABLE master_evaluation_table (key int, value int); -SELECT create_distributed_table('master_evaluation_table', 'key'); +CREATE TABLE coordinator_evaluation_table (key int, value int); +SELECT create_distributed_table('coordinator_evaluation_table', 'key'); create_distributed_table --------------------------------------------------------------------- @@ -44,16 +44,16 @@ SELECT get_local_node_id_volatile(); (1 row) -- 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 -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? --------------------------------------------------------------------- t (1 row) -- 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); ?column? --------------------------------------------------------------------- @@ -103,13 +103,13 @@ execute fast_path_router_with_param(8); (1 row) -- 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? --------------------------------------------------------------------- t (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); ?column? --------------------------------------------------------------------- @@ -159,21 +159,21 @@ execute router_with_param(8); (1 row) -- 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? --------------------------------------------------------------------- t (1 row) -- 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? --------------------------------------------------------------------- 101 | t | t (1 row) -- 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? --------------------------------------------------------------------- 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 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? --------------------------------------------------------------------- 101 | f @@ -189,13 +189,13 @@ SELECT (SELECT count(*) FROM master_evaluation_table), get_local_node_id_volatil RESET citus.task_assignment_policy; -- 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? --------------------------------------------------------------------- t (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? --------------------------------------------------------------------- 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 -- for modifications, we expect the evaluation to happen on the coordinator -- 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); key | value --------------------------------------------------------------------- @@ -249,7 +249,7 @@ EXECUTE insert_with_param_expression(0); -- for modifications, we expect the evaluation to happen on the coordinator -- 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()); key | value --------------------------------------------------------------------- @@ -292,7 +292,7 @@ EXECUTE insert_with_param(0 + get_local_node_id_volatile()); 0 | 0 (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 -- this means that the query should be hitting multiple workers SET client_min_messages TO DEBUG2; @@ -353,7 +353,7 @@ DEBUG: Router planner cannot handle multi-shard select queries t (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 -- on the coordinator 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) -- 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); DEBUG: Router planner cannot handle multi-shard select queries bool_and @@ -514,8 +514,8 @@ DEBUG: Router planner cannot handle multi-shard select queries RESET client_min_messages; RESET citus.log_remote_commands; -- numeric has different casting affects, so some tests on that -CREATE TABLE master_evaluation_table_2 (key numeric, value numeric); -SELECT create_distributed_table('master_evaluation_table_2', 'key'); +CREATE TABLE coordinator_evaluation_table_2 (key numeric, value numeric); +SELECT create_distributed_table('coordinator_evaluation_table_2', 'key'); create_distributed_table --------------------------------------------------------------------- @@ -529,13 +529,13 @@ BEGIN RETURN trunc(random() * (end_int-start_int) + start_int); END; $$ LANGUAGE 'plpgsql' STRICT; -CREATE OR REPLACE PROCEDURE master_evaluation.test_procedure(int) +CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int) LANGUAGE plpgsql AS $procedure$ DECLARE filterKey INTEGER; BEGIN - filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; - PERFORM DISTINCT value FROM master_evaluation_table_2 WHERE key = filterKey; + filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1; + PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey; END; $procedure$; -- 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: Plan is router executable 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 AS $procedure$ DECLARE filterKey INTEGER; BEGIN - filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; - INSERT INTO master_evaluation_table_2 VALUES (filterKey, filterKey); + filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1; + INSERT INTO coordinator_evaluation_table_2 VALUES (filterKey, filterKey); END; $procedure$; 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); -SELECT count(*) FROM master_evaluation_table_2 WHERE key = 101; +SELECT count(*) FROM coordinator_evaluation_table_2 WHERE key = 101; count --------------------------------------------------------------------- 7 (1 row) SET client_min_messages TO ERROR; -DROP SCHEMA master_evaluation CASCADE; +DROP SCHEMA coordinator_evaluation CASCADE; diff --git a/src/test/regress/expected/master_evaluation_modify.out b/src/test/regress/expected/coordinator_evaluation_modify.out similarity index 57% rename from src/test/regress/expected/master_evaluation_modify.out rename to src/test/regress/expected/coordinator_evaluation_modify.out index 5571bce2e..60d2b35c3 100644 --- a/src/test/regress/expected/master_evaluation_modify.out +++ b/src/test/regress/expected/coordinator_evaluation_modify.out @@ -1,7 +1,7 @@ -- This test relies on metadata being synced -- that's why is should be executed on MX schedule -CREATE SCHEMA master_evaluation_combinations_modify; -SET search_path TO master_evaluation_combinations_modify; +CREATE SCHEMA coordinator_evaluation_combinations_modify; +SET search_path TO coordinator_evaluation_combinations_modify; -- in this test, we are considering combinations of -- several Citus features, and there is one prepared -- statement for the combinations of following: @@ -9,8 +9,8 @@ SET search_path TO master_evaluation_combinations_modify; -- (b) Local Execution vs Remote Execution -- (c) Parameters on distribution key vs Parameters on non-dist key -- vs Non-parametrized queries --- (d) Master Function Evaluation Required vs --- Master Function Evaluation Not Required +-- (d) Coordinator Function Evaluation Required vs +-- Coordinator Function Evaluation Not Required SET citus.next_shard_id TO 1180000; -- create a volatile function that returns the local node id CREATE OR REPLACE FUNCTION get_local_node_id_stable() @@ -819,7 +819,7 @@ EXECUTE router_with_only_function; \c - - - :worker_2_port 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 CREATE OR REPLACE FUNCTION get_constant_stable() RETURNS INT AS $$ @@ -834,56 +834,56 @@ INSERT INTO user_info_data (user_id, u_data) VALUES -- make sure that it is also true for fast-path router queries with paramaters PREPARE fast_path_router_with_param(int) AS DELETE FROM user_info_data WHERE user_id = $1 RETURNING user_id, u_data; execute fast_path_router_with_param(3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 3) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test3',3) (1 row) execute fast_path_router_with_param(4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 4) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 4) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 4 | ('test4',4) (1 row) execute fast_path_router_with_param(7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 7) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 7) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 7 | ('test7',7) (1 row) execute fast_path_router_with_param(9); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 9) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 9) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 9 | ('test9',9) (1 row) execute fast_path_router_with_param(11); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 11) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 11) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 11 | ('test11',11) (1 row) execute fast_path_router_with_param(12); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 12) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 12) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 12 | ('test12',12) (1 row) execute fast_path_router_with_param(14); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 14) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 14) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 14 | ('test14',14) (1 row) execute fast_path_router_with_param(16); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 16) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (user_id OPERATOR(pg_catalog.=) 16) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 16 | ('test16',16) @@ -896,55 +896,55 @@ INSERT INTO user_info_data (user_id, u_data) VALUES -- make sure that it is also true for fast-path router queries with paramaters PREPARE fast_path_router_with_param_and_func(int) AS DELETE FROM user_info_data WHERE u_data = ('''test''', get_constant_stable())::user_data AND user_id = $1 RETURNING user_id, u_data; execute fast_path_router_with_param_and_func(3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) execute fast_path_router_with_param_and_func(4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 4)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 4)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 4 | ('test',2) (1 row) execute fast_path_router_with_param_and_func(7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 7)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 7)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 7 | ('test',2) (1 row) execute fast_path_router_with_param_and_func(9); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 9)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 9)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- (0 rows) execute fast_path_router_with_param_and_func(11); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 11)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 11)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 11 | ('test',2) (1 row) execute fast_path_router_with_param_and_func(12); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 12)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 12)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 12 | ('test',2) (1 row) execute fast_path_router_with_param_and_func(14); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 14)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 14)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 14 | ('test',2) (1 row) execute fast_path_router_with_param_and_func(16); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 16)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 16)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 16 | ('test',2) @@ -955,56 +955,56 @@ NOTICE: executing the copy locally for shard xxxxx PREPARE fast_path_router_with_param_and_func_on_non_dist_key(int) AS DELETE FROM user_info_data WHERE user_id = 3 AND user_index = $1 AND u_data = ('test', (get_local_node_id_stable() > 0)::int)::user_data RETURNING user_id, user_index; EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(0); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 0) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 0) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 0 (1 row) EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(1); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 1) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 1) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 1 (1 row) EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(2); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 2) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 2) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 2 (1 row) EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 3 (1 row) EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 4) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 4) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 4 (1 row) EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(5); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 5) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 5) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 5 (1 row) EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(6); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 6) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 6) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 6 (1 row) EXECUTE fast_path_router_with_param_and_func_on_non_dist_key(7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 7) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 7) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 7 @@ -1012,63 +1012,63 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio PREPARE fast_path_router_with_param_on_non_dist_key_and_func(user_data) AS DELETE FROM user_info_data WHERE u_data = $1 AND user_id = 3 RETURNING user_id, u_data; INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) @@ -1076,63 +1076,63 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio PREPARE fast_path_router_with_param_on_non_dist_key(user_data) AS DELETE FROM user_info_data WHERE u_data = $1 AND user_id = 3 RETURNING user_id, u_data; INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) @@ -1144,56 +1144,56 @@ INSERT INTO user_info_data (user_id, u_data) VALUES (14, ('test', 2)), (16, ('test', 2)); PREPARE fast_path_router_with_two_params(user_data, int) AS DELETE FROM user_info_data WHERE u_data = $1 AND user_id = $2 RETURNING user_id, u_data; EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 4)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 4)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 4 | (test,2) (1 row) EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 7)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 7)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 7 | (test,2) (1 row) EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 9); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 9)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 9)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 9 | (test,2) (1 row) EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 11); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 11)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 11)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 11 | (test,2) (1 row) EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 12); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 12)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 12)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 12 | (test,2) (1 row) EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 14); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 14)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 14)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 14 | (test,2) (1 row) EXECUTE fast_path_router_with_two_params(('test', 2)::user_data, 16); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 16)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(test,2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 16)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 16 | (test,2) @@ -1201,63 +1201,63 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio PREPARE fast_path_router_with_only_function AS DELETE FROM user_info_data WHERE get_constant_stable() = 2AND user_id = 3 RETURNING user_id, u_data; INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE fast_path_router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) @@ -1265,49 +1265,49 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio PREPARE insert_with_function_and_param(user_data) AS INSERT INTO user_info_data VALUES (3, $1, (get_local_node_id_stable() > 0)::int) RETURNING user_id; EXECUTE insert_with_function_and_param(('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::master_evaluation_combinations_modify.user_data, 1) RETURNING user_id +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::coordinator_evaluation_combinations_modify.user_data, 1) RETURNING user_id user_id --------------------------------------------------------------------- 3 (1 row) EXECUTE insert_with_function_and_param(('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::master_evaluation_combinations_modify.user_data, 1) RETURNING user_id +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::coordinator_evaluation_combinations_modify.user_data, 1) RETURNING user_id user_id --------------------------------------------------------------------- 3 (1 row) EXECUTE insert_with_function_and_param(('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::master_evaluation_combinations_modify.user_data, 1) RETURNING user_id +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::coordinator_evaluation_combinations_modify.user_data, 1) RETURNING user_id user_id --------------------------------------------------------------------- 3 (1 row) EXECUTE insert_with_function_and_param(('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::master_evaluation_combinations_modify.user_data, 1) RETURNING user_id +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::coordinator_evaluation_combinations_modify.user_data, 1) RETURNING user_id user_id --------------------------------------------------------------------- 3 (1 row) EXECUTE insert_with_function_and_param(('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::master_evaluation_combinations_modify.user_data, 1) RETURNING user_id +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::coordinator_evaluation_combinations_modify.user_data, 1) RETURNING user_id user_id --------------------------------------------------------------------- 3 (1 row) EXECUTE insert_with_function_and_param(('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::master_evaluation_combinations_modify.user_data, 1) RETURNING user_id +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::coordinator_evaluation_combinations_modify.user_data, 1) RETURNING user_id user_id --------------------------------------------------------------------- 3 (1 row) EXECUTE insert_with_function_and_param(('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::master_evaluation_combinations_modify.user_data, 1) RETURNING user_id +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data, user_index) VALUES (3, '(test,1)'::coordinator_evaluation_combinations_modify.user_data, 1) RETURNING user_id user_id --------------------------------------------------------------------- 3 @@ -1322,56 +1322,56 @@ INSERT INTO user_info_data (user_id, u_data) VALUES -- make sure that it is also true for fast-path router queries with paramaters PREPARE router_with_param(int) AS DELETE FROM user_info_data WHERE user_id = $1 AND user_id = $1 RETURNING user_id, u_data; execute router_with_param(3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test3',3) (1 row) execute router_with_param(4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 4 | ('test4',4) (1 row) execute router_with_param(7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 7 | ('test7',7) (1 row) execute router_with_param(9); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 9 | ('test9',9) (1 row) execute router_with_param(11); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 11 | ('test11',11) (1 row) execute router_with_param(12); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 12 | ('test12',12) (1 row) execute router_with_param(14); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 14 | ('test14',14) (1 row) execute router_with_param(16); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) $1) AND (user_id OPERATOR(pg_catalog.=) $1)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 16 | ('test16',16) @@ -1384,55 +1384,55 @@ INSERT INTO user_info_data (user_id, u_data) VALUES -- make sure that it is also true for fast-path router queries with paramaters PREPARE router_with_param_and_func(int) AS DELETE FROM user_info_data WHERE u_data = ('''test''', get_constant_stable())::user_data AND user_id = $1 AND user_id = $1 RETURNING user_id, u_data; execute router_with_param_and_func(3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) execute router_with_param_and_func(4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 4) AND (user_id OPERATOR(pg_catalog.=) 4)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 4) AND (user_id OPERATOR(pg_catalog.=) 4)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 4 | ('test',2) (1 row) execute router_with_param_and_func(7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 7) AND (user_id OPERATOR(pg_catalog.=) 7)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 7) AND (user_id OPERATOR(pg_catalog.=) 7)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 7 | ('test',2) (1 row) execute router_with_param_and_func(9); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 9) AND (user_id OPERATOR(pg_catalog.=) 9)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 9) AND (user_id OPERATOR(pg_catalog.=) 9)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- (0 rows) execute router_with_param_and_func(11); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 11) AND (user_id OPERATOR(pg_catalog.=) 11)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 11) AND (user_id OPERATOR(pg_catalog.=) 11)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 11 | ('test',2) (1 row) execute router_with_param_and_func(12); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 12) AND (user_id OPERATOR(pg_catalog.=) 12)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 12) AND (user_id OPERATOR(pg_catalog.=) 12)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 12 | ('test',2) (1 row) execute router_with_param_and_func(14); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 14) AND (user_id OPERATOR(pg_catalog.=) 14)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 14) AND (user_id OPERATOR(pg_catalog.=) 14)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 14 | ('test',2) (1 row) execute router_with_param_and_func(16); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 16) AND (user_id OPERATOR(pg_catalog.=) 16)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 16) AND (user_id OPERATOR(pg_catalog.=) 16)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 16 | ('test',2) @@ -1443,56 +1443,56 @@ NOTICE: executing the copy locally for shard xxxxx PREPARE router_with_param_and_func_on_non_dist_key(int) AS DELETE FROM user_info_data WHERE user_id = 3 AND user_id = 3 AND user_index = $1 AND u_data = ('test', (get_local_node_id_stable() > 0)::int)::user_data RETURNING user_id, user_index; EXECUTE router_with_param_and_func_on_non_dist_key(0); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 0) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 0) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 0 (1 row) EXECUTE router_with_param_and_func_on_non_dist_key(1); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 1) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 1) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 1 (1 row) EXECUTE router_with_param_and_func_on_non_dist_key(2); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 2) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 2) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 2 (1 row) EXECUTE router_with_param_and_func_on_non_dist_key(3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 3) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 3 (1 row) EXECUTE router_with_param_and_func_on_non_dist_key(4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 4) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 4) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 4 (1 row) EXECUTE router_with_param_and_func_on_non_dist_key(5); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 5) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 5) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 5 (1 row) EXECUTE router_with_param_and_func_on_non_dist_key(6); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 6) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 6) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 6 (1 row) EXECUTE router_with_param_and_func_on_non_dist_key(7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 7) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::master_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_index OPERATOR(pg_catalog.=) 7) AND (u_data OPERATOR(pg_catalog.=) '(test,1)'::coordinator_evaluation_combinations_modify.user_data)) RETURNING user_id, user_index user_id | user_index --------------------------------------------------------------------- 3 | 7 @@ -1500,63 +1500,63 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio PREPARE router_with_param_on_non_dist_key_and_func(user_data) AS DELETE FROM user_info_data WHERE u_data = $1 AND user_id = 3 AND user_id = 3 RETURNING user_id, u_data; INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, '(''test'', 2)'); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, '(''test'',2)'::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key_and_func(('''test''', get_constant_stable())::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | ('test',2) @@ -1564,63 +1564,63 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio PREPARE router_with_param_on_non_dist_key(user_data) AS DELETE FROM user_info_data WHERE u_data = $1 AND user_id = 3 AND user_id = 3 RETURNING user_id, u_data; INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 1)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 1)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_param_on_non_dist_key(('test', 1)::user_data); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) 3) AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,1) @@ -1632,56 +1632,56 @@ INSERT INTO user_info_data (user_id, u_data) VALUES (14, ('test', 2)), (16, ('test', 2)); PREPARE router_with_two_params(user_data, int) AS DELETE FROM user_info_data WHERE u_data = $1 AND user_id = $2 AND user_id = $2 RETURNING user_id, u_data; EXECUTE router_with_two_params(('test', 2)::user_data, 3); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) EXECUTE router_with_two_params(('test', 2)::user_data, 4); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 4 | (test,2) (1 row) EXECUTE router_with_two_params(('test', 2)::user_data, 7); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 7 | (test,2) (1 row) EXECUTE router_with_two_params(('test', 2)::user_data, 9); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 9 | (test,2) (1 row) EXECUTE router_with_two_params(('test', 2)::user_data, 11); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 11 | (test,2) (1 row) EXECUTE router_with_two_params(('test', 2)::user_data, 12); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180003 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 12 | (test,2) (1 row) EXECUTE router_with_two_params(('test', 2)::user_data, 14); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 14 | (test,2) (1 row) EXECUTE router_with_two_params(('test', 2)::user_data, 16); -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::master_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE ((u_data OPERATOR(pg_catalog.=) $1::coordinator_evaluation_combinations_modify.user_data) AND (user_id OPERATOR(pg_catalog.=) $2) AND (user_id OPERATOR(pg_catalog.=) $2)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 16 | (test,2) @@ -1689,63 +1689,63 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio PREPARE router_with_only_function AS DELETE FROM user_info_data WHERE get_constant_stable() = 2AND user_id = 3 RETURNING user_id, u_data; INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) (1 row) INSERT INTO user_info_data (user_id, u_data) VALUES (3, ('test', 2)::user_data); -NOTICE: executing the command locally: INSERT INTO master_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::master_evaluation_combinations_modify.user_data) +NOTICE: executing the command locally: INSERT INTO coordinator_evaluation_combinations_modify.user_info_data_1180001 (user_id, u_data) VALUES (3, ROW('test'::text, 2)::coordinator_evaluation_combinations_modify.user_data) EXECUTE router_with_only_function; -NOTICE: executing the command locally: DELETE FROM master_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data +NOTICE: executing the command locally: DELETE FROM coordinator_evaluation_combinations_modify.user_info_data_1180001 user_info_data WHERE (true AND (user_id OPERATOR(pg_catalog.=) 3)) RETURNING user_id, u_data user_id | u_data --------------------------------------------------------------------- 3 | (test,2) @@ -1754,4 +1754,4 @@ NOTICE: executing the command locally: DELETE FROM master_evaluation_combinatio -- suppress notices \c - - - :master_port SET client_min_messages TO ERROR; -DROP SCHEMA master_evaluation_combinations_modify CASCADE; +DROP SCHEMA coordinator_evaluation_combinations_modify CASCADE; diff --git a/src/test/regress/expected/master_evaluation_select.out b/src/test/regress/expected/coordinator_evaluation_select.out similarity index 57% rename from src/test/regress/expected/master_evaluation_select.out rename to src/test/regress/expected/coordinator_evaluation_select.out index d646ae832..c837090d6 100644 --- a/src/test/regress/expected/master_evaluation_select.out +++ b/src/test/regress/expected/coordinator_evaluation_select.out @@ -7,10 +7,10 @@ -- (b) Local Execution vs Remote Execution -- (c) Parameters on distribution key vs Parameters on non-dist key -- vs Non-parametrized queries --- (d) Master Function Evaluation Required vs --- Master Function Evaluation Not Required -CREATE SCHEMA master_evaluation_combinations; -SET search_path TO master_evaluation_combinations; +-- (d) Coordinator Function Evaluation Required vs +-- Coordinator Function Evaluation Not Required +CREATE SCHEMA coordinator_evaluation_combinations; +SET search_path TO coordinator_evaluation_combinations; SET citus.next_shard_id TO 1170000; -- create a volatile function that returns the local node id CREATE OR REPLACE FUNCTION get_local_node_id_volatile() @@ -824,10 +824,10 @@ EXECUTE router_with_only_function; \c - - - :worker_2_port 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 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 --------------------------------------------------------------------- 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 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); -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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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? --------------------------------------------------------------------- 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 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); -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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- 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 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); -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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- 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; 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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 --------------------------------------------------------------------- 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; 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 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; 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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 --------------------------------------------------------------------- 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 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); -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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (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; -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? --------------------------------------------------------------------- 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; 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- 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 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); -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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t @@ -1394,21 +1394,21 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g -- 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; -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? --------------------------------------------------------------------- t (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; -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 --------------------------------------------------------------------- 1 (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; -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 --------------------------------------------------------------------- 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; 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (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; -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? --------------------------------------------------------------------- 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; 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (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; -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 --------------------------------------------------------------------- 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; 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (1 row) 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 --------------------------------------------------------------------- 1 (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; -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? --------------------------------------------------------------------- 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; 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t (1 row) 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? --------------------------------------------------------------------- t @@ -1687,4 +1687,4 @@ NOTICE: executing the command locally: SELECT (master_evaluation_combinations.g -- suppress notices \c - - - :master_port SET client_min_messages TO ERROR; -DROP SCHEMA master_evaluation_combinations CASCADE; +DROP SCHEMA coordinator_evaluation_combinations CASCADE; diff --git a/src/test/regress/multi_mx_schedule b/src/test/regress/multi_mx_schedule index 8c6f81536..3767c302b 100644 --- a/src/test/regress/multi_mx_schedule +++ b/src/test/regress/multi_mx_schedule @@ -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_join_w1 multi_mx_repartition_join_w2 multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2 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_function_call_delegation test: multi_mx_modifications local_shard_execution diff --git a/src/test/regress/sql/master_evaluation.sql b/src/test/regress/sql/coordinator_evaluation.sql similarity index 73% rename from src/test/regress/sql/master_evaluation.sql rename to src/test/regress/sql/coordinator_evaluation.sql index b55b2513e..76b1fc47f 100644 --- a/src/test/regress/sql/master_evaluation.sql +++ b/src/test/regress/sql/coordinator_evaluation.sql @@ -1,7 +1,7 @@ -- This test relies on metadata being synced -- that's why is should be executed on MX schedule -CREATE SCHEMA master_evaluation; -SET search_path TO master_evaluation; +CREATE SCHEMA coordinator_evaluation; +SET search_path TO coordinator_evaluation; -- create a volatile function that returns the local node id 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)'); -CREATE TABLE master_evaluation_table (key int, value int); -SELECT create_distributed_table('master_evaluation_table', 'key'); +CREATE TABLE coordinator_evaluation_table (key int, value int); +SELECT create_distributed_table('coordinator_evaluation_table', 'key'); -- show that local id is 0, we'll use this information SELECT get_local_node_id_volatile(); -- 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 -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 -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(2); @@ -48,9 +48,9 @@ execute fast_path_router_with_param(7); execute fast_path_router_with_param(8); -- 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(2); @@ -62,29 +62,29 @@ execute router_with_param(7); execute router_with_param(8); -- 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 -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 -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 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; -- 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 count(*) > 0 FROM master_evaluation_table WHERE value >= get_local_node_id_volatile(); +SELECT min(get_local_node_id_volatile()) > 0 FROM coordinator_evaluation_table; +SELECT count(*) > 0 FROM coordinator_evaluation_table WHERE value >= get_local_node_id_volatile(); -- let's have some tests around expressions -- for modifications, we expect the evaluation to happen on the coordinator -- 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); @@ -95,7 +95,7 @@ EXECUTE insert_with_param_expression(0); -- for modifications, we expect the evaluation to happen on the coordinator -- 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()); @@ -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()); -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 -- 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); -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 -- 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()); -- 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); @@ -153,8 +153,8 @@ RESET client_min_messages; RESET citus.log_remote_commands; -- numeric has different casting affects, so some tests on that -CREATE TABLE master_evaluation_table_2 (key numeric, value numeric); -SELECT create_distributed_table('master_evaluation_table_2', 'key'); +CREATE TABLE coordinator_evaluation_table_2 (key numeric, value numeric); +SELECT create_distributed_table('coordinator_evaluation_table_2', 'key'); CREATE OR REPLACE FUNCTION TEST_RANDOM (INTEGER, INTEGER) RETURNS INTEGER AS $$ DECLARE @@ -165,13 +165,13 @@ BEGIN END; $$ LANGUAGE 'plpgsql' STRICT; -CREATE OR REPLACE PROCEDURE master_evaluation.test_procedure(int) +CREATE OR REPLACE PROCEDURE coordinator_evaluation.test_procedure(int) LANGUAGE plpgsql AS $procedure$ DECLARE filterKey INTEGER; BEGIN - filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; - PERFORM DISTINCT value FROM master_evaluation_table_2 WHERE key = filterKey; + filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1; + PERFORM DISTINCT value FROM coordinator_evaluation_table_2 WHERE key = filterKey; END; $procedure$; @@ -187,13 +187,13 @@ 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 AS $procedure$ DECLARE filterKey INTEGER; BEGIN - filterKey := round(master_evaluation.TEST_RANDOM(1,1)) + $1; - INSERT INTO master_evaluation_table_2 VALUES (filterKey, filterKey); + filterKey := round(coordinator_evaluation.TEST_RANDOM(1,1)) + $1; + INSERT INTO coordinator_evaluation_table_2 VALUES (filterKey, filterKey); END; $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); -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; -DROP SCHEMA master_evaluation CASCADE; +DROP SCHEMA coordinator_evaluation CASCADE; diff --git a/src/test/regress/sql/master_evaluation_modify.sql b/src/test/regress/sql/coordinator_evaluation_modify.sql similarity index 98% rename from src/test/regress/sql/master_evaluation_modify.sql rename to src/test/regress/sql/coordinator_evaluation_modify.sql index 1fe8cf136..0b2f7fd19 100644 --- a/src/test/regress/sql/master_evaluation_modify.sql +++ b/src/test/regress/sql/coordinator_evaluation_modify.sql @@ -1,8 +1,8 @@ -- This test relies on metadata being synced -- that's why is should be executed on MX schedule -CREATE SCHEMA master_evaluation_combinations_modify; -SET search_path TO master_evaluation_combinations_modify; +CREATE SCHEMA coordinator_evaluation_combinations_modify; +SET search_path TO coordinator_evaluation_combinations_modify; -- in this test, we are considering combinations of -- 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 -- (c) Parameters on distribution key vs Parameters on non-dist key -- vs Non-parametrized queries --- (d) Master Function Evaluation Required vs --- Master Function Evaluation Not Required +-- (d) Coordinator Function Evaluation Required vs +-- Coordinator Function Evaluation Not Required SET citus.next_shard_id TO 1180000; @@ -270,7 +270,7 @@ EXECUTE router_with_only_function; \c - - - :worker_2_port 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 CREATE OR REPLACE FUNCTION get_constant_stable() @@ -538,4 +538,4 @@ EXECUTE router_with_only_function; -- suppress notices \c - - - :master_port SET client_min_messages TO ERROR; -DROP SCHEMA master_evaluation_combinations_modify CASCADE; +DROP SCHEMA coordinator_evaluation_combinations_modify CASCADE; diff --git a/src/test/regress/sql/master_evaluation_select.sql b/src/test/regress/sql/coordinator_evaluation_select.sql similarity index 98% rename from src/test/regress/sql/master_evaluation_select.sql rename to src/test/regress/sql/coordinator_evaluation_select.sql index 575e22c39..5195a04c9 100644 --- a/src/test/regress/sql/master_evaluation_select.sql +++ b/src/test/regress/sql/coordinator_evaluation_select.sql @@ -9,11 +9,11 @@ -- (b) Local Execution vs Remote Execution -- (c) Parameters on distribution key vs Parameters on non-dist key -- vs Non-parametrized queries --- (d) Master Function Evaluation Required vs --- Master Function Evaluation Not Required +-- (d) Coordinator Function Evaluation Required vs +-- Coordinator Function Evaluation Not Required -CREATE SCHEMA master_evaluation_combinations; -SET search_path TO master_evaluation_combinations; +CREATE SCHEMA coordinator_evaluation_combinations; +SET search_path TO coordinator_evaluation_combinations; SET citus.next_shard_id TO 1170000; @@ -232,7 +232,7 @@ EXECUTE router_with_only_function; \c - - - :worker_2_port 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 SELECT count(*) FROM user_info_data WHERE user_id = 3; @@ -416,4 +416,4 @@ EXECUTE router_with_only_function; -- suppress notices \c - - - :master_port SET client_min_messages TO ERROR; -DROP SCHEMA master_evaluation_combinations CASCADE; +DROP SCHEMA coordinator_evaluation_combinations CASCADE;