1
0
mirror of https://github.com/postgres/postgres.git synced 2025-11-07 19:06:32 +03:00

Implement SQL-compliant treatment of row comparisons for < <= > >= cases

(previously we only did = and <> correctly).  Also, allow row comparisons
with any operators that are in btree opclasses, not only those with these
specific names.  This gets rid of a whole lot of indefensible assumptions
about the behavior of particular operators based on their names ... though
it's still true that IN and NOT IN expand to "= ANY".  The patch adds a
RowCompareExpr expression node type, and makes some changes in the
representation of ANY/ALL/ROWCOMPARE SubLinks so that they can share code
with RowCompareExpr.

I have not yet done anything about making RowCompareExpr an indexable
operator, but will look at that soon.

initdb forced due to changes in stored rules.
This commit is contained in:
Tom Lane
2005-12-28 01:30:02 +00:00
parent a37422e042
commit 6e07709760
26 changed files with 1452 additions and 659 deletions

View File

@@ -49,7 +49,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/optimizer/path/costsize.c,v 1.151 2005/11/26 22:14:56 tgl Exp $
* $PostgreSQL: pgsql/src/backend/optimizer/path/costsize.c,v 1.152 2005/12/28 01:29:59 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -1609,6 +1609,13 @@ cost_qual_eval_walker(Node *node, QualCost *total)
total->per_tuple +=
cpu_operator_cost * estimate_array_length(arraynode) * 0.5;
}
else if (IsA(node, RowCompareExpr))
{
/* Conservatively assume we will check all the columns */
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
total->per_tuple += cpu_operator_cost * list_length(rcexpr->opnos);
}
else if (IsA(node, SubLink))
{
/* This routine should not be applied to un-planned expressions */
@@ -1624,7 +1631,6 @@ cost_qual_eval_walker(Node *node, QualCost *total)
*
* An exception occurs when we have decided we can implement the
* subplan by hashing.
*
*/
SubPlan *subplan = (SubPlan *) node;
Plan *plan = subplan->plan;
@@ -1643,7 +1649,7 @@ cost_qual_eval_walker(Node *node, QualCost *total)
/*
* The per-tuple costs include the cost of evaluating the lefthand
* expressions, plus the cost of probing the hashtable. Recursion
* into the exprs list will handle the lefthand expressions
* into the testexpr will handle the lefthand expressions
* properly, and will count one cpu_operator_cost for each
* comparison operator. That is probably too low for the probing
* cost, but it's hard to make a better estimate, so live with it

View File

@@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/optimizer/plan/subselect.c,v 1.102 2005/11/26 22:14:57 tgl Exp $
* $PostgreSQL: pgsql/src/backend/optimizer/plan/subselect.c,v 1.103 2005/12/28 01:29:59 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -19,14 +19,12 @@
#include "nodes/makefuncs.h"
#include "nodes/params.h"
#include "optimizer/clauses.h"
#include "optimizer/cost.h"
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
#include "optimizer/subselect.h"
#include "optimizer/var.h"
#include "parser/parsetree.h"
#include "parser/parse_expr.h"
#include "parser/parse_oper.h"
#include "parser/parse_relation.h"
#include "rewrite/rewriteManip.h"
#include "utils/builtins.h"
@@ -74,6 +72,12 @@ typedef struct PlannerParamItem
} PlannerParamItem;
typedef struct convert_testexpr_context
{
int rtindex; /* RT index for Vars, or 0 for Params */
List *righthandIds; /* accumulated list of Vars or Param IDs */
} convert_testexpr_context;
typedef struct finalize_primnode_context
{
Bitmapset *paramids; /* Set of PARAM_EXEC paramids found */
@@ -81,10 +85,13 @@ typedef struct finalize_primnode_context
} finalize_primnode_context;
static List *convert_sublink_opers(List *lefthand, List *operOids,
List *targetlist, int rtindex,
List **righthandIds);
static Node *convert_testexpr(Node *testexpr,
int rtindex,
List **righthandIds);
static Node *convert_testexpr_mutator(Node *node,
convert_testexpr_context *context);
static bool subplan_is_hashable(SubLink *slink, SubPlan *node);
static bool hash_ok_operator(OpExpr *expr);
static Node *replace_correlation_vars_mutator(Node *node, void *context);
static Node *process_sublinks_mutator(Node *node, bool *isTopQual);
static Bitmapset *finalize_plan(Plan *plan, List *rtable,
@@ -228,20 +235,20 @@ generate_new_param(Oid paramtype, int32 paramtypmod)
}
/*
* Convert a bare SubLink (as created by the parser) into a SubPlan.
* Convert a SubLink (as created by the parser) into a SubPlan.
*
* We are given the raw SubLink and the already-processed lefthand argument
* list (use this instead of the SubLink's own field). We are also told if
* We are given the original SubLink and the already-processed testexpr
* (use this instead of the SubLink's own field). We are also told if
* this expression appears at top level of a WHERE/HAVING qual.
*
* The result is whatever we need to substitute in place of the SubLink
* node in the executable expression. This will be either the SubPlan
* node (if we have to do the subplan as a subplan), or a Param node
* representing the result of an InitPlan, or possibly an AND or OR tree
* containing InitPlan Param nodes.
* representing the result of an InitPlan, or a row comparison expression
* tree containing InitPlan Param nodes.
*/
static Node *
make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
make_subplan(SubLink *slink, Node *testexpr, bool isTopQual)
{
SubPlan *node = makeNode(SubPlan);
Query *subquery = (Query *) (slink->subselect);
@@ -264,7 +271,7 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
* first tuple will be retrieved. For ALL and ANY subplans, we will be
* able to stop evaluating if the test condition fails, so very often not
* all the tuples will be retrieved; for lack of a better idea, specify
* 50% retrieval. For EXPR and MULTIEXPR subplans, use default behavior
* 50% retrieval. For EXPR and ROWCOMPARE subplans, use default behavior
* (we're only expecting one row out, anyway).
*
* NOTE: if you change these numbers, also change cost_qual_eval_walker()
@@ -300,8 +307,7 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
* Initialize other fields of the SubPlan node.
*/
node->subLinkType = slink->subLinkType;
node->useOr = slink->useOr;
node->exprs = NIL;
node->testexpr = NULL;
node->paramIds = NIL;
node->useHashTable = false;
/* At top level of a qual, can treat UNKNOWN the same as FALSE */
@@ -326,11 +332,11 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
/*
* Un-correlated or undirect correlated plans of EXISTS, EXPR, ARRAY, or
* MULTIEXPR types can be used as initPlans. For EXISTS, EXPR, or ARRAY,
* ROWCOMPARE types can be used as initPlans. For EXISTS, EXPR, or ARRAY,
* we just produce a Param referring to the result of evaluating the
* initPlan. For MULTIEXPR, we must build an AND or OR-clause of the
* individual comparison operators, using the appropriate lefthand side
* expressions and Params for the initPlan's target items.
* initPlan. For ROWCOMPARE, we must modify the testexpr tree to contain
* PARAM_EXEC Params instead of the PARAM_SUBLINK Params emitted by the
* parser.
*/
if (node->parParam == NIL && slink->subLinkType == EXISTS_SUBLINK)
{
@@ -369,34 +375,30 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
PlannerInitPlan = lappend(PlannerInitPlan, node);
result = (Node *) prm;
}
else if (node->parParam == NIL && slink->subLinkType == MULTIEXPR_SUBLINK)
else if (node->parParam == NIL && slink->subLinkType == ROWCOMPARE_SUBLINK)
{
List *exprs;
/* Convert the lefthand exprs and oper OIDs into executable exprs */
exprs = convert_sublink_opers(lefthand,
slink->operOids,
plan->targetlist,
0,
&node->paramIds);
/* Adjust the Params */
result = convert_testexpr(testexpr,
0,
&node->paramIds);
node->setParam = list_copy(node->paramIds);
PlannerInitPlan = lappend(PlannerInitPlan, node);
/*
* The executable expressions are returned to become part of the outer
* plan's expression tree; they are not kept in the initplan node.
* The executable expression is returned to become part of the outer
* plan's expression tree; it is not kept in the initplan node.
*/
if (list_length(exprs) > 1)
result = (Node *) (node->useOr ? make_orclause(exprs) :
make_andclause(exprs));
else
result = (Node *) linitial(exprs);
}
else
{
List *args;
ListCell *l;
/* Adjust the Params */
node->testexpr = convert_testexpr(testexpr,
0,
&node->paramIds);
/*
* We can't convert subplans of ALL_SUBLINK or ANY_SUBLINK types to
* initPlans, even when they are uncorrelated or undirect correlated,
@@ -434,13 +436,6 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
node->plan = plan = materialize_finished_plan(plan);
}
/* Convert the lefthand exprs and oper OIDs into executable exprs */
node->exprs = convert_sublink_opers(lefthand,
slink->operOids,
plan->targetlist,
0,
&node->paramIds);
/*
* Make node->args from parParam.
*/
@@ -465,10 +460,9 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
}
/*
* convert_sublink_opers: given a lefthand-expressions list and a list of
* operator OIDs, build a list of actually executable expressions. The
* righthand sides of the expressions are Params or Vars representing the
* results of the sub-select.
* convert_testexpr: convert the testexpr given by the parser into
* actually executable form. This entails replacing PARAM_SUBLINK Params
* with Params or Vars representing the results of the sub-select:
*
* If rtindex is 0, we build Params to represent the sub-select outputs.
* The paramids of the Params created are returned in the *righthandIds list.
@@ -476,90 +470,84 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
* If rtindex is not 0, we build Vars using that rtindex as varno. Copies
* of the Var nodes are returned in *righthandIds (this is a bit of a type
* cheat, but we can get away with it).
*
* The given testexpr has already been recursively processed by
* process_sublinks_mutator. Hence it can no longer contain any
* PARAM_SUBLINK Params for lower SubLink nodes; we can safely assume that
* any we find are for our own level of SubLink.
*/
static List *
convert_sublink_opers(List *lefthand, List *operOids,
List *targetlist, int rtindex,
List **righthandIds)
static Node *
convert_testexpr(Node *testexpr,
int rtindex,
List **righthandIds)
{
List *result = NIL;
ListCell *l,
*lefthand_item,
*tlist_item;
*righthandIds = NIL;
lefthand_item = list_head(lefthand);
tlist_item = list_head(targetlist);
foreach(l, operOids)
{
Oid opid = lfirst_oid(l);
Node *leftop = (Node *) lfirst(lefthand_item);
TargetEntry *te = (TargetEntry *) lfirst(tlist_item);
Node *rightop;
Operator tup;
Assert(!te->resjunk);
if (rtindex)
{
/* Make the Var node representing the subplan's result */
rightop = (Node *) makeVar(rtindex,
te->resno,
exprType((Node *) te->expr),
exprTypmod((Node *) te->expr),
0);
/*
* Copy it for caller. NB: we need a copy to avoid having
* doubly-linked substructure in the modified parse tree.
*/
*righthandIds = lappend(*righthandIds, copyObject(rightop));
}
else
{
/* Make the Param node representing the subplan's result */
Param *prm;
prm = generate_new_param(exprType((Node *) te->expr),
exprTypmod((Node *) te->expr));
/* Record its ID */
*righthandIds = lappend_int(*righthandIds, prm->paramid);
rightop = (Node *) prm;
}
/* Look up the operator to pass to make_op_expr */
tup = SearchSysCache(OPEROID,
ObjectIdGetDatum(opid),
0, 0, 0);
if (!HeapTupleIsValid(tup))
elog(ERROR, "cache lookup failed for operator %u", opid);
/*
* Make the expression node.
*
* Note: we use make_op_expr in case runtime type conversion function
* calls must be inserted for this operator! (But we are not
* expecting to have to resolve unknown Params, so it's okay to pass a
* null pstate.)
*/
result = lappend(result,
make_op_expr(NULL,
tup,
leftop,
rightop,
exprType(leftop),
exprType((Node *) te->expr)));
ReleaseSysCache(tup);
lefthand_item = lnext(lefthand_item);
tlist_item = lnext(tlist_item);
}
Node *result;
convert_testexpr_context context;
context.rtindex = rtindex;
context.righthandIds = NIL;
result = convert_testexpr_mutator(testexpr, &context);
*righthandIds = context.righthandIds;
return result;
}
static Node *
convert_testexpr_mutator(Node *node,
convert_testexpr_context *context)
{
if (node == NULL)
return NULL;
if (IsA(node, Param))
{
Param *param = (Param *) node;
if (param->paramkind == PARAM_SUBLINK)
{
/*
* We expect to encounter the Params in column-number sequence.
* We could handle non-sequential order if necessary, but for now
* there's no need. (This is also a useful cross-check that we
* aren't finding any unexpected Params.)
*/
if (param->paramid != list_length(context->righthandIds) + 1)
elog(ERROR, "unexpected PARAM_SUBLINK ID: %d", param->paramid);
if (context->rtindex)
{
/* Make the Var node representing the subplan's result */
Var *newvar;
newvar = makeVar(context->rtindex,
param->paramid,
param->paramtype,
-1,
0);
/*
* Copy it for caller. NB: we need a copy to avoid having
* doubly-linked substructure in the modified parse tree.
*/
context->righthandIds = lappend(context->righthandIds,
copyObject(newvar));
return (Node *) newvar;
}
else
{
/* Make the Param node representing the subplan's result */
Param *newparam;
newparam = generate_new_param(param->paramtype, -1);
/* Record its ID */
context->righthandIds = lappend_int(context->righthandIds,
newparam->paramid);
return (Node *) newparam;
}
}
}
return expression_tree_mutator(node,
convert_testexpr_mutator,
(void *) context);
}
/*
* subplan_is_hashable: decide whether we can implement a subplan by hashing
*
@@ -573,15 +561,19 @@ subplan_is_hashable(SubLink *slink, SubPlan *node)
ListCell *l;
/*
* The sublink type must be "= ANY" --- that is, an IN operator. (We
* require the operator name to be unqualified, which may be overly
* paranoid, or may not be.) XXX since we also check that the operators
* are hashable, the test on operator name may be redundant?
* The sublink type must be "= ANY" --- that is, an IN operator. We
* expect that the test expression will be either a single OpExpr, or an
* AND-clause containing OpExprs. (If it's anything else then the parser
* must have determined that the operators have non-equality-like
* semantics. In the OpExpr case we can't be sure what the operator's
* semantics are like, but the test below for hashability will reject
* anything that's not equality.)
*/
if (slink->subLinkType != ANY_SUBLINK)
return false;
if (list_length(slink->operName) != 1 ||
strcmp(strVal(linitial(slink->operName)), "=") != 0)
if (slink->testexpr == NULL ||
(!IsA(slink->testexpr, OpExpr) &&
!and_clause(slink->testexpr)))
return false;
/*
@@ -614,26 +606,47 @@ subplan_is_hashable(SubLink *slink, SubPlan *node)
* could be relaxed by using two different sets of operators with the hash
* table, but there is no obvious usefulness to that at present.)
*/
foreach(l, slink->operOids)
if (IsA(slink->testexpr, OpExpr))
{
Oid opid = lfirst_oid(l);
HeapTuple tup;
Form_pg_operator optup;
tup = SearchSysCache(OPEROID,
ObjectIdGetDatum(opid),
0, 0, 0);
if (!HeapTupleIsValid(tup))
elog(ERROR, "cache lookup failed for operator %u", opid);
optup = (Form_pg_operator) GETSTRUCT(tup);
if (!optup->oprcanhash || optup->oprcom != opid ||
!func_strict(optup->oprcode))
{
ReleaseSysCache(tup);
if (!hash_ok_operator((OpExpr *) slink->testexpr))
return false;
}
ReleaseSysCache(tup);
}
else
{
foreach(l, ((BoolExpr *) slink->testexpr)->args)
{
Node *andarg = (Node *) lfirst(l);
if (!IsA(andarg, OpExpr))
return false; /* probably can't happen */
if (!hash_ok_operator((OpExpr *) andarg))
return false;
}
}
return true;
}
static bool
hash_ok_operator(OpExpr *expr)
{
Oid opid = expr->opno;
HeapTuple tup;
Form_pg_operator optup;
tup = SearchSysCache(OPEROID,
ObjectIdGetDatum(opid),
0, 0, 0);
if (!HeapTupleIsValid(tup))
elog(ERROR, "cache lookup failed for operator %u", opid);
optup = (Form_pg_operator) GETSTRUCT(tup);
if (!optup->oprcanhash || optup->oprcom != opid ||
!func_strict(optup->oprcode))
{
ReleaseSysCache(tup);
return false;
}
ReleaseSysCache(tup);
return true;
}
@@ -659,17 +672,28 @@ convert_IN_to_join(PlannerInfo *root, SubLink *sublink)
RangeTblEntry *rte;
RangeTblRef *rtr;
InClauseInfo *ininfo;
List *exprs;
/*
* The sublink type must be "= ANY" --- that is, an IN operator. (We
* require the operator name to be unqualified, which may be overly
* paranoid, or may not be.)
* The sublink type must be "= ANY" --- that is, an IN operator. We
* expect that the test expression will be either a single OpExpr, or an
* AND-clause containing OpExprs. (If it's anything else then the parser
* must have determined that the operators have non-equality-like
* semantics. In the OpExpr case we can't be sure what the operator's
* semantics are like, and must check for ourselves.)
*/
if (sublink->subLinkType != ANY_SUBLINK)
return NULL;
if (list_length(sublink->operName) != 1 ||
strcmp(strVal(linitial(sublink->operName)), "=") != 0)
if (sublink->testexpr && IsA(sublink->testexpr, OpExpr))
{
List *opclasses;
List *opstrats;
get_op_btree_interpretation(((OpExpr *) sublink->testexpr)->opno,
&opclasses, &opstrats);
if (!list_member_int(opstrats, ROWCOMPARE_EQ))
return NULL;
}
else if (!and_clause(sublink->testexpr))
return NULL;
/*
@@ -683,16 +707,14 @@ convert_IN_to_join(PlannerInfo *root, SubLink *sublink)
* The left-hand expressions must contain some Vars of the current query,
* else it's not gonna be a join.
*/
left_varnos = pull_varnos((Node *) sublink->lefthand);
left_varnos = pull_varnos(sublink->testexpr);
if (bms_is_empty(left_varnos))
return NULL;
/*
* The left-hand expressions mustn't be volatile. (Perhaps we should test
* the combining operators, too? We'd only need to point the function
* directly at the sublink ...)
* The combining operators and left-hand expressions mustn't be volatile.
*/
if (contain_volatile_functions((Node *) sublink->lefthand))
if (contain_volatile_functions(sublink->testexpr))
return NULL;
/*
@@ -722,16 +744,13 @@ convert_IN_to_join(PlannerInfo *root, SubLink *sublink)
root->in_info_list = lappend(root->in_info_list, ininfo);
/*
* Build the result qual expressions. As a side effect,
* Build the result qual expression. As a side effect,
* ininfo->sub_targetlist is filled with a list of Vars representing the
* subselect outputs.
*/
exprs = convert_sublink_opers(sublink->lefthand,
sublink->operOids,
subselect->targetList,
rtindex,
&ininfo->sub_targetlist);
return (Node *) make_ands_explicit(exprs);
return convert_testexpr(sublink->testexpr,
rtindex,
&ininfo->sub_targetlist);
}
/*
@@ -802,19 +821,18 @@ process_sublinks_mutator(Node *node, bool *isTopQual)
if (IsA(node, SubLink))
{
SubLink *sublink = (SubLink *) node;
List *lefthand;
Node *testexpr;
/*
* First, recursively process the lefthand-side expressions, if any.
*/
locTopQual = false;
lefthand = (List *)
process_sublinks_mutator((Node *) sublink->lefthand, &locTopQual);
testexpr = process_sublinks_mutator(sublink->testexpr, &locTopQual);
/*
* Now build the SubPlan node and make the expr to return.
*/
return make_subplan(sublink, lefthand, *isTopQual);
return make_subplan(sublink, testexpr, *isTopQual);
}
/*

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/optimizer/util/clauses.c,v 1.204 2005/12/20 02:30:36 tgl Exp $
* $PostgreSQL: pgsql/src/backend/optimizer/util/clauses.c,v 1.205 2005/12/28 01:30:00 tgl Exp $
*
* HISTORY
* AUTHOR DATE MAJOR EVENT
@@ -540,6 +540,8 @@ expression_returns_set_walker(Node *node, void *context)
return false;
if (IsA(node, RowExpr))
return false;
if (IsA(node, RowCompareExpr))
return false;
if (IsA(node, CoalesceExpr))
return false;
if (IsA(node, MinMaxExpr))
@@ -651,12 +653,12 @@ contain_mutable_functions_walker(Node *node, void *context)
return true;
/* else fall through to check args */
}
if (IsA(node, SubLink))
if (IsA(node, RowCompareExpr))
{
SubLink *sublink = (SubLink *) node;
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
ListCell *opid;
foreach(opid, sublink->operOids)
foreach(opid, rcexpr->opnos)
{
if (op_volatile(lfirst_oid(opid)) != PROVOLATILE_IMMUTABLE)
return true;
@@ -734,12 +736,13 @@ contain_volatile_functions_walker(Node *node, void *context)
return true;
/* else fall through to check args */
}
if (IsA(node, SubLink))
if (IsA(node, RowCompareExpr))
{
SubLink *sublink = (SubLink *) node;
/* RowCompare probably can't have volatile ops, but check anyway */
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
ListCell *opid;
foreach(opid, sublink->operOids)
foreach(opid, rcexpr->opnos)
{
if (op_volatile(lfirst_oid(opid)) == PROVOLATILE_VOLATILE)
return true;
@@ -847,6 +850,8 @@ contain_nonstrict_functions_walker(Node *node, void *context)
return true;
if (IsA(node, RowExpr))
return true;
if (IsA(node, RowCompareExpr))
return true;
if (IsA(node, CoalesceExpr))
return true;
if (IsA(node, MinMaxExpr))
@@ -2857,8 +2862,8 @@ evaluate_expr(Expr *expr, Oid result_type)
* FromExpr, JoinExpr, and SetOperationStmt nodes are handled, so that query
* jointrees and setOperation trees can be processed without additional code.
*
* expression_tree_walker will handle SubLink nodes by recursing normally into
* the "lefthand" arguments (which are expressions belonging to the outer
* expression_tree_walker will handle SubLink nodes by recursing normally
* into the "testexpr" subtree (which is an expression belonging to the outer
* plan). It will also call the walker on the sub-Query node; however, when
* expression_tree_walker itself is called on a Query node, it does nothing
* and returns "false". The net effect is that unless the walker does
@@ -2882,7 +2887,7 @@ evaluate_expr(Expr *expr, Oid result_type)
* walker on all the expression subtrees of the given Query node.
*
* expression_tree_walker will handle SubPlan nodes by recursing normally
* into the "exprs" and "args" lists (which are expressions belonging to
* into the "testexpr" and the "args" list (which are expressions belonging to
* the outer plan). It will not touch the completed subplan, however. Since
* there is no link to the original Query, it is not possible to recurse into
* subselects of an already-planned expression tree. This is OK for current
@@ -2992,7 +2997,7 @@ expression_tree_walker(Node *node,
{
SubLink *sublink = (SubLink *) node;
if (expression_tree_walker((Node *) sublink->lefthand,
if (expression_tree_walker(sublink->testexpr,
walker, context))
return true;
@@ -3007,8 +3012,8 @@ expression_tree_walker(Node *node,
{
SubPlan *subplan = (SubPlan *) node;
/* recurse into the exprs list, but not into the Plan */
if (expression_tree_walker((Node *) subplan->exprs,
/* recurse into the testexpr, but not into the Plan */
if (expression_tree_walker(subplan->testexpr,
walker, context))
return true;
/* also examine args list */
@@ -3058,6 +3063,16 @@ expression_tree_walker(Node *node,
return walker(((ArrayExpr *) node)->elements, context);
case T_RowExpr:
return walker(((RowExpr *) node)->args, context);
case T_RowCompareExpr:
{
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
if (walker(rcexpr->largs, context))
return true;
if (walker(rcexpr->rargs, context))
return true;
}
break;
case T_CoalesceExpr:
return walker(((CoalesceExpr *) node)->args, context);
case T_MinMaxExpr:
@@ -3263,7 +3278,7 @@ range_table_walker(List *rtable,
* and qualifier clauses during the planning stage.
*
* expression_tree_mutator will handle SubLink nodes by recursing normally
* into the "lefthand" arguments (which are expressions belonging to the outer
* into the "testexpr" subtree (which is an expression belonging to the outer
* plan). It will also call the mutator on the sub-Query node; however, when
* expression_tree_mutator itself is called on a Query node, it does nothing
* and returns the unmodified Query node. The net effect is that unless the
@@ -3272,8 +3287,8 @@ range_table_walker(List *rtable,
* SubLink node. Mutators that want to descend into sub-selects will usually
* do so by recognizing Query nodes and calling query_tree_mutator (below).
*
* expression_tree_mutator will handle a SubPlan node by recursing into
* the "exprs" and "args" lists (which belong to the outer plan), but it
* expression_tree_mutator will handle a SubPlan node by recursing into the
* "testexpr" and the "args" list (which belong to the outer plan), but it
* will simply copy the link to the inner plan, since that's typically what
* expression tree mutators want. A mutator that wants to modify the subplan
* can force appropriate behavior by recognizing SubPlan expression nodes
@@ -3404,7 +3419,7 @@ expression_tree_mutator(Node *node,
SubLink *newnode;
FLATCOPY(newnode, sublink, SubLink);
MUTATE(newnode->lefthand, sublink->lefthand, List *);
MUTATE(newnode->testexpr, sublink->testexpr, Node *);
/*
* Also invoke the mutator on the sublink's Query node, so it
@@ -3420,8 +3435,8 @@ expression_tree_mutator(Node *node,
SubPlan *newnode;
FLATCOPY(newnode, subplan, SubPlan);
/* transform exprs list */
MUTATE(newnode->exprs, subplan->exprs, List *);
/* transform testexpr */
MUTATE(newnode->testexpr, subplan->testexpr, Node *);
/* transform args list (params to be passed to subplan) */
MUTATE(newnode->args, subplan->args, List *);
/* but not the sub-Plan itself, which is referenced as-is */
@@ -3513,6 +3528,17 @@ expression_tree_mutator(Node *node,
return (Node *) newnode;
}
break;
case T_RowCompareExpr:
{
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
RowCompareExpr *newnode;
FLATCOPY(newnode, rcexpr, RowCompareExpr);
MUTATE(newnode->largs, rcexpr->largs, List *);
MUTATE(newnode->rargs, rcexpr->rargs, List *);
return (Node *) newnode;
}
break;
case T_CoalesceExpr:
{
CoalesceExpr *coalesceexpr = (CoalesceExpr *) node;