First phase of work on array improvements. ARRAY[x,y,z] constructor

expressions, ARRAY(sub-SELECT) expressions, some array functions.
Polymorphic functions using ANYARRAY/ANYELEMENT argument and return
types.  Some regression tests in place, documentation is lacking.
Joe Conway, with some kibitzing from Tom Lane.
This commit is contained in:
Tom Lane 2003-04-08 23:20:04 +00:00
parent 6fb5115850
commit 730840c9b6
47 changed files with 2597 additions and 479 deletions

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/catalog/pg_proc.c,v 1.95 2002/12/12 15:49:24 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/catalog/pg_proc.c,v 1.96 2003/04/08 23:20:00 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -86,6 +86,29 @@ ProcedureCreate(const char *procedureName,
elog(ERROR, "functions cannot have more than %d arguments",
FUNC_MAX_ARGS);
/*
* Do not allow return type ANYARRAY or ANYELEMENT unless at least one
* argument is also ANYARRAY or ANYELEMENT
*/
if (returnType == ANYARRAYOID || returnType == ANYELEMENTOID)
{
bool genericParam = false;
for (i = 0; i < parameterCount; i++)
{
if (parameterTypes[i] == ANYARRAYOID ||
parameterTypes[i] == ANYELEMENTOID)
{
genericParam = true;
break;
}
}
if (!genericParam)
elog(ERROR, "functions returning ANYARRAY or ANYELEMENT must " \
"have at least one argument of either type");
}
/* Make sure we have a zero-padded param type array */
MemSet(typev, 0, FUNC_MAX_ARGS * sizeof(Oid));
if (parameterCount > 0)

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/execQual.c,v 1.127 2003/03/27 16:51:27 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/execQual.c,v 1.128 2003/04/08 23:20:00 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -75,6 +75,9 @@ static Datum ExecEvalAnd(BoolExprState *andExpr, ExprContext *econtext,
bool *isNull);
static Datum ExecEvalCase(CaseExprState *caseExpr, ExprContext *econtext,
bool *isNull, ExprDoneCond *isDone);
static Datum ExecEvalArray(ArrayExprState *astate,
ExprContext *econtext,
bool *isNull);
static Datum ExecEvalCoalesce(CoalesceExprState *coalesceExpr,
ExprContext *econtext,
bool *isNull);
@ -246,38 +249,38 @@ ExecEvalArrayRef(ArrayRefExprState *astate,
resultArray = array_set(array_source, i,
upper.indx,
sourceData,
arrayRef->refattrlength,
arrayRef->refelemlength,
arrayRef->refelembyval,
arrayRef->refelemalign,
astate->refattrlength,
astate->refelemlength,
astate->refelembyval,
astate->refelemalign,
isNull);
else
resultArray = array_set_slice(array_source, i,
upper.indx, lower.indx,
(ArrayType *) DatumGetPointer(sourceData),
arrayRef->refattrlength,
arrayRef->refelemlength,
arrayRef->refelembyval,
arrayRef->refelemalign,
astate->refattrlength,
astate->refelemlength,
astate->refelembyval,
astate->refelemalign,
isNull);
return PointerGetDatum(resultArray);
}
if (lIndex == NULL)
return array_ref(array_source, i, upper.indx,
arrayRef->refattrlength,
arrayRef->refelemlength,
arrayRef->refelembyval,
arrayRef->refelemalign,
astate->refattrlength,
astate->refelemlength,
astate->refelembyval,
astate->refelemalign,
isNull);
else
{
resultArray = array_get_slice(array_source, i,
upper.indx, lower.indx,
arrayRef->refattrlength,
arrayRef->refelemlength,
arrayRef->refelembyval,
arrayRef->refelemalign,
astate->refattrlength,
astate->refelemlength,
astate->refelembyval,
astate->refelemalign,
isNull);
return PointerGetDatum(resultArray);
}
@ -613,6 +616,7 @@ init_fcache(Oid foid, FuncExprState *fcache, MemoryContext fcacheCxt)
/* Initialize additional info */
fcache->setArgsValid = false;
fcache->func.fn_expr = (Node *) fcache->xprstate.expr;
}
/*
@ -1426,6 +1430,158 @@ ExecEvalCase(CaseExprState *caseExpr, ExprContext *econtext,
return (Datum) 0;
}
/* ----------------------------------------------------------------
* ExecEvalArray - ARRAY[] expressions
* ----------------------------------------------------------------
*/
static Datum
ExecEvalArray(ArrayExprState *astate, ExprContext *econtext,
bool *isNull)
{
ArrayExpr *arrayExpr = (ArrayExpr *) astate->xprstate.expr;
ArrayType *result;
List *element;
Oid element_type = arrayExpr->element_typeid;
int ndims = arrayExpr->ndims;
int dims[MAXDIM];
int lbs[MAXDIM];
if (ndims == 1)
{
int nelems;
Datum *dvalues;
int i = 0;
nelems = length(astate->elements);
/* Shouldn't happen here, but if length is 0, return NULL */
if (nelems == 0)
{
*isNull = true;
return (Datum) 0;
}
dvalues = (Datum *) palloc(nelems * sizeof(Datum));
/* loop through and build array of datums */
foreach(element, astate->elements)
{
ExprState *e = (ExprState *) lfirst(element);
bool eisnull;
dvalues[i++] = ExecEvalExpr(e, econtext, &eisnull, NULL);
if (eisnull)
elog(ERROR, "Arrays cannot have NULL elements");
}
/* setup for 1-D array of the given length */
dims[0] = nelems;
lbs[0] = 1;
result = construct_md_array(dvalues, ndims, dims, lbs,
element_type,
astate->elemlength,
astate->elembyval,
astate->elemalign);
}
else
{
char *dat = NULL;
Size ndatabytes = 0;
int nbytes;
int outer_nelems = length(astate->elements);
int elem_ndims = 0;
int *elem_dims = NULL;
int *elem_lbs = NULL;
bool firstone = true;
int i;
if (ndims <= 0 || ndims > MAXDIM)
elog(ERROR, "Arrays cannot have more than %d dimensions", MAXDIM);
/* loop through and get data area from each element */
foreach(element, astate->elements)
{
ExprState *e = (ExprState *) lfirst(element);
bool eisnull;
Datum arraydatum;
ArrayType *array;
int elem_ndatabytes;
arraydatum = ExecEvalExpr(e, econtext, &eisnull, NULL);
if (eisnull)
elog(ERROR, "Arrays cannot have NULL elements");
array = DatumGetArrayTypeP(arraydatum);
if (firstone)
{
/* Get sub-array details from first member */
elem_ndims = ARR_NDIM(array);
elem_dims = (int *) palloc(elem_ndims * sizeof(int));
memcpy(elem_dims, ARR_DIMS(array), elem_ndims * sizeof(int));
elem_lbs = (int *) palloc(elem_ndims * sizeof(int));
memcpy(elem_lbs, ARR_LBOUND(array), elem_ndims * sizeof(int));
firstone = false;
}
else
{
/* Check other sub-arrays are compatible */
if (elem_ndims != ARR_NDIM(array))
elog(ERROR, "Multiple dimension arrays must have array "
"expressions with matching number of dimensions");
if (memcmp(elem_dims, ARR_DIMS(array),
elem_ndims * sizeof(int)) != 0)
elog(ERROR, "Multiple dimension arrays must have array "
"expressions with matching dimensions");
if (memcmp(elem_lbs, ARR_LBOUND(array),
elem_ndims * sizeof(int)) != 0)
elog(ERROR, "Multiple dimension arrays must have array "
"expressions with matching dimensions");
}
elem_ndatabytes = ARR_SIZE(array) - ARR_OVERHEAD(elem_ndims);
ndatabytes += elem_ndatabytes;
if (dat == NULL)
dat = (char *) palloc(ndatabytes);
else
dat = (char *) repalloc(dat, ndatabytes);
memcpy(dat + (ndatabytes - elem_ndatabytes),
ARR_DATA_PTR(array),
elem_ndatabytes);
}
/* setup for multi-D array */
dims[0] = outer_nelems;
lbs[0] = 1;
for (i = 1; i < ndims; i++)
{
dims[i] = elem_dims[i - 1];
lbs[i] = elem_lbs[i - 1];
}
nbytes = ndatabytes + ARR_OVERHEAD(ndims);
result = (ArrayType *) palloc(nbytes);
result->size = nbytes;
result->ndim = ndims;
result->flags = 0;
result->elemtype = element_type;
memcpy(ARR_DIMS(result), dims, ndims * sizeof(int));
memcpy(ARR_LBOUND(result), lbs, ndims * sizeof(int));
if (ndatabytes > 0)
memcpy(ARR_DATA_PTR(result), dat, ndatabytes);
if (dat != NULL)
pfree(dat);
}
return PointerGetDatum(result);
}
/* ----------------------------------------------------------------
* ExecEvalCoalesce
* ----------------------------------------------------------------
@ -1908,6 +2064,11 @@ ExecEvalExpr(ExprState *expression,
isNull,
isDone);
break;
case T_ArrayExpr:
retDatum = ExecEvalArray((ArrayExprState *) expression,
econtext,
isNull);
break;
case T_CoalesceExpr:
retDatum = ExecEvalCoalesce((CoalesceExprState *) expression,
econtext,
@ -2060,6 +2221,12 @@ ExecInitExpr(Expr *node, PlanState *parent)
astate->refexpr = ExecInitExpr(aref->refexpr, parent);
astate->refassgnexpr = ExecInitExpr(aref->refassgnexpr,
parent);
/* do one-time catalog lookups for type info */
astate->refattrlength = get_typlen(aref->refarraytype);
get_typlenbyvalalign(aref->refelemtype,
&astate->refelemlength,
&astate->refelembyval,
&astate->refelemalign);
state = (ExprState *) astate;
}
break;
@ -2174,6 +2341,30 @@ ExecInitExpr(Expr *node, PlanState *parent)
state = (ExprState *) cstate;
}
break;
case T_ArrayExpr:
{
ArrayExpr *arrayexpr = (ArrayExpr *) node;
ArrayExprState *astate = makeNode(ArrayExprState);
List *outlist = NIL;
List *inlist;
foreach(inlist, arrayexpr->elements)
{
Expr *e = (Expr *) lfirst(inlist);
ExprState *estate;
estate = ExecInitExpr(e, parent);
outlist = lappend(outlist, estate);
}
astate->elements = outlist;
/* do one-time catalog lookup for type info */
get_typlenbyvalalign(arrayexpr->element_typeid,
&astate->elemlength,
&astate->elembyval,
&astate->elemalign);
state = (ExprState *) astate;
}
break;
case T_CoalesceExpr:
{
CoalesceExpr *coalesceexpr = (CoalesceExpr *) node;

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSubplan.c,v 1.44 2003/02/09 00:30:39 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSubplan.c,v 1.45 2003/04/08 23:20:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -24,9 +24,27 @@
#include "executor/nodeSubplan.h"
#include "nodes/makefuncs.h"
#include "parser/parse_expr.h"
#include "tcop/pquery.h"
#include "utils/array.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
typedef struct ArrayBuildState
{
MemoryContext mcontext; /* where all the temp stuff is kept */
Datum *dvalues; /* array of accumulated Datums */
/*
* The allocated size of dvalues[] is always a multiple of
* ARRAY_ELEMS_CHUNKSIZE
*/
#define ARRAY_ELEMS_CHUNKSIZE 64
int nelems; /* number of valid Datums in dvalues[] */
Oid element_type; /* data type of the Datums */
int16 typlen; /* needed info about datatype */
bool typbyval;
char typalign;
} ArrayBuildState;
static Datum ExecHashSubPlan(SubPlanState *node,
ExprContext *econtext,
bool *isNull);
@ -36,6 +54,12 @@ static Datum ExecScanSubPlan(SubPlanState *node,
static void buildSubPlanHash(SubPlanState *node);
static bool findPartialMatch(TupleHashTable hashtable, TupleTableSlot *slot);
static bool tupleAllNulls(HeapTuple tuple);
static ArrayBuildState *accumArrayResult(ArrayBuildState *astate,
Datum dvalue, bool disnull,
Oid element_type,
MemoryContext rcontext);
static Datum makeArrayResult(ArrayBuildState *astate,
MemoryContext rcontext);
/* ----------------------------------------------------------------
@ -206,6 +230,7 @@ ExecScanSubPlan(SubPlanState *node,
bool found = false; /* TRUE if got at least one subplan tuple */
List *pvar;
List *lst;
ArrayBuildState *astate = NULL;
/*
* We are probably in a short-lived expression-evaluation context.
@ -236,11 +261,11 @@ ExecScanSubPlan(SubPlanState *node,
ExecReScan(planstate, NULL);
/*
* For all sublink types except EXPR_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
* 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.)
* NULL results from the combining operators are handled according to
@ -249,9 +274,10 @@ ExecScanSubPlan(SubPlanState *node,
* MULTIEXPR_SUBLINK.
*
* For EXPR_SUBLINK we require the subplan to produce no more than one
* tuple, else an error is raised. If zero tuples are produced, we
* return NULL. Assuming we get a tuple, we just return its first
* column (there can be only one non-junk column in this case).
* tuple, else an error is raised. For ARRAY_SUBLINK we allow the subplan
* to produce more than one tuple. In either case, if zero tuples are
* produced, we return NULL. Assuming we get a tuple, we just use its
* first column (there can be only one non-junk column in this case).
*/
result = BoolGetDatum(subLinkType == ALL_SUBLINK);
*isNull = false;
@ -301,6 +327,21 @@ ExecScanSubPlan(SubPlanState *node,
continue;
}
if (subLinkType == ARRAY_SUBLINK)
{
Datum dvalue;
bool disnull;
found = true;
/* stash away current value */
dvalue = heap_getattr(tup, 1, tdesc, &disnull);
astate = accumArrayResult(astate, dvalue, disnull,
tdesc->attrs[0]->atttypid,
oldcontext);
/* keep scanning subplan to collect all values */
continue;
}
/* cannot allow multiple input tuples for MULTIEXPR sublink either */
if (subLinkType == MULTIEXPR_SUBLINK && found)
elog(ERROR, "More than one tuple returned by a subselect used as an expression.");
@ -407,15 +448,23 @@ ExecScanSubPlan(SubPlanState *node,
{
/*
* deal with empty subplan result. result/isNull were previously
* initialized correctly for all sublink types except EXPR and
* initialized correctly for all sublink types except EXPR, ARRAY, and
* MULTIEXPR; for those, return NULL.
*/
if (subLinkType == EXPR_SUBLINK || subLinkType == MULTIEXPR_SUBLINK)
if (subLinkType == EXPR_SUBLINK ||
subLinkType == ARRAY_SUBLINK ||
subLinkType == MULTIEXPR_SUBLINK)
{
result = (Datum) 0;
*isNull = true;
}
}
else if (subLinkType == ARRAY_SUBLINK)
{
Assert(astate != NULL);
/* We return the result in the caller's context */
result = makeArrayResult(astate, oldcontext);
}
MemoryContextSwitchTo(oldcontext);
@ -797,6 +846,7 @@ ExecInitSubPlan(SubPlanState *node, EState *estate)
/* Lookup the combining function */
fmgr_info(opexpr->opfuncid, &node->eqfunctions[i-1]);
node->eqfunctions[i-1].fn_expr = (Node *) opexpr;
i++;
}
@ -857,6 +907,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
TupleTableSlot *slot;
List *lst;
bool found = false;
ArrayBuildState *astate = NULL;
/*
* Must switch to child query's per-query memory context.
@ -891,6 +942,21 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
break;
}
if (subLinkType == ARRAY_SUBLINK)
{
Datum dvalue;
bool disnull;
found = true;
/* stash away current value */
dvalue = heap_getattr(tup, 1, tdesc, &disnull);
astate = accumArrayResult(astate, dvalue, disnull,
tdesc->attrs[0]->atttypid,
oldcontext);
/* keep scanning subplan to collect all values */
continue;
}
if (found &&
(subLinkType == EXPR_SUBLINK ||
subLinkType == MULTIEXPR_SUBLINK))
@ -951,6 +1017,18 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
}
}
}
else if (subLinkType == ARRAY_SUBLINK)
{
/* There can be only one param... */
int paramid = lfirsti(subplan->setParam);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
Assert(astate != NULL);
prm->execPlan = NULL;
/* We build the result in query context so it won't disappear */
prm->value = makeArrayResult(astate, econtext->ecxt_per_query_memory);
prm->isnull = false;
}
MemoryContextSwitchTo(oldcontext);
}
@ -1007,3 +1085,101 @@ ExecReScanSetParamPlan(SubPlanState *node, PlanState *parent)
parent->chgParam = bms_add_member(parent->chgParam, paramid);
}
}
/*
* accumArrayResult - accumulate one (more) Datum for an ARRAY_SUBLINK
*
* astate is working state (NULL on first call)
* rcontext is where to keep working state
*/
static ArrayBuildState *
accumArrayResult(ArrayBuildState *astate,
Datum dvalue, bool disnull,
Oid element_type,
MemoryContext rcontext)
{
MemoryContext arr_context,
oldcontext;
if (astate == NULL)
{
/* First time through --- initialize */
/* Make a temporary context to hold all the junk */
arr_context = AllocSetContextCreate(rcontext,
"ARRAY_SUBLINK Result",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
oldcontext = MemoryContextSwitchTo(arr_context);
astate = (ArrayBuildState *) palloc(sizeof(ArrayBuildState));
astate->mcontext = arr_context;
astate->dvalues = (Datum *)
palloc(ARRAY_ELEMS_CHUNKSIZE * sizeof(Datum));
astate->nelems = 0;
astate->element_type = element_type;
get_typlenbyvalalign(element_type,
&astate->typlen,
&astate->typbyval,
&astate->typalign);
}
else
{
oldcontext = MemoryContextSwitchTo(astate->mcontext);
Assert(astate->element_type == element_type);
/* enlarge dvalues[] if needed */
if ((astate->nelems % ARRAY_ELEMS_CHUNKSIZE) == 0)
astate->dvalues = (Datum *)
repalloc(astate->dvalues,
(astate->nelems + ARRAY_ELEMS_CHUNKSIZE) * sizeof(Datum));
}
if (disnull)
elog(ERROR, "NULL elements not allowed in Arrays");
/* Use datumCopy to ensure pass-by-ref stuff is copied into mcontext */
astate->dvalues[astate->nelems++] =
datumCopy(dvalue, astate->typbyval, astate->typlen);
MemoryContextSwitchTo(oldcontext);
return astate;
}
/*
* makeArrayResult - produce final result of ARRAY_SUBLINK
*
* astate is working state (not NULL)
* rcontext is where to construct result
*/
static Datum
makeArrayResult(ArrayBuildState *astate,
MemoryContext rcontext)
{
ArrayType *result;
int dims[1];
int lbs[1];
MemoryContext oldcontext;
/* Build the final array result in rcontext */
oldcontext = MemoryContextSwitchTo(rcontext);
dims[0] = astate->nelems;
lbs[0] = 1;
result = construct_md_array(astate->dvalues,
1,
dims,
lbs,
astate->element_type,
astate->typlen,
astate->typbyval,
astate->typalign);
MemoryContextSwitchTo(oldcontext);
/* Clean up all the junk */
MemoryContextDelete(astate->mcontext);
return PointerGetDatum(result);
}

View File

@ -15,7 +15,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/nodes/copyfuncs.c,v 1.247 2003/03/20 07:02:08 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/nodes/copyfuncs.c,v 1.248 2003/04/08 23:20:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -738,10 +738,8 @@ _copyArrayRef(ArrayRef *from)
ArrayRef *newnode = makeNode(ArrayRef);
COPY_SCALAR_FIELD(refrestype);
COPY_SCALAR_FIELD(refattrlength);
COPY_SCALAR_FIELD(refelemlength);
COPY_SCALAR_FIELD(refelembyval);
COPY_SCALAR_FIELD(refelemalign);
COPY_SCALAR_FIELD(refarraytype);
COPY_SCALAR_FIELD(refelemtype);
COPY_NODE_FIELD(refupperindexpr);
COPY_NODE_FIELD(reflowerindexpr);
COPY_NODE_FIELD(refexpr);
@ -919,6 +917,22 @@ _copyCaseWhen(CaseWhen *from)
return newnode;
}
/*
* _copyArrayExpr
*/
static ArrayExpr *
_copyArrayExpr(ArrayExpr *from)
{
ArrayExpr *newnode = makeNode(ArrayExpr);
COPY_SCALAR_FIELD(array_typeid);
COPY_SCALAR_FIELD(element_typeid);
COPY_NODE_FIELD(elements);
COPY_SCALAR_FIELD(ndims);
return newnode;
}
/*
* _copyCoalesceExpr
*/
@ -2537,6 +2551,9 @@ copyObject(void *from)
case T_CaseWhen:
retval = _copyCaseWhen(from);
break;
case T_ArrayExpr:
retval = _copyArrayExpr(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
* $Header: /cvsroot/pgsql/src/backend/nodes/equalfuncs.c,v 1.190 2003/03/20 07:02:08 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/nodes/equalfuncs.c,v 1.191 2003/04/08 23:20:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -212,10 +212,8 @@ static bool
_equalArrayRef(ArrayRef *a, ArrayRef *b)
{
COMPARE_SCALAR_FIELD(refrestype);
COMPARE_SCALAR_FIELD(refattrlength);
COMPARE_SCALAR_FIELD(refelemlength);
COMPARE_SCALAR_FIELD(refelembyval);
COMPARE_SCALAR_FIELD(refelemalign);
COMPARE_SCALAR_FIELD(refarraytype);
COMPARE_SCALAR_FIELD(refelemtype);
COMPARE_NODE_FIELD(refupperindexpr);
COMPARE_NODE_FIELD(reflowerindexpr);
COMPARE_NODE_FIELD(refexpr);
@ -378,6 +376,17 @@ _equalCaseWhen(CaseWhen *a, CaseWhen *b)
return true;
}
static bool
_equalArrayExpr(ArrayExpr *a, ArrayExpr *b)
{
COMPARE_SCALAR_FIELD(array_typeid);
COMPARE_SCALAR_FIELD(element_typeid);
COMPARE_NODE_FIELD(elements);
COMPARE_SCALAR_FIELD(ndims);
return true;
}
static bool
_equalCoalesceExpr(CoalesceExpr *a, CoalesceExpr *b)
{
@ -1661,6 +1670,9 @@ equal(void *a, void *b)
case T_CaseWhen:
retval = _equalCaseWhen(a, b);
break;
case T_ArrayExpr:
retval = _equalArrayExpr(a, b);
break;
case T_CoalesceExpr:
retval = _equalCoalesceExpr(a, b);
break;

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/nodes/outfuncs.c,v 1.201 2003/03/10 03:53:49 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/nodes/outfuncs.c,v 1.202 2003/04/08 23:20:01 tgl Exp $
*
* NOTES
* Every node type that can appear in stored rules' parsetrees *must*
@ -607,10 +607,8 @@ _outArrayRef(StringInfo str, ArrayRef *node)
WRITE_NODE_TYPE("ARRAYREF");
WRITE_OID_FIELD(refrestype);
WRITE_INT_FIELD(refattrlength);
WRITE_INT_FIELD(refelemlength);
WRITE_BOOL_FIELD(refelembyval);
WRITE_CHAR_FIELD(refelemalign);
WRITE_OID_FIELD(refarraytype);
WRITE_OID_FIELD(refelemtype);
WRITE_NODE_FIELD(refupperindexpr);
WRITE_NODE_FIELD(reflowerindexpr);
WRITE_NODE_FIELD(refexpr);
@ -753,6 +751,17 @@ _outCaseWhen(StringInfo str, CaseWhen *node)
WRITE_NODE_FIELD(result);
}
static void
_outArrayExpr(StringInfo str, ArrayExpr *node)
{
WRITE_NODE_TYPE("ARRAY");
WRITE_OID_FIELD(array_typeid);
WRITE_OID_FIELD(element_typeid);
WRITE_NODE_FIELD(elements);
WRITE_INT_FIELD(ndims);
}
static void
_outCoalesceExpr(StringInfo str, CoalesceExpr *node)
{
@ -1610,6 +1619,9 @@ _outNode(StringInfo str, void *obj)
case T_CaseWhen:
_outCaseWhen(str, obj);
break;
case T_ArrayExpr:
_outArrayExpr(str, obj);
break;
case T_CoalesceExpr:
_outCoalesceExpr(str, obj);
break;

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/nodes/readfuncs.c,v 1.150 2003/03/10 03:53:49 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/nodes/readfuncs.c,v 1.151 2003/04/08 23:20:01 tgl Exp $
*
* NOTES
* Path and Plan nodes do not have any readfuncs support, because we
@ -427,10 +427,8 @@ _readArrayRef(void)
READ_LOCALS(ArrayRef);
READ_OID_FIELD(refrestype);
READ_INT_FIELD(refattrlength);
READ_INT_FIELD(refelemlength);
READ_BOOL_FIELD(refelembyval);
READ_CHAR_FIELD(refelemalign);
READ_OID_FIELD(refarraytype);
READ_OID_FIELD(refelemtype);
READ_NODE_FIELD(refupperindexpr);
READ_NODE_FIELD(reflowerindexpr);
READ_NODE_FIELD(refexpr);
@ -619,6 +617,22 @@ _readCaseWhen(void)
READ_DONE();
}
/*
* _readArrayExpr
*/
static ArrayExpr *
_readArrayExpr(void)
{
READ_LOCALS(ArrayExpr);
READ_OID_FIELD(array_typeid);
READ_OID_FIELD(element_typeid);
READ_NODE_FIELD(elements);
READ_INT_FIELD(ndims);
READ_DONE();
}
/*
* _readCoalesceExpr
*/
@ -947,6 +961,8 @@ parseNodeString(void)
return_value = _readCaseExpr();
else if (MATCH("WHEN", 4))
return_value = _readCaseWhen();
else if (MATCH("ARRAY", 5))
return_value = _readArrayExpr();
else if (MATCH("COALESCE", 8))
return_value = _readCoalesceExpr();
else if (MATCH("NULLIFEXPR", 10))

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/subselect.c,v 1.73 2003/03/10 03:53:50 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/subselect.c,v 1.74 2003/04/08 23:20:01 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -28,6 +28,7 @@
#include "parser/parse_expr.h"
#include "parser/parse_oper.h"
#include "parser/parse_relation.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@ -264,9 +265,9 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
bms_free(tmpset);
/*
* Un-correlated or undirect correlated plans of EXISTS, EXPR, or
* MULTIEXPR types can be used as initPlans. For EXISTS or EXPR, we
* just produce a Param referring to the result of evaluating the
* Un-correlated or undirect correlated plans of EXISTS, EXPR, ARRAY, or
* MULTIEXPR 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.
@ -291,6 +292,22 @@ make_subplan(SubLink *slink, List *lefthand, bool isTopQual)
PlannerInitPlan = lappend(PlannerInitPlan, node);
result = (Node *) prm;
}
else if (node->parParam == NIL && slink->subLinkType == ARRAY_SUBLINK)
{
TargetEntry *te = lfirst(plan->targetlist);
Oid arraytype;
Param *prm;
Assert(!te->resdom->resjunk);
arraytype = get_array_type(te->resdom->restype);
if (!OidIsValid(arraytype))
elog(ERROR, "Cannot find array type for datatype %s",
format_type_be(te->resdom->restype));
prm = generate_new_param(arraytype, -1);
node->setParam = makeListi1(prm->paramid);
PlannerInitPlan = lappend(PlannerInitPlan, node);
result = (Node *) prm;
}
else if (node->parParam == NIL && slink->subLinkType == MULTIEXPR_SUBLINK)
{
List *exprs;
@ -441,9 +458,6 @@ convert_sublink_opers(List *lefthand, List *operOids,
TargetEntry *te = lfirst(targetlist);
Node *rightop;
Operator tup;
Form_pg_operator opform;
Node *left,
*right;
Assert(!te->resdom->resjunk);
@ -470,28 +484,25 @@ convert_sublink_opers(List *lefthand, List *operOids,
rightop = (Node *) prm;
}
/* Look up the operator to get its declared input types */
/* 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);
opform = (Form_pg_operator) GETSTRUCT(tup);
/*
* Make the expression node.
*
* Note: we use make_operand in case runtime type conversion
* Note: we use make_op_expr in case runtime type conversion
* function calls must be inserted for this operator!
*/
left = make_operand(leftop, exprType(leftop), opform->oprleft);
right = make_operand(rightop, te->resdom->restype, opform->oprright);
result = lappend(result,
make_opclause(opid,
opform->oprresult,
false, /* set-result not allowed */
(Expr *) left,
(Expr *) right));
make_op_expr(tup,
leftop,
rightop,
exprType(leftop),
te->resdom->restype));
ReleaseSysCache(tup);

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/clauses.c,v 1.133 2003/03/22 01:49:38 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/clauses.c,v 1.134 2003/04/08 23:20:01 tgl Exp $
*
* HISTORY
* AUTHOR DATE MAJOR EVENT
@ -59,15 +59,17 @@ static bool contain_mutable_functions_walker(Node *node, void *context);
static bool contain_volatile_functions_walker(Node *node, void *context);
static bool contain_nonstrict_functions_walker(Node *node, void *context);
static Node *eval_const_expressions_mutator(Node *node, List *active_fns);
static Expr *simplify_function(Oid funcid, List *args, bool allow_inline,
List *active_fns);
static Expr *evaluate_function(Oid funcid, List *args, HeapTuple func_tuple);
static Expr *inline_function(Oid funcid, List *args, HeapTuple func_tuple,
List *active_fns);
static Expr *simplify_function(Oid funcid, Oid result_type, List *args,
bool allow_inline, List *active_fns);
static Expr *evaluate_function(Oid funcid, Oid result_type, List *args,
HeapTuple func_tuple);
static Expr *inline_function(Oid funcid, Oid result_type, List *args,
HeapTuple func_tuple, List *active_fns);
static Node *substitute_actual_parameters(Node *expr, int nargs, List *args,
int *usecounts);
static Node *substitute_actual_parameters_mutator(Node *node,
substitute_actual_parameters_context *context);
static Expr *evaluate_expr(Expr *expr, Oid result_type);
/*****************************************************************************
@ -464,6 +466,8 @@ expression_returns_set_walker(Node *node, void *context)
return false;
if (IsA(node, SubPlan))
return false;
if (IsA(node, ArrayExpr))
return false;
if (IsA(node, CoalesceExpr))
return false;
if (IsA(node, NullIfExpr))
@ -722,6 +726,7 @@ contain_nonstrict_functions_walker(Node *node, void *context)
}
if (IsA(node, CaseExpr))
return true;
/* NB: ArrayExpr might someday be nonstrict */
if (IsA(node, CoalesceExpr))
return true;
if (IsA(node, NullIfExpr))
@ -1028,7 +1033,8 @@ eval_const_expressions_mutator(Node *node, List *active_fns)
* Code for op/func reduction is pretty bulky, so split it out
* as a separate function.
*/
simple = simplify_function(expr->funcid, args, true, active_fns);
simple = simplify_function(expr->funcid, expr->funcresulttype, args,
true, active_fns);
if (simple) /* successfully simplified it */
return (Node *) simple;
/*
@ -1068,7 +1074,8 @@ eval_const_expressions_mutator(Node *node, List *active_fns)
* Code for op/func reduction is pretty bulky, so split it out
* as a separate function.
*/
simple = simplify_function(expr->opfuncid, args, true, active_fns);
simple = simplify_function(expr->opfuncid, expr->opresulttype, args,
true, active_fns);
if (simple) /* successfully simplified it */
return (Node *) simple;
/*
@ -1143,8 +1150,8 @@ eval_const_expressions_mutator(Node *node, List *active_fns)
* Code for op/func reduction is pretty bulky, so split it out
* as a separate function.
*/
simple = simplify_function(expr->opfuncid, args,
false, active_fns);
simple = simplify_function(expr->opfuncid, expr->opresulttype,
args, false, active_fns);
if (simple) /* successfully simplified it */
{
/*
@ -1442,6 +1449,37 @@ eval_const_expressions_mutator(Node *node, List *active_fns)
newcase->defresult = (Expr *) defresult;
return (Node *) newcase;
}
if (IsA(node, ArrayExpr))
{
ArrayExpr *arrayexpr = (ArrayExpr *) node;
ArrayExpr *newarray;
bool all_const = true;
List *newelems = NIL;
List *element;
foreach(element, arrayexpr->elements)
{
Node *e;
e = eval_const_expressions_mutator((Node *) lfirst(element),
active_fns);
if (!IsA(e, Const))
all_const = false;
newelems = lappend(newelems, e);
}
newarray = makeNode(ArrayExpr);
newarray->array_typeid = arrayexpr->array_typeid;
newarray->element_typeid = arrayexpr->element_typeid;
newarray->elements = newelems;
newarray->ndims = arrayexpr->ndims;
if (all_const)
return (Node *) evaluate_expr((Expr *) newarray,
newarray->array_typeid);
return (Node *) newarray;
}
if (IsA(node, CoalesceExpr))
{
CoalesceExpr *coalesceexpr = (CoalesceExpr *) node;
@ -1513,14 +1551,16 @@ eval_const_expressions_mutator(Node *node, List *active_fns)
* Subroutine for eval_const_expressions: try to simplify a function call
* (which might originally have been an operator; we don't care)
*
* Inputs are the function OID and the pre-simplified argument list;
* Inputs are the function OID, actual result type OID (which is needed for
* polymorphic functions), and the pre-simplified argument list;
* also a list of already-active inline function expansions.
*
* Returns a simplified expression if successful, or NULL if cannot
* simplify the function call.
*/
static Expr *
simplify_function(Oid funcid, List *args, bool allow_inline, List *active_fns)
simplify_function(Oid funcid, Oid result_type, List *args,
bool allow_inline, List *active_fns)
{
HeapTuple func_tuple;
Expr *newexpr;
@ -1539,10 +1579,11 @@ simplify_function(Oid funcid, List *args, bool allow_inline, List *active_fns)
if (!HeapTupleIsValid(func_tuple))
elog(ERROR, "Function OID %u does not exist", funcid);
newexpr = evaluate_function(funcid, args, func_tuple);
newexpr = evaluate_function(funcid, result_type, args, func_tuple);
if (!newexpr && allow_inline)
newexpr = inline_function(funcid, args, func_tuple, active_fns);
newexpr = inline_function(funcid, result_type, args,
func_tuple, active_fns);
ReleaseSysCache(func_tuple);
@ -1560,21 +1601,14 @@ simplify_function(Oid funcid, List *args, bool allow_inline, List *active_fns)
* simplify the function.
*/
static Expr *
evaluate_function(Oid funcid, List *args, HeapTuple func_tuple)
evaluate_function(Oid funcid, Oid result_type, List *args,
HeapTuple func_tuple)
{
Form_pg_proc funcform = (Form_pg_proc) GETSTRUCT(func_tuple);
Oid result_typeid = funcform->prorettype;
int16 resultTypLen;
bool resultTypByVal;
bool has_nonconst_input = false;
bool has_null_input = false;
FuncExpr *newexpr;
ExprState *newexprstate;
EState *estate;
MemoryContext oldcontext;
Datum const_val;
bool const_is_null;
List *arg;
FuncExpr *newexpr;
/*
* Can't simplify if it returns a set.
@ -1600,7 +1634,7 @@ evaluate_function(Oid funcid, List *args, HeapTuple func_tuple)
* and even if the function is not otherwise immutable.
*/
if (funcform->proisstrict && has_null_input)
return (Expr *) makeNullConst(result_typeid);
return (Expr *) makeNullConst(result_type);
/*
* Otherwise, can simplify only if the function is immutable and
@ -1614,61 +1648,16 @@ evaluate_function(Oid funcid, List *args, HeapTuple func_tuple)
/*
* OK, looks like we can simplify this operator/function.
*
* We use the executor's routine ExecEvalExpr() to avoid duplication of
* code and ensure we get the same result as the executor would get.
* To use the executor, we need an EState.
*/
estate = CreateExecutorState();
/* We can use the estate's working context to avoid memory leaks. */
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
/*
* Build a new FuncExpr node containing the already-simplified arguments.
*/
newexpr = makeNode(FuncExpr);
newexpr->funcid = funcid;
newexpr->funcresulttype = result_typeid;
newexpr->funcresulttype = result_type;
newexpr->funcretset = false;
newexpr->funcformat = COERCE_EXPLICIT_CALL; /* doesn't matter */
newexpr->args = args;
/*
* Prepare it for execution.
*/
newexprstate = ExecPrepareExpr((Expr *) newexpr, estate);
/*
* And evaluate it.
*
* It is OK to use a default econtext because none of the
* ExecEvalExpr() code used in this situation will use econtext. That
* might seem fortuitous, but it's not so unreasonable --- a constant
* expression does not depend on context, by definition, n'est ce pas?
*/
const_val = ExecEvalExprSwitchContext(newexprstate,
GetPerTupleExprContext(estate),
&const_is_null, NULL);
/* Get info needed about result datatype */
get_typlenbyval(result_typeid, &resultTypLen, &resultTypByVal);
/* Get back to outer memory context */
MemoryContextSwitchTo(oldcontext);
/* Must copy result out of sub-context used by expression eval */
if (!const_is_null)
const_val = datumCopy(const_val, resultTypByVal, resultTypLen);
/* Release all the junk we just created */
FreeExecutorState(estate);
/*
* Make the constant result node.
*/
return (Expr *) makeConst(result_typeid, resultTypLen,
const_val, const_is_null,
resultTypByVal);
return evaluate_expr((Expr *) newexpr, result_type);
}
/*
@ -1693,11 +1682,10 @@ evaluate_function(Oid funcid, List *args, HeapTuple func_tuple)
* simplify the function.
*/
static Expr *
inline_function(Oid funcid, List *args, HeapTuple func_tuple,
List *active_fns)
inline_function(Oid funcid, Oid result_type, List *args,
HeapTuple func_tuple, List *active_fns)
{
Form_pg_proc funcform = (Form_pg_proc) GETSTRUCT(func_tuple);
Oid result_typeid = funcform->prorettype;
char result_typtype;
char *src;
Datum tmp;
@ -1723,8 +1711,8 @@ inline_function(Oid funcid, List *args, HeapTuple func_tuple,
funcform->pronargs != length(args))
return NULL;
/* Forget it if return type is tuple or void */
result_typtype = get_typtype(result_typeid);
/* Forget it if declared return type is tuple or void */
result_typtype = get_typtype(funcform->prorettype);
if (result_typtype != 'b' &&
result_typtype != 'd')
return NULL;
@ -1928,6 +1916,69 @@ substitute_actual_parameters_mutator(Node *node,
(void *) context);
}
/*
* evaluate_expr: pre-evaluate a constant expression
*
* We use the executor's routine ExecEvalExpr() to avoid duplication of
* code and ensure we get the same result as the executor would get.
*/
static Expr *
evaluate_expr(Expr *expr, Oid result_type)
{
EState *estate;
ExprState *exprstate;
MemoryContext oldcontext;
Datum const_val;
bool const_is_null;
int16 resultTypLen;
bool resultTypByVal;
/*
* To use the executor, we need an EState.
*/
estate = CreateExecutorState();
/* We can use the estate's working context to avoid memory leaks. */
oldcontext = MemoryContextSwitchTo(estate->es_query_cxt);
/*
* Prepare expr for execution.
*/
exprstate = ExecPrepareExpr(expr, estate);
/*
* And evaluate it.
*
* It is OK to use a default econtext because none of the
* ExecEvalExpr() code used in this situation will use econtext. That
* might seem fortuitous, but it's not so unreasonable --- a constant
* expression does not depend on context, by definition, n'est ce pas?
*/
const_val = ExecEvalExprSwitchContext(exprstate,
GetPerTupleExprContext(estate),
&const_is_null, NULL);
/* Get info needed about result datatype */
get_typlenbyval(result_type, &resultTypLen, &resultTypByVal);
/* Get back to outer memory context */
MemoryContextSwitchTo(oldcontext);
/* Must copy result out of sub-context used by expression eval */
if (!const_is_null)
const_val = datumCopy(const_val, resultTypByVal, resultTypLen);
/* Release all the junk we just created */
FreeExecutorState(estate);
/*
* Make the constant result node.
*/
return (Expr *) makeConst(result_type, resultTypLen,
const_val, const_is_null,
resultTypByVal);
}
/*
* Standard expression-tree walking support
@ -2159,6 +2210,8 @@ expression_tree_walker(Node *node,
return true;
}
break;
case T_ArrayExpr:
return walker(((ArrayExpr *) node)->elements, context);
case T_CoalesceExpr:
return walker(((CoalesceExpr *) node)->args, context);
case T_NullIfExpr:
@ -2535,6 +2588,16 @@ expression_tree_mutator(Node *node,
return (Node *) newnode;
}
break;
case T_ArrayExpr:
{
ArrayExpr *arrayexpr = (ArrayExpr *) node;
ArrayExpr *newnode;
FLATCOPY(newnode, arrayexpr, ArrayExpr);
MUTATE(newnode->elements, arrayexpr->elements, List *);
return (Node *) newnode;
}
break;
case T_CoalesceExpr:
{
CoalesceExpr *coalesceexpr = (CoalesceExpr *) node;

View File

@ -11,7 +11,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/gram.y,v 2.410 2003/04/01 23:42:55 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/gram.y,v 2.411 2003/04/08 23:20:01 tgl Exp $
*
* HISTORY
* AUTHOR DATE MAJOR EVENT
@ -265,8 +265,8 @@ static void doNegateFloat(Value *v);
%type <defelt> def_elem
%type <node> def_arg columnElem where_clause insert_column_item
a_expr b_expr c_expr r_expr AexprConst
in_expr having_clause func_table
%type <list> row row_descriptor type_list
in_expr having_clause func_table array_expr
%type <list> row row_descriptor type_list array_expr_list
%type <node> case_expr case_arg when_clause case_default
%type <list> when_clause_list
%type <ival> sub_type
@ -323,7 +323,7 @@ static void doNegateFloat(Value *v);
/* ordinary key words in alphabetical order */
%token <keyword> ABORT_P ABSOLUTE ACCESS ACTION ADD AFTER
AGGREGATE ALL ALTER ANALYSE ANALYZE AND ANY AS ASC
AGGREGATE ALL ALTER ANALYSE ANALYZE AND ANY ARRAY AS ASC
ASSERTION ASSIGNMENT AT AUTHORIZATION
BACKWARD BEFORE BEGIN_P BETWEEN BIGINT BINARY BIT
@ -4986,6 +4986,19 @@ Typename: SimpleTypename opt_array_bounds
$$->arrayBounds = $3;
$$->setof = TRUE;
}
| SimpleTypename ARRAY '[' Iconst ']'
{
/* SQL99's redundant syntax */
$$ = $1;
$$->arrayBounds = makeList1(makeInteger($4));
}
| SETOF SimpleTypename ARRAY '[' Iconst ']'
{
/* SQL99's redundant syntax */
$$ = $2;
$$->arrayBounds = makeList1(makeInteger($5));
$$->setof = TRUE;
}
;
opt_array_bounds:
@ -6057,7 +6070,6 @@ c_expr: columnref { $$ = (Node *) $1; }
n->indirection = $3;
$$ = (Node *)n;
}
| '(' a_expr ')' { $$ = $2; }
| '(' a_expr ')' attrs opt_indirection
{
ExprFieldSelect *n = makeNode(ExprFieldSelect);
@ -6066,6 +6078,19 @@ c_expr: columnref { $$ = (Node *) $1; }
n->indirection = $5;
$$ = (Node *)n;
}
| '(' a_expr ')' opt_indirection
{
if ($4)
{
ExprFieldSelect *n = makeNode(ExprFieldSelect);
n->arg = $2;
n->fields = NIL;
n->indirection = $4;
$$ = (Node *)n;
}
else
$$ = $2;
}
| case_expr
{ $$ = $1; }
| func_name '(' ')'
@ -6509,6 +6534,17 @@ c_expr: columnref { $$ = (Node *) $1; }
n->subselect = $2;
$$ = (Node *)n;
}
| ARRAY select_with_parens
{
SubLink *n = makeNode(SubLink);
n->subLinkType = ARRAY_SUBLINK;
n->lefthand = NIL;
n->operName = NIL;
n->subselect = $2;
$$ = (Node *)n;
}
| ARRAY array_expr
{ $$ = $2; }
;
/*
@ -6559,6 +6595,26 @@ type_list: type_list ',' Typename
}
;
array_expr_list: array_expr
{ $$ = makeList1($1); }
| array_expr_list ',' array_expr
{ $$ = lappend($1, $3); }
;
array_expr: '[' expr_list ']'
{
ArrayExpr *n = makeNode(ArrayExpr);
n->elements = $2;
$$ = (Node *)n;
}
| '[' array_expr_list ']'
{
ArrayExpr *n = makeNode(ArrayExpr);
n->elements = $2;
$$ = (Node *)n;
}
;
/* Allow delimited string SCONST in extract_arg as an SQL extension.
* - thomas 2001-04-12
*/
@ -7346,6 +7402,7 @@ reserved_keyword:
| ANALYZE
| AND
| ANY
| ARRAY
| AS
| ASC
| BOTH

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/keywords.c,v 1.137 2003/03/27 16:51:28 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/keywords.c,v 1.138 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -43,6 +43,7 @@ static const ScanKeyword ScanKeywords[] = {
{"analyze", ANALYZE},
{"and", AND},
{"any", ANY},
{"array", ARRAY},
{"as", AS},
{"asc", ASC},
{"assertion", ASSERTION},

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/parse_coerce.c,v 2.93 2003/02/09 06:56:28 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/parse_coerce.c,v 2.94 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -189,7 +189,8 @@ coerce_type(Node *node, Oid inputTypeId, Oid targetTypeId,
ReleaseSysCache(targetType);
}
else if (targetTypeId == ANYOID ||
targetTypeId == ANYARRAYOID)
targetTypeId == ANYARRAYOID ||
targetTypeId == ANYELEMENTOID)
{
/* assume can_coerce_type verified that implicit coercion is okay */
/* NB: we do NOT want a RelabelType here */
@ -295,6 +296,7 @@ bool
can_coerce_type(int nargs, Oid *input_typeids, Oid *target_typeids,
CoercionContext ccontext)
{
bool have_generics = false;
int i;
/* run through argument list... */
@ -329,29 +331,12 @@ can_coerce_type(int nargs, Oid *input_typeids, Oid *target_typeids,
if (targetTypeId == ANYOID)
continue;
/*
* if target is ANYARRAY and source is a varlena array type,
* accept
*/
if (targetTypeId == ANYARRAYOID)
/* accept if target is ANYARRAY or ANYELEMENT, for now */
if (targetTypeId == ANYARRAYOID ||
targetTypeId == ANYELEMENTOID)
{
Oid typOutput;
Oid typElem;
bool typIsVarlena;
if (getTypeOutputInfo(inputTypeId, &typOutput, &typElem,
&typIsVarlena))
{
if (OidIsValid(typElem) && typIsVarlena)
continue;
}
/*
* Otherwise reject; this assumes there are no explicit
* coercion paths to ANYARRAY. If we don't reject then
* parse_coerce would have to repeat the above test.
*/
return false;
have_generics = true; /* do more checking later */
continue;
}
/*
@ -374,6 +359,14 @@ can_coerce_type(int nargs, Oid *input_typeids, Oid *target_typeids,
return false;
}
/* If we found any generic argument types, cross-check them */
if (have_generics)
{
if (!check_generic_type_consistency(input_typeids, target_typeids,
nargs))
return false;
}
return true;
}
@ -644,6 +637,260 @@ coerce_to_common_type(Node *node, Oid targetTypeId, const char *context)
return node;
}
/*
* check_generic_type_consistency()
* Are the actual arguments potentially compatible with a
* polymorphic function?
*
* The argument consistency rules are:
*
* 1) All arguments declared ANYARRAY must have matching datatypes,
* and must in fact be varlena arrays.
* 2) All arguments declared ANYELEMENT must have matching datatypes.
* 3) If there are arguments of both ANYELEMENT and ANYARRAY, make sure
* the actual ANYELEMENT datatype is in fact the element type for
* the actual ANYARRAY datatype.
*
* If we have UNKNOWN input (ie, an untyped literal) for any ANYELEMENT
* or ANYARRAY argument, assume it is okay.
*
* We do not elog here, but just return FALSE if a rule is violated.
*/
bool
check_generic_type_consistency(Oid *actual_arg_types,
Oid *declared_arg_types,
int nargs)
{
int j;
Oid elem_typeid = InvalidOid;
Oid array_typeid = InvalidOid;
Oid array_typelem;
/*
* Loop through the arguments to see if we have any that are
* ANYARRAY or ANYELEMENT. If so, require the actual types to be
* self-consistent
*/
for (j = 0; j < nargs; j++)
{
Oid actual_type = actual_arg_types[j];
if (declared_arg_types[j] == ANYELEMENTOID)
{
if (actual_type == UNKNOWNOID)
continue;
if (OidIsValid(elem_typeid) && actual_type != elem_typeid)
return false;
elem_typeid = actual_type;
}
else if (declared_arg_types[j] == ANYARRAYOID)
{
if (actual_type == UNKNOWNOID)
continue;
if (OidIsValid(array_typeid) && actual_type != array_typeid)
return false;
array_typeid = actual_type;
}
}
/* Get the element type based on the array type, if we have one */
if (OidIsValid(array_typeid))
{
array_typelem = get_element_type(array_typeid);
if (!OidIsValid(array_typelem))
return false; /* should be an array, but isn't */
if (!OidIsValid(elem_typeid))
{
/* if we don't have an element type yet, use the one we just got */
elem_typeid = array_typelem;
}
else if (array_typelem != elem_typeid)
{
/* otherwise, they better match */
return false;
}
}
/* Looks valid */
return true;
}
/*
* enforce_generic_type_consistency()
* Make sure a polymorphic function is legally callable, and
* deduce actual argument and result types.
*
* If ANYARRAY or ANYELEMENT is used for a function's arguments or
* return type, we make sure the actual data types are consistent with
* each other. The argument consistency rules are shown above for
* check_generic_type_consistency().
*
* If we have UNKNOWN input (ie, an untyped literal) for any ANYELEMENT
* or ANYARRAY argument, we attempt to deduce the actual type it should
* have. If successful, we alter that position of declared_arg_types[]
* so that make_fn_arguments will coerce the literal to the right thing.
*
* Rules are applied to the function's return type (possibly altering it)
* if it is declared ANYARRAY or ANYELEMENT:
*
* 1) If return type is ANYARRAY, and any argument is ANYARRAY, use the
* argument's actual type as the function's return type.
* 2) If return type is ANYARRAY, no argument is ANYARRAY, but any argument
* is ANYELEMENT, use the actual type of the argument to determine
* the function's return type, i.e. the element type's corresponding
* array type.
* 3) If return type is ANYARRAY, no argument is ANYARRAY or ANYELEMENT,
* generate an ERROR. This condition is prevented by CREATE FUNCTION
* and is therefore not expected here.
* 4) If return type is ANYELEMENT, and any argument is ANYELEMENT, use the
* argument's actual type as the function's return type.
* 5) If return type is ANYELEMENT, no argument is ANYELEMENT, but any
* argument is ANYARRAY, use the actual type of the argument to determine
* the function's return type, i.e. the array type's corresponding
* element type.
* 6) If return type is ANYELEMENT, no argument is ANYARRAY or ANYELEMENT,
* generate an ERROR. This condition is prevented by CREATE FUNCTION
* and is therefore not expected here.
*/
Oid
enforce_generic_type_consistency(Oid *actual_arg_types,
Oid *declared_arg_types,
int nargs,
Oid rettype)
{
int j;
bool have_generics = false;
bool have_unknowns = false;
Oid elem_typeid = InvalidOid;
Oid array_typeid = InvalidOid;
Oid array_typelem = InvalidOid;
/*
* Loop through the arguments to see if we have any that are
* ANYARRAY or ANYELEMENT. If so, require the actual types to be
* self-consistent
*/
for (j = 0; j < nargs; j++)
{
Oid actual_type = actual_arg_types[j];
if (declared_arg_types[j] == ANYELEMENTOID)
{
have_generics = true;
if (actual_type == UNKNOWNOID)
{
have_unknowns = true;
continue;
}
if (OidIsValid(elem_typeid) && actual_type != elem_typeid)
elog(ERROR, "Arguments declared ANYELEMENT are not all alike: %s vs %s",
format_type_be(elem_typeid),
format_type_be(actual_type));
elem_typeid = actual_type;
}
else if (declared_arg_types[j] == ANYARRAYOID)
{
have_generics = true;
if (actual_type == UNKNOWNOID)
{
have_unknowns = true;
continue;
}
if (OidIsValid(array_typeid) && actual_type != array_typeid)
elog(ERROR, "Arguments declared ANYARRAY are not all alike: %s vs %s",
format_type_be(array_typeid),
format_type_be(actual_type));
array_typeid = actual_type;
}
}
/*
* Fast Track: if none of the arguments are ANYARRAY or ANYELEMENT,
* return the unmodified rettype.
*/
if (!have_generics)
return rettype;
/* Get the element type based on the array type, if we have one */
if (OidIsValid(array_typeid))
{
array_typelem = get_element_type(array_typeid);
if (!OidIsValid(array_typelem))
elog(ERROR, "Argument declared ANYARRAY is not an array: %s",
format_type_be(array_typeid));
if (!OidIsValid(elem_typeid))
{
/* if we don't have an element type yet, use the one we just got */
elem_typeid = array_typelem;
}
else if (array_typelem != elem_typeid)
{
/* otherwise, they better match */
elog(ERROR, "Argument declared ANYARRAY is not consistent with "
"argument declared ANYELEMENT: %s vs %s",
format_type_be(array_typeid),
format_type_be(elem_typeid));
}
}
else if (!OidIsValid(elem_typeid))
{
/* Only way to get here is if all the generic args are UNKNOWN */
elog(ERROR, "Cannot determine ANYARRAY/ANYELEMENT type because input is UNKNOWN");
}
/*
* If we had any unknown inputs, re-scan to assign correct types
*/
if (have_unknowns)
{
for (j = 0; j < nargs; j++)
{
Oid actual_type = actual_arg_types[j];
if (actual_type != UNKNOWNOID)
continue;
if (declared_arg_types[j] == ANYELEMENTOID)
{
declared_arg_types[j] = elem_typeid;
}
else if (declared_arg_types[j] == ANYARRAYOID)
{
if (!OidIsValid(array_typeid))
{
array_typeid = get_array_type(elem_typeid);
if (!OidIsValid(array_typeid))
elog(ERROR, "Cannot find array type for datatype %s",
format_type_be(elem_typeid));
}
declared_arg_types[j] = array_typeid;
}
}
}
/* if we return ANYARRAYOID use the appropriate argument type */
if (rettype == ANYARRAYOID)
{
if (!OidIsValid(array_typeid))
{
array_typeid = get_array_type(elem_typeid);
if (!OidIsValid(array_typeid))
elog(ERROR, "Cannot find array type for datatype %s",
format_type_be(elem_typeid));
}
return array_typeid;
}
/* if we return ANYELEMENTOID use the appropriate argument type */
if (rettype == ANYELEMENTOID)
return elem_typeid;
/* we don't return a generic type; send back the original return type */
return rettype;
}
/* TypeCategory()
* Assign a category to the specified OID.
@ -727,6 +974,19 @@ TypeCategory(Oid inType)
result = UNKNOWN_TYPE;
break;
case (RECORDOID):
case (CSTRINGOID):
case (ANYOID):
case (ANYARRAYOID):
case (VOIDOID):
case (TRIGGEROID):
case (LANGUAGE_HANDLEROID):
case (INTERNALOID):
case (OPAQUEOID):
case (ANYELEMENTOID):
result = GENERIC_TYPE;
break;
default:
result = USER_TYPE;
break;
@ -761,6 +1021,12 @@ PreferredType(CATEGORY category, Oid type)
switch (category)
{
case (INVALID_TYPE):
case (UNKNOWN_TYPE):
case (GENERIC_TYPE):
result = UNKNOWNOID;
break;
case (BOOLEAN_TYPE):
result = BOOLOID;
break;
@ -797,16 +1063,20 @@ PreferredType(CATEGORY category, Oid type)
result = INTERVALOID;
break;
case (GEOMETRIC_TYPE):
result = type;
break;
case (NETWORK_TYPE):
result = INETOID;
break;
case (GEOMETRIC_TYPE):
case (USER_TYPE):
result = type;
break;
default:
elog(ERROR, "PreferredType: unknown category");
result = UNKNOWNOID;
break;
}
@ -897,7 +1167,7 @@ find_coercion_pathway(Oid targetTypeId, Oid sourceTypeId,
if (sourceTypeId == targetTypeId)
return true;
/* Else look in pg_cast */
/* Look in pg_cast */
tuple = SearchSysCache(CASTSOURCETARGET,
ObjectIdGetDatum(sourceTypeId),
ObjectIdGetDatum(targetTypeId),
@ -936,6 +1206,28 @@ find_coercion_pathway(Oid targetTypeId, Oid sourceTypeId,
ReleaseSysCache(tuple);
}
else
{
/*
* If there's no pg_cast entry, perhaps we are dealing with a
* pair of array types. If so, and if the element types have
* a suitable cast, use array_type_coerce().
*/
Oid targetElemType;
Oid sourceElemType;
Oid elemfuncid;
if ((targetElemType = get_element_type(targetTypeId)) != InvalidOid &&
(sourceElemType = get_element_type(sourceTypeId)) != InvalidOid)
{
if (find_coercion_pathway(targetElemType, sourceElemType,
ccontext, &elemfuncid))
{
*funcid = F_ARRAY_TYPE_COERCE;
result = true;
}
}
}
return result;
}

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/parse_expr.c,v 1.146 2003/02/16 02:30:38 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/parse_expr.c,v 1.147 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -395,7 +395,8 @@ transformExpr(ParseState *pstate, Node *expr)
sublink->operOids = NIL;
sublink->useOr = FALSE;
}
else if (sublink->subLinkType == EXPR_SUBLINK)
else if (sublink->subLinkType == EXPR_SUBLINK ||
sublink->subLinkType == ARRAY_SUBLINK)
{
List *tlist = qtree->targetList;
@ -413,8 +414,8 @@ transformExpr(ParseState *pstate, Node *expr)
}
/*
* EXPR needs no lefthand or combining operator. These
* fields should be NIL already, but make sure.
* EXPR and ARRAY need no lefthand or combining operator.
* These fields should be NIL already, but make sure.
*/
sublink->lefthand = NIL;
sublink->operName = NIL;
@ -633,6 +634,98 @@ transformExpr(ParseState *pstate, Node *expr)
break;
}
case T_ArrayExpr:
{
ArrayExpr *a = (ArrayExpr *) expr;
ArrayExpr *newa = makeNode(ArrayExpr);
List *newelems = NIL;
List *newcoercedelems = NIL;
List *typeids = NIL;
List *element;
Oid array_type;
Oid element_type;
int ndims;
/* Transform the element expressions */
foreach(element, a->elements)
{
Node *e = (Node *) lfirst(element);
Node *newe;
newe = transformExpr(pstate, e);
newelems = lappend(newelems, newe);
typeids = lappendo(typeids, exprType(newe));
}
/* Select a common type for the elements */
element_type = select_common_type(typeids, "ARRAY");
/* Coerce arguments to common type if necessary */
foreach(element, newelems)
{
Node *e = (Node *) lfirst(element);
Node *newe;
newe = coerce_to_common_type(e, element_type, "ARRAY");
newcoercedelems = lappend(newcoercedelems, newe);
}
/* Do we have an array type to use? */
array_type = get_array_type(element_type);
if (array_type != InvalidOid)
{
/* Elements are presumably of scalar type */
ndims = 1;
}
else
{
/* Must be nested array expressions */
array_type = element_type;
element_type = get_element_type(array_type);
if (!OidIsValid(element_type))
elog(ERROR, "Cannot find array type for datatype %s",
format_type_be(array_type));
/*
* make sure the element expressions all have the same
* number of dimensions
*/
ndims = 0;
foreach(element, newcoercedelems)
{
ArrayExpr *e = (ArrayExpr *) lfirst(element);
if (!IsA(e, ArrayExpr))
elog(ERROR, "Multi-dimensional ARRAY[] must be built from nested array expressions");
if (ndims == 0)
ndims = e->ndims;
else if (e->ndims != ndims)
elog(ERROR, "Nested array expressions must have "
"common number of dimensions");
if (e->element_typeid != element_type)
elog(ERROR, "Nested array expressions must have "
"common element type");
}
/* increment the number of dimensions */
ndims++;
/* make sure we don't have too many dimensions now */
if (ndims > MAXDIM)
elog(ERROR, "Number of array dimensions, %d, "
"exceeds the maximum allowed %d",
ndims, MAXDIM);
}
newa->array_typeid = array_type;
newa->element_typeid = element_type;
newa->elements = newcoercedelems;
newa->ndims = ndims;
result = (Node *) newa;
break;
}
case T_CoalesceExpr:
{
CoalesceExpr *c = (CoalesceExpr *) expr;
@ -1018,7 +1111,8 @@ exprType(Node *expr)
{
SubLink *sublink = (SubLink *) expr;
if (sublink->subLinkType == EXPR_SUBLINK)
if (sublink->subLinkType == EXPR_SUBLINK ||
sublink->subLinkType == ARRAY_SUBLINK)
{
/* get the type of the subselect's first target column */
Query *qtree = (Query *) sublink->subselect;
@ -1029,7 +1123,15 @@ exprType(Node *expr)
tent = (TargetEntry *) lfirst(qtree->targetList);
Assert(IsA(tent, TargetEntry));
Assert(!tent->resdom->resjunk);
type = tent->resdom->restype;
if (sublink->subLinkType == EXPR_SUBLINK)
type = tent->resdom->restype;
else /* ARRAY_SUBLINK */
{
type = get_array_type(tent->resdom->restype);
if (!OidIsValid(type))
elog(ERROR, "Cannot find array type for datatype %s",
format_type_be(tent->resdom->restype));
}
}
else
{
@ -1047,7 +1149,8 @@ exprType(Node *expr)
*/
SubPlan *subplan = (SubPlan *) expr;
if (subplan->subLinkType == EXPR_SUBLINK)
if (subplan->subLinkType == EXPR_SUBLINK ||
subplan->subLinkType == ARRAY_SUBLINK)
{
/* get the type of the subselect's first target column */
TargetEntry *tent;
@ -1055,7 +1158,15 @@ exprType(Node *expr)
tent = (TargetEntry *) lfirst(subplan->plan->targetlist);
Assert(IsA(tent, TargetEntry));
Assert(!tent->resdom->resjunk);
type = tent->resdom->restype;
if (subplan->subLinkType == EXPR_SUBLINK)
type = tent->resdom->restype;
else /* ARRAY_SUBLINK */
{
type = get_array_type(tent->resdom->restype);
if (!OidIsValid(type))
elog(ERROR, "Cannot find array type for datatype %s",
format_type_be(tent->resdom->restype));
}
}
else
{
@ -1076,6 +1187,9 @@ exprType(Node *expr)
case T_CaseWhen:
type = exprType((Node *) ((CaseWhen *) expr)->result);
break;
case T_ArrayExpr:
type = ((ArrayExpr *) expr)->array_typeid;
break;
case T_CoalesceExpr:
type = ((CoalesceExpr *) expr)->coalescetype;
break;

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/parse_func.c,v 1.144 2003/02/09 06:56:28 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/parse_func.c,v 1.145 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -37,10 +37,6 @@ static Oid **argtype_inherit(int nargs, Oid *argtypes);
static int find_inheritors(Oid relid, Oid **supervec);
static Oid **gen_cross_product(InhPaths *arginh, int nargs);
static void make_arguments(int nargs,
List *fargs,
Oid *input_typeids,
Oid *function_typeids);
static int match_argtypes(int nargs,
Oid *input_typeids,
FuncCandidateList function_typeids,
@ -81,8 +77,8 @@ ParseFuncOrColumn(ParseState *pstate, List *funcname, List *fargs,
Node *first_arg = NULL;
int nargs = length(fargs);
int argn;
Oid oid_array[FUNC_MAX_ARGS];
Oid *true_oid_array;
Oid actual_arg_types[FUNC_MAX_ARGS];
Oid *declared_arg_types;
Node *retval;
bool retset;
FuncDetailCode fdresult;
@ -145,7 +141,7 @@ ParseFuncOrColumn(ParseState *pstate, List *funcname, List *fargs,
* function. Extract arg type info and transform RangeVar arguments
* into varnodes of the appropriate form.
*/
MemSet(oid_array, 0, FUNC_MAX_ARGS * sizeof(Oid));
MemSet(actual_arg_types, 0, FUNC_MAX_ARGS * sizeof(Oid));
argn = 0;
foreach(i, fargs)
@ -238,7 +234,7 @@ ParseFuncOrColumn(ParseState *pstate, List *funcname, List *fargs,
else
toid = exprType(arg);
oid_array[argn++] = toid;
actual_arg_types[argn++] = toid;
}
/*
@ -248,16 +244,16 @@ ParseFuncOrColumn(ParseState *pstate, List *funcname, List *fargs,
* function's return value. it also returns the true argument types
* to the function.
*/
fdresult = func_get_detail(funcname, fargs, nargs, oid_array,
fdresult = func_get_detail(funcname, fargs, nargs, actual_arg_types,
&funcid, &rettype, &retset,
&true_oid_array);
&declared_arg_types);
if (fdresult == FUNCDETAIL_COERCION)
{
/*
* We can do it as a trivial coercion. coerce_type can handle
* these cases, so why duplicate code...
*/
return coerce_type(lfirst(fargs), oid_array[0], rettype,
return coerce_type(lfirst(fargs), actual_arg_types[0], rettype,
COERCION_EXPLICIT, COERCE_EXPLICIT_CALL);
}
else if (fdresult == FUNCDETAIL_NORMAL)
@ -303,14 +299,24 @@ ParseFuncOrColumn(ParseState *pstate, List *funcname, List *fargs,
/*
* Else generate a detailed complaint for a function
*/
func_error(NULL, funcname, nargs, oid_array,
func_error(NULL, funcname, nargs, actual_arg_types,
"Unable to identify a function that satisfies the "
"given argument types"
"\n\tYou may need to add explicit typecasts");
}
/*
* enforce consistency with ANYARRAY and ANYELEMENT argument and
* return types, possibly adjusting return type or declared_arg_types
* (which will be used as the cast destination by make_fn_arguments)
*/
rettype = enforce_generic_type_consistency(actual_arg_types,
declared_arg_types,
nargs,
rettype);
/* perform the necessary typecasting of arguments */
make_arguments(nargs, fargs, oid_array, true_oid_array);
make_fn_arguments(fargs, actual_arg_types, declared_arg_types);
/* build the appropriate output structure */
if (fdresult == FUNCDETAIL_NORMAL)
@ -1130,32 +1136,36 @@ typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId)
}
/* make_arguments()
* Given the number and types of arguments to a function, and the
* actual arguments and argument types, do the necessary typecasting.
/*
* make_fn_arguments()
*
* Given the actual argument expressions for a function, and the desired
* input types for the function, add any necessary typecasting to the
* expression tree. Caller should already have verified that casting is
* allowed.
*
* Caution: given argument list is modified in-place.
*/
static void
make_arguments(int nargs,
List *fargs,
Oid *input_typeids,
Oid *function_typeids)
void
make_fn_arguments(List *fargs,
Oid *actual_arg_types,
Oid *declared_arg_types)
{
List *current_fargs;
int i;
int i = 0;
for (i = 0, current_fargs = fargs;
i < nargs;
i++, current_fargs = lnext(current_fargs))
foreach(current_fargs, fargs)
{
/* types don't match? then force coercion using a function call... */
if (input_typeids[i] != function_typeids[i])
if (actual_arg_types[i] != declared_arg_types[i])
{
lfirst(current_fargs) = coerce_type(lfirst(current_fargs),
input_typeids[i],
function_typeids[i],
actual_arg_types[i],
declared_arg_types[i],
COERCION_IMPLICIT,
COERCE_IMPLICIT_CAST);
}
i++;
}
}

View File

@ -8,25 +8,21 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/parse_node.c,v 1.76 2002/12/12 20:35:13 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/parse_node.c,v 1.77 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/heapam.h"
#include "catalog/pg_operator.h"
#include "catalog/pg_type.h"
#include "nodes/makefuncs.h"
#include "parser/parsetree.h"
#include "parser/parse_coerce.h"
#include "parser/parse_expr.h"
#include "parser/parse_node.h"
#include "parser/parse_oper.h"
#include "parser/parse_relation.h"
#include "utils/builtins.h"
#include "utils/int8.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
#include "utils/varbit.h"
@ -49,99 +45,6 @@ make_parsestate(ParseState *parentParseState)
}
/* make_operand()
* Ensure argument type match by forcing conversion of constants.
*/
Node *
make_operand(Node *tree, Oid orig_typeId, Oid target_typeId)
{
Node *result;
if (tree != NULL)
{
/* must coerce? */
if (target_typeId != orig_typeId)
result = coerce_type(tree, orig_typeId, target_typeId,
COERCION_IMPLICIT, COERCE_IMPLICIT_CAST);
else
result = tree;
}
else
{
/* otherwise, this is a NULL value */
result = (Node *) makeNullConst(target_typeId);
}
return result;
} /* make_operand() */
/* make_op()
* Operator construction.
*
* Transform operator expression ensuring type compatibility.
* This is where some type conversion happens.
*/
Expr *
make_op(List *opname, Node *ltree, Node *rtree)
{
Oid ltypeId,
rtypeId;
Operator tup;
Form_pg_operator opform;
Node *left,
*right;
OpExpr *result;
ltypeId = (ltree == NULL) ? UNKNOWNOID : exprType(ltree);
rtypeId = (rtree == NULL) ? UNKNOWNOID : exprType(rtree);
/* right operator? */
if (rtree == NULL)
{
tup = right_oper(opname, ltypeId, false);
opform = (Form_pg_operator) GETSTRUCT(tup);
left = make_operand(ltree, ltypeId, opform->oprleft);
right = NULL;
}
/* left operator? */
else if (ltree == NULL)
{
tup = left_oper(opname, rtypeId, false);
opform = (Form_pg_operator) GETSTRUCT(tup);
right = make_operand(rtree, rtypeId, opform->oprright);
left = NULL;
}
/* otherwise, binary operator */
else
{
tup = oper(opname, ltypeId, rtypeId, false);
opform = (Form_pg_operator) GETSTRUCT(tup);
left = make_operand(ltree, ltypeId, opform->oprleft);
right = make_operand(rtree, rtypeId, opform->oprright);
}
result = makeNode(OpExpr);
result->opno = oprid(tup);
result->opfuncid = InvalidOid;
result->opresulttype = opform->oprresult;
result->opretset = get_func_retset(opform->oprcode);
if (!left)
result->args = makeList1(right);
else if (!right)
result->args = makeList1(left);
else
result->args = makeList2(left, right);
ReleaseSysCache(tup);
return (Expr *) result;
} /* make_op() */
/*
* make_var
* Build a Var node for an attribute identified by RTE and attrno
@ -193,10 +96,8 @@ transformArraySubscripts(ParseState *pstate,
{
Oid elementType,
resultType;
HeapTuple type_tuple_array,
type_tuple_element;
Form_pg_type type_struct_array,
type_struct_element;
HeapTuple type_tuple_array;
Form_pg_type type_struct_array;
bool isSlice = forceSlice;
List *upperIndexpr = NIL;
List *lowerIndexpr = NIL;
@ -217,15 +118,6 @@ transformArraySubscripts(ParseState *pstate,
elog(ERROR, "transformArraySubscripts: type %s is not an array",
NameStr(type_struct_array->typname));
/* Get the type tuple for the array element type */
type_tuple_element = SearchSysCache(TYPEOID,
ObjectIdGetDatum(elementType),
0, 0, 0);
if (!HeapTupleIsValid(type_tuple_element))
elog(ERROR, "transformArraySubscripts: Cache lookup failed for array element type %u",
elementType);
type_struct_element = (Form_pg_type) GETSTRUCT(type_tuple_element);
/*
* A list containing only single subscripts refers to a single array
* element. If any of the items are double subscripts (lower:upper),
@ -330,19 +222,15 @@ transformArraySubscripts(ParseState *pstate,
* Ready to build the ArrayRef node.
*/
aref = makeNode(ArrayRef);
aref->refrestype = resultType; /* XXX should save element type
* OID too */
aref->refattrlength = type_struct_array->typlen;
aref->refelemlength = type_struct_element->typlen;
aref->refelembyval = type_struct_element->typbyval;
aref->refelemalign = type_struct_element->typalign;
aref->refrestype = resultType;
aref->refarraytype = arrayType;
aref->refelemtype = elementType;
aref->refupperindexpr = upperIndexpr;
aref->reflowerindexpr = lowerIndexpr;
aref->refexpr = (Expr *) arrayBase;
aref->refassgnexpr = (Expr *) assignFrom;
ReleaseSysCache(type_tuple_array);
ReleaseSysCache(type_tuple_element);
return aref;
}

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/parse_oper.c,v 1.61 2002/11/29 21:39:11 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/parse_oper.c,v 1.62 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -22,13 +22,16 @@
#include "catalog/namespace.h"
#include "catalog/pg_operator.h"
#include "parser/parse_coerce.h"
#include "parser/parse_expr.h"
#include "parser/parse_func.h"
#include "parser/parse_oper.h"
#include "parser/parse_type.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
static Oid binary_oper_exact(Oid arg1, Oid arg2,
FuncCandidateList candidates);
static Oid oper_select_candidate(int nargs, Oid *input_typeids,
@ -1008,3 +1011,118 @@ unary_op_error(List *op, Oid arg, bool is_left_op)
NameListToString(op), format_type_be(arg));
}
}
/*
* make_op()
* Operator expression construction.
*
* Transform operator expression ensuring type compatibility.
* This is where some type conversion happens.
*/
Expr *
make_op(List *opname, Node *ltree, Node *rtree)
{
Oid ltypeId,
rtypeId;
Operator tup;
Expr *result;
/* Select the operator */
if (rtree == NULL)
{
/* right operator */
ltypeId = exprType(ltree);
rtypeId = InvalidOid;
tup = right_oper(opname, ltypeId, false);
}
else if (ltree == NULL)
{
/* left operator */
rtypeId = exprType(rtree);
ltypeId = InvalidOid;
tup = left_oper(opname, rtypeId, false);
}
else
{
/* otherwise, binary operator */
ltypeId = exprType(ltree);
rtypeId = exprType(rtree);
tup = oper(opname, ltypeId, rtypeId, false);
}
/* Do typecasting and build the expression tree */
result = make_op_expr(tup, ltree, rtree, ltypeId, rtypeId);
ReleaseSysCache(tup);
return result;
}
/*
* make_op_expr()
* Build operator expression using an already-looked-up operator.
*/
Expr *
make_op_expr(Operator op, Node *ltree, Node *rtree,
Oid ltypeId, Oid rtypeId)
{
Form_pg_operator opform = (Form_pg_operator) GETSTRUCT(op);
Oid actual_arg_types[2];
Oid declared_arg_types[2];
int nargs;
List *args;
Oid rettype;
OpExpr *result;
if (rtree == NULL)
{
/* right operator */
args = makeList1(ltree);
actual_arg_types[0] = ltypeId;
declared_arg_types[0] = opform->oprleft;
nargs = 1;
}
else if (ltree == NULL)
{
/* left operator */
args = makeList1(rtree);
actual_arg_types[0] = rtypeId;
declared_arg_types[0] = opform->oprright;
nargs = 1;
}
else
{
/* otherwise, binary operator */
args = makeList2(ltree, rtree);
actual_arg_types[0] = ltypeId;
actual_arg_types[1] = rtypeId;
declared_arg_types[0] = opform->oprleft;
declared_arg_types[1] = opform->oprright;
nargs = 2;
}
/*
* enforce consistency with ANYARRAY and ANYELEMENT argument and
* return types, possibly adjusting return type or declared_arg_types
* (which will be used as the cast destination by make_fn_arguments)
*/
rettype = enforce_generic_type_consistency(actual_arg_types,
declared_arg_types,
nargs,
opform->oprresult);
/* perform the necessary typecasting of arguments */
make_fn_arguments(args, actual_arg_types, declared_arg_types);
/* and build the expression node */
result = makeNode(OpExpr);
result->opno = oprid(op);
result->opfuncid = InvalidOid;
result->opresulttype = rettype;
result->opretset = get_func_retset(opform->oprcode);
result->args = args;
return (Expr *) result;
}

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/parser/parse_target.c,v 1.98 2003/02/16 02:30:38 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/parser/parse_target.c,v 1.99 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -470,13 +470,19 @@ FigureColnameInternal(Node *node, char **name)
break;
case T_ExprFieldSelect:
{
char *fname = strVal(llast(((ExprFieldSelect *) node)->fields));
ExprFieldSelect *efs = (ExprFieldSelect *) node;
if (strcmp(fname, "*") != 0)
if (efs->fields)
{
*name = fname;
return 2;
char *fname = strVal(llast(efs->fields));
if (strcmp(fname, "*") != 0)
{
*name = fname;
return 2;
}
}
return FigureColnameInternal(efs->arg, name);
}
break;
case T_FuncCall:
@ -518,6 +524,10 @@ FigureColnameInternal(Node *node, char **name)
return 1;
}
break;
case T_ArrayExpr:
/* make ARRAY[] act like a function */
*name = "array";
return 2;
case T_CoalesceExpr:
/* make coalesce() act like a regular function */
*name = "coalesce";

View File

@ -1,7 +1,7 @@
#
# Makefile for utils/adt
#
# $Header: /cvsroot/pgsql/src/backend/utils/adt/Makefile,v 1.53 2002/08/22 00:01:43 tgl Exp $
# $Header: /cvsroot/pgsql/src/backend/utils/adt/Makefile,v 1.54 2003/04/08 23:20:02 tgl Exp $
#
subdir = src/backend/utils/adt
@ -15,8 +15,8 @@ override CFLAGS+= -mieee
endif
endif
OBJS = acl.o arrayfuncs.o arrayutils.o bool.o cash.o char.o \
date.o datetime.o datum.o float.o format_type.o \
OBJS = acl.o arrayfuncs.o array_userfuncs.o arrayutils.o bool.o \
cash.o char.o date.o datetime.o datum.o float.o format_type.o \
geo_ops.o geo_selfuncs.o int.o int8.o like.o lockfuncs.o \
misc.o nabstime.o name.o not_in.o numeric.o numutils.o \
oid.o oracle_compat.o pseudotypes.o \

View File

@ -0,0 +1,436 @@
/*-------------------------------------------------------------------------
*
* array_userfuncs.c
* Misc user-visible array support functions
*
* Copyright (c) 2003, PostgreSQL Global Development Group
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/adt/array_userfuncs.c,v 1.1 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
#include "utils/array.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
/*-----------------------------------------------------------------------------
* singleton_array :
* Form a multi-dimensional array given one starting element.
*
* - first argument is the datum with which to build the array
* - second argument is the number of dimensions the array should have;
* defaults to 1 if no second argument is provided
*----------------------------------------------------------------------------
*/
Datum
singleton_array(PG_FUNCTION_ARGS)
{
Oid elem_type = get_fn_expr_argtype(fcinfo, 0);
int ndims;
if (elem_type == InvalidOid)
elog(ERROR, "Cannot determine input datatype");
if (PG_NARGS() == 2)
ndims = PG_GETARG_INT32(1);
else
ndims = 1;
PG_RETURN_ARRAYTYPE_P(create_singleton_array(elem_type,
PG_GETARG_DATUM(0),
ndims));
}
/*-----------------------------------------------------------------------------
* array_push :
* push an element onto either end of a one-dimensional array
*----------------------------------------------------------------------------
*/
Datum
array_push(PG_FUNCTION_ARGS)
{
ArrayType *v;
Datum newelem;
int *dimv,
*lb;
ArrayType *result;
int indx;
bool isNull;
Oid element_type;
int16 typlen;
bool typbyval;
char typalign;
Oid arg0_typeid = get_fn_expr_argtype(fcinfo, 0);
Oid arg1_typeid = get_fn_expr_argtype(fcinfo, 1);
Oid arg0_elemid;
Oid arg1_elemid;
if (arg0_typeid == InvalidOid || arg1_typeid == InvalidOid)
elog(ERROR, "array_push: cannot determine input data types");
arg0_elemid = get_element_type(arg0_typeid);
arg1_elemid = get_element_type(arg1_typeid);
if (arg0_elemid != InvalidOid)
{
v = PG_GETARG_ARRAYTYPE_P(0);
element_type = ARR_ELEMTYPE(v);
newelem = PG_GETARG_DATUM(1);
}
else if (arg1_elemid != InvalidOid)
{
v = PG_GETARG_ARRAYTYPE_P(1);
element_type = ARR_ELEMTYPE(v);
newelem = PG_GETARG_DATUM(0);
}
else
{
/* Shouldn't get here given proper type checking in parser */
elog(ERROR, "array_push: neither input type is an array");
PG_RETURN_NULL(); /* keep compiler quiet */
}
/* Sanity check: do we have a one-dimensional array */
if (ARR_NDIM(v) != 1)
elog(ERROR, "Arrays greater than one-dimension are not supported");
lb = ARR_LBOUND(v);
dimv = ARR_DIMS(v);
if (arg0_elemid != InvalidOid)
{
/* append newelem */
int ub = dimv[0] + lb[0] - 1;
indx = ub + 1;
}
else
{
/* prepend newelem */
indx = lb[0] - 1;
}
get_typlenbyvalalign(element_type, &typlen, &typbyval, &typalign);
result = array_set(v, 1, &indx, newelem, -1,
typlen, typbyval, typalign, &isNull);
PG_RETURN_ARRAYTYPE_P(result);
}
/*-----------------------------------------------------------------------------
* array_cat :
* concatenate two nD arrays to form an (n+1)D array, or
* push an (n-1)D array onto the end of an nD array
*----------------------------------------------------------------------------
*/
Datum
array_cat(PG_FUNCTION_ARGS)
{
ArrayType *v1, *v2;
int *dims, *lbs, ndims, ndatabytes, nbytes;
int *dims1, *lbs1, ndims1, ndatabytes1;
int *dims2, *lbs2, ndims2, ndatabytes2;
char *dat1, *dat2;
Oid element_type;
Oid element_type1;
Oid element_type2;
ArrayType *result;
v1 = PG_GETARG_ARRAYTYPE_P(0);
v2 = PG_GETARG_ARRAYTYPE_P(1);
/*
* We must have one of the following combinations of inputs:
* 1) two arrays with ndims1 == ndims2
* 2) ndims1 == ndims2 - 1
* 3) ndims1 == ndims2 + 1
*/
ndims1 = ARR_NDIM(v1);
ndims2 = ARR_NDIM(v2);
if (ndims1 != ndims2 && ndims1 != ndims2 - 1 && ndims1 != ndims2 + 1)
elog(ERROR, "Cannot concatenate incompatible arrays of %d and "
"%d dimensions", ndims1, ndims2);
element_type1 = ARR_ELEMTYPE(v1);
element_type2 = ARR_ELEMTYPE(v2);
/* Do we have a matching element types */
if (element_type1 != element_type2)
elog(ERROR, "Cannot concatenate incompatible arrays with element "
"type %u and %u", element_type1, element_type2);
/* OK, use it */
element_type = element_type1;
/* get argument array details */
lbs1 = ARR_LBOUND(v1);
lbs2 = ARR_LBOUND(v2);
dims1 = ARR_DIMS(v1);
dims2 = ARR_DIMS(v2);
dat1 = ARR_DATA_PTR(v1);
dat2 = ARR_DATA_PTR(v2);
ndatabytes1 = ARR_SIZE(v1) - ARR_OVERHEAD(ndims1);
ndatabytes2 = ARR_SIZE(v2) - ARR_OVERHEAD(ndims2);
if (ndims1 == ndims2)
{
/*
* resulting array has two element outer array made up of input
* argument arrays
*/
int i;
ndims = ndims1 + 1;
dims = (int *) palloc(ndims * sizeof(int));
lbs = (int *) palloc(ndims * sizeof(int));
dims[0] = 2; /* outer array made up of two input arrays */
lbs[0] = 1; /* start lower bound at 1 */
for (i = 0; i < ndims1; i++)
{
if (dims1[i] != dims2[i] || lbs1[i] != lbs2[i])
elog(ERROR, "Cannot concatenate arrays with differing dimensions");
dims[i + 1] = dims1[i];
lbs[i + 1] = lbs1[i];
}
}
else if (ndims1 == ndims2 - 1)
{
/*
* resulting array has the second argument as the outer array,
* with the first argument appended to the front of the outer
* dimension
*/
int i;
ndims = ndims2;
dims = dims2;
lbs = lbs2;
/* increment number of elements in outer array */
dims[0] += 1;
/* make sure the added element matches our existing elements */
for (i = 0; i < ndims1; i++)
{
if (dims1[i] != dims[i + 1] || lbs1[i] != lbs[i + 1])
elog(ERROR, "Cannot concatenate arrays with differing dimensions");
}
}
else /* (ndims1 == ndims2 + 1) */
{
/*
* resulting array has the first argument as the outer array,
* with the second argument appended to the end of the outer
* dimension
*/
int i;
ndims = ndims1;
dims = dims1;
lbs = lbs1;
/* increment number of elements in outer array */
dims[0] += 1;
/* make sure the added element matches our existing elements */
for (i = 0; i < ndims2; i++)
{
if (dims2[i] != dims[i + 1] || lbs2[i] != lbs[i + 1])
elog(ERROR, "Cannot concatenate arrays with differing dimensions");
}
}
/* build the result array */
ndatabytes = ndatabytes1 + ndatabytes2;
nbytes = ndatabytes + ARR_OVERHEAD(ndims);
result = (ArrayType *) palloc(nbytes);
result->size = nbytes;
result->ndim = ndims;
result->flags = 0;
result->elemtype = element_type;
memcpy(ARR_DIMS(result), dims, ndims * sizeof(int));
memcpy(ARR_LBOUND(result), lbs, ndims * sizeof(int));
/* data area is arg1 then arg2 */
memcpy(ARR_DATA_PTR(result), dat1, ndatabytes1);
memcpy(ARR_DATA_PTR(result) + ndatabytes1, dat2, ndatabytes2);
PG_RETURN_ARRAYTYPE_P(result);
}
/*----------------------------------------------------------------------------
* array_accum :
* accumulator to build a 1-D array from input values -- this can be used
* to create custom aggregates.
*
* This function is not marked strict, so we have to be careful about nulls.
*----------------------------------------------------------------------------
*/
Datum
array_accum(PG_FUNCTION_ARGS)
{
/* return NULL if both arguments are NULL */
if (PG_ARGISNULL(0) && PG_ARGISNULL(1))
PG_RETURN_NULL();
/* create a new 1-D array from the new element if the array is NULL */
if (PG_ARGISNULL(0))
{
Oid tgt_type = get_fn_expr_rettype(fcinfo);
Oid tgt_elem_type;
if (tgt_type == InvalidOid)
elog(ERROR, "Cannot determine target array type");
tgt_elem_type = get_element_type(tgt_type);
if (tgt_elem_type == InvalidOid)
elog(ERROR, "Target type is not an array");
PG_RETURN_ARRAYTYPE_P(create_singleton_array(tgt_elem_type,
PG_GETARG_DATUM(1),
1));
}
/* return the array if the new element is NULL */
if (PG_ARGISNULL(1))
PG_RETURN_ARRAYTYPE_P(PG_GETARG_ARRAYTYPE_P_COPY(0));
/*
* Otherwise this is equivalent to array_push. We hack the call a little
* so that array_push can see the fn_expr information.
*/
return array_push(fcinfo);
}
/*-----------------------------------------------------------------------------
* array_assign :
* assign an element of an array to a new value and return the
* redefined array
*----------------------------------------------------------------------------
*/
Datum
array_assign(PG_FUNCTION_ARGS)
{
ArrayType *v;
int idx_to_chg;
Datum newelem;
int *dimv,
*lb, ub;
ArrayType *result;
bool isNull;
Oid element_type;
int16 typlen;
bool typbyval;
char typalign;
v = PG_GETARG_ARRAYTYPE_P(0);
idx_to_chg = PG_GETARG_INT32(1);
newelem = PG_GETARG_DATUM(2);
/* Sanity check: do we have a one-dimensional array */
if (ARR_NDIM(v) != 1)
elog(ERROR, "Arrays greater than one-dimension are not supported");
lb = ARR_LBOUND(v);
dimv = ARR_DIMS(v);
ub = dimv[0] + lb[0] - 1;
if (idx_to_chg < lb[0] || idx_to_chg > ub)
elog(ERROR, "Cannot alter nonexistent array element: %d", idx_to_chg);
element_type = ARR_ELEMTYPE(v);
/* Sanity check: do we have a non-zero element type */
if (element_type == 0)
elog(ERROR, "Invalid array element type: %u", element_type);
get_typlenbyvalalign(element_type, &typlen, &typbyval, &typalign);
result = array_set(v, 1, &idx_to_chg, newelem, -1,
typlen, typbyval, typalign, &isNull);
PG_RETURN_ARRAYTYPE_P(result);
}
/*-----------------------------------------------------------------------------
* array_subscript :
* return specific element of an array
*----------------------------------------------------------------------------
*/
Datum
array_subscript(PG_FUNCTION_ARGS)
{
ArrayType *v;
int idx;
int *dimv,
*lb, ub;
Datum result;
bool isNull;
Oid element_type;
int16 typlen;
bool typbyval;
char typalign;
v = PG_GETARG_ARRAYTYPE_P(0);
idx = PG_GETARG_INT32(1);
/* Sanity check: do we have a one-dimensional array */
if (ARR_NDIM(v) != 1)
elog(ERROR, "Arrays greater than one-dimension are not supported");
lb = ARR_LBOUND(v);
dimv = ARR_DIMS(v);
ub = dimv[0] + lb[0] - 1;
if (idx < lb[0] || idx > ub)
elog(ERROR, "Cannot return nonexistent array element: %d", idx);
element_type = ARR_ELEMTYPE(v);
/* Sanity check: do we have a non-zero element type */
if (element_type == 0)
elog(ERROR, "Invalid array element type: %u", element_type);
get_typlenbyvalalign(element_type, &typlen, &typbyval, &typalign);
result = array_ref(v, 1, &idx, -1, typlen, typbyval, typalign, &isNull);
PG_RETURN_DATUM(result);
}
/*
* actually does the work for singleton_array(), and array_accum() if it is
* given a null input array.
*/
ArrayType *
create_singleton_array(Oid element_type, Datum element, int ndims)
{
Datum dvalues[1];
int16 typlen;
bool typbyval;
char typalign;
int dims[MAXDIM];
int lbs[MAXDIM];
int i;
if (element_type == 0)
elog(ERROR, "Invalid array element type: %u", element_type);
if (ndims < 1 || ndims > MAXDIM)
elog(ERROR, "Invalid number of dimensions %d", ndims);
dvalues[0] = element;
for (i = 0; i < ndims; i++)
{
dims[i] = 1;
lbs[i] = 1;
}
get_typlenbyvalalign(element_type, &typlen, &typbyval, &typalign);
return construct_md_array(dvalues, ndims, dims, lbs, element_type,
typlen, typbyval, typalign);
}

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/adt/arrayfuncs.c,v 1.86 2003/01/29 01:28:33 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/utils/adt/arrayfuncs.c,v 1.87 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -21,7 +21,9 @@
#include "catalog/pg_type.h"
#include "parser/parse_coerce.h"
#include "utils/array.h"
#include "utils/builtins.h"
#include "utils/memutils.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
@ -763,7 +765,11 @@ array_length_coerce(PG_FUNCTION_ARGS)
int32 len = PG_GETARG_INT32(1);
bool isExplicit = PG_GETARG_BOOL(2);
FmgrInfo *fmgr_info = fcinfo->flinfo;
FmgrInfo *element_finfo;
typedef struct {
Oid elemtype;
FmgrInfo coerce_finfo;
} alc_extra;
alc_extra *my_extra;
FunctionCallInfoData locfcinfo;
/* If no typmod is provided, shortcircuit the whole thing */
@ -772,33 +778,38 @@ array_length_coerce(PG_FUNCTION_ARGS)
/*
* We arrange to look up the element type's coercion function only
* once per series of calls.
* once per series of calls, assuming the element type doesn't change
* underneath us.
*/
if (fmgr_info->fn_extra == NULL)
my_extra = (alc_extra *) fmgr_info->fn_extra;
if (my_extra == NULL)
{
fmgr_info->fn_extra = MemoryContextAlloc(fmgr_info->fn_mcxt,
sizeof(alc_extra));
my_extra = (alc_extra *) fmgr_info->fn_extra;
my_extra->elemtype = InvalidOid;
}
if (my_extra->elemtype != ARR_ELEMTYPE(v))
{
Oid funcId;
int nargs;
fmgr_info->fn_extra = MemoryContextAlloc(fmgr_info->fn_mcxt,
sizeof(FmgrInfo));
element_finfo = (FmgrInfo *) fmgr_info->fn_extra;
funcId = find_typmod_coercion_function(ARR_ELEMTYPE(v), &nargs);
if (OidIsValid(funcId))
fmgr_info_cxt(funcId, element_finfo, fmgr_info->fn_mcxt);
fmgr_info_cxt(funcId, &my_extra->coerce_finfo, fmgr_info->fn_mcxt);
else
element_finfo->fn_oid = InvalidOid;
my_extra->coerce_finfo.fn_oid = InvalidOid;
my_extra->elemtype = ARR_ELEMTYPE(v);
}
else
element_finfo = (FmgrInfo *) fmgr_info->fn_extra;
/*
* If we didn't find a coercion function, return the array unmodified
* (this should not happen in the normal course of things, but might
* happen if this function is called manually).
*/
if (element_finfo->fn_oid == InvalidOid)
if (my_extra->coerce_finfo.fn_oid == InvalidOid)
PG_RETURN_ARRAYTYPE_P(v);
/*
@ -807,7 +818,7 @@ array_length_coerce(PG_FUNCTION_ARGS)
* Note: we pass isExplicit whether or not the function wants it ...
*/
MemSet(&locfcinfo, 0, sizeof(locfcinfo));
locfcinfo.flinfo = element_finfo;
locfcinfo.flinfo = &my_extra->coerce_finfo;
locfcinfo.nargs = 3;
locfcinfo.arg[0] = PointerGetDatum(v);
locfcinfo.arg[1] = Int32GetDatum(len);
@ -1617,19 +1628,63 @@ array_map(FunctionCallInfo fcinfo, Oid inpType, Oid retType)
* NULL element values are not supported.
*
* NOTE: it would be cleaner to look up the elmlen/elmbval/elmalign info
* from the system catalogs, given the elmtype. However, in most current
* uses the type is hard-wired into the caller and so we can save a lookup
* cycle by hard-wiring the type info as well.
* from the system catalogs, given the elmtype. However, the caller is
* in a better position to cache this info across multiple uses, or even
* to hard-wire values if the element type is hard-wired.
*----------
*/
ArrayType *
construct_array(Datum *elems, int nelems,
Oid elmtype,
int elmlen, bool elmbyval, char elmalign)
{
int dims[1];
int lbs[1];
dims[0] = nelems;
lbs[0] = 1;
return construct_md_array(elems, 1, dims, lbs,
elmtype, elmlen, elmbyval, elmalign);
}
/*----------
* construct_md_array --- simple method for constructing an array object
* with arbitrary dimensions
*
* elems: array of Datum items to become the array contents
* ndims: number of dimensions
* dims: integer array with size of each dimension
* lbs: integer array with lower bound of each dimension
* elmtype, elmlen, elmbyval, elmalign: info for the datatype of the items
*
* A palloc'd ndims-D array object is constructed and returned. Note that
* elem values will be copied into the object even if pass-by-ref type.
* NULL element values are not supported.
*
* NOTE: it would be cleaner to look up the elmlen/elmbval/elmalign info
* from the system catalogs, given the elmtype. However, the caller is
* in a better position to cache this info across multiple uses, or even
* to hard-wire values if the element type is hard-wired.
*----------
*/
ArrayType *
construct_md_array(Datum *elems,
int ndims,
int *dims,
int *lbs,
Oid elmtype, int elmlen, bool elmbyval, char elmalign)
{
ArrayType *result;
int nbytes;
int i;
int nelems;
if (ndims < 1 || ndims > MAXDIM)
elog(ERROR, "Number of array dimensions, %d, exceeds the maximum allowed (%d)",
ndims, MAXDIM);
nelems = ArrayGetNItems(ndims, dims);
/* compute required space */
if (elmlen > 0)
@ -1648,17 +1703,16 @@ construct_array(Datum *elems, int nelems,
}
}
/* Allocate and initialize 1-D result array */
nbytes += ARR_OVERHEAD(1);
/* Allocate and initialize ndims-D result array */
nbytes += ARR_OVERHEAD(ndims);
result = (ArrayType *) palloc(nbytes);
result->size = nbytes;
result->ndim = 1;
result->ndim = ndims;
result->flags = 0;
result->elemtype = elmtype;
ARR_DIMS(result)[0] = nelems;
ARR_LBOUND(result)[0] = 1;
memcpy((char *) ARR_DIMS(result), (char *) dims, ndims * sizeof(int));
memcpy((char *) ARR_LBOUND(result), (char *) lbs, ndims * sizeof(int));
CopyArrayEls(ARR_DATA_PTR(result), elems, nelems,
elmlen, elmbyval, elmalign, false);
@ -2035,3 +2089,82 @@ array_insert_slice(int ndim,
/* don't miss any data at the end */
memcpy(destPtr, origPtr, origEndpoint - origPtr);
}
/*
* array_type_coerce -- allow explicit or assignment coercion from
* one array type to another.
*
* Caller should have already verified that the source element type can be
* coerced into the target element type.
*/
Datum
array_type_coerce(PG_FUNCTION_ARGS)
{
ArrayType *src = PG_GETARG_ARRAYTYPE_P(0);
Oid src_elem_type = ARR_ELEMTYPE(src);
FmgrInfo *fmgr_info = fcinfo->flinfo;
typedef struct {
Oid srctype;
Oid desttype;
FmgrInfo coerce_finfo;
} atc_extra;
atc_extra *my_extra;
FunctionCallInfoData locfcinfo;
/*
* We arrange to look up the coercion function only once per series of
* calls, assuming the input data type doesn't change underneath us.
* (Output type can't change.)
*/
my_extra = (atc_extra *) fmgr_info->fn_extra;
if (my_extra == NULL)
{
fmgr_info->fn_extra = MemoryContextAlloc(fmgr_info->fn_mcxt,
sizeof(atc_extra));
my_extra = (atc_extra *) fmgr_info->fn_extra;
my_extra->srctype = InvalidOid;
}
if (my_extra->srctype != src_elem_type)
{
Oid tgt_type = get_fn_expr_rettype(fcinfo);
Oid tgt_elem_type;
Oid funcId;
if (tgt_type == InvalidOid)
elog(ERROR, "Cannot determine target array type");
tgt_elem_type = get_element_type(tgt_type);
if (tgt_elem_type == InvalidOid)
elog(ERROR, "Target type is not an array");
if (!find_coercion_pathway(tgt_elem_type, src_elem_type,
COERCION_EXPLICIT, &funcId))
{
/* should never happen, but check anyway */
elog(ERROR, "no conversion function from %s to %s",
format_type_be(src_elem_type), format_type_be(tgt_elem_type));
}
if (OidIsValid(funcId))
fmgr_info_cxt(funcId, &my_extra->coerce_finfo, fmgr_info->fn_mcxt);
else
my_extra->coerce_finfo.fn_oid = InvalidOid;
my_extra->srctype = src_elem_type;
my_extra->desttype = tgt_elem_type;
}
/*
* If it's binary-compatible, return the array unmodified.
*/
if (my_extra->coerce_finfo.fn_oid == InvalidOid)
PG_RETURN_ARRAYTYPE_P(src);
/*
* Use array_map to apply the function to each array element.
*/
MemSet(&locfcinfo, 0, sizeof(locfcinfo));
locfcinfo.flinfo = &my_extra->coerce_finfo;
locfcinfo.nargs = 1;
locfcinfo.arg[0] = PointerGetDatum(src);
return array_map(&locfcinfo, my_extra->srctype, my_extra->desttype);
}

View File

@ -16,7 +16,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/adt/pseudotypes.c,v 1.4 2002/09/04 20:31:28 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/utils/adt/pseudotypes.c,v 1.5 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -238,3 +238,26 @@ opaque_out(PG_FUNCTION_ARGS)
PG_RETURN_VOID(); /* keep compiler quiet */
}
/*
* anyelement_in - input routine for pseudo-type ANYELEMENT.
*/
Datum
anyelement_in(PG_FUNCTION_ARGS)
{
elog(ERROR, "Cannot accept a constant of type %s", "ANYELEMENT");
PG_RETURN_VOID(); /* keep compiler quiet */
}
/*
* anyelement_out - output routine for pseudo-type ANYELEMENT.
*/
Datum
anyelement_out(PG_FUNCTION_ARGS)
{
elog(ERROR, "Cannot display a value of type %s", "ANYELEMENT");
PG_RETURN_VOID(); /* keep compiler quiet */
}

View File

@ -3,7 +3,7 @@
* back to source text
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/adt/ruleutils.c,v 1.137 2003/03/20 18:58:02 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/utils/adt/ruleutils.c,v 1.138 2003/04/08 23:20:02 tgl Exp $
*
* This software is copyrighted by Jan Wieck - Hamburg.
*
@ -2217,6 +2217,7 @@ get_rule_expr(Node *node, deparse_context *context,
{
ArrayRef *aref = (ArrayRef *) node;
bool savevarprefix = context->varprefix;
bool need_parens;
List *lowlist;
List *uplist;
@ -2229,7 +2230,16 @@ get_rule_expr(Node *node, deparse_context *context,
*/
if (aref->refassgnexpr)
context->varprefix = false;
/*
* Parenthesize the argument unless it's a simple Var.
*/
need_parens = (aref->refassgnexpr == NULL) &&
!IsA(aref->refexpr, Var);
if (need_parens)
appendStringInfoChar(buf, '(');
get_rule_expr((Node *) aref->refexpr, context, showimplicit);
if (need_parens)
appendStringInfoChar(buf, ')');
context->varprefix = savevarprefix;
lowlist = aref->reflowerindexpr;
foreach(uplist, aref->refupperindexpr)
@ -2421,6 +2431,26 @@ get_rule_expr(Node *node, deparse_context *context,
}
break;
case T_ArrayExpr:
{
ArrayExpr *arrayexpr = (ArrayExpr *) node;
List *element;
char *sep;
appendStringInfo(buf, "ARRAY[");
sep = "";
foreach(element, arrayexpr->elements)
{
Node *e = (Node *) lfirst(element);
appendStringInfo(buf, sep);
get_rule_expr(e, context, true);
sep = ", ";
}
appendStringInfo(buf, "]");
}
break;
case T_CoalesceExpr:
{
CoalesceExpr *coalesceexpr = (CoalesceExpr *) node;
@ -2906,7 +2936,10 @@ get_sublink_expr(SubLink *sublink, deparse_context *context)
char *sep;
bool need_paren;
appendStringInfoChar(buf, '(');
if (sublink->subLinkType == ARRAY_SUBLINK)
appendStringInfo(buf, "ARRAY(");
else
appendStringInfoChar(buf, '(');
if (sublink->lefthand != NIL)
{
@ -2967,6 +3000,7 @@ get_sublink_expr(SubLink *sublink, deparse_context *context)
break;
case EXPR_SUBLINK:
case ARRAY_SUBLINK:
need_paren = false;
break;

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/cache/lsyscache.c,v 1.91 2003/03/23 05:14:36 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/utils/cache/lsyscache.c,v 1.92 2003/04/08 23:20:02 tgl Exp $
*
* NOTES
* Eventually, the index information should go through here, too.
@ -1265,6 +1265,87 @@ get_typ_typrelid(Oid typid)
return InvalidOid;
}
/*
* get_element_type
*
* Given the type OID, get the typelem (InvalidOid if not an array type).
*
* NB: this only considers varlena arrays to be true arrays; InvalidOid is
* returned if the input is a fixed-length array type.
*/
Oid
get_element_type(Oid typid)
{
HeapTuple tp;
tp = SearchSysCache(TYPEOID,
ObjectIdGetDatum(typid),
0, 0, 0);
if (HeapTupleIsValid(tp))
{
Form_pg_type typtup = (Form_pg_type) GETSTRUCT(tp);
Oid result;
if (typtup->typlen == -1)
result = typtup->typelem;
else
result = InvalidOid;
ReleaseSysCache(tp);
return result;
}
else
return InvalidOid;
}
/*
* get_array_type
*
* Given the type OID, get the corresponding array type.
* Returns InvalidOid if no array type can be found.
*
* NB: this only considers varlena arrays to be true arrays.
*/
Oid
get_array_type(Oid typid)
{
HeapTuple tp;
tp = SearchSysCache(TYPEOID,
ObjectIdGetDatum(typid),
0, 0, 0);
if (HeapTupleIsValid(tp))
{
Form_pg_type typtup = (Form_pg_type) GETSTRUCT(tp);
char *array_typename;
Oid namespaceId;
array_typename = makeArrayTypeName(NameStr(typtup->typname));
namespaceId = typtup->typnamespace;
ReleaseSysCache(tp);
tp = SearchSysCache(TYPENAMENSP,
PointerGetDatum(array_typename),
ObjectIdGetDatum(namespaceId),
0, 0);
pfree(array_typename);
if (HeapTupleIsValid(tp))
{
Oid result;
typtup = (Form_pg_type) GETSTRUCT(tp);
if (typtup->typlen == -1 && typtup->typelem == typid)
result = HeapTupleGetOid(tp);
else
result = InvalidOid;
ReleaseSysCache(tp);
return result;
}
}
return InvalidOid;
}
/*
* getTypeInputInfo
*

View File

@ -67,6 +67,7 @@ typedef struct
bool fn_retset; /* function returns a set (over multiple calls) */
void *fn_extra; /* extra space for use by handler */
MemoryContext fn_mcxt; /* memory context to store fn_extra in */
Node *fn_expr; /* expression parse tree for call, or NULL */
} FmgrInfo;
For an ordinary built-in function, fn_addr is just the address of the C
@ -80,7 +81,10 @@ a function handler could set it to avoid making repeated lookups of its
own when the same FmgrInfo is used repeatedly during a query.) fn_nargs
is the number of arguments expected by the function, fn_strict is its
strictness flag, and fn_retset shows whether it returns a set; all of
these values come from the function's pg_proc entry.
these values come from the function's pg_proc entry. If the function is
being called as part of a SQL expression, fn_expr will point to the
expression parse tree for the function call; this can be used to extract
parse-time knowledge about the actual arguments.
FmgrInfo already exists in the current code, but has fewer fields. This
change should be transparent at the source-code level.

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/utils/fmgr/fmgr.c,v 1.67 2002/12/05 04:04:44 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/utils/fmgr/fmgr.c,v 1.68 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -20,6 +20,7 @@
#include "catalog/pg_proc.h"
#include "executor/functions.h"
#include "miscadmin.h"
#include "parser/parse_expr.h"
#include "utils/builtins.h"
#include "utils/fmgrtab.h"
#include "utils/lsyscache.h"
@ -164,6 +165,7 @@ fmgr_info_cxt_security(Oid functionId, FmgrInfo *finfo, MemoryContext mcxt,
finfo->fn_oid = InvalidOid;
finfo->fn_extra = NULL;
finfo->fn_mcxt = mcxt;
finfo->fn_expr = NULL; /* caller may set this later */
if ((fbp = fmgr_isbuiltin(functionId)) != NULL)
{
@ -1611,3 +1613,63 @@ pg_detoast_datum_slice(struct varlena * datum, int32 first, int32 count)
/* Only get the specified portion from the toast rel */
return (struct varlena *) heap_tuple_untoast_attr_slice((varattrib *) datum, first, count);
}
/*-------------------------------------------------------------------------
* Support routines for extracting info from fn_expr parse tree
*-------------------------------------------------------------------------
*/
/*
* Get the OID of the function return type
*
* Returns InvalidOid if information is not available
*/
Oid
get_fn_expr_rettype(FunctionCallInfo fcinfo)
{
Node *expr;
/*
* can't return anything useful if we have no FmgrInfo or if
* its fn_expr node has not been initialized
*/
if (!fcinfo || !fcinfo->flinfo || !fcinfo->flinfo->fn_expr)
return InvalidOid;
expr = fcinfo->flinfo->fn_expr;
return exprType(expr);
}
/*
* Get the type OID of a specific function argument (counting from 0)
*
* Returns InvalidOid if information is not available
*/
Oid
get_fn_expr_argtype(FunctionCallInfo fcinfo, int argnum)
{
Node *expr;
List *args;
/*
* can't return anything useful if we have no FmgrInfo or if
* its fn_expr node has not been initialized
*/
if (!fcinfo || !fcinfo->flinfo || !fcinfo->flinfo->fn_expr)
return InvalidOid;
expr = fcinfo->flinfo->fn_expr;
if (IsA(expr, FuncExpr))
args = ((FuncExpr *) expr)->args;
else if (IsA(expr, OpExpr))
args = ((OpExpr *) expr)->args;
else
return InvalidOid;
if (argnum < 0 || argnum >= length(args))
return InvalidOid;
return exprType((Node *) nth(argnum, args));
}

View File

@ -37,7 +37,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: catversion.h,v 1.183 2003/03/23 05:14:37 tgl Exp $
* $Id: catversion.h,v 1.184 2003/04/08 23:20:02 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -53,6 +53,6 @@
*/
/* yyyymmddN */
#define CATALOG_VERSION_NO 200303221
#define CATALOG_VERSION_NO 200304071
#endif

View File

@ -8,7 +8,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: pg_operator.h,v 1.111 2003/03/10 22:28:19 tgl Exp $
* $Id: pg_operator.h,v 1.112 2003/04/08 23:20:03 tgl Exp $
*
* NOTES
* the genbki.sh script reads this file and generates .bki
@ -116,31 +116,10 @@ DATA(insert OID = 96 ( "=" PGNSP PGUID b t 23 23 16 96 518 97 97 97 521 int
DATA(insert OID = 97 ( "<" PGNSP PGUID b f 23 23 16 521 525 0 0 0 0 int4lt scalarltsel scalarltjoinsel ));
DATA(insert OID = 98 ( "=" PGNSP PGUID b t 25 25 16 98 531 664 664 664 666 texteq eqsel eqjoinsel ));
DATA(insert OID = 329 ( "=" PGNSP PGUID b f 1000 1000 16 329 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 349 ( "=" PGNSP PGUID b f 1001 1001 16 349 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 374 ( "=" PGNSP PGUID b f 1002 1002 16 374 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 375 ( "=" PGNSP PGUID b f 1003 1003 16 375 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 377 ( "=" PGNSP PGUID b f 1005 1005 16 377 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 378 ( "=" PGNSP PGUID b f 1006 1006 16 378 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 379 ( "=" PGNSP PGUID b f 1007 1007 16 379 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 380 ( "=" PGNSP PGUID b f 1008 1008 16 380 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 381 ( "=" PGNSP PGUID b f 1009 1009 16 381 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 382 ( "=" PGNSP PGUID b f 1028 1028 16 382 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 383 ( "=" PGNSP PGUID b f 1010 1010 16 383 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 384 ( "=" PGNSP PGUID b f 1011 1011 16 384 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 385 ( "=" PGNSP PGUID b f 1012 1012 16 385 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 386 ( "=" PGNSP PGUID b f 1013 1013 16 386 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 390 ( "=" PGNSP PGUID b f 1017 1017 16 390 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 391 ( "=" PGNSP PGUID b f 1018 1018 16 391 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 392 ( "=" PGNSP PGUID b f 1019 1019 16 392 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 393 ( "=" PGNSP PGUID b f 1020 1020 16 393 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 394 ( "=" PGNSP PGUID b f 1021 1021 16 394 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 395 ( "=" PGNSP PGUID b f 1022 1022 16 395 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 396 ( "=" PGNSP PGUID b f 1023 1023 16 396 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 397 ( "=" PGNSP PGUID b f 1024 1024 16 397 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 398 ( "=" PGNSP PGUID b f 1025 1025 16 398 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 400 ( "=" PGNSP PGUID b f 1027 1027 16 400 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 401 ( "=" PGNSP PGUID b f 1034 1034 16 401 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 329 ( "=" PGNSP PGUID b f 2277 2277 16 329 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 349 ( "||" PGNSP PGUID b f 2277 2283 2277 0 0 0 0 0 0 array_append - - ));
DATA(insert OID = 374 ( "||" PGNSP PGUID b f 2283 2277 2277 0 0 0 0 0 0 array_prepend - - ));
DATA(insert OID = 375 ( "||" PGNSP PGUID b f 2277 2277 2277 0 0 0 0 0 0 array_cat - - ));
DATA(insert OID = 352 ( "=" PGNSP PGUID b t 28 28 16 352 0 0 0 0 0 xideq eqsel eqjoinsel ));
DATA(insert OID = 353 ( "=" PGNSP PGUID b t 28 23 16 0 0 0 0 0 0 xideq eqsel eqjoinsel ));
@ -835,33 +814,6 @@ DATA(insert OID = 2066 ( "+" PGNSP PGUID b f 1114 1186 1114 0 0 0 0 0 0 tim
DATA(insert OID = 2067 ( "-" PGNSP PGUID b f 1114 1114 1186 0 0 0 0 0 0 timestamp_mi - - ));
DATA(insert OID = 2068 ( "-" PGNSP PGUID b f 1114 1186 1114 0 0 0 0 0 0 timestamp_mi_span - - ));
/* array equality operators */
DATA(insert OID = 2222 ( "=" PGNSP PGUID b f 2207 2207 16 2222 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2223 ( "=" PGNSP PGUID b f 2208 2208 16 2223 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2224 ( "=" PGNSP PGUID b f 2209 2209 16 2224 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2225 ( "=" PGNSP PGUID b f 2210 2210 16 2225 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2226 ( "=" PGNSP PGUID b f 2211 2211 16 2226 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2227 ( "=" PGNSP PGUID b f 629 629 16 2227 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2228 ( "=" PGNSP PGUID b f 651 651 16 2228 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2229 ( "=" PGNSP PGUID b f 719 719 16 2229 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2230 ( "=" PGNSP PGUID b f 791 791 16 2230 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2231 ( "=" PGNSP PGUID b f 1014 1014 16 2231 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2232 ( "=" PGNSP PGUID b f 1015 1015 16 2232 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2233 ( "=" PGNSP PGUID b f 1016 1016 16 2233 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2234 ( "=" PGNSP PGUID b f 1040 1040 16 2234 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2235 ( "=" PGNSP PGUID b f 1041 1041 16 2235 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2236 ( "=" PGNSP PGUID b f 1115 1115 16 2236 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2237 ( "=" PGNSP PGUID b f 1182 1182 16 2237 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2238 ( "=" PGNSP PGUID b f 1183 1183 16 2238 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2239 ( "=" PGNSP PGUID b f 1185 1185 16 2239 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2240 ( "=" PGNSP PGUID b f 1187 1187 16 2240 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2241 ( "=" PGNSP PGUID b f 1231 1231 16 2241 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2242 ( "=" PGNSP PGUID b f 1270 1270 16 2242 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2243 ( "=" PGNSP PGUID b f 1561 1561 16 2243 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2244 ( "=" PGNSP PGUID b f 1563 1563 16 2244 0 0 0 0 0 array_eq eqsel eqjoinsel ));
DATA(insert OID = 2245 ( "=" PGNSP PGUID b f 2201 2201 16 2245 0 0 0 0 0 array_eq eqsel eqjoinsel ));
/*
* function prototypes

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: pg_proc.h,v 1.291 2003/04/04 03:03:54 tgl Exp $
* $Id: pg_proc.h,v 1.292 2003/04/08 23:20:03 tgl Exp $
*
* NOTES
* The script catalog/genbki.sh reads this file and generates .bki
@ -1000,6 +1000,22 @@ DATA(insert OID = 2091 ( array_lower PGNSP PGUID 12 f f t f i 2 23 "2277 23"
DESCR("array lower dimension");
DATA(insert OID = 2092 ( array_upper PGNSP PGUID 12 f f t f i 2 23 "2277 23" array_upper - _null_ ));
DESCR("array upper dimension");
DATA(insert OID = 377 ( singleton_array PGNSP PGUID 12 f f t f i 1 2277 "2283" singleton_array - _null_ ));
DESCR("create array from single element");
DATA(insert OID = 378 ( array_append PGNSP PGUID 12 f f t f i 2 2277 "2277 2283" array_push - _null_ ));
DESCR("append element onto end of array");
DATA(insert OID = 379 ( array_prepend PGNSP PGUID 12 f f t f i 2 2277 "2283 2277" array_push - _null_ ));
DESCR("prepend element onto front of array");
DATA(insert OID = 380 ( array_accum PGNSP PGUID 12 f f f f i 2 2277 "2277 2283" array_accum - _null_ ));
DESCR("push element onto end of array, creating array if needed");
DATA(insert OID = 381 ( array_assign PGNSP PGUID 12 f f t f i 3 2277 "2277 23 2283" array_assign - _null_ ));
DESCR("assign specific array element");
DATA(insert OID = 382 ( array_subscript PGNSP PGUID 12 f f t f i 2 2283 "2277 23" array_subscript - _null_ ));
DESCR("return specific array element");
DATA(insert OID = 383 ( array_cat PGNSP PGUID 12 f f t f i 2 2277 "2277 2277" array_cat - _null_ ));
DESCR("concatenate two arrays");
DATA(insert OID = 384 ( array_coerce PGNSP PGUID 12 f f t f i 1 2277 "2277" array_type_coerce - _null_ ));
DESCR("coerce array type to another array type");
DATA(insert OID = 760 ( smgrin PGNSP PGUID 12 f f t f s 1 210 "2275" smgrin - _null_ ));
DESCR("storage manager(internal)");
@ -3127,11 +3143,16 @@ DATA(insert OID = 2306 ( opaque_in PGNSP PGUID 12 f f t f i 1 2282 "2275" opa
DESCR("(internal)");
DATA(insert OID = 2307 ( opaque_out PGNSP PGUID 12 f f t f i 1 2275 "2282" opaque_out - _null_ ));
DESCR("(internal)");
DATA(insert OID = 2312 ( anyelement_in PGNSP PGUID 12 f f t f i 1 2283 "2275" anyelement_in - _null_ ));
DESCR("(internal)");
DATA(insert OID = 2313 ( anyelement_out PGNSP PGUID 12 f f t f i 1 2275 "2283" anyelement_out - _null_ ));
DESCR("(internal)");
/* cryptographic */
DATA(insert OID = 2311 ( md5 PGNSP PGUID 12 f f t f i 1 25 "25" md5_text - _null_ ));
DESCR("calculates md5 hash");
/*
* Symbolic values for provolatile column: these indicate whether the result
* of a function is dependent *only* on the values of its explicit arguments,

View File

@ -8,7 +8,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: pg_type.h,v 1.139 2003/01/23 23:39:06 petere Exp $
* $Id: pg_type.h,v 1.140 2003/04/08 23:20:04 tgl Exp $
*
* NOTES
* the genbki.sh script reads this file and generates .bki
@ -533,6 +533,8 @@ DATA(insert OID = 2281 ( internal PGNSP PGUID 4 t p t \054 0 0 internal_in int
#define INTERNALOID 2281
DATA(insert OID = 2282 ( opaque PGNSP PGUID 4 t p t \054 0 0 opaque_in opaque_out i p f 0 -1 0 _null_ _null_ ));
#define OPAQUEOID 2282
DATA(insert OID = 2283 ( anyelement PGNSP PGUID 4 t p t \054 0 0 anyelement_in anyelement_out i p f 0 -1 0 _null_ _null_ ));
#define ANYELEMENTOID 2283
/*
* prototypes for functions in pg_type.c

View File

@ -11,7 +11,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: fmgr.h,v 1.26 2002/10/25 22:17:32 tgl Exp $
* $Id: fmgr.h,v 1.27 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -48,6 +48,7 @@ typedef struct FmgrInfo
bool fn_retset; /* function returns a set */
void *fn_extra; /* extra space for use by handler */
MemoryContext fn_mcxt; /* memory context to store fn_extra in */
struct Node *fn_expr; /* expression parse tree for call, or NULL */
} FmgrInfo;
/*
@ -377,6 +378,8 @@ extern Datum OidFunctionCall9(Oid functionId, Datum arg1, Datum arg2,
*/
extern Pg_finfo_record *fetch_finfo_record(void *filehandle, char *funcname);
extern Oid fmgr_internal_function(const char *proname);
extern Oid get_fn_expr_rettype(FunctionCallInfo fcinfo);
extern Oid get_fn_expr_argtype(FunctionCallInfo fcinfo, int argnum);
/*
* Routines in dfmgr.c

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: execnodes.h,v 1.96 2003/03/09 02:19:13 tgl Exp $
* $Id: execnodes.h,v 1.97 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -427,6 +427,10 @@ typedef struct AggrefExprState
/* ----------------
* ArrayRefExprState node
*
* Note: array types can be fixed-length (typlen > 0), but only when the
* element type is itself fixed-length. Otherwise they are varlena structures
* and have typlen = -1. In any case, an array type is never pass-by-value.
* ----------------
*/
typedef struct ArrayRefExprState
@ -436,6 +440,10 @@ typedef struct ArrayRefExprState
List *reflowerindexpr;
ExprState *refexpr;
ExprState *refassgnexpr;
int16 refattrlength; /* typlen of array type */
int16 refelemlength; /* typlen of the array element type */
bool refelembyval; /* is the element type pass-by-value? */
char refelemalign; /* typalign of the element type */
} ArrayRefExprState;
/* ----------------
@ -540,6 +548,22 @@ typedef struct CaseWhenState
ExprState *result; /* substitution result */
} CaseWhenState;
/* ----------------
* ArrayExprState node
*
* Note: ARRAY[] expressions always produce varlena arrays, never fixed-length
* arrays.
* ----------------
*/
typedef struct ArrayExprState
{
ExprState xprstate;
List *elements; /* states for child nodes */
int16 elemlength; /* typlen of the array element type */
bool elembyval; /* is the element type pass-by-value? */
char elemalign; /* typalign of the element type */
} ArrayExprState;
/* ----------------
* CoalesceExprState node
* ----------------

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: nodes.h,v 1.139 2003/03/20 07:02:11 momjian Exp $
* $Id: nodes.h,v 1.140 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -112,6 +112,7 @@ typedef enum NodeTag
T_RelabelType,
T_CaseExpr,
T_CaseWhen,
T_ArrayExpr,
T_CoalesceExpr,
T_NullIfExpr,
T_NullTest,
@ -138,6 +139,7 @@ typedef enum NodeTag
T_SubPlanState,
T_CaseExprState,
T_CaseWhenState,
T_ArrayExprState,
T_CoalesceExprState,
T_CoerceToDomainState,
T_DomainConstraintState,

View File

@ -10,7 +10,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: primnodes.h,v 1.80 2003/02/16 02:30:39 tgl Exp $
* $Id: primnodes.h,v 1.81 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -237,14 +237,8 @@ typedef struct Aggref
* reflowerindexpr must be the same length as refupperindexpr when it
* is not NIL.
*
* Note: array types can be fixed-length (refattrlength > 0), but only
* when the element type is itself fixed-length. Otherwise they are
* varlena structures and have refattrlength = -1. In any case,
* an array type is never pass-by-value.
*
* Note: refrestype is NOT the element type, but the array type,
* when doing subarray fetch or either type of store. It might be a good
* idea to include a refelemtype field as well.
* when doing subarray fetch or either type of store.
* ----------------
*/
typedef struct ArrayRef
@ -252,10 +246,8 @@ typedef struct ArrayRef
Expr xpr;
Oid refrestype; /* type of the result of the ArrayRef
* operation */
int refattrlength; /* typlen of array type */
int refelemlength; /* typlen of the array element type */
bool refelembyval; /* is the element type pass-by-value? */
char refelemalign; /* typalign of the element type */
Oid refarraytype; /* type of the array proper */
Oid refelemtype; /* type of the array elements */
List *refupperindexpr;/* expressions that evaluate to upper
* array indexes */
List *reflowerindexpr;/* expressions that evaluate to lower
@ -366,12 +358,15 @@ typedef struct BoolExpr
* ANY_SUBLINK (lefthand) op ANY (SELECT ...)
* MULTIEXPR_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
* 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.
* 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
@ -390,14 +385,19 @@ typedef struct BoolExpr
* And subselect is transformed to a Query. This is the representation
* seen in saved rules and in the rewriter.
*
* In EXISTS and EXPR SubLinks, lefthand, operName, and operOids are unused
* and are always NIL. useOr is not significant either for these sublink
* types.
* 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.
* ----------------
*/
typedef enum SubLinkType
{
EXISTS_SUBLINK, ALL_SUBLINK, ANY_SUBLINK, MULTIEXPR_SUBLINK, EXPR_SUBLINK
EXISTS_SUBLINK,
ALL_SUBLINK,
ANY_SUBLINK,
MULTIEXPR_SUBLINK,
EXPR_SUBLINK,
ARRAY_SUBLINK
} SubLinkType;
@ -537,6 +537,21 @@ typedef struct CaseWhen
Expr *result; /* substitution result */
} CaseWhen;
/*
* ArrayExpr - an ARRAY[] expression
*
* Note: if ndims > 1, then the array elements are all ArrayExprs of the
* same type and ndims one less.
*/
typedef struct ArrayExpr
{
Expr xpr;
Oid array_typeid; /* type of expression result */
Oid element_typeid; /* common type of expression elements */
List *elements; /* the array elements */
int ndims; /* number of array dimensions */
} ArrayExpr;
/*
* CoalesceExpr - a COALESCE expression
*/

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: parse_coerce.h,v 1.49 2003/02/03 21:15:44 tgl Exp $
* $Id: parse_coerce.h,v 1.50 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -21,6 +21,7 @@ typedef enum CATEGORY
{
INVALID_TYPE,
UNKNOWN_TYPE,
GENERIC_TYPE,
BOOLEAN_TYPE,
STRING_TYPE,
BITSTRING_TYPE,
@ -54,6 +55,14 @@ extern Oid select_common_type(List *typeids, const char *context);
extern Node *coerce_to_common_type(Node *node, Oid targetTypeId,
const char *context);
extern bool check_generic_type_consistency(Oid *actual_arg_types,
Oid *declared_arg_types,
int nargs);
extern Oid enforce_generic_type_consistency(Oid *actual_arg_types,
Oid *declared_arg_types,
int nargs,
Oid rettype);
extern bool find_coercion_pathway(Oid targetTypeId, Oid sourceTypeId,
CoercionContext ccontext,
Oid *funcid);

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: parse_func.h,v 1.43 2002/09/04 20:31:45 momjian Exp $
* $Id: parse_func.h,v 1.44 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -49,6 +49,10 @@ extern FuncDetailCode func_get_detail(List *funcname, List *fargs,
extern bool typeInheritsFrom(Oid subclassTypeId, Oid superclassTypeId);
extern void make_fn_arguments(List *fargs,
Oid *actual_arg_types,
Oid *declared_arg_types);
extern void func_error(const char *caller, List *funcname,
int nargs, const Oid *argtypes,
const char *msg);

View File

@ -6,7 +6,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: parse_node.h,v 1.33 2002/12/12 20:35:16 tgl Exp $
* $Id: parse_node.h,v 1.34 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -52,8 +52,6 @@ typedef struct ParseState
} ParseState;
extern ParseState *make_parsestate(ParseState *parentParseState);
extern Expr *make_op(List *opname, Node *ltree, Node *rtree);
extern Node *make_operand(Node *tree, Oid orig_typeId, Oid target_typeId);
extern Var *make_var(ParseState *pstate, RangeTblEntry *rte, int attrno);
extern ArrayRef *transformArraySubscripts(ParseState *pstate,
Node *arrayBase,

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: parse_oper.h,v 1.23 2002/11/29 21:39:12 tgl Exp $
* $Id: parse_oper.h,v 1.24 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -49,4 +49,9 @@ extern Oid ordering_oper_opid(Oid argtype);
extern Oid oprid(Operator op);
extern Oid oprfuncid(Operator op);
/* Build expression tree for an operator invocation */
extern Expr *make_op(List *opname, Node *ltree, Node *rtree);
extern Expr *make_op_expr(Operator op, Node *ltree, Node *rtree,
Oid ltypeId, Oid rtypeId);
#endif /* PARSE_OPER_H */

View File

@ -10,7 +10,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: array.h,v 1.36 2002/11/08 17:27:03 momjian Exp $
* $Id: array.h,v 1.37 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -87,6 +87,9 @@ extern Datum array_eq(PG_FUNCTION_ARGS);
extern Datum array_dims(PG_FUNCTION_ARGS);
extern Datum array_lower(PG_FUNCTION_ARGS);
extern Datum array_upper(PG_FUNCTION_ARGS);
extern Datum array_assign(PG_FUNCTION_ARGS);
extern Datum array_subscript(PG_FUNCTION_ARGS);
extern Datum array_type_coerce(PG_FUNCTION_ARGS);
extern Datum array_ref(ArrayType *array, int nSubscripts, int *indx,
int arraylen, int elmlen, bool elmbyval, char elmalign,
@ -110,6 +113,11 @@ extern Datum array_map(FunctionCallInfo fcinfo, Oid inpType, Oid retType);
extern ArrayType *construct_array(Datum *elems, int nelems,
Oid elmtype,
int elmlen, bool elmbyval, char elmalign);
extern ArrayType *construct_md_array(Datum *elems,
int ndims,
int *dims,
int *lbs,
Oid elmtype, int elmlen, bool elmbyval, char elmalign);
extern void deconstruct_array(ArrayType *array,
Oid elmtype,
int elmlen, bool elmbyval, char elmalign,
@ -128,4 +136,16 @@ extern void mda_get_prod(int n, int *range, int *prod);
extern void mda_get_offset_values(int n, int *dist, int *prod, int *span);
extern int mda_next_tuple(int n, int *curr, int *span);
/*
* prototypes for functions defined in array_userfuncs.c
*/
extern Datum singleton_array(PG_FUNCTION_ARGS);
extern Datum array_push(PG_FUNCTION_ARGS);
extern Datum array_accum(PG_FUNCTION_ARGS);
extern Datum array_cat(PG_FUNCTION_ARGS);
extern ArrayType *create_singleton_array(Oid element_type,
Datum element,
int ndims);
#endif /* ARRAY_H */

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: builtins.h,v 1.210 2003/03/21 23:18:51 tgl Exp $
* $Id: builtins.h,v 1.211 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -362,6 +362,8 @@ extern Datum internal_in(PG_FUNCTION_ARGS);
extern Datum internal_out(PG_FUNCTION_ARGS);
extern Datum opaque_in(PG_FUNCTION_ARGS);
extern Datum opaque_out(PG_FUNCTION_ARGS);
extern Datum anyelement_in(PG_FUNCTION_ARGS);
extern Datum anyelement_out(PG_FUNCTION_ARGS);
/* regexp.c */
extern Datum nameregexeq(PG_FUNCTION_ARGS);

View File

@ -6,7 +6,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: lsyscache.h,v 1.67 2003/02/03 21:15:45 tgl Exp $
* $Id: lsyscache.h,v 1.68 2003/04/08 23:20:04 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -58,6 +58,8 @@ extern int32 get_typtypmod(Oid typid);
extern Node *get_typdefault(Oid typid);
extern char get_typtype(Oid typid);
extern Oid get_typ_typrelid(Oid typid);
extern Oid get_element_type(Oid typid);
extern Oid get_array_type(Oid typid);
extern void getTypeInputInfo(Oid type, Oid *typInput, Oid *typElem);
extern bool getTypeOutputInfo(Oid type, Oid *typOutput, Oid *typElem,
bool *typIsVarlena);
@ -75,6 +77,8 @@ extern void free_attstatsslot(Oid atttype,
extern char *get_namespace_name(Oid nspid);
extern int32 get_usesysid(const char *username);
#define is_array_type(typid) (get_element_type(typid) != InvalidOid)
#define TypeIsToastable(typid) (get_typstorage(typid) != 'p')
#endif /* LSYSCACHE_H */

View File

@ -3,7 +3,7 @@
* procedural language
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.84 2003/03/27 16:51:29 momjian Exp $
* $Header: /cvsroot/pgsql/src/pl/plpgsql/src/pl_exec.c,v 1.85 2003/04/08 23:20:04 tgl Exp $
*
* This software is copyrighted by Jan Wieck - Hamburg.
*
@ -3605,6 +3605,16 @@ exec_simple_check_node(Node *node)
return TRUE;
}
case T_ArrayExpr:
{
ArrayExpr *expr = (ArrayExpr *) node;
if (!exec_simple_check_node((Node *) expr->elements))
return FALSE;
return TRUE;
}
case T_CoalesceExpr:
{
CoalesceExpr *expr = (CoalesceExpr *) node;

View File

@ -117,3 +117,204 @@ SELECT a[1:3],
{16,25,23} | | {foobar,new_word} | {{elt2}}
(3 rows)
--
-- array expressions and operators
--
-- table creation and INSERTs
CREATE TEMP TABLE arrtest2 (i integer ARRAY[4], f float8[], n numeric[], t text[], d timestamp[]);
INSERT INTO arrtest2 VALUES(
ARRAY[[[113,142],[1,147]]],
ARRAY[1.1,1.2,1.3]::float8[],
ARRAY[1.1,1.2,1.3],
ARRAY[[['aaa','aab'],['aba','abb'],['aca','acb']],[['baa','bab'],['bba','bbb'],['bca','bcb']]],
ARRAY['19620326','19931223','19970117']::timestamp[]
);
-- some more test data
CREATE TEMP TABLE arrtest_f (f0 int, f1 text, f2 float8);
insert into arrtest_f values(1,'cat1',1.21);
insert into arrtest_f values(2,'cat1',1.24);
insert into arrtest_f values(3,'cat1',1.18);
insert into arrtest_f values(4,'cat1',1.26);
insert into arrtest_f values(5,'cat1',1.15);
insert into arrtest_f values(6,'cat2',1.15);
insert into arrtest_f values(7,'cat2',1.26);
insert into arrtest_f values(8,'cat2',1.32);
insert into arrtest_f values(9,'cat2',1.30);
CREATE TEMP TABLE arrtest_i (f0 int, f1 text, f2 int);
insert into arrtest_i values(1,'cat1',21);
insert into arrtest_i values(2,'cat1',24);
insert into arrtest_i values(3,'cat1',18);
insert into arrtest_i values(4,'cat1',26);
insert into arrtest_i values(5,'cat1',15);
insert into arrtest_i values(6,'cat2',15);
insert into arrtest_i values(7,'cat2',26);
insert into arrtest_i values(8,'cat2',32);
insert into arrtest_i values(9,'cat2',30);
-- expressions
SELECT t.f[1][3][1] AS "131", t.f[2][2][1] AS "221" FROM (
SELECT ARRAY[[[111,112],[121,122],[131,132]],[[211,212],[221,122],[231,232]]] AS f
) AS t;
131 | 221
-----+-----
131 | 221
(1 row)
SELECT ARRAY[[[[[['hello'],['world']]]]]];
array
---------------------------
{{{{{{hello},{world}}}}}}
(1 row)
SELECT ARRAY[ARRAY['hello'],ARRAY['world']];
array
-------------------
{{hello},{world}}
(1 row)
SELECT ARRAY(select f2 from arrtest_f order by f2) AS "ARRAY";
ARRAY
-----------------------------------------------
{1.15,1.15,1.18,1.21,1.24,1.26,1.26,1.3,1.32}
(1 row)
-- functions
SELECT singleton_array(42) AS "{42}";
{42}
------
{42}
(1 row)
SELECT array_append(singleton_array(42), 6) AS "{42,6}";
{42,6}
--------
{42,6}
(1 row)
SELECT array_prepend(6, singleton_array(42)) AS "{6,42}";
{6,42}
--------
{6,42}
(1 row)
SELECT array_cat(ARRAY[1,2], ARRAY[3,4]) AS "{{1,2},{3,4}}";
{{1,2},{3,4}}
---------------
{{1,2},{3,4}}
(1 row)
SELECT array_cat(ARRAY[1,2], ARRAY[[3,4],[5,6]]) AS "{{1,2},{3,4},{5,6}}";
{{1,2},{3,4},{5,6}}
---------------------
{{1,2},{3,4},{5,6}}
(1 row)
SELECT array_cat(ARRAY[[3,4],[5,6]], ARRAY[1,2]) AS "{{3,4},{5,6},{1,2}}";
{{3,4},{5,6},{1,2}}
---------------------
{{3,4},{5,6},{1,2}}
(1 row)
SELECT array_subscript(n, 2) AS "1.2" FROM arrtest2;
1.2
-----
1.2
(1 row)
SELECT array_assign(n, 2, 9.99) AS "{1.1,9.99,1.3}" FROM arrtest2;
{1.1,9.99,1.3}
----------------
{1.1,9.99,1.3}
(1 row)
SELECT array_subscript(array_assign(n, 2, 9.99), 2) AS "9.99" FROM arrtest2;
9.99
------
9.99
(1 row)
-- operators
SELECT a FROM arrtest WHERE b = ARRAY[[[113,142],[1,147]]];
a
---------------
{16,25,3,4,5}
(1 row)
SELECT NOT ARRAY[1.1,1.2,1.3] = ARRAY[1.1,1.2,1.3] AS "FALSE";
FALSE
-------
f
(1 row)
SELECT ARRAY[1,2] || 3 AS "{1,2,3}";
{1,2,3}
---------
{1,2,3}
(1 row)
SELECT 0 || ARRAY[1,2] AS "{0,1,2}";
{0,1,2}
---------
{0,1,2}
(1 row)
SELECT ARRAY[1,2] || ARRAY[3,4] AS "{{1,2},{3,4}}";
{{1,2},{3,4}}
---------------
{{1,2},{3,4}}
(1 row)
SELECT ARRAY[[['hello','world']]] || ARRAY[[['happy','birthday']]] AS "ARRAY";
ARRAY
------------------------------------------
{{{{hello,world}}},{{{happy,birthday}}}}
(1 row)
SELECT ARRAY[[1,2],[3,4]] || ARRAY[5,6] AS "{{1,2},{3,4},{5,6}}";
{{1,2},{3,4},{5,6}}
---------------------
{{1,2},{3,4},{5,6}}
(1 row)
SELECT ARRAY[0,0] || ARRAY[1,1] || ARRAY[2,2] AS "{{0,0},{1,1},{2,2}}";
{{0,0},{1,1},{2,2}}
---------------------
{{0,0},{1,1},{2,2}}
(1 row)
SELECT 0 || ARRAY[1,2] || 3 AS "{0,1,2,3}";
{0,1,2,3}
-----------
{0,1,2,3}
(1 row)
-- array casts
SELECT ARRAY[1,2,3]::text[]::int[]::float8[] AS "{1,2,3}";
{1,2,3}
---------
{1,2,3}
(1 row)
SELECT ARRAY[1,2,3]::text[]::int[]::float8[] is of (float8[]) as "TRUE";
TRUE
------
t
(1 row)
SELECT ARRAY[['a','bc'],['def','hijk']]::text[]::varchar[] AS "{{a,bc},{def,hijk}}";
{{a,bc},{def,hijk}}
---------------------
{{a,bc},{def,hijk}}
(1 row)
SELECT ARRAY[['a','bc'],['def','hijk']]::text[]::varchar[] is of (varchar[]) as "TRUE";
TRUE
------
t
(1 row)
SELECT CAST(ARRAY[[[[[['a','bb','ccc']]]]]] as text[]) as "{{{{{{a,bb,ccc}}}}}}";
{{{{{{a,bb,ccc}}}}}}
----------------------
{{{{{{a,bb,ccc}}}}}}
(1 row)

View File

@ -73,10 +73,12 @@ WHERE p1.oid != p2.oid AND
-- Look for uses of different type OIDs in the argument/result type fields
-- for different aliases of the same built-in function.
-- This indicates that the types are being presumed to be binary-equivalent.
-- This indicates that the types are being presumed to be binary-equivalent,
-- or that the built-in function is prepared to deal with different types.
-- That's not wrong, necessarily, but we make lists of all the types being
-- so treated. Note that the expected output of this part of the test will
-- need to be modified whenever new pairs of types are made binary-equivalent!
-- need to be modified whenever new pairs of types are made binary-equivalent,
-- or when new polymorphic built-in functions are added!
-- Note: ignore aggregate functions here, since they all point to the same
-- dummy built-in function.
SELECT DISTINCT p1.prorettype, p2.prorettype
@ -104,7 +106,8 @@ WHERE p1.oid != p2.oid AND
25 | 1043
1114 | 1184
1560 | 1562
(3 rows)
2277 | 2283
(4 rows)
SELECT DISTINCT p1.proargtypes[1], p2.proargtypes[1]
FROM pg_proc AS p1, pg_proc AS p2
@ -117,7 +120,8 @@ WHERE p1.oid != p2.oid AND
-------------+-------------
1114 | 1184
1560 | 1562
(2 rows)
2277 | 2283
(3 rows)
SELECT DISTINCT p1.proargtypes[2], p2.proargtypes[2]
FROM pg_proc AS p1, pg_proc AS p2

View File

@ -70,16 +70,6 @@ WHERE p1.typtype in ('b') AND p1.typname NOT LIKE '\\_%' AND NOT EXISTS
705 | unknown
(3 rows)
-- Look for array types that don't have an equality operator.
SELECT p1.oid, p1.typname
FROM pg_type as p1
WHERE p1.typtype != 'c' AND p1.typname LIKE '\\_%' AND NOT EXISTS
(SELECT 1 FROM pg_operator
WHERE oprname = '=' AND oprleft = p1.oid AND oprright = p1.oid);
oid | typname
-----+---------
(0 rows)
-- Conversion routines must be provided except in 'c' entries.
SELECT p1.oid, p1.typname
FROM pg_type as p1

View File

@ -83,3 +83,77 @@ SELECT a[1:3],
c[1:2],
d[1:1][2:2]
FROM arrtest;
--
-- array expressions and operators
--
-- table creation and INSERTs
CREATE TEMP TABLE arrtest2 (i integer ARRAY[4], f float8[], n numeric[], t text[], d timestamp[]);
INSERT INTO arrtest2 VALUES(
ARRAY[[[113,142],[1,147]]],
ARRAY[1.1,1.2,1.3]::float8[],
ARRAY[1.1,1.2,1.3],
ARRAY[[['aaa','aab'],['aba','abb'],['aca','acb']],[['baa','bab'],['bba','bbb'],['bca','bcb']]],
ARRAY['19620326','19931223','19970117']::timestamp[]
);
-- some more test data
CREATE TEMP TABLE arrtest_f (f0 int, f1 text, f2 float8);
insert into arrtest_f values(1,'cat1',1.21);
insert into arrtest_f values(2,'cat1',1.24);
insert into arrtest_f values(3,'cat1',1.18);
insert into arrtest_f values(4,'cat1',1.26);
insert into arrtest_f values(5,'cat1',1.15);
insert into arrtest_f values(6,'cat2',1.15);
insert into arrtest_f values(7,'cat2',1.26);
insert into arrtest_f values(8,'cat2',1.32);
insert into arrtest_f values(9,'cat2',1.30);
CREATE TEMP TABLE arrtest_i (f0 int, f1 text, f2 int);
insert into arrtest_i values(1,'cat1',21);
insert into arrtest_i values(2,'cat1',24);
insert into arrtest_i values(3,'cat1',18);
insert into arrtest_i values(4,'cat1',26);
insert into arrtest_i values(5,'cat1',15);
insert into arrtest_i values(6,'cat2',15);
insert into arrtest_i values(7,'cat2',26);
insert into arrtest_i values(8,'cat2',32);
insert into arrtest_i values(9,'cat2',30);
-- expressions
SELECT t.f[1][3][1] AS "131", t.f[2][2][1] AS "221" FROM (
SELECT ARRAY[[[111,112],[121,122],[131,132]],[[211,212],[221,122],[231,232]]] AS f
) AS t;
SELECT ARRAY[[[[[['hello'],['world']]]]]];
SELECT ARRAY[ARRAY['hello'],ARRAY['world']];
SELECT ARRAY(select f2 from arrtest_f order by f2) AS "ARRAY";
-- functions
SELECT singleton_array(42) AS "{42}";
SELECT array_append(singleton_array(42), 6) AS "{42,6}";
SELECT array_prepend(6, singleton_array(42)) AS "{6,42}";
SELECT array_cat(ARRAY[1,2], ARRAY[3,4]) AS "{{1,2},{3,4}}";
SELECT array_cat(ARRAY[1,2], ARRAY[[3,4],[5,6]]) AS "{{1,2},{3,4},{5,6}}";
SELECT array_cat(ARRAY[[3,4],[5,6]], ARRAY[1,2]) AS "{{3,4},{5,6},{1,2}}";
SELECT array_subscript(n, 2) AS "1.2" FROM arrtest2;
SELECT array_assign(n, 2, 9.99) AS "{1.1,9.99,1.3}" FROM arrtest2;
SELECT array_subscript(array_assign(n, 2, 9.99), 2) AS "9.99" FROM arrtest2;
-- operators
SELECT a FROM arrtest WHERE b = ARRAY[[[113,142],[1,147]]];
SELECT NOT ARRAY[1.1,1.2,1.3] = ARRAY[1.1,1.2,1.3] AS "FALSE";
SELECT ARRAY[1,2] || 3 AS "{1,2,3}";
SELECT 0 || ARRAY[1,2] AS "{0,1,2}";
SELECT ARRAY[1,2] || ARRAY[3,4] AS "{{1,2},{3,4}}";
SELECT ARRAY[[['hello','world']]] || ARRAY[[['happy','birthday']]] AS "ARRAY";
SELECT ARRAY[[1,2],[3,4]] || ARRAY[5,6] AS "{{1,2},{3,4},{5,6}}";
SELECT ARRAY[0,0] || ARRAY[1,1] || ARRAY[2,2] AS "{{0,0},{1,1},{2,2}}";
SELECT 0 || ARRAY[1,2] || 3 AS "{0,1,2,3}";
-- array casts
SELECT ARRAY[1,2,3]::text[]::int[]::float8[] AS "{1,2,3}";
SELECT ARRAY[1,2,3]::text[]::int[]::float8[] is of (float8[]) as "TRUE";
SELECT ARRAY[['a','bc'],['def','hijk']]::text[]::varchar[] AS "{{a,bc},{def,hijk}}";
SELECT ARRAY[['a','bc'],['def','hijk']]::text[]::varchar[] is of (varchar[]) as "TRUE";
SELECT CAST(ARRAY[[[[[['a','bb','ccc']]]]]] as text[]) as "{{{{{{a,bb,ccc}}}}}}";

View File

@ -69,10 +69,12 @@ WHERE p1.oid != p2.oid AND
-- Look for uses of different type OIDs in the argument/result type fields
-- for different aliases of the same built-in function.
-- This indicates that the types are being presumed to be binary-equivalent.
-- This indicates that the types are being presumed to be binary-equivalent,
-- or that the built-in function is prepared to deal with different types.
-- That's not wrong, necessarily, but we make lists of all the types being
-- so treated. Note that the expected output of this part of the test will
-- need to be modified whenever new pairs of types are made binary-equivalent!
-- need to be modified whenever new pairs of types are made binary-equivalent,
-- or when new polymorphic built-in functions are added!
-- Note: ignore aggregate functions here, since they all point to the same
-- dummy built-in function.

View File

@ -59,14 +59,6 @@ WHERE p1.typtype in ('b') AND p1.typname NOT LIKE '\\_%' AND NOT EXISTS
WHERE p2.typname = ('_' || p1.typname)::name AND
p2.typelem = p1.oid);
-- Look for array types that don't have an equality operator.
SELECT p1.oid, p1.typname
FROM pg_type as p1
WHERE p1.typtype != 'c' AND p1.typname LIKE '\\_%' AND NOT EXISTS
(SELECT 1 FROM pg_operator
WHERE oprname = '=' AND oprleft = p1.oid AND oprright = p1.oid);
-- Conversion routines must be provided except in 'c' entries.
SELECT p1.oid, p1.typname