Commit 6e077097 authored by Tom Lane's avatar Tom Lane

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.
parent a37422e0
This diff is collapsed.
......@@ -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,22 +1129,27 @@ find_expr_references_walker(Node *node,
&context->addrs);
/* fall through to examine arguments */
}
if (IsA(node, SubLink))
if (is_subplan(node))
{
SubLink *sublink = (SubLink *) node;
ListCell *opid;
/* 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(opid, sublink->operOids)
foreach(l, rcexpr->opnos)
{
add_object_address(OCLASS_OPERATOR, lfirst_oid(opid), 0,
add_object_address(OCLASS_OPERATOR, lfirst_oid(l), 0,
&context->addrs);
}
/* fall through to examine arguments */
}
if (is_subplan(node))
foreach(l, rcexpr->opclasses)
{
/* Extra work needed here if we ever need this case */
elog(ERROR, "already-planned subqueries not supported");
add_object_address(OCLASS_OPCLASS, lfirst_oid(l), 0,
&context->addrs);
}
/* fall through to examine arguments */
}
if (IsA(node, Query))
{
......
......@@ -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;
......
......@@ -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")));
......
......@@ -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;
......
......@@ -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;
......
......@@ -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;
......
......@@ -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))
......
......@@ -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
......
This diff is collapsed.
......@@ -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;
......
......@@ -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;
......
This diff is collapsed.
......@@ -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)
......
......@@ -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.
*
......
......@@ -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 ---------- */
/*
......
......@@ -37,7 +37,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/catalog/catversion.h,v 1.307 2005/11/17 22:14:54 tgl Exp $
* $PostgreSQL: pgsql/src/include/catalog/catversion.h,v 1.308 2005/12/28 01:30:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -53,6 +53,6 @@
*/
/* yyyymmddN */
#define CATALOG_VERSION_NO 200511171
#define CATALOG_VERSION_NO 200512271
#endif
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/nodes/execnodes.h,v 1.146 2005/12/02 20:03:42 tgl Exp $
* $PostgreSQL: pgsql/src/include/nodes/execnodes.h,v 1.147 2005/12/28 01:30:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -563,7 +563,7 @@ typedef struct SubPlanState
ExprState xprstate;
EState *sub_estate; /* subselect plan has its own EState */
struct PlanState *planstate; /* subselect plan's state tree */
List *exprs; /* states of combining expression(s) */
ExprState *testexpr; /* state of combining expression */
List *args; /* states of argument expression(s) */
bool needShutdown; /* TRUE = need to shutdown subplan */
HeapTuple curTuple; /* copy of most recent tuple from subplan */
......@@ -671,6 +671,18 @@ typedef struct RowExprState
TupleDesc tupdesc; /* descriptor for result tuples */
} RowExprState;
/* ----------------
* RowCompareExprState node
* ----------------
*/
typedef struct RowCompareExprState
{
ExprState xprstate;
List *largs; /* the left-hand input arguments */
List *rargs; /* the right-hand input arguments */
FmgrInfo *funcs; /* array of comparison function info */
} RowCompareExprState;
/* ----------------
* CoalesceExprState node
* ----------------
......
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/nodes/nodes.h,v 1.179 2005/12/20 02:30:36 tgl Exp $
* $PostgreSQL: pgsql/src/include/nodes/nodes.h,v 1.180 2005/12/28 01:30:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -125,6 +125,7 @@ typedef enum NodeTag
T_CaseTestExpr,
T_ArrayExpr,
T_RowExpr,
T_RowCompareExpr,
T_CoalesceExpr,
T_MinMaxExpr,
T_NullIfExpr,
......@@ -159,6 +160,7 @@ typedef enum NodeTag
T_CaseWhenState,
T_ArrayExprState,
T_RowExprState,
T_RowCompareExprState,
T_CoalesceExprState,
T_MinMaxExprState,
T_CoerceToDomainState,
......
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/nodes/params.h,v 1.28 2004/12/31 22:03:34 pgsql Exp $
* $PostgreSQL: pgsql/src/include/nodes/params.h,v 1.29 2005/12/28 01:30:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -32,6 +32,11 @@
* PARAM_EXEC: The parameter is an internal executor parameter.
* It has a number contained in the `paramid' field.
*
* PARAM_SUBLINK: The parameter represents an output column of a SubLink
* node's sub-select. The column number is contained in the
* `paramid' field. (This type of Param is converted to
* PARAM_EXEC during planning.)
*
* PARAM_INVALID should never appear in a Param node; it's used to mark
* the end of a ParamListInfo array.
*
......@@ -44,6 +49,7 @@
#define PARAM_NAMED 11
#define PARAM_NUM 12
#define PARAM_EXEC 15
#define PARAM_SUBLINK 16
#define PARAM_INVALID 100
......
......@@ -10,7 +10,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/nodes/primnodes.h,v 1.110 2005/12/20 02:30:36 tgl Exp $
* $PostgreSQL: pgsql/src/include/nodes/primnodes.h,v 1.111 2005/12/28 01:30:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -158,6 +158,11 @@ typedef struct Const
*
* PARAM_EXEC: The parameter is an internal executor parameter.
* It has a number contained in the `paramid' field.
*
* PARAM_SUBLINK: The parameter represents an output column of a SubLink
* node's sub-select. The column number is contained in the
* `paramid' field. (This type of Param is converted to
* PARAM_EXEC during planning.)
* ----------------
*/
typedef struct Param
......@@ -329,7 +334,7 @@ typedef struct BoolExpr
List *args; /* arguments to this expression */
} BoolExpr;
/* ----------------
/*
* SubLink
*
* A SubLink represents a subselect appearing in an expression, and in some
......@@ -338,46 +343,42 @@ typedef struct BoolExpr
* EXISTS_SUBLINK EXISTS(SELECT ...)
* ALL_SUBLINK (lefthand) op ALL (SELECT ...)
* ANY_SUBLINK (lefthand) op ANY (SELECT ...)
* MULTIEXPR_SUBLINK (lefthand) op (SELECT ...)
* ROWCOMPARE_SUBLINK (lefthand) op (SELECT ...)
* EXPR_SUBLINK (SELECT with single targetlist item ...)
* ARRAY_SUBLINK ARRAY(SELECT with single targetlist item ...)
* For ALL, ANY, and MULTIEXPR, the lefthand is a list of expressions of the
* same length as the subselect's targetlist. MULTIEXPR will *always* have
* For ALL, ANY, and ROWCOMPARE, the lefthand is a list of expressions of the
* same length as the subselect's targetlist. ROWCOMPARE will *always* have
* a list with more than one entry; if the subselect has just one target
* then the parser will create an EXPR_SUBLINK instead (and any operator
* above the subselect will be represented separately). Note that both
* MULTIEXPR and EXPR require the subselect to deliver only one row.
* ROWCOMPARE and EXPR require the subselect to deliver only one row.
* ALL, ANY, and ROWCOMPARE require the combining operators to deliver boolean
* results. ALL and ANY combine the per-row results using AND and OR
* semantics respectively.
* ARRAY requires just one target column, and creates an array of the target
* column's type using one or more rows resulting from the subselect.
* ALL, ANY, and MULTIEXPR require the combining operators to deliver boolean
* results. These are reduced to one result per row using OR or AND semantics
* depending on the "useOr" flag. ALL and ANY combine the per-row results
* using AND and OR semantics respectively.
*
* SubLink is classed as an Expr node, but it is not actually executable;
* it must be replaced in the expression tree by a SubPlan node during
* planning.
*
* NOTE: in the raw output of gram.y, lefthand contains a list of raw
* expressions; useOr and operOids are not filled in yet. Also, subselect
* is a raw parsetree. During parse analysis, the parser transforms the
* lefthand expression list using normal expression transformation rules.
* It fills operOids with the OIDs representing the specific operator(s)
* to apply to each pair of lefthand and targetlist expressions.
* And subselect is transformed to a Query. This is the representation
* seen in saved rules and in the rewriter.
*
* In EXISTS, EXPR, and ARRAY SubLinks, lefthand, operName, and operOids are
* unused and are always NIL. useOr is not significant either for these
* sublink types.
* ----------------
* NOTE: in the raw output of gram.y, testexpr contains just the raw form
* of the lefthand expression (if any), and operName is the String name of
* the combining operator. Also, subselect is a raw parsetree. During parse
* analysis, the parser transforms testexpr into a complete boolean expression
* that compares the lefthand value(s) to PARAM_SUBLINK nodes representing the
* output columns of the subselect. And subselect is transformed to a Query.
* This is the representation seen in saved rules and in the rewriter.
*
* In EXISTS, EXPR, and ARRAY SubLinks, testexpr and operName are unused and
* are always null.
*/
typedef enum SubLinkType
{
EXISTS_SUBLINK,
ALL_SUBLINK,
ANY_SUBLINK,
MULTIEXPR_SUBLINK,
ROWCOMPARE_SUBLINK,
EXPR_SUBLINK,
ARRAY_SUBLINK
} SubLinkType;
......@@ -386,12 +387,9 @@ typedef enum SubLinkType
typedef struct SubLink
{
Expr xpr;
SubLinkType subLinkType; /* EXISTS, ALL, ANY, MULTIEXPR, EXPR */
bool useOr; /* TRUE to combine column results with "OR"
* not "AND" */
List *lefthand; /* list of outer-query expressions on the left */
SubLinkType subLinkType; /* see above */
Node *testexpr; /* outer-query test for ALL/ANY/ROWCOMPARE */
List *operName; /* originally specified operator name */
List *operOids; /* OIDs of actual combining operators */
Node *subselect; /* subselect as Query* or parsetree */
} SubLink;
......@@ -402,14 +400,18 @@ typedef struct SubLink
* nodes after it has finished planning the subquery. SubPlan contains
* a sub-plantree and rtable instead of a sub-Query.
*
* In an ordinary subplan, "exprs" points to a list of executable expressions
* (OpExpr trees) for the combining operators; their left-hand arguments are
* the original lefthand expressions, and their right-hand arguments are
* PARAM_EXEC Param nodes representing the outputs of the sub-select.
* (NOTE: runtime coercion functions may be inserted as well.) But if the
* sub-select becomes an initplan rather than a subplan, these executable
* expressions are part of the outer plan's expression tree (and the SubPlan
* node itself is not). In this case "exprs" is NIL to avoid duplication.
* In an ordinary subplan, testexpr points to an executable expression
* (OpExpr, an AND/OR tree of OpExprs, or RowCompareExpr) for the combining
* operator(s); the left-hand arguments are the original lefthand expressions,
* and the right-hand arguments are PARAM_EXEC Param nodes representing the
* outputs of the sub-select. (NOTE: runtime coercion functions may be
* inserted as well.) This is just the same expression tree as testexpr in
* the original SubLink node, but the PARAM_SUBLINK nodes are replaced by
* suitably numbered PARAM_EXEC nodes.
*
* If the sub-select becomes an initplan rather than a subplan, the executable
* expression is part of the outer plan's expression tree (and the SubPlan
* node itself is not). In this case testexpr is NULL to avoid duplication.
*
* The planner also derives lists of the values that need to be passed into
* and out of the subplan. Input values are represented as a list "args" of
......@@ -426,13 +428,10 @@ typedef struct SubPlan
{
Expr xpr;
/* Fields copied from original SubLink: */
SubLinkType subLinkType; /* EXISTS, ALL, ANY, MULTIEXPR, EXPR */
bool useOr; /* TRUE to combine column results with "OR"
* not "AND" */
/* The combining operators, transformed to executable expressions: */
List *exprs; /* list of OpExpr expression trees */
SubLinkType subLinkType; /* see above */
/* The combining operators, transformed to an executable expression: */
Node *testexpr; /* OpExpr or RowCompareExpr expression tree */
List *paramIds; /* IDs of Params embedded in the above */
/* Note: paramIds has a one-to-one correspondence to the exprs list */
/* The subselect, transformed to a Plan: */
struct Plan *plan; /* subselect plan itself */
int plan_id; /* dummy thing because of we haven't equal
......@@ -642,6 +641,41 @@ typedef struct RowExpr
CoercionForm row_format; /* how to display this node */
} RowExpr;
/*
* RowCompareExpr - row-wise comparison, such as (a, b) <= (1, 2)
*
* We support row comparison for any operator that can be determined to
* act like =, <>, <, <=, >, or >= (we determine this by looking for the
* operator in btree opclasses). Note that the same operator name might
* map to a different operator for each pair of row elements, since the
* element datatypes can vary.
*
* A RowCompareExpr node is only generated for the < <= > >= cases;
* the = and <> cases are translated to simple AND or OR combinations
* of the pairwise comparisons. However, we include = and <> in the
* RowCompareType enum for the convenience of parser logic.
*/
typedef enum RowCompareType
{
/* Values of this enum are chosen to match btree strategy numbers */
ROWCOMPARE_LT = 1, /* BTLessStrategyNumber */
ROWCOMPARE_LE = 2, /* BTLessEqualStrategyNumber */
ROWCOMPARE_EQ = 3, /* BTEqualStrategyNumber */
ROWCOMPARE_GE = 4, /* BTGreaterEqualStrategyNumber */
ROWCOMPARE_GT = 5, /* BTGreaterStrategyNumber */
ROWCOMPARE_NE = 6 /* no such btree strategy */
} RowCompareType;
typedef struct RowCompareExpr
{
Expr xpr;
RowCompareType rctype; /* LT LE GE or GT, never EQ or NE */
List *opnos; /* OID list of pairwise comparison ops */
List *opclasses; /* OID list of containing operator classes */
List *largs; /* the left-hand input arguments */
List *rargs; /* the right-hand input arguments */
} RowCompareExpr;
/*
* CoalesceExpr - a COALESCE expression
*/
......
......@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/parser/parse_oper.h,v 1.36 2004/12/31 22:03:38 pgsql Exp $
* $PostgreSQL: pgsql/src/include/parser/parse_oper.h,v 1.37 2005/12/28 01:30:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -59,8 +59,5 @@ extern Expr *make_op(ParseState *pstate, List *opname,
extern Expr *make_scalar_array_op(ParseState *pstate, List *opname,
bool useOr,
Node *ltree, Node *rtree);
extern Expr *make_op_expr(ParseState *pstate, Operator op,
Node *ltree, Node *rtree,
Oid ltypeId, Oid rtypeId);
#endif /* PARSE_OPER_H */
......@@ -6,7 +6,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/utils/lsyscache.h,v 1.101 2005/10/15 02:49:46 momjian Exp $
* $PostgreSQL: pgsql/src/include/utils/lsyscache.h,v 1.102 2005/12/28 01:30:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
......@@ -31,6 +31,8 @@ extern void get_op_opclass_properties(Oid opno, Oid opclass,
bool *recheck);
extern Oid get_opclass_member(Oid opclass, Oid subtype, int16 strategy);
extern Oid get_op_hash_function(Oid opno);
extern void get_op_btree_interpretation(Oid opno,
List **opclasses, List **opstrats);
extern Oid get_opclass_proc(Oid opclass, Oid subtype, int16 procnum);
extern char *get_attname(Oid relid, AttrNumber attnum);
extern char *get_relid_attribute_name(Oid relid, AttrNumber attnum);
......@@ -41,6 +43,8 @@ extern void get_atttypetypmod(Oid relid, AttrNumber attnum,
Oid *typid, int32 *typmod);
extern bool opclass_is_btree(Oid opclass);
extern bool opclass_is_hash(Oid opclass);
extern bool opclass_is_default(Oid opclass);
extern Oid get_opclass_input_type(Oid opclass);
extern RegProcedure get_opcode(Oid opno);
extern char *get_opname(Oid opno);
extern void op_input_types(Oid opno, Oid *lefttype, Oid *righttype);
......
......@@ -3,7 +3,7 @@
* procedural language
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.157 2005/11/22 18:17:33 momjian Exp $
* $PostgreSQL: pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.158 2005/12/28 01:30:01 tgl Exp $
*
* This software is copyrighted by Jan Wieck - Hamburg.
*
......@@ -4283,6 +4283,18 @@ exec_simple_check_node(Node *node)
return TRUE;
}
case T_RowCompareExpr:
{
RowCompareExpr *expr = (RowCompareExpr *) node;
if (!exec_simple_check_node((Node *) expr->largs))
return FALSE;
if (!exec_simple_check_node((Node *) expr->rargs))
return FALSE;
return TRUE;
}
case T_CoalesceExpr:
{
CoalesceExpr *expr = (CoalesceExpr *) node;
......
......@@ -117,3 +117,125 @@ select (fn).first, substr((fn).last, 1, 20), length((fn).last) from people;
Jim | abcdefghijklabcdefgh | 1200000
(2 rows)
-- Test row comparison semantics. Prior to PG 8.2 we did this in a totally
-- non-spec-compliant way.
select ROW(1,2) < ROW(1,3) as true;
true
------
t
(1 row)
select ROW(1,2) < ROW(1,1) as false;
false
-------
f
(1 row)
select ROW(1,2) < ROW(1,NULL) as null;
null
------
(1 row)
select ROW(1,2,3) < ROW(1,3,NULL) as true; -- the NULL is not examined
true
------
t
(1 row)
select ROW(11,'ABC') < ROW(11,'DEF') as true;
true
------
t
(1 row)
select ROW(11,'ABC') > ROW(11,'DEF') as false;
false
-------
f
(1 row)
select ROW(12,'ABC') > ROW(11,'DEF') as true;
true
------
t
(1 row)
-- = and <> have different NULL-behavior than < etc
select ROW(1,2,3) < ROW(1,NULL,4) as null;
null
------
(1 row)
select ROW(1,2,3) = ROW(1,NULL,4) as false;
false
-------
f
(1 row)
select ROW(1,2,3) <> ROW(1,NULL,4) as true;
true
------
t
(1 row)
-- We allow operators beyond the six standard ones, if they have btree
-- operator classes.
select ROW('ABC','DEF') ~<=~ ROW('DEF','ABC') as true;
true
------
t
(1 row)
select ROW('ABC','DEF') ~>=~ ROW('DEF','ABC') as false;
false
-------
f
(1 row)
select ROW('ABC','DEF') ~~ ROW('DEF','ABC') as fail;
ERROR: could not determine interpretation of row comparison operator ~~
HINT: Row comparison operators must be associated with btree operator classes.
-- Check row comparison with a subselect
select unique1, unique2 from tenk1
where (unique1, unique2) < any (select ten, ten from tenk1 where hundred < 3);
unique1 | unique2
---------+---------
1 | 2838
0 | 9998
(2 rows)
-- Also check row comparison with an indexable condition
select thousand, tenthous from tenk1
where (thousand, tenthous) >= (997, 5000)
order by thousand, tenthous;
thousand | tenthous
----------+----------
997 | 5997
997 | 6997
997 | 7997
997 | 8997
997 | 9997
998 | 998
998 | 1998
998 | 2998
998 | 3998
998 | 4998
998 | 5998
998 | 6998
998 | 7998
998 | 8998
998 | 9998
999 | 999
999 | 1999
999 | 2999
999 | 3999
999 | 4999
999 | 5999
999 | 6999
999 | 7999
999 | 8999
999 | 9999
(25 rows)
......@@ -72,3 +72,35 @@ insert into pp values (repeat('abcdefghijkl', 100000));
insert into people select ('Jim', f1, null)::fullname, current_date from pp;
select (fn).first, substr((fn).last, 1, 20), length((fn).last) from people;
-- Test row comparison semantics. Prior to PG 8.2 we did this in a totally
-- non-spec-compliant way.
select ROW(1,2) < ROW(1,3) as true;
select ROW(1,2) < ROW(1,1) as false;
select ROW(1,2) < ROW(1,NULL) as null;
select ROW(1,2,3) < ROW(1,3,NULL) as true; -- the NULL is not examined
select ROW(11,'ABC') < ROW(11,'DEF') as true;
select ROW(11,'ABC') > ROW(11,'DEF') as false;
select ROW(12,'ABC') > ROW(11,'DEF') as true;
-- = and <> have different NULL-behavior than < etc
select ROW(1,2,3) < ROW(1,NULL,4) as null;
select ROW(1,2,3) = ROW(1,NULL,4) as false;
select ROW(1,2,3) <> ROW(1,NULL,4) as true;
-- We allow operators beyond the six standard ones, if they have btree
-- operator classes.
select ROW('ABC','DEF') ~<=~ ROW('DEF','ABC') as true;
select ROW('ABC','DEF') ~>=~ ROW('DEF','ABC') as false;
select ROW('ABC','DEF') ~~ ROW('DEF','ABC') as fail;
-- Check row comparison with a subselect
select unique1, unique2 from tenk1
where (unique1, unique2) < any (select ten, ten from tenk1 where hundred < 3);
-- Also check row comparison with an indexable condition
select thousand, tenthous from tenk1
where (thousand, tenthous) >= (997, 5000)
order by thousand, tenthous;
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment