1
0
mirror of https://github.com/postgres/postgres.git synced 2025-07-03 20:02:46 +03:00

SQL-language functions are now callable in ordinary fmgr contexts ...

for example, an SQL function can be used in a functional index.  (I make
no promises about speed, but it'll work ;-).)  Clean up and simplify
handling of functions returning sets.
This commit is contained in:
Tom Lane
2000-08-24 03:29:15 +00:00
parent 87523ab8db
commit 782c16c6a1
35 changed files with 889 additions and 921 deletions

View File

@ -9,7 +9,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/Attic/execFlatten.c,v 1.12 2000/01/26 05:56:21 momjian Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/Attic/execFlatten.c,v 1.13 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -36,11 +36,12 @@ static bool FjoinBumpOuterNodes(TargetEntry *tlist, ExprContext *econtext,
#endif
Datum
ExecEvalIter(Iter *iterNode,
ExprContext *econtext,
bool *resultIsNull,
bool *iterIsDone)
bool *isNull,
ExprDoneCond *isDone)
{
Node *expression;
@ -52,14 +53,14 @@ ExecEvalIter(Iter *iterNode,
* only worrying about postquel functions, c functions will come
* later.
*/
return ExecEvalExpr(expression, econtext, resultIsNull, iterIsDone);
return ExecEvalExpr(expression, econtext, isNull, isDone);
}
void
ExecEvalFjoin(TargetEntry *tlist,
ExprContext *econtext,
bool *isNullVect,
bool *fj_isDone)
ExprDoneCond *fj_isDone)
{
#ifdef SETS_FIXED
@ -72,7 +73,7 @@ ExecEvalFjoin(TargetEntry *tlist,
BoolPtr alwaysDone = fjNode->fj_alwaysDone;
if (fj_isDone)
*fj_isDone = false;
*fj_isDone = ExprMultipleResult;
/*
* For the next tuple produced by the plan, we need to re-initialize

File diff suppressed because it is too large Load Diff

View File

@ -12,7 +12,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/execScan.c,v 1.13 2000/07/17 03:04:53 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/execScan.c,v 1.14 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -50,11 +50,10 @@ ExecScan(Scan *node,
{
CommonScanState *scanstate;
EState *estate;
List *qual;
bool isDone;
TupleTableSlot *resultSlot;
ExprContext *econtext;
ProjectionInfo *projInfo;
List *qual;
ExprDoneCond isDone;
TupleTableSlot *resultSlot;
/* ----------------
* Fetch data from node
@ -65,13 +64,6 @@ ExecScan(Scan *node,
econtext = scanstate->cstate.cs_ExprContext;
qual = node->plan.qual;
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* Check to see if we're still projecting out tuples from a previous
* scan tuple (because there is a function-returning-set in the
@ -80,14 +72,21 @@ ExecScan(Scan *node,
*/
if (scanstate->cstate.cs_TupFromTlist)
{
projInfo = scanstate->cstate.cs_ProjInfo;
resultSlot = ExecProject(projInfo, &isDone);
if (!isDone)
resultSlot = ExecProject(scanstate->cstate.cs_ProjInfo, &isDone);
if (isDone == ExprMultipleResult)
return resultSlot;
/* Done with that source tuple... */
scanstate->cstate.cs_TupFromTlist = false;
}
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle. Note this can't
* happen until we're done projecting out tuples from a scan tuple.
* ----------------
*/
ResetExprContext(econtext);
/*
* get a tuple from the access method loop until we obtain a tuple
* which passes the qualification.
@ -121,8 +120,6 @@ ExecScan(Scan *node,
/* ----------------
* check that the current tuple satisfies the qual-clause
* if our qualification succeeds then we may
* leave the loop.
*
* check for non-nil qual here to avoid a function call to
* ExecQual() when the qual is nil ... saves only a few cycles,
@ -130,7 +127,22 @@ ExecScan(Scan *node,
* ----------------
*/
if (!qual || ExecQual(qual, econtext, false))
break;
{
/* ----------------
* Found a satisfactory scan tuple.
*
* Form a projection tuple, store it in the result tuple
* slot and return it --- unless we find we can project no
* tuples from this scan tuple, in which case continue scan.
* ----------------
*/
resultSlot = ExecProject(scanstate->cstate.cs_ProjInfo, &isDone);
if (isDone != ExprEndResult)
{
scanstate->cstate.cs_TupFromTlist = (isDone == ExprMultipleResult);
return resultSlot;
}
}
/* ----------------
* Tuple fails qual, so free per-tuple memory and try again.
@ -138,18 +150,4 @@ ExecScan(Scan *node,
*/
ResetExprContext(econtext);
}
/* ----------------
* Found a satisfactory scan tuple.
*
* Form a projection tuple, store it in the result tuple
* slot and return it.
* ----------------
*/
projInfo = scanstate->cstate.cs_ProjInfo;
resultSlot = ExecProject(projInfo, &isDone);
scanstate->cstate.cs_TupFromTlist = !isDone;
return resultSlot;
}

View File

@ -8,22 +8,29 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/functions.c,v 1.37 2000/08/08 15:41:22 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/functions.c,v 1.38 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/heapam.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h"
#include "executor/execdefs.h"
#include "executor/executor.h"
#include "executor/functions.h"
#include "tcop/pquery.h"
#include "tcop/tcopprot.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
#include "utils/datum.h"
#include "utils/syscache.h"
/*
* We have an execution_state record for each query in the function.
*/
typedef enum
{
F_EXEC_START, F_EXEC_RUN, F_EXEC_DONE
@ -39,15 +46,40 @@ typedef struct local_es
#define LAST_POSTQUEL_COMMAND(es) ((es)->next == (execution_state *) NULL)
/*
* An SQLFunctionCache record is built during the first call,
* and linked to from the fn_extra field of the FmgrInfo struct.
*/
typedef struct
{
int typlen; /* length of the return type */
bool typbyval; /* true if return type is pass by value */
bool returnsTuple; /* true if return type is a tuple */
TupleTableSlot *funcSlot; /* if one result we need to copy it before
* we end execution of the function and
* free stuff */
/* head of linked list of execution_state records */
execution_state *func_state;
} SQLFunctionCache;
typedef SQLFunctionCache *SQLFunctionCachePtr;
/* non-export function prototypes */
static execution_state *init_execution_state(char *src,
Oid *argOidVect, int nargs);
static void init_sql_fcache(FmgrInfo *finfo);
static TupleDesc postquel_start(execution_state *es);
static execution_state *init_execution_state(FunctionCachePtr fcache);
static TupleTableSlot *postquel_getnext(execution_state *es);
static void postquel_end(execution_state *es);
static void postquel_sub_params(execution_state *es, FunctionCallInfo fcinfo);
static Datum postquel_execute(execution_state *es,
FunctionCallInfo fcinfo,
FunctionCachePtr fcache);
SQLFunctionCachePtr fcache);
static Datum
@ -69,21 +101,19 @@ ProjectAttribute(HeapTuple tup,
}
static execution_state *
init_execution_state(FunctionCachePtr fcache)
init_execution_state(char *src, Oid *argOidVect, int nargs)
{
execution_state *newes;
execution_state *nextes;
execution_state *preves;
List *queryTree_list,
*qtl_item;
int nargs = fcache->nargs;
newes = (execution_state *) palloc(sizeof(execution_state));
nextes = newes;
preves = (execution_state *) NULL;
queryTree_list = pg_parse_and_rewrite(fcache->src,
fcache->argOidVect, nargs);
queryTree_list = pg_parse_and_rewrite(src, argOidVect, nargs);
foreach(qtl_item, queryTree_list)
{
@ -138,6 +168,134 @@ init_execution_state(FunctionCachePtr fcache)
return newes;
}
static void
init_sql_fcache(FmgrInfo *finfo)
{
Oid foid = finfo->fn_oid;
HeapTuple procedureTuple;
HeapTuple typeTuple;
Form_pg_proc procedureStruct;
Form_pg_type typeStruct;
SQLFunctionCachePtr fcache;
Oid *argOidVect;
char *src;
int nargs;
Datum tmp;
bool isNull;
/* ----------------
* get the procedure tuple corresponding to the given function Oid
*
* NB: use SearchSysCacheTupleCopy to ensure tuple lives long enough
* ----------------
*/
procedureTuple = SearchSysCacheTupleCopy(PROCOID,
ObjectIdGetDatum(foid),
0, 0, 0);
if (!HeapTupleIsValid(procedureTuple))
elog(ERROR, "init_sql_fcache: Cache lookup failed for procedure %u",
foid);
procedureStruct = (Form_pg_proc) GETSTRUCT(procedureTuple);
/* ----------------
* get the return type from the procedure tuple
* ----------------
*/
typeTuple = SearchSysCacheTuple(TYPEOID,
ObjectIdGetDatum(procedureStruct->prorettype),
0, 0, 0);
if (!HeapTupleIsValid(typeTuple))
elog(ERROR, "init_sql_fcache: Cache lookup failed for type %u",
procedureStruct->prorettype);
typeStruct = (Form_pg_type) GETSTRUCT(typeTuple);
fcache = (SQLFunctionCachePtr) palloc(sizeof(SQLFunctionCache));
MemSet(fcache, 0, sizeof(SQLFunctionCache));
/* ----------------
* get the type length and by-value flag from the type tuple
* ----------------
*/
fcache->typlen = typeStruct->typlen;
if (typeStruct->typrelid == InvalidOid)
{
/* The return type is not a relation, so just use byval */
fcache->typbyval = typeStruct->typbyval;
fcache->returnsTuple = false;
}
else
{
/*
* This is a hack. We assume here that any function returning a
* tuple returns it by reference. This needs to be fixed, since
* actually the mechanism isn't quite like return-by-reference.
*/
fcache->typbyval = false;
fcache->returnsTuple = true;
}
/*
* If we are returning exactly one result then we have to copy tuples
* and by reference results because we have to end the execution
* before we return the results. When you do this everything
* allocated by the executor (i.e. slots and tuples) is freed.
*/
if (!finfo->fn_retset && !fcache->typbyval)
{
TupleTableSlot *slot;
slot = makeNode(TupleTableSlot);
slot->val = (HeapTuple) NULL;
slot->ttc_shouldFree = true;
slot->ttc_descIsNew = true;
slot->ttc_tupleDescriptor = (TupleDesc) NULL;
slot->ttc_buffer = InvalidBuffer;
slot->ttc_whichplan = -1;
fcache->funcSlot = slot;
}
else
fcache->funcSlot = NULL;
nargs = procedureStruct->pronargs;
if (nargs > 0)
{
argOidVect = (Oid *) palloc(nargs * sizeof(Oid));
memcpy(argOidVect,
procedureStruct->proargtypes,
nargs * sizeof(Oid));
}
else
{
argOidVect = (Oid *) NULL;
}
tmp = SysCacheGetAttr(PROCOID,
procedureTuple,
Anum_pg_proc_prosrc,
&isNull);
if (isNull)
elog(ERROR, "init_sql_fcache: null prosrc for procedure %u",
foid);
src = DatumGetCString(DirectFunctionCall1(textout, tmp));
fcache->func_state = init_execution_state(src, argOidVect, nargs);
pfree(src);
heap_freetuple(procedureTuple);
finfo->fn_extra = (void *) fcache;
}
static TupleDesc
postquel_start(execution_state *es)
{
@ -208,7 +366,7 @@ postquel_sub_params(execution_state *es, FunctionCallInfo fcinfo)
}
static TupleTableSlot *
copy_function_result(FunctionCachePtr fcache,
copy_function_result(SQLFunctionCachePtr fcache,
TupleTableSlot *resultSlot)
{
TupleTableSlot *funcSlot;
@ -219,10 +377,10 @@ copy_function_result(FunctionCachePtr fcache,
Assert(!TupIsNull(resultSlot));
resultTuple = resultSlot->val;
funcSlot = (TupleTableSlot *) fcache->funcSlot;
funcSlot = fcache->funcSlot;
if (funcSlot == (TupleTableSlot *) NULL)
return resultSlot;
if (funcSlot == NULL)
return resultSlot; /* no need to copy result */
/*
* If first time through, we have to initialize the funcSlot's
@ -243,7 +401,7 @@ copy_function_result(FunctionCachePtr fcache,
static Datum
postquel_execute(execution_state *es,
FunctionCallInfo fcinfo,
FunctionCachePtr fcache)
SQLFunctionCachePtr fcache)
{
TupleTableSlot *slot;
Datum value;
@ -319,7 +477,7 @@ postquel_execute(execution_state *es,
* If this is a single valued function we have to end the function
* execution now.
*/
if (!fcache->returnsSet)
if (!fcinfo->flinfo->fn_retset)
{
postquel_end(es);
es->status = F_EXEC_DONE;
@ -338,11 +496,10 @@ postquel_execute(execution_state *es,
}
Datum
postquel_function(FunctionCallInfo fcinfo,
FunctionCachePtr fcache,
bool *isDone)
fmgr_sql(PG_FUNCTION_ARGS)
{
MemoryContext oldcontext;
SQLFunctionCachePtr fcache;
execution_state *es;
Datum result = 0;
CommandId savedId;
@ -352,7 +509,7 @@ postquel_function(FunctionCallInfo fcinfo,
* parsetrees, plans, etc, will have sufficient lifetime. The
* sub-executor is responsible for deleting per-tuple information.
*/
oldcontext = MemoryContextSwitchTo(fcache->fcacheCxt);
oldcontext = MemoryContextSwitchTo(fcinfo->flinfo->fn_mcxt);
/*
* Before we start do anything we must save CurrentScanCommandId to
@ -362,13 +519,21 @@ postquel_function(FunctionCallInfo fcinfo,
savedId = GetScanCommandId();
SetScanCommandId(GetCurrentCommandId());
es = (execution_state *) fcache->func_state;
if (es == NULL)
/*
* Initialize fcache and execution state if first time through.
*/
fcache = (SQLFunctionCachePtr) fcinfo->flinfo->fn_extra;
if (fcache == NULL)
{
es = init_execution_state(fcache);
fcache->func_state = (char *) es;
init_sql_fcache(fcinfo->flinfo);
fcache = (SQLFunctionCachePtr) fcinfo->flinfo->fn_extra;
}
es = fcache->func_state;
Assert(es);
/*
* Find first unfinished query in function.
*/
while (es && es->status == F_EXEC_DONE)
es = es->next;
@ -401,7 +566,7 @@ postquel_function(FunctionCallInfo fcinfo,
/*
* Reset the execution states to start over again
*/
es = (execution_state *) fcache->func_state;
es = fcache->func_state;
while (es)
{
es->status = F_EXEC_START;
@ -411,9 +576,21 @@ postquel_function(FunctionCallInfo fcinfo,
/*
* Let caller know we're finished.
*/
*isDone = true;
if (fcinfo->flinfo->fn_retset)
{
ReturnSetInfo *rsi = (ReturnSetInfo *) fcinfo->resultinfo;
if (rsi && IsA(rsi, ReturnSetInfo))
rsi->isDone = ExprEndResult;
else
elog(ERROR, "Set-valued function called in context that cannot accept a set");
fcinfo->isnull = true;
result = (Datum) 0;
}
MemoryContextSwitchTo(oldcontext);
return (fcache->returnsSet) ? (Datum) NULL : result;
return result;
}
/*
@ -422,7 +599,18 @@ postquel_function(FunctionCallInfo fcinfo,
*/
Assert(LAST_POSTQUEL_COMMAND(es));
*isDone = false;
/*
* Let caller know we're not finished.
*/
if (fcinfo->flinfo->fn_retset)
{
ReturnSetInfo *rsi = (ReturnSetInfo *) fcinfo->resultinfo;
if (rsi && IsA(rsi, ReturnSetInfo))
rsi->isDone = ExprMultipleResult;
else
elog(ERROR, "Set-valued function called in context that cannot accept a set");
}
MemoryContextSwitchTo(oldcontext);

View File

@ -34,7 +34,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeAgg.c,v 1.70 2000/07/17 03:04:53 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeAgg.c,v 1.71 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -451,7 +451,6 @@ ExecAgg(Agg *node)
TupleTableSlot *resultSlot;
HeapTuple inputTuple;
int aggno;
bool isDone;
bool isNull;
/* ---------------------
@ -523,7 +522,7 @@ ExecAgg(Agg *node)
Datum newVal;
newVal = ExecEvalExpr(aggref->target, econtext,
&isNull, &isDone);
&isNull, NULL);
if (aggref->aggdistinct)
{
@ -677,8 +676,9 @@ ExecAgg(Agg *node)
/*
* Form a projection tuple using the aggregate results and the
* representative input tuple. Store it in the result tuple slot.
* Note we do not support aggregates returning sets ...
*/
resultSlot = ExecProject(projInfo, &isDone);
resultSlot = ExecProject(projInfo, NULL);
/*
* If the completed tuple does not match the qualifications, it is

View File

@ -15,7 +15,7 @@
* locate group boundaries.
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeGroup.c,v 1.37 2000/07/12 02:37:03 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeGroup.c,v 1.38 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -73,7 +73,6 @@ ExecGroupEveryTuple(Group *node)
TupleTableSlot *outerslot;
ProjectionInfo *projInfo;
TupleTableSlot *resultSlot;
bool isDone;
/* ---------------------
* get state info from node
@ -163,7 +162,7 @@ ExecGroupEveryTuple(Group *node)
projInfo = grpstate->csstate.cstate.cs_ProjInfo;
econtext->ecxt_scantuple = grpstate->csstate.css_ScanTupleSlot;
resultSlot = ExecProject(projInfo, &isDone);
resultSlot = ExecProject(projInfo, NULL);
return resultSlot;
}
@ -185,7 +184,6 @@ ExecGroupOneTuple(Group *node)
TupleTableSlot *outerslot;
ProjectionInfo *projInfo;
TupleTableSlot *resultSlot;
bool isDone;
/* ---------------------
* get state info from node
@ -258,7 +256,7 @@ ExecGroupOneTuple(Group *node)
grpstate->csstate.css_ScanTupleSlot,
InvalidBuffer, false);
econtext->ecxt_scantuple = grpstate->csstate.css_ScanTupleSlot;
resultSlot = ExecProject(projInfo, &isDone);
resultSlot = ExecProject(projInfo, NULL);
/* save outerTuple if we are not done yet */
if (!grpstate->grp_done)

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
*
* $Id: nodeHash.c,v 1.51 2000/08/22 04:06:19 tgl Exp $
* $Id: nodeHash.c,v 1.52 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -524,7 +524,6 @@ ExecHashGetBucket(HashJoinTable hashtable,
int bucketno;
Datum keyval;
bool isNull;
bool isDone;
/* ----------------
* Get the join attribute value of the tuple
@ -535,8 +534,7 @@ ExecHashGetBucket(HashJoinTable hashtable,
*/
ResetExprContext(econtext);
keyval = ExecEvalExprSwitchContext(hashkey, econtext,
&isNull, &isDone);
keyval = ExecEvalExprSwitchContext(hashkey, econtext, &isNull, NULL);
/* ------------------
* compute the hash function

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeHashjoin.c,v 1.32 2000/07/17 03:04:53 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeHashjoin.c,v 1.33 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -55,6 +55,7 @@ ExecHashJoin(HashJoin *node)
TupleTableSlot *inntuple;
Node *outerVar;
ExprContext *econtext;
ExprDoneCond isDone;
HashJoinTable hashtable;
HeapTuple curtuple;
TupleTableSlot *outerTupleSlot;
@ -83,13 +84,6 @@ ExecHashJoin(HashJoin *node)
hashtable = hjstate->hj_HashTable;
econtext = hjstate->jstate.cs_ExprContext;
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* Check to see if we're still projecting out tuples from a previous
* join tuple (because there is a function-returning-set in the
@ -99,15 +93,22 @@ ExecHashJoin(HashJoin *node)
if (hjstate->jstate.cs_TupFromTlist)
{
TupleTableSlot *result;
bool isDone;
result = ExecProject(hjstate->jstate.cs_ProjInfo, &isDone);
if (!isDone)
if (isDone == ExprMultipleResult)
return result;
/* Done with that source tuple... */
hjstate->jstate.cs_TupFromTlist = false;
}
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle. Note this can't
* happen until we're done projecting out tuples from a join tuple.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* if this is the first call, build the hash table for inner relation
* ----------------
@ -241,15 +242,15 @@ ExecHashJoin(HashJoin *node)
*/
if (ExecQual(qual, econtext, false))
{
ProjectionInfo *projInfo;
TupleTableSlot *result;
bool isDone;
hjstate->jstate.cs_OuterTupleSlot = outerTupleSlot;
projInfo = hjstate->jstate.cs_ProjInfo;
result = ExecProject(projInfo, &isDone);
hjstate->jstate.cs_TupFromTlist = !isDone;
return result;
result = ExecProject(hjstate->jstate.cs_ProjInfo, &isDone);
if (isDone != ExprEndResult)
{
hjstate->jstate.cs_TupFromTlist = (isDone == ExprMultipleResult);
return result;
}
}
}

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeIndexscan.c,v 1.53 2000/08/13 02:50:03 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeIndexscan.c,v 1.54 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -334,7 +334,6 @@ ExecIndexReScan(IndexScan *node, ExprContext *exprCtxt, Plan *parent)
Node *scanexpr;
Datum scanvalue;
bool isNull;
bool isDone;
estate = node->scan.plan.state;
indexstate = node->indxstate;
@ -411,14 +410,10 @@ ExecIndexReScan(IndexScan *node, ExprContext *exprCtxt, Plan *parent)
(Node *) get_rightop(clause) :
(Node *) get_leftop(clause);
/*
* pass in isDone but ignore it. We don't iterate in
* quals
*/
scanvalue = ExecEvalExprSwitchContext(scanexpr,
econtext,
&isNull,
&isDone);
NULL);
scan_keys[j].sk_argument = scanvalue;
if (isNull)
scan_keys[j].sk_flags |= SK_ISNULL;

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeMergejoin.c,v 1.36 2000/07/12 02:37:03 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeMergejoin.c,v 1.37 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -226,18 +226,16 @@ MergeCompare(List *eqQual, List *compareQual, ExprContext *econtext)
{
Datum const_value;
bool isNull;
bool isDone;
/* ----------------
* first test if our compare clause is satisfied.
* if so then return true.
*
* A NULL result is considered false.
* ignore isDone, don't iterate in quals.
* ----------------
*/
const_value = ExecEvalExpr((Node *) lfirst(clause), econtext,
&isNull, &isDone);
&isNull, NULL);
if (DatumGetBool(const_value) && !isNull)
{
@ -254,7 +252,7 @@ MergeCompare(List *eqQual, List *compareQual, ExprContext *econtext)
const_value = ExecEvalExpr((Node *) lfirst(eqclause),
econtext,
&isNull,
&isDone);
NULL);
if (! DatumGetBool(const_value) || isNull)
break; /* return false */
@ -447,13 +445,6 @@ ExecMergeJoin(MergeJoin *node)
innerSkipQual = mergestate->mj_OuterSkipQual;
}
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* Check to see if we're still projecting out tuples from a previous
* join tuple (because there is a function-returning-set in the
@ -463,15 +454,23 @@ ExecMergeJoin(MergeJoin *node)
if (mergestate->jstate.cs_TupFromTlist)
{
TupleTableSlot *result;
bool isDone;
ExprDoneCond isDone;
result = ExecProject(mergestate->jstate.cs_ProjInfo, &isDone);
if (!isDone)
if (isDone == ExprMultipleResult)
return result;
/* Done with that source tuple... */
mergestate->jstate.cs_TupFromTlist = false;
}
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle. Note this can't
* happen until we're done projecting out tuples from a join tuple.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* ok, everything is setup.. let's go to work
* ----------------
@ -599,17 +598,19 @@ ExecMergeJoin(MergeJoin *node)
* projection tuple and return the slot containing it.
* ----------------
*/
ProjectionInfo *projInfo;
TupleTableSlot *result;
bool isDone;
ExprDoneCond isDone;
MJ_printf("ExecMergeJoin: **** returning tuple ****\n");
projInfo = mergestate->jstate.cs_ProjInfo;
result = ExecProject(mergestate->jstate.cs_ProjInfo,
&isDone);
result = ExecProject(projInfo, &isDone);
mergestate->jstate.cs_TupFromTlist = !isDone;
return result;
if (isDone != ExprEndResult)
{
mergestate->jstate.cs_TupFromTlist = (isDone == ExprMultipleResult);
return result;
}
}
break;

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeNestloop.c,v 1.19 2000/08/13 02:50:03 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeNestloop.c,v 1.20 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -87,13 +87,6 @@ ExecNestLoop(NestLoop *node)
outerTupleSlot = nlstate->jstate.cs_OuterTupleSlot;
econtext->ecxt_outertuple = outerTupleSlot;
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* Check to see if we're still projecting out tuples from a previous
* join tuple (because there is a function-returning-set in the
@ -103,15 +96,23 @@ ExecNestLoop(NestLoop *node)
if (nlstate->jstate.cs_TupFromTlist)
{
TupleTableSlot *result;
bool isDone;
ExprDoneCond isDone;
result = ExecProject(nlstate->jstate.cs_ProjInfo, &isDone);
if (!isDone)
if (isDone == ExprMultipleResult)
return result;
/* Done with that source tuple... */
nlstate->jstate.cs_TupFromTlist = false;
}
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle. Note this can't
* happen until we're done projecting out tuples from a join tuple.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* Ok, everything is setup for the join so now loop until
* we return a qualifying join tuple..
@ -219,16 +220,18 @@ ExecNestLoop(NestLoop *node)
* using ExecProject().
* ----------------
*/
ProjectionInfo *projInfo;
TupleTableSlot *result;
bool isDone;
ExprDoneCond isDone;
ENL1_printf("qualification succeeded, projecting tuple");
projInfo = nlstate->jstate.cs_ProjInfo;
result = ExecProject(projInfo, &isDone);
nlstate->jstate.cs_TupFromTlist = !isDone;
return result;
result = ExecProject(nlstate->jstate.cs_ProjInfo, &isDone);
if (isDone != ExprEndResult)
{
nlstate->jstate.cs_TupFromTlist = (isDone == ExprMultipleResult);
return result;
}
}
/* ----------------

View File

@ -34,7 +34,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeResult.c,v 1.15 2000/07/17 03:04:53 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeResult.c,v 1.16 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -67,8 +67,7 @@ ExecResult(Result *node)
TupleTableSlot *resultSlot;
Plan *outerPlan;
ExprContext *econtext;
bool isDone;
ProjectionInfo *projInfo;
ExprDoneCond isDone;
/* ----------------
* initialize the result node's state
@ -77,13 +76,6 @@ ExecResult(Result *node)
resstate = node->resstate;
econtext = resstate->cstate.cs_ExprContext;
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* check constant qualifications like (2 > 1), if not already done
* ----------------
@ -111,12 +103,20 @@ ExecResult(Result *node)
if (resstate->cstate.cs_TupFromTlist)
{
resultSlot = ExecProject(resstate->cstate.cs_ProjInfo, &isDone);
if (!isDone)
if (isDone == ExprMultipleResult)
return resultSlot;
/* Done with that source tuple... */
resstate->cstate.cs_TupFromTlist = false;
}
/* ----------------
* Reset per-tuple memory context to free any expression evaluation
* storage allocated in the previous tuple cycle. Note this can't
* happen until we're done projecting out tuples from a scan tuple.
* ----------------
*/
ResetExprContext(econtext);
/* ----------------
* if rs_done is true then it means that we were asked to return
* a constant tuple and we already did the last time ExecResult()
@ -124,7 +124,7 @@ ExecResult(Result *node)
* Either way, now we are through.
* ----------------
*/
if (!resstate->rs_done)
while (!resstate->rs_done)
{
outerPlan = outerPlan(node);
@ -159,13 +159,18 @@ ExecResult(Result *node)
}
/* ----------------
* form the result tuple using ExecProject(), and return it.
* form the result tuple using ExecProject(), and return it
* --- unless the projection produces an empty set, in which case
* we must loop back to see if there are more outerPlan tuples.
* ----------------
*/
projInfo = resstate->cstate.cs_ProjInfo;
resultSlot = ExecProject(projInfo, &isDone);
resstate->cstate.cs_TupFromTlist = !isDone;
return resultSlot;
resultSlot = ExecProject(resstate->cstate.cs_ProjInfo, &isDone);
if (isDone != ExprEndResult)
{
resstate->cstate.cs_TupFromTlist = (isDone == ExprMultipleResult);
return resultSlot;
}
}
return NULL;

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.26 2000/07/12 02:37:04 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSubplan.c,v 1.27 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -40,7 +40,6 @@ ExecSubPlan(SubPlan *node, List *pvar, ExprContext *econtext, bool *isNull)
MemoryContext oldcontext;
TupleTableSlot *slot;
Datum result;
bool isDone;
bool found = false; /* TRUE if got at least one subplan tuple */
List *lst;
@ -67,9 +66,7 @@ ExecSubPlan(SubPlan *node, List *pvar, ExprContext *econtext, bool *isNull)
prm->value = ExecEvalExprSwitchContext((Node *) lfirst(pvar),
econtext,
&(prm->isnull),
&isDone);
if (!isDone)
elog(ERROR, "ExecSubPlan: set values not supported for params");
NULL);
pvar = lnext(pvar);
}
plan->chgParam = nconc(plan->chgParam, listCopy(node->parParam));
@ -189,9 +186,7 @@ ExecSubPlan(SubPlan *node, List *pvar, ExprContext *econtext, bool *isNull)
* Now we can eval the combining operator for this column.
*/
expresult = ExecEvalExprSwitchContext((Node *) expr, econtext,
&expnull, &isDone);
if (!isDone)
elog(ERROR, "ExecSubPlan: set values not supported for combining operators");
&expnull, NULL);
/*
* Combine the result into the row result as appropriate.

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeTidscan.c,v 1.11 2000/08/03 19:19:30 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeTidscan.c,v 1.12 2000/08/24 03:29:03 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -38,7 +38,6 @@ TidListCreate(List *evalList, ExprContext *econtext, ItemPointer *tidList)
List *lst;
ItemPointer itemptr;
bool isNull;
bool isDone;
int numTids = 0;
foreach(lst, evalList)
@ -47,8 +46,8 @@ TidListCreate(List *evalList, ExprContext *econtext, ItemPointer *tidList)
DatumGetPointer(ExecEvalExprSwitchContext(lfirst(lst),
econtext,
&isNull,
&isDone));
if (itemptr && ItemPointerIsValid(itemptr))
NULL));
if (!isNull && itemptr && ItemPointerIsValid(itemptr))
{
tidList[numTids] = itemptr;
numTids++;