1
0
mirror of https://github.com/postgres/postgres.git synced 2025-11-10 17:42:29 +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

@@ -8,7 +8,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/catalog/dependency.c,v 1.48 2005/11/22 18:17:07 momjian Exp $
* $PostgreSQL: pgsql/src/backend/catalog/dependency.c,v 1.49 2005/12/28 01:29:59 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -1129,23 +1129,28 @@ find_expr_references_walker(Node *node,
&context->addrs);
/* fall through to examine arguments */
}
if (IsA(node, SubLink))
{
SubLink *sublink = (SubLink *) node;
ListCell *opid;
foreach(opid, sublink->operOids)
{
add_object_address(OCLASS_OPERATOR, lfirst_oid(opid), 0,
&context->addrs);
}
/* fall through to examine arguments */
}
if (is_subplan(node))
{
/* Extra work needed here if we ever need this case */
elog(ERROR, "already-planned subqueries not supported");
}
if (IsA(node, RowCompareExpr))
{
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
ListCell *l;
foreach(l, rcexpr->opnos)
{
add_object_address(OCLASS_OPERATOR, lfirst_oid(l), 0,
&context->addrs);
}
foreach(l, rcexpr->opclasses)
{
add_object_address(OCLASS_OPCLASS, lfirst_oid(l), 0,
&context->addrs);
}
/* fall through to examine arguments */
}
if (IsA(node, Query))
{
/* Recurse into RTE subquery or not-yet-planned sublink subquery */

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/executor/execQual.c,v 1.186 2005/12/14 16:28:32 tgl Exp $
* $PostgreSQL: pgsql/src/backend/executor/execQual.c,v 1.187 2005/12/28 01:29:59 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -37,6 +37,7 @@
#include "postgres.h"
#include "access/heapam.h"
#include "access/nbtree.h"
#include "catalog/pg_type.h"
#include "commands/typecmds.h"
#include "executor/execdebug.h"
@@ -104,6 +105,9 @@ static Datum ExecEvalArray(ArrayExprState *astate,
static Datum ExecEvalRow(RowExprState *rstate,
ExprContext *econtext,
bool *isNull, ExprDoneCond *isDone);
static Datum ExecEvalRowCompare(RowCompareExprState *rstate,
ExprContext *econtext,
bool *isNull, ExprDoneCond *isDone);
static Datum ExecEvalCoalesce(CoalesceExprState *coalesceExpr,
ExprContext *econtext,
bool *isNull, ExprDoneCond *isDone);
@@ -2306,6 +2310,76 @@ ExecEvalRow(RowExprState *rstate,
return HeapTupleGetDatum(tuple);
}
/* ----------------------------------------------------------------
* ExecEvalRowCompare - ROW() comparison-op ROW()
* ----------------------------------------------------------------
*/
static Datum
ExecEvalRowCompare(RowCompareExprState *rstate,
ExprContext *econtext,
bool *isNull, ExprDoneCond *isDone)
{
bool result;
RowCompareType rctype = ((RowCompareExpr *) rstate->xprstate.expr)->rctype;
int32 cmpresult = 0;
ListCell *l;
ListCell *r;
int i;
if (isDone)
*isDone = ExprSingleResult;
*isNull = true; /* until we get a result */
i = 0;
forboth(l, rstate->largs, r, rstate->rargs)
{
ExprState *le = (ExprState *) lfirst(l);
ExprState *re = (ExprState *) lfirst(r);
FunctionCallInfoData locfcinfo;
InitFunctionCallInfoData(locfcinfo, &(rstate->funcs[i]), 2,
NULL, NULL);
locfcinfo.arg[0] = ExecEvalExpr(le, econtext,
&locfcinfo.argnull[0], NULL);
locfcinfo.arg[1] = ExecEvalExpr(re, econtext,
&locfcinfo.argnull[1], NULL);
if (rstate->funcs[i].fn_strict &&
(locfcinfo.argnull[0] || locfcinfo.argnull[1]))
return (Datum) 0; /* force NULL result */
locfcinfo.isnull = false;
cmpresult = DatumGetInt32(FunctionCallInvoke(&locfcinfo));
if (locfcinfo.isnull)
return (Datum) 0; /* force NULL result */
if (cmpresult != 0)
break; /* no need to compare remaining columns */
i++;
}
switch (rctype)
{
/* EQ and NE cases aren't allowed here */
case ROWCOMPARE_LT:
result = (cmpresult < 0);
break;
case ROWCOMPARE_LE:
result = (cmpresult <= 0);
break;
case ROWCOMPARE_GE:
result = (cmpresult >= 0);
break;
case ROWCOMPARE_GT:
result = (cmpresult > 0);
break;
default:
elog(ERROR, "unrecognized RowCompareType: %d", (int) rctype);
result = 0; /* keep compiler quiet */
break;
}
*isNull = false;
return BoolGetDatum(result);
}
/* ----------------------------------------------------------------
* ExecEvalCoalesce
* ----------------------------------------------------------------
@@ -3118,8 +3192,8 @@ ExecInitExpr(Expr *node, PlanState *parent)
sstate->sub_estate = NULL;
sstate->planstate = NULL;
sstate->exprs = (List *)
ExecInitExpr((Expr *) subplan->exprs, parent);
sstate->testexpr =
ExecInitExpr((Expr *) subplan->testexpr, parent);
sstate->args = (List *)
ExecInitExpr((Expr *) subplan->args, parent);
@@ -3336,6 +3410,66 @@ ExecInitExpr(Expr *node, PlanState *parent)
state = (ExprState *) rstate;
}
break;
case T_RowCompareExpr:
{
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
RowCompareExprState *rstate = makeNode(RowCompareExprState);
int nopers = list_length(rcexpr->opnos);
List *outlist;
ListCell *l;
ListCell *l2;
int i;
rstate->xprstate.evalfunc = (ExprStateEvalFunc) ExecEvalRowCompare;
Assert(list_length(rcexpr->largs) == nopers);
outlist = NIL;
foreach(l, rcexpr->largs)
{
Expr *e = (Expr *) lfirst(l);
ExprState *estate;
estate = ExecInitExpr(e, parent);
outlist = lappend(outlist, estate);
}
rstate->largs = outlist;
Assert(list_length(rcexpr->rargs) == nopers);
outlist = NIL;
foreach(l, rcexpr->rargs)
{
Expr *e = (Expr *) lfirst(l);
ExprState *estate;
estate = ExecInitExpr(e, parent);
outlist = lappend(outlist, estate);
}
rstate->rargs = outlist;
Assert(list_length(rcexpr->opclasses) == nopers);
rstate->funcs = (FmgrInfo *) palloc(nopers * sizeof(FmgrInfo));
i = 0;
forboth(l, rcexpr->opnos, l2, rcexpr->opclasses)
{
Oid opno = lfirst_oid(l);
Oid opclass = lfirst_oid(l2);
int strategy;
Oid subtype;
bool recheck;
Oid proc;
get_op_opclass_properties(opno, opclass,
&strategy, &subtype, &recheck);
proc = get_opclass_proc(opclass, subtype, BTORDER_PROC);
/*
* If we enforced permissions checks on index support
* functions, we'd need to make a check here. But the
* index support machinery doesn't do that, and neither
* does this code.
*/
fmgr_info(proc, &(rstate->funcs[i]));
i++;
}
state = (ExprState *) rstate;
}
break;
case T_CoalesceExpr:
{
CoalesceExpr *coalesceexpr = (CoalesceExpr *) node;
@@ -3382,6 +3516,12 @@ ExecInitExpr(Expr *node, PlanState *parent)
(errcode(ERRCODE_UNDEFINED_FUNCTION),
errmsg("could not identify a comparison function for type %s",
format_type_be(minmaxexpr->minmaxtype))));
/*
* If we enforced permissions checks on index support
* functions, we'd need to make a check here. But the
* index support machinery doesn't do that, and neither
* does this code.
*/
fmgr_info(typentry->cmp_proc, &(mstate->cfunc));
state = (ExprState *) mstate;
}
@@ -3484,7 +3624,7 @@ ExecInitExprInitPlan(SubPlan *node, PlanState *parent)
sstate->sub_estate = NULL;
sstate->planstate = NULL;
sstate->exprs = (List *) ExecInitExpr((Expr *) node->exprs, parent);
sstate->testexpr = ExecInitExpr((Expr *) node->testexpr, parent);
sstate->args = (List *) ExecInitExpr((Expr *) node->args, parent);
sstate->xprstate.expr = (Expr *) node;

View File

@@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/executor/nodeSubplan.c,v 1.71 2005/11/22 18:17:10 momjian Exp $
* $PostgreSQL: pgsql/src/backend/executor/nodeSubplan.c,v 1.72 2005/12/28 01:29:59 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -23,6 +23,7 @@
#include "executor/executor.h"
#include "executor/nodeSubplan.h"
#include "nodes/makefuncs.h"
#include "optimizer/clauses.h"
#include "parser/parse_expr.h"
#include "utils/array.h"
#include "utils/datum.h"
@@ -205,7 +206,6 @@ ExecScanSubPlan(SubPlanState *node,
SubPlan *subplan = (SubPlan *) node->xprstate.expr;
PlanState *planstate = node->planstate;
SubLinkType subLinkType = subplan->subLinkType;
bool useOr = subplan->useOr;
MemoryContext oldcontext;
TupleTableSlot *slot;
Datum result;
@@ -245,15 +245,13 @@ ExecScanSubPlan(SubPlanState *node,
/*
* For all sublink types except EXPR_SUBLINK and ARRAY_SUBLINK, the result
* is boolean as are the results of the combining operators. We combine
* results within a tuple (if there are multiple columns) using OR
* semantics if "useOr" is true, AND semantics if not. We then combine
* results across tuples (if the subplan produces more than one) using OR
* semantics for ANY_SUBLINK or AND semantics for ALL_SUBLINK.
* (MULTIEXPR_SUBLINK doesn't allow multiple tuples from the subplan.)
* (ROWCOMPARE_SUBLINK doesn't allow multiple tuples from the subplan.)
* NULL results from the combining operators are handled according to the
* usual SQL semantics for OR and AND. The result for no input tuples is
* FALSE for ANY_SUBLINK, TRUE for ALL_SUBLINK, NULL for
* MULTIEXPR_SUBLINK.
* ROWCOMPARE_SUBLINK.
*
* For EXPR_SUBLINK we require the subplan to produce no more than one
* tuple, else an error is raised. For ARRAY_SUBLINK we allow the subplan
@@ -269,9 +267,9 @@ ExecScanSubPlan(SubPlanState *node,
slot = ExecProcNode(planstate))
{
TupleDesc tdesc = slot->tts_tupleDescriptor;
Datum rowresult = BoolGetDatum(!useOr);
bool rownull = false;
int col = 1;
Datum rowresult;
bool rownull;
int col;
ListCell *plst;
if (subLinkType == EXISTS_SUBLINK)
@@ -304,7 +302,7 @@ ExecScanSubPlan(SubPlanState *node,
node->curTuple = ExecCopySlotTuple(slot);
MemoryContextSwitchTo(node->sub_estate->es_query_cxt);
result = heap_getattr(node->curTuple, col, tdesc, isNull);
result = heap_getattr(node->curTuple, 1, tdesc, isNull);
/* keep scanning subplan to make sure there's only one tuple */
continue;
}
@@ -324,8 +322,8 @@ ExecScanSubPlan(SubPlanState *node,
continue;
}
/* cannot allow multiple input tuples for MULTIEXPR sublink either */
if (subLinkType == MULTIEXPR_SUBLINK && found)
/* cannot allow multiple input tuples for ROWCOMPARE sublink either */
if (subLinkType == ROWCOMPARE_SUBLINK && found)
ereport(ERROR,
(errcode(ERRCODE_CARDINALITY_VIOLATION),
errmsg("more than one row returned by a subquery used as an expression")));
@@ -333,69 +331,25 @@ ExecScanSubPlan(SubPlanState *node,
found = true;
/*
* For ALL, ANY, and MULTIEXPR sublinks, iterate over combining
* operators for columns of tuple.
* For ALL, ANY, and ROWCOMPARE sublinks, load up the Params
* representing the columns of the sub-select, and then evaluate
* the combining expression.
*/
Assert(list_length(node->exprs) == list_length(subplan->paramIds));
forboth(l, node->exprs, plst, subplan->paramIds)
col = 1;
foreach(plst, subplan->paramIds)
{
ExprState *exprstate = (ExprState *) lfirst(l);
int paramid = lfirst_int(plst);
ParamExecData *prmdata;
Datum expresult;
bool expnull;
/*
* Load up the Param representing this column of the sub-select.
*/
prmdata = &(econtext->ecxt_param_exec_vals[paramid]);
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col,
&(prmdata->isnull));
/*
* Now we can eval the combining operator for this column.
*/
expresult = ExecEvalExprSwitchContext(exprstate, econtext,
&expnull, NULL);
/*
* Combine the result into the row result as appropriate.
*/
if (col == 1)
{
rowresult = expresult;
rownull = expnull;
}
else if (useOr)
{
/* combine within row per OR semantics */
if (expnull)
rownull = true;
else if (DatumGetBool(expresult))
{
rowresult = BoolGetDatum(true);
rownull = false;
break; /* needn't look at any more columns */
}
}
else
{
/* combine within row per AND semantics */
if (expnull)
rownull = true;
else if (!DatumGetBool(expresult))
{
rowresult = BoolGetDatum(false);
rownull = false;
break; /* needn't look at any more columns */
}
}
prmdata->value = slot_getattr(slot, col, &(prmdata->isnull));
col++;
}
rowresult = ExecEvalExprSwitchContext(node->testexpr, econtext,
&rownull, NULL);
if (subLinkType == ANY_SUBLINK)
{
/* combine across rows per OR semantics */
@@ -422,7 +376,7 @@ ExecScanSubPlan(SubPlanState *node,
}
else
{
/* must be MULTIEXPR_SUBLINK */
/* must be ROWCOMPARE_SUBLINK */
result = rowresult;
*isNull = rownull;
}
@@ -433,11 +387,11 @@ ExecScanSubPlan(SubPlanState *node,
/*
* deal with empty subplan result. result/isNull were previously
* initialized correctly for all sublink types except EXPR, ARRAY, and
* MULTIEXPR; for those, return NULL.
* ROWCOMPARE; for those, return NULL.
*/
if (subLinkType == EXPR_SUBLINK ||
subLinkType == ARRAY_SUBLINK ||
subLinkType == MULTIEXPR_SUBLINK)
subLinkType == ROWCOMPARE_SUBLINK)
{
result = (Datum) 0;
*isNull = true;
@@ -463,7 +417,7 @@ buildSubPlanHash(SubPlanState *node)
{
SubPlan *subplan = (SubPlan *) node->xprstate.expr;
PlanState *planstate = node->planstate;
int ncols = list_length(node->exprs);
int ncols = list_length(subplan->paramIds);
ExprContext *innerecontext = node->innerecontext;
MemoryContext tempcxt = innerecontext->ecxt_per_tuple_memory;
MemoryContext oldcontext;
@@ -471,7 +425,6 @@ buildSubPlanHash(SubPlanState *node)
TupleTableSlot *slot;
Assert(subplan->subLinkType == ANY_SUBLINK);
Assert(!subplan->useOr);
/*
* If we already had any hash tables, destroy 'em; then create empty hash
@@ -764,11 +717,12 @@ ExecInitSubPlan(SubPlanState *node, EState *estate)
TupleDesc tupDesc;
TupleTable tupTable;
TupleTableSlot *slot;
List *lefttlist,
List *oplist,
*lefttlist,
*righttlist,
*leftptlist,
*rightptlist;
ListCell *lexpr;
ListCell *l;
/* We need a memory context to hold the hash table(s) */
node->tablecxt =
@@ -780,7 +734,7 @@ ExecInitSubPlan(SubPlanState *node, EState *estate)
/* and a short-lived exprcontext for function evaluation */
node->innerecontext = CreateExprContext(estate);
/* Silly little array of column numbers 1..n */
ncols = list_length(node->exprs);
ncols = list_length(subplan->paramIds);
node->keyColIdx = (AttrNumber *) palloc(ncols * sizeof(AttrNumber));
for (i = 0; i < ncols; i++)
node->keyColIdx[i] = i + 1;
@@ -799,14 +753,34 @@ ExecInitSubPlan(SubPlanState *node, EState *estate)
* We also extract the combining operators themselves to initialize
* the equality and hashing functions for the hash tables.
*/
if (IsA(node->testexpr->expr, OpExpr))
{
/* single combining operator */
oplist = list_make1(node->testexpr);
}
else if (and_clause((Node *) node->testexpr->expr))
{
/* multiple combining operators */
Assert(IsA(node->testexpr, BoolExprState));
oplist = ((BoolExprState *) node->testexpr)->args;
}
else
{
/* shouldn't see anything else in a hashable subplan */
elog(ERROR, "unrecognized testexpr type: %d",
(int) nodeTag(node->testexpr->expr));
oplist = NIL; /* keep compiler quiet */
}
Assert(list_length(oplist) == ncols);
lefttlist = righttlist = NIL;
leftptlist = rightptlist = NIL;
node->eqfunctions = (FmgrInfo *) palloc(ncols * sizeof(FmgrInfo));
node->hashfunctions = (FmgrInfo *) palloc(ncols * sizeof(FmgrInfo));
i = 1;
foreach(lexpr, node->exprs)
foreach(l, oplist)
{
FuncExprState *fstate = (FuncExprState *) lfirst(lexpr);
FuncExprState *fstate = (FuncExprState *) lfirst(l);
OpExpr *opexpr = (OpExpr *) fstate->xprstate.expr;
ExprState *exstate;
Expr *expr;
@@ -967,7 +941,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
if (found &&
(subLinkType == EXPR_SUBLINK ||
subLinkType == MULTIEXPR_SUBLINK))
subLinkType == ROWCOMPARE_SUBLINK))
ereport(ERROR,
(errcode(ERRCODE_CARDINALITY_VIOLATION),
errmsg("more than one row returned by a subquery used as an expression")));

View File

@@ -15,7 +15,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/nodes/copyfuncs.c,v 1.323 2005/12/20 02:30:35 tgl Exp $
* $PostgreSQL: pgsql/src/backend/nodes/copyfuncs.c,v 1.324 2005/12/28 01:29:59 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -862,10 +862,8 @@ _copySubLink(SubLink *from)
SubLink *newnode = makeNode(SubLink);
COPY_SCALAR_FIELD(subLinkType);
COPY_SCALAR_FIELD(useOr);
COPY_NODE_FIELD(lefthand);
COPY_NODE_FIELD(testexpr);
COPY_NODE_FIELD(operName);
COPY_NODE_FIELD(operOids);
COPY_NODE_FIELD(subselect);
return newnode;
@@ -880,8 +878,7 @@ _copySubPlan(SubPlan *from)
SubPlan *newnode = makeNode(SubPlan);
COPY_SCALAR_FIELD(subLinkType);
COPY_SCALAR_FIELD(useOr);
COPY_NODE_FIELD(exprs);
COPY_NODE_FIELD(testexpr);
COPY_NODE_FIELD(paramIds);
COPY_NODE_FIELD(plan);
COPY_SCALAR_FIELD(plan_id);
@@ -1033,6 +1030,23 @@ _copyRowExpr(RowExpr *from)
return newnode;
}
/*
* _copyRowCompareExpr
*/
static RowCompareExpr *
_copyRowCompareExpr(RowCompareExpr *from)
{
RowCompareExpr *newnode = makeNode(RowCompareExpr);
COPY_SCALAR_FIELD(rctype);
COPY_NODE_FIELD(opnos);
COPY_NODE_FIELD(opclasses);
COPY_NODE_FIELD(largs);
COPY_NODE_FIELD(rargs);
return newnode;
}
/*
* _copyCoalesceExpr
*/
@@ -2876,6 +2890,9 @@ copyObject(void *from)
case T_RowExpr:
retval = _copyRowExpr(from);
break;
case T_RowCompareExpr:
retval = _copyRowCompareExpr(from);
break;
case T_CoalesceExpr:
retval = _copyCoalesceExpr(from);
break;

View File

@@ -18,7 +18,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/nodes/equalfuncs.c,v 1.259 2005/12/20 02:30:35 tgl Exp $
* $PostgreSQL: pgsql/src/backend/nodes/equalfuncs.c,v 1.260 2005/12/28 01:29:59 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -156,6 +156,7 @@ _equalParam(Param *a, Param *b)
break;
case PARAM_NUM:
case PARAM_EXEC:
case PARAM_SUBLINK:
COMPARE_SCALAR_FIELD(paramid);
break;
default:
@@ -295,10 +296,8 @@ static bool
_equalSubLink(SubLink *a, SubLink *b)
{
COMPARE_SCALAR_FIELD(subLinkType);
COMPARE_SCALAR_FIELD(useOr);
COMPARE_NODE_FIELD(lefthand);
COMPARE_NODE_FIELD(testexpr);
COMPARE_NODE_FIELD(operName);
COMPARE_NODE_FIELD(operOids);
COMPARE_NODE_FIELD(subselect);
return true;
@@ -308,8 +307,7 @@ static bool
_equalSubPlan(SubPlan *a, SubPlan *b)
{
COMPARE_SCALAR_FIELD(subLinkType);
COMPARE_SCALAR_FIELD(useOr);
COMPARE_NODE_FIELD(exprs);
COMPARE_NODE_FIELD(testexpr);
COMPARE_NODE_FIELD(paramIds);
/* should compare plans, but have to settle for comparing plan IDs */
COMPARE_SCALAR_FIELD(plan_id);
@@ -440,6 +438,18 @@ _equalRowExpr(RowExpr *a, RowExpr *b)
return true;
}
static bool
_equalRowCompareExpr(RowCompareExpr *a, RowCompareExpr *b)
{
COMPARE_SCALAR_FIELD(rctype);
COMPARE_NODE_FIELD(opnos);
COMPARE_NODE_FIELD(opclasses);
COMPARE_NODE_FIELD(largs);
COMPARE_NODE_FIELD(rargs);
return true;
}
static bool
_equalCoalesceExpr(CoalesceExpr *a, CoalesceExpr *b)
{
@@ -1919,6 +1929,9 @@ equal(void *a, void *b)
case T_RowExpr:
retval = _equalRowExpr(a, b);
break;
case T_RowCompareExpr:
retval = _equalRowCompareExpr(a, b);
break;
case T_CoalesceExpr:
retval = _equalCoalesceExpr(a, b);
break;

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/nodes/outfuncs.c,v 1.265 2005/12/20 02:30:35 tgl Exp $
* $PostgreSQL: pgsql/src/backend/nodes/outfuncs.c,v 1.266 2005/12/28 01:29:59 tgl Exp $
*
* NOTES
* Every node type that can appear in stored rules' parsetrees *must*
@@ -736,10 +736,8 @@ _outSubLink(StringInfo str, SubLink *node)
WRITE_NODE_TYPE("SUBLINK");
WRITE_ENUM_FIELD(subLinkType, SubLinkType);
WRITE_BOOL_FIELD(useOr);
WRITE_NODE_FIELD(lefthand);
WRITE_NODE_FIELD(testexpr);
WRITE_NODE_FIELD(operName);
WRITE_NODE_FIELD(operOids);
WRITE_NODE_FIELD(subselect);
}
@@ -749,8 +747,7 @@ _outSubPlan(StringInfo str, SubPlan *node)
WRITE_NODE_TYPE("SUBPLAN");
WRITE_ENUM_FIELD(subLinkType, SubLinkType);
WRITE_BOOL_FIELD(useOr);
WRITE_NODE_FIELD(exprs);
WRITE_NODE_FIELD(testexpr);
WRITE_NODE_FIELD(paramIds);
WRITE_NODE_FIELD(plan);
WRITE_INT_FIELD(plan_id);
@@ -855,6 +852,18 @@ _outRowExpr(StringInfo str, RowExpr *node)
WRITE_ENUM_FIELD(row_format, CoercionForm);
}
static void
_outRowCompareExpr(StringInfo str, RowCompareExpr *node)
{
WRITE_NODE_TYPE("ROWCOMPARE");
WRITE_ENUM_FIELD(rctype, RowCompareType);
WRITE_NODE_FIELD(opnos);
WRITE_NODE_FIELD(opclasses);
WRITE_NODE_FIELD(largs);
WRITE_NODE_FIELD(rargs);
}
static void
_outCoalesceExpr(StringInfo str, CoalesceExpr *node)
{
@@ -1936,6 +1945,9 @@ _outNode(StringInfo str, void *obj)
case T_RowExpr:
_outRowExpr(str, obj);
break;
case T_RowCompareExpr:
_outRowCompareExpr(str, obj);
break;
case T_CoalesceExpr:
_outCoalesceExpr(str, obj);
break;

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/nodes/readfuncs.c,v 1.182 2005/10/15 02:49:19 momjian Exp $
* $PostgreSQL: pgsql/src/backend/nodes/readfuncs.c,v 1.183 2005/12/28 01:29:59 tgl Exp $
*
* NOTES
* Path and Plan nodes do not have any readfuncs support, because we
@@ -494,10 +494,8 @@ _readSubLink(void)
READ_LOCALS(SubLink);
READ_ENUM_FIELD(subLinkType, SubLinkType);
READ_BOOL_FIELD(useOr);
READ_NODE_FIELD(lefthand);
READ_NODE_FIELD(testexpr);
READ_NODE_FIELD(operName);
READ_NODE_FIELD(operOids);
READ_NODE_FIELD(subselect);
READ_DONE();
@@ -645,6 +643,23 @@ _readRowExpr(void)
READ_DONE();
}
/*
* _readRowCompareExpr
*/
static RowCompareExpr *
_readRowCompareExpr(void)
{
READ_LOCALS(RowCompareExpr);
READ_ENUM_FIELD(rctype, RowCompareType);
READ_NODE_FIELD(opnos);
READ_NODE_FIELD(opclasses);
READ_NODE_FIELD(largs);
READ_NODE_FIELD(rargs);
READ_DONE();
}
/*
* _readCoalesceExpr
*/
@@ -996,6 +1011,8 @@ parseNodeString(void)
return_value = _readArrayExpr();
else if (MATCH("ROW", 3))
return_value = _readRowExpr();
else if (MATCH("ROWCOMPARE", 10))
return_value = _readRowCompareExpr();
else if (MATCH("COALESCE", 8))
return_value = _readCoalesceExpr();
else if (MATCH("MINMAX", 6))

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;

View File

@@ -11,7 +11,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/parser/gram.y,v 2.519 2005/12/27 04:00:07 momjian Exp $
* $PostgreSQL: pgsql/src/backend/parser/gram.y,v 2.520 2005/12/28 01:30:00 tgl Exp $
*
* HISTORY
* AUTHOR DATE MAJOR EVENT
@@ -6774,10 +6774,7 @@ a_expr: c_expr { $$ = $1; }
/* generate foo = ANY (subquery) */
SubLink *n = (SubLink *) $3;
n->subLinkType = ANY_SUBLINK;
if (IsA($1, RowExpr))
n->lefthand = ((RowExpr *) $1)->args;
else
n->lefthand = list_make1($1);
n->testexpr = $1;
n->operName = list_make1(makeString("="));
$$ = (Node *)n;
}
@@ -6796,10 +6793,7 @@ a_expr: c_expr { $$ = $1; }
/* Make an = ANY node */
SubLink *n = (SubLink *) $4;
n->subLinkType = ANY_SUBLINK;
if (IsA($1, RowExpr))
n->lefthand = ((RowExpr *) $1)->args;
else
n->lefthand = list_make1($1);
n->testexpr = $1;
n->operName = list_make1(makeString("="));
/* Stick a NOT on top */
$$ = (Node *) makeA_Expr(AEXPR_NOT, NIL, NULL, (Node *) n);
@@ -6814,10 +6808,7 @@ a_expr: c_expr { $$ = $1; }
{
SubLink *n = makeNode(SubLink);
n->subLinkType = $3;
if (IsA($1, RowExpr))
n->lefthand = ((RowExpr *) $1)->args;
else
n->lefthand = list_make1($1);
n->testexpr = $1;
n->operName = $2;
n->subselect = $4;
$$ = (Node *)n;
@@ -6950,7 +6941,7 @@ c_expr: columnref { $$ = $1; }
{
SubLink *n = makeNode(SubLink);
n->subLinkType = EXPR_SUBLINK;
n->lefthand = NIL;
n->testexpr = NULL;
n->operName = NIL;
n->subselect = $1;
$$ = (Node *)n;
@@ -6959,7 +6950,7 @@ c_expr: columnref { $$ = $1; }
{
SubLink *n = makeNode(SubLink);
n->subLinkType = EXISTS_SUBLINK;
n->lefthand = NIL;
n->testexpr = NULL;
n->operName = NIL;
n->subselect = $2;
$$ = (Node *)n;
@@ -6968,7 +6959,7 @@ c_expr: columnref { $$ = $1; }
{
SubLink *n = makeNode(SubLink);
n->subLinkType = ARRAY_SUBLINK;
n->lefthand = NIL;
n->testexpr = NULL;
n->operName = NIL;
n->subselect = $2;
$$ = (Node *)n;

View File

@@ -8,21 +8,20 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/parser/parse_expr.c,v 1.188 2005/11/28 04:35:31 tgl Exp $
* $PostgreSQL: pgsql/src/backend/parser/parse_expr.c,v 1.189 2005/12/28 01:30:00 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_proc.h"
#include "commands/dbcommands.h"
#include "mb/pg_wchar.h"
#include "miscadmin.h"
#include "nodes/makefuncs.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
#include "optimizer/clauses.h"
#include "parser/analyze.h"
#include "parser/gramparse.h"
#include "parser/parse_coerce.h"
@@ -33,7 +32,7 @@
#include "parser/parse_type.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
bool Transform_null_equals = false;
@@ -64,8 +63,8 @@ static Node *transformIndirection(ParseState *pstate, Node *basenode,
List *indirection);
static Node *typecast_expression(ParseState *pstate, Node *expr,
TypeName *typename);
static Node *make_row_op(ParseState *pstate, List *opname,
RowExpr *lrow, RowExpr *rrow);
static Node *make_row_comparison_op(ParseState *pstate, List *opname,
List *largs, List *rargs);
static Node *make_row_distinct_op(ParseState *pstate, List *opname,
RowExpr *lrow, RowExpr *rrow);
static Expr *make_distinct_op(ParseState *pstate, List *opname,
@@ -592,14 +591,14 @@ transformAExprOp(ParseState *pstate, A_Expr *a)
((SubLink *) rexpr)->subLinkType == EXPR_SUBLINK)
{
/*
* Convert "row op subselect" into a MULTIEXPR sublink. Formerly the
* Convert "row op subselect" into a ROWCOMPARE sublink. Formerly the
* grammar did this, but now that a row construct is allowed anywhere
* in expressions, it's easier to do it here.
*/
SubLink *s = (SubLink *) rexpr;
s->subLinkType = MULTIEXPR_SUBLINK;
s->lefthand = ((RowExpr *) lexpr)->args;
s->subLinkType = ROWCOMPARE_SUBLINK;
s->testexpr = lexpr;
s->operName = a->name;
result = transformExpr(pstate, (Node *) s);
}
@@ -612,10 +611,10 @@ transformAExprOp(ParseState *pstate, A_Expr *a)
Assert(IsA(lexpr, RowExpr));
Assert(IsA(rexpr, RowExpr));
result = make_row_op(pstate,
a->name,
(RowExpr *) lexpr,
(RowExpr *) rexpr);
result = make_row_comparison_op(pstate,
a->name,
((RowExpr *) lexpr)->args,
((RowExpr *) rexpr)->args);
}
else
{
@@ -885,10 +884,10 @@ transformAExprIn(ParseState *pstate, A_Expr *a)
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("arguments of row IN must all be row expressions")));
cmp = make_row_op(pstate,
a->name,
(RowExpr *) copyObject(lexpr),
(RowExpr *) rexpr);
cmp = make_row_comparison_op(pstate,
a->name,
(List *) copyObject(((RowExpr *) lexpr)->args),
((RowExpr *) rexpr)->args);
}
else
cmp = (Node *) make_op(pstate,
@@ -1080,13 +1079,11 @@ transformSubLink(ParseState *pstate, SubLink *sublink)
if (sublink->subLinkType == EXISTS_SUBLINK)
{
/*
* EXISTS needs no lefthand or combining operator. These fields
* should be NIL already, but make sure.
* EXISTS needs no test expression or combining operator.
* These fields should be null already, but make sure.
*/
sublink->lefthand = NIL;
sublink->testexpr = NULL;
sublink->operName = NIL;
sublink->operOids = NIL;
sublink->useOr = FALSE;
}
else if (sublink->subLinkType == EXPR_SUBLINK ||
sublink->subLinkType == ARRAY_SUBLINK)
@@ -1111,128 +1108,72 @@ transformSubLink(ParseState *pstate, SubLink *sublink)
}
/*
* EXPR and ARRAY need no lefthand or combining operator. These fields
* should be NIL already, but make sure.
* EXPR and ARRAY need no test expression or combining operator.
* These fields should be null already, but make sure.
*/
sublink->lefthand = NIL;
sublink->testexpr = NULL;
sublink->operName = NIL;
sublink->operOids = NIL;
sublink->useOr = FALSE;
}
else
{
/* ALL, ANY, or MULTIEXPR: generate operator list */
List *left_list = sublink->lefthand;
List *right_list = qtree->targetList;
int row_length = list_length(left_list);
bool needNot = false;
List *op = sublink->operName;
char *opname = strVal(llast(op));
/* ALL, ANY, or ROWCOMPARE: generate row-comparing expression */
Node *lefthand;
List *left_list;
List *right_list;
ListCell *l;
ListCell *ll_item;
/* transform lefthand expressions */
foreach(l, left_list)
lfirst(l) = transformExpr(pstate, lfirst(l));
/*
* If the expression is "<> ALL" (with unqualified opname) then
* convert it to "NOT IN". This is a hack to improve efficiency of
* expressions output by pre-7.4 Postgres.
* Transform lefthand expression, and convert to a list
*/
if (sublink->subLinkType == ALL_SUBLINK &&
list_length(op) == 1 && strcmp(opname, "<>") == 0)
{
sublink->subLinkType = ANY_SUBLINK;
opname = pstrdup("=");
op = list_make1(makeString(opname));
sublink->operName = op;
needNot = true;
}
/* Set useOr if op is "<>" (possibly qualified) */
if (strcmp(opname, "<>") == 0)
sublink->useOr = TRUE;
lefthand = transformExpr(pstate, sublink->testexpr);
if (lefthand && IsA(lefthand, RowExpr))
left_list = ((RowExpr *) lefthand)->args;
else
sublink->useOr = FALSE;
/* Combining operators other than =/<> is dubious... */
if (row_length != 1 &&
strcmp(opname, "=") != 0 &&
strcmp(opname, "<>") != 0)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("row comparison cannot use operator %s",
opname)));
left_list = list_make1(lefthand);
/*
* To build the list of combining operator OIDs, we must scan
* subquery's targetlist to find values that will be matched against
* lefthand values. We need to ignore resjunk targets, so doing the
* outer iteration over right_list is easier than doing it over
* left_list.
* Build a list of PARAM_SUBLINK nodes representing the
* output columns of the subquery.
*/
sublink->operOids = NIL;
ll_item = list_head(left_list);
foreach(l, right_list)
right_list = NIL;
foreach(l, qtree->targetList)
{
TargetEntry *tent = (TargetEntry *) lfirst(l);
Node *lexpr;
Operator optup;
Form_pg_operator opform;
Param *param;
if (tent->resjunk)
continue;
if (ll_item == NULL)
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("subquery has too many columns")));
lexpr = lfirst(ll_item);
ll_item = lnext(ll_item);
param = makeNode(Param);
param->paramkind = PARAM_SUBLINK;
param->paramid = (AttrNumber) tent->resno;
param->paramtype = exprType((Node *) tent->expr);
/*
* It's OK to use oper() not compatible_oper() here, because
* make_subplan() will insert type coercion calls if needed.
*/
optup = oper(op,
exprType(lexpr),
exprType((Node *) tent->expr),
false);
opform = (Form_pg_operator) GETSTRUCT(optup);
if (opform->oprresult != BOOLOID)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
errmsg("operator %s must return type boolean, not type %s",
opname,
format_type_be(opform->oprresult)),
errhint("The operator of a quantified predicate subquery must return type boolean.")));
if (get_func_retset(opform->oprcode))
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
errmsg("operator %s must not return a set",
opname),
errhint("The operator of a quantified predicate subquery must return type boolean.")));
sublink->operOids = lappend_oid(sublink->operOids,
oprid(optup));
ReleaseSysCache(optup);
right_list = lappend(right_list, param);
}
if (ll_item != NULL)
/*
* We could rely on make_row_comparison_op to complain if the
* list lengths differ, but we prefer to generate a more specific
* error message.
*/
if (list_length(left_list) < list_length(right_list))
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("subquery has too many columns")));
if (list_length(left_list) > list_length(right_list))
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("subquery has too few columns")));
if (needNot)
{
result = coerce_to_boolean(pstate, result, "NOT");
result = (Node *) makeBoolExpr(NOT_EXPR,
list_make1(result));
}
/*
* Identify the combining operator(s) and generate a suitable
* row-comparison expression.
*/
sublink->testexpr = make_row_comparison_op(pstate,
sublink->operName,
left_list,
right_list);
}
return result;
@@ -1673,6 +1614,9 @@ exprType(Node *expr)
case T_RowExpr:
type = ((RowExpr *) expr)->row_typeid;
break;
case T_RowCompareExpr:
type = BOOLOID;
break;
case T_CoalesceExpr:
type = ((CoalesceExpr *) expr)->coalescetype;
break;
@@ -1953,76 +1897,258 @@ typecast_expression(ParseState *pstate, Node *expr, TypeName *typename)
}
/*
* Transform a "row op row" construct
* Transform a "row compare-op row" construct
*
* The input RowExprs are already transformed
* The inputs are lists of already-transformed expressions.
* As with coerce_type, pstate may be NULL if no special unknown-Param
* processing is wanted.
*
* The output may be a single OpExpr, an AND or OR combination of OpExprs,
* or a RowCompareExpr. In all cases it is guaranteed to return boolean.
* The AND, OR, and RowCompareExpr cases further imply things about the
* behavior of the operators (ie, they behave as =, <>, or < <= > >=).
*/
static Node *
make_row_op(ParseState *pstate, List *opname,
RowExpr *lrow, RowExpr *rrow)
make_row_comparison_op(ParseState *pstate, List *opname,
List *largs, List *rargs)
{
Node *result = NULL;
List *largs = lrow->args;
List *rargs = rrow->args;
RowCompareExpr *rcexpr;
RowCompareType rctype;
List *opexprs;
List *opnos;
List *opclasses;
ListCell *l,
*r;
char *oprname;
BoolExprType boolop;
List **opclass_lists;
List **opstrat_lists;
Bitmapset *strats;
int nopers;
int i;
if (list_length(largs) != list_length(rargs))
nopers = list_length(largs);
if (nopers != list_length(rargs))
ereport(ERROR,
(errcode(ERRCODE_SYNTAX_ERROR),
errmsg("unequal number of entries in row expressions")));
/*
* XXX it's really wrong to generate a simple AND combination for < <= >
* >=. We probably need to invent a new runtime node type to handle those
* correctly. For the moment, though, keep on doing this ...
* We can't compare zero-length rows because there is no principled
* basis for figuring out what the operator is.
*/
oprname = strVal(llast(opname));
if ((strcmp(oprname, "=") == 0) ||
(strcmp(oprname, "<") == 0) ||
(strcmp(oprname, "<=") == 0) ||
(strcmp(oprname, ">") == 0) ||
(strcmp(oprname, ">=") == 0))
boolop = AND_EXPR;
else if (strcmp(oprname, "<>") == 0)
boolop = OR_EXPR;
else
{
if (nopers == 0)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("operator %s is not supported for row expressions",
oprname)));
boolop = 0; /* keep compiler quiet */
}
errmsg("cannot compare rows of zero length")));
/*
* Identify all the pairwise operators, using make_op so that
* behavior is the same as in the simple scalar case.
*/
opexprs = NIL;
forboth(l, largs, r, rargs)
{
Node *larg = (Node *) lfirst(l);
Node *rarg = (Node *) lfirst(r);
Node *cmp;
OpExpr *cmp;
cmp = (Node *) make_op(pstate, opname, larg, rarg);
cmp = coerce_to_boolean(pstate, cmp, "row comparison");
if (result == NULL)
result = cmp;
else
result = (Node *) makeBoolExpr(boolop,
list_make2(result, cmp));
cmp = (OpExpr *) make_op(pstate, opname, larg, rarg);
Assert(IsA(cmp, OpExpr));
/*
* We don't use coerce_to_boolean here because we insist on the
* operator yielding boolean directly, not via coercion. If it
* doesn't yield bool it won't be in any index opclasses...
*/
if (cmp->opresulttype != BOOLOID)
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
errmsg("row comparison operator must yield type boolean, "
"not type %s",
format_type_be(cmp->opresulttype))));
if (expression_returns_set((Node *) cmp))
ereport(ERROR,
(errcode(ERRCODE_DATATYPE_MISMATCH),
errmsg("row comparison operator must not return a set")));
opexprs = lappend(opexprs, cmp);
}
if (result == NULL)
/*
* If rows are length 1, just return the single operator. In this
* case we don't insist on identifying btree semantics for the operator
* (but we still require it to return boolean).
*/
if (nopers == 1)
return (Node *) linitial(opexprs);
/*
* Now we must determine which row comparison semantics (= <> < <= > >=)
* apply to this set of operators. We look for btree opclasses containing
* the operators, and see which interpretations (strategy numbers) exist
* for each operator.
*/
opclass_lists = (List **) palloc(nopers * sizeof(List *));
opstrat_lists = (List **) palloc(nopers * sizeof(List *));
strats = NULL;
i = 0;
foreach(l, opexprs)
{
/* zero-length rows? Generate constant TRUE or FALSE */
if (boolop == AND_EXPR)
result = makeBoolConst(true, false);
Bitmapset *this_strats;
ListCell *j;
get_op_btree_interpretation(((OpExpr *) lfirst(l))->opno,
&opclass_lists[i], &opstrat_lists[i]);
/*
* convert strategy number list to a Bitmapset to make the intersection
* calculation easy.
*/
this_strats = NULL;
foreach(j, opstrat_lists[i])
{
this_strats = bms_add_member(this_strats, lfirst_int(j));
}
if (i == 0)
strats = this_strats;
else
result = makeBoolConst(false, false);
strats = bms_int_members(strats, this_strats);
i++;
}
return result;
switch (bms_membership(strats))
{
case BMS_EMPTY_SET:
/* No common interpretation, so fail */
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("could not determine interpretation of row comparison operator %s",
strVal(llast(opname))),
errhint("Row comparison operators must be associated with btree operator classes.")));
rctype = 0; /* keep compiler quiet */
break;
case BMS_SINGLETON:
/* Simple case: just one possible interpretation */
rctype = bms_singleton_member(strats);
break;
case BMS_MULTIPLE:
default: /* keep compiler quiet */
{
/*
* Prefer the interpretation with the most default opclasses.
*/
int best_defaults = 0;
bool multiple_best = false;
int this_rctype;
rctype = 0; /* keep compiler quiet */
while ((this_rctype = bms_first_member(strats)) >= 0)
{
int ndefaults = 0;
for (i = 0; i < nopers; i++)
{
forboth(l, opclass_lists[i], r, opstrat_lists[i])
{
Oid opclass = lfirst_oid(l);
int opstrat = lfirst_int(r);
if (opstrat == this_rctype &&
opclass_is_default(opclass))
ndefaults++;
}
}
if (ndefaults > best_defaults)
{
best_defaults = ndefaults;
rctype = this_rctype;
multiple_best = false;
}
else if (ndefaults == best_defaults)
multiple_best = true;
}
if (best_defaults == 0 || multiple_best)
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("could not determine interpretation of row comparison operator %s",
strVal(llast(opname))),
errdetail("There are multiple equally-plausible candidates.")));
break;
}
}
/*
* For = and <> cases, we just combine the pairwise operators with
* AND or OR respectively.
*
* Note: this is presently the only place where the parser generates
* BoolExpr with more than two arguments. Should be OK since the
* rest of the system thinks BoolExpr is N-argument anyway.
*/
if (rctype == ROWCOMPARE_EQ)
return (Node *) makeBoolExpr(AND_EXPR, opexprs);
if (rctype == ROWCOMPARE_NE)
return (Node *) makeBoolExpr(OR_EXPR, opexprs);
/*
* Otherwise we need to determine exactly which opclass to associate
* with each operator.
*/
opclasses = NIL;
for (i = 0; i < nopers; i++)
{
Oid best_opclass = 0;
int ndefault = 0;
int nmatch = 0;
forboth(l, opclass_lists[i], r, opstrat_lists[i])
{
Oid opclass = lfirst_oid(l);
int opstrat = lfirst_int(r);
if (opstrat == rctype)
{
if (ndefault == 0)
best_opclass = opclass;
if (opclass_is_default(opclass))
ndefault++;
else
nmatch++;
}
}
if (ndefault == 1 || (ndefault == 0 && nmatch == 1))
opclasses = lappend_oid(opclasses, best_opclass);
else
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("could not determine interpretation of row comparison operator %s",
strVal(llast(opname))),
errdetail("There are multiple equally-plausible candidates.")));
}
/*
* Now deconstruct the OpExprs and create a RowCompareExpr.
*
* Note: can't just reuse the passed largs/rargs lists, because of
* possibility that make_op inserted coercion operations.
*/
opnos = NIL;
largs = NIL;
rargs = NIL;
foreach(l, opexprs)
{
OpExpr *cmp = (OpExpr *) lfirst(l);
opnos = lappend_oid(opnos, cmp->opno);
largs = lappend(largs, linitial(cmp->args));
rargs = lappend(rargs, lsecond(cmp->args));
}
rcexpr = makeNode(RowCompareExpr);
rcexpr->rctype = rctype;
rcexpr->opnos = opnos;
rcexpr->opclasses = opclasses;
rcexpr->largs = largs;
rcexpr->rargs = rargs;
return (Node *) rcexpr;
}
/*

View File

@@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/parser/parse_oper.c,v 1.83 2005/11/22 18:17:16 momjian Exp $
* $PostgreSQL: pgsql/src/backend/parser/parse_oper.c,v 1.84 2005/12/28 01:30:00 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@@ -39,6 +39,9 @@ static const char *op_signature_string(List *op, char oprkind,
Oid arg1, Oid arg2);
static void op_error(List *op, char oprkind, Oid arg1, Oid arg2,
FuncDetailCode fdresult);
static Expr *make_op_expr(ParseState *pstate, Operator op,
Node *ltree, Node *rtree,
Oid ltypeId, Oid rtypeId);
/*
@@ -942,7 +945,7 @@ make_scalar_array_op(ParseState *pstate, List *opname,
* As with coerce_type, pstate may be NULL if no special unknown-Param
* processing is wanted.
*/
Expr *
static Expr *
make_op_expr(ParseState *pstate, Operator op,
Node *ltree, Node *rtree,
Oid ltypeId, Oid rtypeId)

View File

@@ -3,7 +3,7 @@
* back to source text
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/utils/adt/ruleutils.c,v 1.210 2005/12/10 19:21:03 tgl Exp $
* $PostgreSQL: pgsql/src/backend/utils/adt/ruleutils.c,v 1.211 2005/12/28 01:30:00 tgl Exp $
*
* This software is copyrighted by Jan Wieck - Hamburg.
*
@@ -215,7 +215,6 @@ static void printSubscripts(ArrayRef *aref, deparse_context *context);
static char *generate_relation_name(Oid relid);
static char *generate_function_name(Oid funcid, int nargs, Oid *argtypes);
static char *generate_operator_name(Oid operid, Oid arg1, Oid arg2);
static void print_operator_name(StringInfo buf, List *opname);
static text *string_to_text(char *str);
#define only_marker(rte) ((rte)->inh ? "" : "ONLY ")
@@ -3106,6 +3105,7 @@ get_rule_expr(Node *node, deparse_context *context,
break;
case PARAM_NUM:
case PARAM_EXEC:
case PARAM_SUBLINK:
appendStringInfo(buf, "$%d", param->paramid);
break;
default:
@@ -3514,6 +3514,50 @@ get_rule_expr(Node *node, deparse_context *context,
}
break;
case T_RowCompareExpr:
{
RowCompareExpr *rcexpr = (RowCompareExpr *) node;
ListCell *arg;
char *sep;
/*
* SQL99 allows "ROW" to be omitted when there is more than
* one column, but for simplicity we always print it.
*/
appendStringInfo(buf, "(ROW(");
sep = "";
foreach(arg, rcexpr->largs)
{
Node *e = (Node *) lfirst(arg);
appendStringInfoString(buf, sep);
get_rule_expr(e, context, true);
sep = ", ";
}
/*
* We assume that the name of the first-column operator
* will do for all the rest too. This is definitely
* open to failure, eg if some but not all operators
* were renamed since the construct was parsed, but there
* seems no way to be perfect.
*/
appendStringInfo(buf, ") %s ROW(",
generate_operator_name(linitial_oid(rcexpr->opnos),
exprType(linitial(rcexpr->largs)),
exprType(linitial(rcexpr->rargs))));
sep = "";
foreach(arg, rcexpr->rargs)
{
Node *e = (Node *) lfirst(arg);
appendStringInfoString(buf, sep);
get_rule_expr(e, context, true);
sep = ", ";
}
appendStringInfo(buf, "))");
}
break;
case T_CoalesceExpr:
{
CoalesceExpr *coalesceexpr = (CoalesceExpr *) node;
@@ -3967,6 +4011,7 @@ get_sublink_expr(SubLink *sublink, deparse_context *context)
{
StringInfo buf = context->buf;
Query *query = (Query *) (sublink->subselect);
char *opname = NULL;
bool need_paren;
if (sublink->subLinkType == ARRAY_SUBLINK)
@@ -3974,25 +4019,67 @@ get_sublink_expr(SubLink *sublink, deparse_context *context)
else
appendStringInfoChar(buf, '(');
if (sublink->lefthand != NIL)
/*
* Note that we print the name of only the first operator, when there
* are multiple combining operators. This is an approximation that
* could go wrong in various scenarios (operators in different schemas,
* renamed operators, etc) but there is not a whole lot we can do about
* it, since the syntax allows only one operator to be shown.
*/
if (sublink->testexpr)
{
need_paren = (list_length(sublink->lefthand) > 1);
if (need_paren)
if (IsA(sublink->testexpr, OpExpr))
{
/* single combining operator */
OpExpr *opexpr = (OpExpr *) sublink->testexpr;
get_rule_expr(linitial(opexpr->args), context, true);
opname = generate_operator_name(opexpr->opno,
exprType(linitial(opexpr->args)),
exprType(lsecond(opexpr->args)));
}
else if (IsA(sublink->testexpr, BoolExpr))
{
/* multiple combining operators, = or <> cases */
char *sep;
ListCell *l;
appendStringInfoChar(buf, '(');
get_rule_expr((Node *) sublink->lefthand, context, true);
if (need_paren)
sep = "";
foreach(l, ((BoolExpr *) sublink->testexpr)->args)
{
OpExpr *opexpr = (OpExpr *) lfirst(l);
Assert(IsA(opexpr, OpExpr));
appendStringInfoString(buf, sep);
get_rule_expr(linitial(opexpr->args), context, true);
if (!opname)
opname = generate_operator_name(opexpr->opno,
exprType(linitial(opexpr->args)),
exprType(lsecond(opexpr->args)));
sep = ", ";
}
appendStringInfoChar(buf, ')');
appendStringInfoChar(buf, ' ');
}
else if (IsA(sublink->testexpr, RowCompareExpr))
{
/* multiple combining operators, < <= > >= cases */
RowCompareExpr *rcexpr = (RowCompareExpr *) sublink->testexpr;
appendStringInfoChar(buf, '(');
get_rule_expr((Node *) rcexpr->largs, context, true);
opname = generate_operator_name(linitial_oid(rcexpr->opnos),
exprType(linitial(rcexpr->largs)),
exprType(linitial(rcexpr->rargs)));
appendStringInfoChar(buf, ')');
}
else
elog(ERROR, "unrecognized testexpr type: %d",
(int) nodeTag(sublink->testexpr));
}
need_paren = true;
/*
* XXX we regurgitate the originally given operator name, with or without
* schema qualification. This is not necessarily 100% right but it's the
* best we can do, since the operators actually used might not all be in
* the same schema.
*/
switch (sublink->subLinkType)
{
case EXISTS_SUBLINK:
@@ -4000,27 +4087,18 @@ get_sublink_expr(SubLink *sublink, deparse_context *context)
break;
case ANY_SUBLINK:
if (list_length(sublink->operName) == 1 &&
strcmp(strVal(linitial(sublink->operName)), "=") == 0)
{
/* Represent = ANY as IN */
appendStringInfo(buf, "IN ");
}
if (strcmp(opname, "=") == 0) /* Represent = ANY as IN */
appendStringInfo(buf, " IN ");
else
{
print_operator_name(buf, sublink->operName);
appendStringInfo(buf, " ANY ");
}
appendStringInfo(buf, " %s ANY ", opname);
break;
case ALL_SUBLINK:
print_operator_name(buf, sublink->operName);
appendStringInfo(buf, " ALL ");
appendStringInfo(buf, " %s ALL ", opname);
break;
case MULTIEXPR_SUBLINK:
print_operator_name(buf, sublink->operName);
appendStringInfoChar(buf, ' ');
case ROWCOMPARE_SUBLINK:
appendStringInfo(buf, " %s ", opname);
break;
case EXPR_SUBLINK:
@@ -4812,30 +4890,6 @@ generate_operator_name(Oid operid, Oid arg1, Oid arg2)
return buf.data;
}
/*
* Print out a possibly-qualified operator name
*/
static void
print_operator_name(StringInfo buf, List *opname)
{
ListCell *op = list_head(opname);
int nnames = list_length(opname);
if (nnames == 1)
appendStringInfoString(buf, strVal(lfirst(op)));
else
{
appendStringInfo(buf, "OPERATOR(");
while (nnames-- > 1)
{
appendStringInfo(buf, "%s.",
quote_identifier(strVal(lfirst(op))));
op = lnext(op);
}
appendStringInfo(buf, "%s)", strVal(lfirst(op)));
}
}
/*
* Given a C string, produce a TEXT datum.
*

View File

@@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/utils/cache/lsyscache.c,v 1.130 2005/11/17 22:14:53 tgl Exp $
* $PostgreSQL: pgsql/src/backend/utils/cache/lsyscache.c,v 1.131 2005/12/28 01:30:01 tgl Exp $
*
* NOTES
* Eventually, the index information should go through here, too.
@@ -183,6 +183,99 @@ get_op_hash_function(Oid opno)
return InvalidOid;
}
/*
* get_op_btree_interpretation
* Given an operator's OID, find out which btree opclasses it belongs to,
* and what strategy number it has within each one. The results are
* returned as an OID list and a parallel integer list.
*
* In addition to the normal btree operators, we consider a <> operator to be
* a "member" of an opclass if its negator is the opclass' equality operator.
* ROWCOMPARE_NE is returned as the strategy number for this case.
*/
void
get_op_btree_interpretation(Oid opno, List **opclasses, List **opstrats)
{
Oid lefttype,
righttype;
CatCList *catlist;
bool op_negated;
int i;
*opclasses = NIL;
*opstrats = NIL;
/*
* Get the nominal left-hand input type of the operator; we will ignore
* opclasses that don't have that as the expected input datatype. This
* is a kluge to avoid being confused by binary-compatible opclasses
* (such as text_ops and varchar_ops, which share the same operators).
*/
op_input_types(opno, &lefttype, &righttype);
Assert(OidIsValid(lefttype));
/*
* Find all the pg_amop entries containing the operator.
*/
catlist = SearchSysCacheList(AMOPOPID, 1,
ObjectIdGetDatum(opno),
0, 0, 0);
/*
* If we can't find any opclass containing the op, perhaps it is a
* <> operator. See if it has a negator that is in an opclass.
*/
op_negated = false;
if (catlist->n_members == 0)
{
Oid op_negator = get_negator(opno);
if (OidIsValid(op_negator))
{
op_negated = true;
ReleaseSysCacheList(catlist);
catlist = SearchSysCacheList(AMOPOPID, 1,
ObjectIdGetDatum(op_negator),
0, 0, 0);
}
}
/* Now search the opclasses */
for (i = 0; i < catlist->n_members; i++)
{
HeapTuple op_tuple = &catlist->members[i]->tuple;
Form_pg_amop op_form = (Form_pg_amop) GETSTRUCT(op_tuple);
Oid opclass_id;
StrategyNumber op_strategy;
opclass_id = op_form->amopclaid;
/* must be btree */
if (!opclass_is_btree(opclass_id))
continue;
/* must match operator input type exactly */
if (get_opclass_input_type(opclass_id) != lefttype)
continue;
/* Get the operator's btree strategy number */
op_strategy = (StrategyNumber) op_form->amopstrategy;
Assert(op_strategy >= 1 && op_strategy <= 5);
if (op_negated)
{
/* Only consider negators that are = */
if (op_strategy != BTEqualStrategyNumber)
continue;
op_strategy = ROWCOMPARE_NE;
}
*opclasses = lappend_oid(*opclasses, opclass_id);
*opstrats = lappend_int(*opstrats, op_strategy);
}
ReleaseSysCacheList(catlist);
}
/* ---------- AMPROC CACHES ---------- */
@@ -433,6 +526,55 @@ opclass_is_hash(Oid opclass)
return result;
}
/*
* opclass_is_default
*
* Returns TRUE iff the specified opclass is the default for its
* index access method and input data type.
*/
bool
opclass_is_default(Oid opclass)
{
HeapTuple tp;
Form_pg_opclass cla_tup;
bool result;
tp = SearchSysCache(CLAOID,
ObjectIdGetDatum(opclass),
0, 0, 0);
if (!HeapTupleIsValid(tp))
elog(ERROR, "cache lookup failed for opclass %u", opclass);
cla_tup = (Form_pg_opclass) GETSTRUCT(tp);
result = cla_tup->opcdefault;
ReleaseSysCache(tp);
return result;
}
/*
* get_opclass_input_type
*
* Returns the OID of the datatype the opclass indexes.
*/
Oid
get_opclass_input_type(Oid opclass)
{
HeapTuple tp;
Form_pg_opclass cla_tup;
Oid result;
tp = SearchSysCache(CLAOID,
ObjectIdGetDatum(opclass),
0, 0, 0);
if (!HeapTupleIsValid(tp))
elog(ERROR, "cache lookup failed for opclass %u", opclass);
cla_tup = (Form_pg_opclass) GETSTRUCT(tp);
result = cla_tup->opcintype;
ReleaseSysCache(tp);
return result;
}
/* ---------- OPERATOR CACHE ---------- */
/*