Commit f6dba10e authored by Tom Lane's avatar Tom Lane

First phase of implementing hash-based grouping/aggregation. An AGG plan

node now does its own grouping of the input rows, and has no need for a
preceding GROUP node in the plan pipeline.  This allows elimination of
the misnamed tuplePerGroup option for GROUP, and actually saves more code
in nodeGroup.c than it costs in nodeAgg.c, as well as being presumably
faster.  Restructure the API of query_planner so that we do not commit to
using a sorted or unsorted plan in query_planner; instead grouping_planner
makes the decision.  (Right now it isn't any smarter than query_planner
was, but that will change as soon as it has the option to select a hash-
based aggregation step.)  Despite all the hackery, no initdb needed since
only in-memory node types changed.
parent a8c18b98
...@@ -5,7 +5,7 @@ ...@@ -5,7 +5,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994-5, Regents of the University of California * Portions Copyright (c) 1994-5, Regents of the University of California
* *
* $Header: /cvsroot/pgsql/src/backend/commands/explain.c,v 1.89 2002/10/14 04:26:54 momjian Exp $ * $Header: /cvsroot/pgsql/src/backend/commands/explain.c,v 1.90 2002/11/06 00:00:43 tgl Exp $
* *
*/ */
...@@ -275,8 +275,22 @@ explain_outNode(StringInfo str, Plan *plan, Plan *outer_plan, ...@@ -275,8 +275,22 @@ explain_outNode(StringInfo str, Plan *plan, Plan *outer_plan,
pname = "Group"; pname = "Group";
break; break;
case T_Agg: case T_Agg:
switch (((Agg *) plan)->aggstrategy)
{
case AGG_PLAIN:
pname = "Aggregate"; pname = "Aggregate";
break; break;
case AGG_SORTED:
pname = "GroupAggregate";
break;
case AGG_HASHED:
pname = "HashAggregate";
break;
default:
pname = "Aggregate ???";
break;
}
break;
case T_Unique: case T_Unique:
pname = "Unique"; pname = "Unique";
break; break;
......
...@@ -46,7 +46,7 @@ ...@@ -46,7 +46,7 @@
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeAgg.c,v 1.90 2002/11/01 19:33:09 tgl Exp $ * $Header: /cvsroot/pgsql/src/backend/executor/nodeAgg.c,v 1.91 2002/11/06 00:00:43 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -58,6 +58,7 @@ ...@@ -58,6 +58,7 @@
#include "catalog/pg_operator.h" #include "catalog/pg_operator.h"
#include "executor/executor.h" #include "executor/executor.h"
#include "executor/nodeAgg.h" #include "executor/nodeAgg.h"
#include "executor/nodeGroup.h"
#include "miscadmin.h" #include "miscadmin.h"
#include "optimizer/clauses.h" #include "optimizer/clauses.h"
#include "parser/parse_coerce.h" #include "parser/parse_coerce.h"
...@@ -159,6 +160,7 @@ typedef struct AggStatePerAggData ...@@ -159,6 +160,7 @@ typedef struct AggStatePerAggData
static void initialize_aggregate(AggStatePerAgg peraggstate); static void initialize_aggregate(AggStatePerAgg peraggstate);
static void advance_transition_function(AggStatePerAgg peraggstate, static void advance_transition_function(AggStatePerAgg peraggstate,
Datum newVal, bool isNull); Datum newVal, bool isNull);
static void advance_aggregates(AggState *aggstate, ExprContext *econtext);
static void process_sorted_aggregate(AggState *aggstate, static void process_sorted_aggregate(AggState *aggstate,
AggStatePerAgg peraggstate); AggStatePerAgg peraggstate);
static void finalize_aggregate(AggStatePerAgg peraggstate, static void finalize_aggregate(AggStatePerAgg peraggstate,
...@@ -313,6 +315,62 @@ advance_transition_function(AggStatePerAgg peraggstate, ...@@ -313,6 +315,62 @@ advance_transition_function(AggStatePerAgg peraggstate,
peraggstate->transValueIsNull = fcinfo.isnull; peraggstate->transValueIsNull = fcinfo.isnull;
} }
/*
* Advance all the aggregates for one input tuple. The input tuple
* has been stored in econtext->ecxt_scantuple, so that it is accessible
* to ExecEvalExpr.
*
* When called, CurrentMemoryContext should be the per-query context.
*/
static void
advance_aggregates(AggState *aggstate, ExprContext *econtext)
{
MemoryContext oldContext;
int aggno;
/*
* Clear and select the current working context for evaluation
* of the input expressions and transition functions at this
* input tuple.
*/
econtext->ecxt_per_tuple_memory = aggstate->agg_cxt[aggstate->which_cxt];
ResetExprContext(econtext);
oldContext = MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory);
for (aggno = 0; aggno < aggstate->numaggs; aggno++)
{
AggStatePerAgg peraggstate = &aggstate->peragg[aggno];
Aggref *aggref = peraggstate->aggref;
Datum newVal;
bool isNull;
newVal = ExecEvalExpr(aggref->target, econtext, &isNull, NULL);
if (aggref->aggdistinct)
{
/* in DISTINCT mode, we may ignore nulls */
if (isNull)
continue;
/* putdatum has to be called in per-query context */
MemoryContextSwitchTo(oldContext);
tuplesort_putdatum(peraggstate->sortstate, newVal, isNull);
MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory);
}
else
{
advance_transition_function(peraggstate, newVal, isNull);
}
}
/*
* Make the other context current so that these transition
* results are preserved.
*/
aggstate->which_cxt = 1 - aggstate->which_cxt;
MemoryContextSwitchTo(oldContext);
}
/* /*
* Run the transition function for a DISTINCT aggregate. This is called * Run the transition function for a DISTINCT aggregate. This is called
* after we have completed entering all the input values into the sort * after we have completed entering all the input values into the sort
...@@ -448,23 +506,18 @@ finalize_aggregate(AggStatePerAgg peraggstate, ...@@ -448,23 +506,18 @@ finalize_aggregate(AggStatePerAgg peraggstate,
} }
/* --------------------------------------- /*
*
* ExecAgg - * ExecAgg -
* *
* ExecAgg receives tuples from its outer subplan and aggregates over * ExecAgg receives tuples from its outer subplan and aggregates over
* the appropriate attribute for each aggregate function use (Aggref * the appropriate attribute for each aggregate function use (Aggref
* node) appearing in the targetlist or qual of the node. The number * node) appearing in the targetlist or qual of the node. The number
* of tuples to aggregate over depends on whether a GROUP BY clause is * of tuples to aggregate over depends on whether grouped or plain
* present. We can produce an aggregate result row per group, or just * aggregation is selected. In grouped aggregation, we produce a result
* one for the whole query. The value of each aggregate is stored in * row for each group; in plain aggregation there's a single result row
* the expression context to be used when ExecProject evaluates the * for the whole query. In either case, the value of each aggregate is
* result tuple. * stored in the expression context to be used when ExecProject evaluates
* * the result tuple.
* If the outer subplan is a Group node, ExecAgg returns as many tuples
* as there are groups.
*
* ------------------------------------------
*/ */
TupleTableSlot * TupleTableSlot *
ExecAgg(Agg *node) ExecAgg(Agg *node)
...@@ -478,10 +531,10 @@ ExecAgg(Agg *node) ...@@ -478,10 +531,10 @@ ExecAgg(Agg *node)
bool *aggnulls; bool *aggnulls;
AggStatePerAgg peragg; AggStatePerAgg peragg;
MemoryContext oldContext; MemoryContext oldContext;
TupleTableSlot *outerslot;
TupleTableSlot *firstSlot;
TupleTableSlot *resultSlot; TupleTableSlot *resultSlot;
HeapTuple inputTuple;
int aggno; int aggno;
bool isNull;
/* /*
* get state info from node * get state info from node
...@@ -494,6 +547,7 @@ ExecAgg(Agg *node) ...@@ -494,6 +547,7 @@ ExecAgg(Agg *node)
aggnulls = econtext->ecxt_aggnulls; aggnulls = econtext->ecxt_aggnulls;
projInfo = aggstate->csstate.cstate.cs_ProjInfo; projInfo = aggstate->csstate.cstate.cs_ProjInfo;
peragg = aggstate->peragg; peragg = aggstate->peragg;
firstSlot = aggstate->csstate.css_ScanTupleSlot;
/* /*
* We loop retrieving groups until we find one matching * We loop retrieving groups until we find one matching
...@@ -504,6 +558,31 @@ ExecAgg(Agg *node) ...@@ -504,6 +558,31 @@ ExecAgg(Agg *node)
if (aggstate->agg_done) if (aggstate->agg_done)
return NULL; return NULL;
/*
* If we don't already have the first tuple of the new group,
* fetch it from the outer plan.
*/
if (aggstate->grp_firstTuple == NULL)
{
outerslot = ExecProcNode(outerPlan, (Plan *) node);
if (!TupIsNull(outerslot))
{
/*
* Make a copy of the first input tuple; we will use this
* for comparisons (in group mode) and for projection.
*/
aggstate->grp_firstTuple = heap_copytuple(outerslot->val);
}
else
{
/* outer plan produced no tuples at all */
aggstate->agg_done = true;
/* If we are grouping, we should produce no tuples too */
if (node->aggstrategy != AGG_PLAIN)
return NULL;
}
}
/* /*
* Clear the per-output-tuple context for each group * Clear the per-output-tuple context for each group
*/ */
...@@ -519,74 +598,61 @@ ExecAgg(Agg *node) ...@@ -519,74 +598,61 @@ ExecAgg(Agg *node)
initialize_aggregate(peraggstate); initialize_aggregate(peraggstate);
} }
inputTuple = NULL; /* no saved input tuple yet */ if (aggstate->grp_firstTuple != NULL)
{
/*
* Store the copied first input tuple in the tuple table slot
* reserved for it. The tuple will be deleted when it is
* cleared from the slot.
*/
ExecStoreTuple(aggstate->grp_firstTuple,
firstSlot,
InvalidBuffer,
true);
aggstate->grp_firstTuple = NULL; /* don't keep two pointers */
/* set up for first advance_aggregates call */
econtext->ecxt_scantuple = firstSlot;
/* /*
* for each tuple from the outer plan, update all the aggregates * Process each outer-plan tuple, and then fetch the next one,
* until we exhaust the outer plan or cross a group boundary.
*/ */
for (;;) for (;;)
{ {
TupleTableSlot *outerslot; advance_aggregates(aggstate, econtext);
outerslot = ExecProcNode(outerPlan, (Plan *) node); outerslot = ExecProcNode(outerPlan, (Plan *) node);
if (TupIsNull(outerslot)) if (TupIsNull(outerslot))
{
/* no more outer-plan tuples available */
aggstate->agg_done = true;
break; break;
}
/* set up for next advance_aggregates call */
econtext->ecxt_scantuple = outerslot; econtext->ecxt_scantuple = outerslot;
/* /*
* Clear and select the current working context for evaluation * If we are grouping, check whether we've crossed a group
* of the input expressions and transition functions at this * boundary.
* input tuple.
*/ */
econtext->ecxt_per_tuple_memory = if (node->aggstrategy == AGG_SORTED)
aggstate->agg_cxt[aggstate->which_cxt];
ResetExprContext(econtext);
oldContext =
MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory);
for (aggno = 0; aggno < aggstate->numaggs; aggno++)
{ {
AggStatePerAgg peraggstate = &peragg[aggno]; if (!execTuplesMatch(firstSlot->val,
Aggref *aggref = peraggstate->aggref; outerslot->val,
Datum newVal; firstSlot->ttc_tupleDescriptor,
node->numCols, node->grpColIdx,
newVal = ExecEvalExpr(aggref->target, econtext, aggstate->eqfunctions,
&isNull, NULL); aggstate->agg_cxt[aggstate->which_cxt]))
if (aggref->aggdistinct)
{ {
/* in DISTINCT mode, we may ignore nulls */ /*
if (isNull) * Save the first input tuple of the next group.
continue; */
/* putdatum has to be called in per-query context */ aggstate->grp_firstTuple = heap_copytuple(outerslot->val);
MemoryContextSwitchTo(oldContext); break;
tuplesort_putdatum(peraggstate->sortstate,
newVal, isNull);
MemoryContextSwitchTo(econtext->ecxt_per_tuple_memory);
} }
else
{
advance_transition_function(peraggstate,
newVal, isNull);
} }
} }
/*
* Make the other context current so that these transition
* results are preserved.
*/
aggstate->which_cxt = 1 - aggstate->which_cxt;
MemoryContextSwitchTo(oldContext);
/*
* Keep a copy of the first input tuple for the projection.
* (We only need one since only the GROUP BY columns in it can
* be referenced, and these will be the same for all tuples
* aggregated over.)
*/
if (!inputTuple)
inputTuple = heap_copytuple(outerslot->val);
} }
/* /*
...@@ -626,32 +692,7 @@ ExecAgg(Agg *node) ...@@ -626,32 +692,7 @@ ExecAgg(Agg *node)
} }
/* /*
* If the outerPlan is a Group node, we will reach here after each * If we have no first tuple (ie, the outerPlan didn't return
* group. We are not done unless the Group node is done (a little
* ugliness here while we reach into the Group's state to find
* out). Furthermore, when grouping we return nothing at all
* unless we had some input tuple(s). By the nature of Group,
* there are no empty groups, so if we get here with no input the
* whole scan is empty.
*
* If the outerPlan isn't a Group, we are done when we get here, and
* we will emit a (single) tuple even if there were no input
* tuples.
*/
if (IsA(outerPlan, Group))
{
/* aggregation over groups */
aggstate->agg_done = ((Group *) outerPlan)->grpstate->grp_done;
/* check for no groups */
if (inputTuple == NULL)
return NULL;
}
else
{
aggstate->agg_done = true;
/*
* If inputtuple==NULL (ie, the outerPlan didn't return
* anything), create a dummy all-nulls input tuple for use by * anything), create a dummy all-nulls input tuple for use by
* ExecProject. 99.44% of the time this is a waste of cycles, * ExecProject. 99.44% of the time this is a waste of cycles,
* because ordinarily the projected output tuple's targetlist * because ordinarily the projected output tuple's targetlist
...@@ -665,37 +706,35 @@ ExecAgg(Agg *node) ...@@ -665,37 +706,35 @@ ExecAgg(Agg *node)
* The values returned for the aggregates will be the initial * The values returned for the aggregates will be the initial
* values of the transition functions. * values of the transition functions.
*/ */
if (inputTuple == NULL) if (TupIsNull(firstSlot))
{ {
TupleDesc tupType; TupleDesc tupType;
Datum *dvalues;
char *dnulls;
tupType = aggstate->csstate.css_ScanTupleSlot->ttc_tupleDescriptor; /* Should only happen in non-grouped mode */
/* watch out for null input tuples, though... */ Assert(node->aggstrategy == AGG_PLAIN);
Assert(aggstate->agg_done);
tupType = firstSlot->ttc_tupleDescriptor;
/* watch out for zero-column input tuples, though... */
if (tupType && tupType->natts > 0) if (tupType && tupType->natts > 0)
{ {
HeapTuple nullsTuple;
Datum *dvalues;
char *dnulls;
dvalues = (Datum *) palloc(sizeof(Datum) * tupType->natts); dvalues = (Datum *) palloc(sizeof(Datum) * tupType->natts);
dnulls = (char *) palloc(sizeof(char) * tupType->natts); dnulls = (char *) palloc(sizeof(char) * tupType->natts);
MemSet(dvalues, 0, sizeof(Datum) * tupType->natts); MemSet(dvalues, 0, sizeof(Datum) * tupType->natts);
MemSet(dnulls, 'n', sizeof(char) * tupType->natts); MemSet(dnulls, 'n', sizeof(char) * tupType->natts);
inputTuple = heap_formtuple(tupType, dvalues, dnulls); nullsTuple = heap_formtuple(tupType, dvalues, dnulls);
ExecStoreTuple(nullsTuple,
firstSlot,
InvalidBuffer,
true);
pfree(dvalues); pfree(dvalues);
pfree(dnulls); pfree(dnulls);
} }
} }
}
/*
* Store the representative input tuple in the tuple table slot
* reserved for it. The tuple will be deleted when it is cleared
* from the slot.
*/
ExecStoreTuple(inputTuple,
aggstate->csstate.css_ScanTupleSlot,
InvalidBuffer,
true);
econtext->ecxt_scantuple = aggstate->csstate.css_ScanTupleSlot;
/* /*
* Do projection and qual check in the per-output-tuple context. * Do projection and qual check in the per-output-tuple context.
...@@ -707,6 +746,7 @@ ExecAgg(Agg *node) ...@@ -707,6 +746,7 @@ ExecAgg(Agg *node)
* representative input tuple. Store it in the result tuple slot. * representative input tuple. Store it in the result tuple slot.
* Note we do not support aggregates returning sets ... * Note we do not support aggregates returning sets ...
*/ */
econtext->ecxt_scantuple = firstSlot;
resultSlot = ExecProject(projInfo, NULL); resultSlot = ExecProject(projInfo, NULL);
/* /*
...@@ -748,6 +788,8 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent) ...@@ -748,6 +788,8 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent)
*/ */
aggstate = makeNode(AggState); aggstate = makeNode(AggState);
node->aggstate = aggstate; node->aggstate = aggstate;
aggstate->eqfunctions = NULL;
aggstate->grp_firstTuple = NULL;
aggstate->agg_done = false; aggstate->agg_done = false;
/* /*
...@@ -765,13 +807,12 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent) ...@@ -765,13 +807,12 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent)
if (numaggs <= 0) if (numaggs <= 0)
{ {
/* /*
* This used to be treated as an error, but we can't do that * This is not an error condition: we might be using the Agg node just
* anymore because constant-expression simplification could * to do hash-based grouping. Even in the regular case,
* optimize away all of the Aggrefs in the targetlist and qual. * constant-expression simplification could optimize away all of the
* So, just make a debug note, and force numaggs positive so that * Aggrefs in the targetlist and qual. So keep going, but force local
* palloc()s below don't choke. * copy of numaggs positive so that palloc()s below don't choke.
*/ */
elog(DEBUG1, "ExecInitAgg: could not find any aggregate functions");
numaggs = 1; numaggs = 1;
} }
...@@ -843,6 +884,17 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent) ...@@ -843,6 +884,17 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent)
ExecAssignResultTypeFromTL((Plan *) node, &aggstate->csstate.cstate); ExecAssignResultTypeFromTL((Plan *) node, &aggstate->csstate.cstate);
ExecAssignProjectionInfo((Plan *) node, &aggstate->csstate.cstate); ExecAssignProjectionInfo((Plan *) node, &aggstate->csstate.cstate);
/*
* If we are grouping, precompute fmgr lookup data for inner loop
*/
if (node->numCols > 0)
{
aggstate->eqfunctions =
execTuplesMatchPrepare(ExecGetScanType(&aggstate->csstate),
node->numCols,
node->grpColIdx);
}
/* /*
* Perform lookups of aggregate function info, and initialize the * Perform lookups of aggregate function info, and initialize the
* unchanging fields of the per-agg data * unchanging fields of the per-agg data
...@@ -1024,6 +1076,11 @@ ExecEndAgg(Agg *node) ...@@ -1024,6 +1076,11 @@ ExecEndAgg(Agg *node)
/* clean up tuple table */ /* clean up tuple table */
ExecClearTuple(aggstate->csstate.css_ScanTupleSlot); ExecClearTuple(aggstate->csstate.css_ScanTupleSlot);
if (aggstate->grp_firstTuple != NULL)
{
heap_freetuple(aggstate->grp_firstTuple);
aggstate->grp_firstTuple = NULL;
}
} }
void void
...@@ -1033,6 +1090,11 @@ ExecReScanAgg(Agg *node, ExprContext *exprCtxt, Plan *parent) ...@@ -1033,6 +1090,11 @@ ExecReScanAgg(Agg *node, ExprContext *exprCtxt, Plan *parent)
ExprContext *econtext = aggstate->csstate.cstate.cs_ExprContext; ExprContext *econtext = aggstate->csstate.cstate.cs_ExprContext;
aggstate->agg_done = false; aggstate->agg_done = false;
if (aggstate->grp_firstTuple != NULL)
{
heap_freetuple(aggstate->grp_firstTuple);
aggstate->grp_firstTuple = NULL;
}
MemSet(econtext->ecxt_aggvalues, 0, sizeof(Datum) * aggstate->numaggs); MemSet(econtext->ecxt_aggvalues, 0, sizeof(Datum) * aggstate->numaggs);
MemSet(econtext->ecxt_aggnulls, 0, sizeof(bool) * aggstate->numaggs); MemSet(econtext->ecxt_aggnulls, 0, sizeof(bool) * aggstate->numaggs);
......
...@@ -15,7 +15,7 @@ ...@@ -15,7 +15,7 @@
* locate group boundaries. * locate group boundaries.
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeGroup.c,v 1.47 2002/06/20 20:29:28 momjian Exp $ * $Header: /cvsroot/pgsql/src/backend/executor/nodeGroup.c,v 1.48 2002/11/06 00:00:43 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -31,147 +31,14 @@ ...@@ -31,147 +31,14 @@
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "utils/syscache.h" #include "utils/syscache.h"
static TupleTableSlot *ExecGroupEveryTuple(Group *node);
static TupleTableSlot *ExecGroupOneTuple(Group *node);
/* --------------------------------------- /*
* ExecGroup - * ExecGroup -
* *
* There are two modes in which tuples are returned by ExecGroup. If * Return one tuple for each group of matching input tuples.
* tuplePerGroup is TRUE, every tuple from the same group will be
* returned, followed by a NULL at the end of each group. This is
* useful for Agg node which needs to aggregate over tuples of the same
* group. (eg. SELECT salary, count(*) FROM emp GROUP BY salary)
*
* If tuplePerGroup is FALSE, only one tuple per group is returned. The
* tuple returned contains only the group columns. NULL is returned only
* at the end when no more groups are present. This is useful when
* the query does not involve aggregates. (eg. SELECT salary FROM emp
* GROUP BY salary)
* ------------------------------------------
*/ */
TupleTableSlot * TupleTableSlot *
ExecGroup(Group *node) ExecGroup(Group *node)
{
if (node->tuplePerGroup)
return ExecGroupEveryTuple(node);
else
return ExecGroupOneTuple(node);
}
/*
* ExecGroupEveryTuple -
* return every tuple with a NULL between each group
*/
static TupleTableSlot *
ExecGroupEveryTuple(Group *node)
{
GroupState *grpstate;
EState *estate;
ExprContext *econtext;
TupleDesc tupdesc;
HeapTuple outerTuple = NULL;
HeapTuple firsttuple;
TupleTableSlot *outerslot;
ProjectionInfo *projInfo;
TupleTableSlot *resultSlot;
/*
* get state info from node
*/
grpstate = node->grpstate;
if (grpstate->grp_done)
return NULL;
estate = node->plan.state;
econtext = grpstate->csstate.cstate.cs_ExprContext;
tupdesc = ExecGetScanType(&grpstate->csstate);
/*
* We need not call ResetExprContext here because execTuplesMatch will
* reset the per-tuple memory context once per input tuple.
*/
/* if we haven't returned first tuple of a new group yet ... */
if (grpstate->grp_useFirstTuple)
{
grpstate->grp_useFirstTuple = FALSE;
/*
* note we rely on subplan to hold ownership of the tuple for as
* long as we need it; we don't copy it.
*/
ExecStoreTuple(grpstate->grp_firstTuple,
grpstate->csstate.css_ScanTupleSlot,
InvalidBuffer, false);
}
else
{
outerslot = ExecProcNode(outerPlan(node), (Plan *) node);
if (TupIsNull(outerslot))
{
grpstate->grp_done = TRUE;
return NULL;
}
outerTuple = outerslot->val;
firsttuple = grpstate->grp_firstTuple;
if (firsttuple == NULL)
{
/* this should occur on the first call only */
grpstate->grp_firstTuple = heap_copytuple(outerTuple);
}
else
{
/*
* Compare with first tuple and see if this tuple is of the
* same group.
*/
if (!execTuplesMatch(firsttuple, outerTuple,
tupdesc,
node->numCols, node->grpColIdx,
grpstate->eqfunctions,
econtext->ecxt_per_tuple_memory))
{
/*
* No; save the tuple to return it next time, and return
* NULL
*/
grpstate->grp_useFirstTuple = TRUE;
heap_freetuple(firsttuple);
grpstate->grp_firstTuple = heap_copytuple(outerTuple);
return NULL; /* signifies the end of the group */
}
}
/*
* note we rely on subplan to hold ownership of the tuple for as
* long as we need it; we don't copy it.
*/
ExecStoreTuple(outerTuple,
grpstate->csstate.css_ScanTupleSlot,
InvalidBuffer, false);
}
/*
* form a projection tuple, store it in the result tuple slot and
* return it.
*/
projInfo = grpstate->csstate.cstate.cs_ProjInfo;
econtext->ecxt_scantuple = grpstate->csstate.css_ScanTupleSlot;
resultSlot = ExecProject(projInfo, NULL);
return resultSlot;
}
/*
* ExecGroupOneTuple -
* returns one tuple per group, a NULL at the end when there are no more
* tuples.
*/
static TupleTableSlot *
ExecGroupOneTuple(Group *node)
{ {
GroupState *grpstate; GroupState *grpstate;
EState *estate; EState *estate;
...@@ -198,10 +65,11 @@ ExecGroupOneTuple(Group *node) ...@@ -198,10 +65,11 @@ ExecGroupOneTuple(Group *node)
* reset the per-tuple memory context once per input tuple. * reset the per-tuple memory context once per input tuple.
*/ */
/* If we don't already have first tuple of group, fetch it */
/* this should occur on the first call only */
firsttuple = grpstate->grp_firstTuple; firsttuple = grpstate->grp_firstTuple;
if (firsttuple == NULL) if (firsttuple == NULL)
{ {
/* this should occur on the first call only */
outerslot = ExecProcNode(outerPlan(node), (Plan *) node); outerslot = ExecProcNode(outerPlan(node), (Plan *) node);
if (TupIsNull(outerslot)) if (TupIsNull(outerslot))
{ {
...@@ -213,7 +81,7 @@ ExecGroupOneTuple(Group *node) ...@@ -213,7 +81,7 @@ ExecGroupOneTuple(Group *node)
} }
/* /*
* find all tuples that belong to a group * Scan over all tuples that belong to this group
*/ */
for (;;) for (;;)
{ {
...@@ -239,22 +107,18 @@ ExecGroupOneTuple(Group *node) ...@@ -239,22 +107,18 @@ ExecGroupOneTuple(Group *node)
} }
/* /*
* form a projection tuple, store it in the result tuple slot and * form a projection tuple based on the (copied) first tuple of the
* return it. * group, and store it in the result tuple slot.
*/
projInfo = grpstate->csstate.cstate.cs_ProjInfo;
/*
* note we rely on subplan to hold ownership of the tuple for as long
* as we need it; we don't copy it.
*/ */
ExecStoreTuple(firsttuple, ExecStoreTuple(firsttuple,
grpstate->csstate.css_ScanTupleSlot, grpstate->csstate.css_ScanTupleSlot,
InvalidBuffer, false); InvalidBuffer,
false);
econtext->ecxt_scantuple = grpstate->csstate.css_ScanTupleSlot; econtext->ecxt_scantuple = grpstate->csstate.css_ScanTupleSlot;
projInfo = grpstate->csstate.cstate.cs_ProjInfo;
resultSlot = ExecProject(projInfo, NULL); resultSlot = ExecProject(projInfo, NULL);
/* save outerTuple if we are not done yet */ /* save first tuple of next group, if we are not done yet */
if (!grpstate->grp_done) if (!grpstate->grp_done)
{ {
heap_freetuple(firsttuple); heap_freetuple(firsttuple);
...@@ -386,14 +250,14 @@ ExecReScanGroup(Group *node, ExprContext *exprCtxt, Plan *parent) ...@@ -386,14 +250,14 @@ ExecReScanGroup(Group *node, ExprContext *exprCtxt, Plan *parent)
} }
/***************************************************************************** /*****************************************************************************
* Code shared with nodeUnique.c * Code shared with nodeUnique.c and nodeAgg.c
*****************************************************************************/ *****************************************************************************/
/* /*
* execTuplesMatch * execTuplesMatch
* Return true if two tuples match in all the indicated fields. * Return true if two tuples match in all the indicated fields.
* This is used to detect group boundaries in nodeGroup, and to * This is used to detect group boundaries in nodeGroup and nodeAgg,
* decide whether two tuples are distinct or not in nodeUnique. * and to decide whether two tuples are distinct or not in nodeUnique.
* *
* tuple1, tuple2: the tuples to compare * tuple1, tuple2: the tuples to compare
* tupdesc: tuple descriptor applying to both tuples * tupdesc: tuple descriptor applying to both tuples
...@@ -425,7 +289,8 @@ execTuplesMatch(HeapTuple tuple1, ...@@ -425,7 +289,8 @@ execTuplesMatch(HeapTuple tuple1,
* We cannot report a match without checking all the fields, but we * We cannot report a match without checking all the fields, but we
* can report a non-match as soon as we find unequal fields. So, * can report a non-match as soon as we find unequal fields. So,
* start comparing at the last field (least significant sort key). * start comparing at the last field (least significant sort key).
* That's the most likely to be different... * That's the most likely to be different if we are dealing with
* sorted input.
*/ */
result = true; result = true;
......
...@@ -15,7 +15,7 @@ ...@@ -15,7 +15,7 @@
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/nodes/copyfuncs.c,v 1.214 2002/10/14 22:14:34 tgl Exp $ * $Header: /cvsroot/pgsql/src/backend/nodes/copyfuncs.c,v 1.215 2002/11/06 00:00:43 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -497,10 +497,10 @@ _copyGroup(Group *from) ...@@ -497,10 +497,10 @@ _copyGroup(Group *from)
CopyPlanFields((Plan *) from, (Plan *) newnode); CopyPlanFields((Plan *) from, (Plan *) newnode);
newnode->tuplePerGroup = from->tuplePerGroup;
newnode->numCols = from->numCols; newnode->numCols = from->numCols;
newnode->grpColIdx = palloc(from->numCols * sizeof(AttrNumber)); newnode->grpColIdx = palloc(from->numCols * sizeof(AttrNumber));
memcpy(newnode->grpColIdx, from->grpColIdx, from->numCols * sizeof(AttrNumber)); memcpy(newnode->grpColIdx, from->grpColIdx,
from->numCols * sizeof(AttrNumber));
return newnode; return newnode;
} }
...@@ -516,6 +516,15 @@ _copyAgg(Agg *from) ...@@ -516,6 +516,15 @@ _copyAgg(Agg *from)
CopyPlanFields((Plan *) from, (Plan *) newnode); CopyPlanFields((Plan *) from, (Plan *) newnode);
newnode->aggstrategy = from->aggstrategy;
newnode->numCols = from->numCols;
if (from->numCols > 0)
{
newnode->grpColIdx = palloc(from->numCols * sizeof(AttrNumber));
memcpy(newnode->grpColIdx, from->grpColIdx,
from->numCols * sizeof(AttrNumber));
}
return newnode; return newnode;
} }
...@@ -1280,6 +1289,29 @@ _copyAppendPath(AppendPath *from) ...@@ -1280,6 +1289,29 @@ _copyAppendPath(AppendPath *from)
return newnode; return newnode;
} }
/* ----------------
* _copyResultPath
* ----------------
*/
static ResultPath *
_copyResultPath(ResultPath *from)
{
ResultPath *newnode = makeNode(ResultPath);
/*
* copy the node superclass fields
*/
CopyPathFields((Path *) from, (Path *) newnode);
/*
* copy remainder of node
*/
Node_Copy(from, newnode, subpath);
Node_Copy(from, newnode, constantqual);
return newnode;
}
/* ---------------- /* ----------------
* CopyJoinPathFields * CopyJoinPathFields
* *
...@@ -2878,6 +2910,9 @@ copyObject(void *from) ...@@ -2878,6 +2910,9 @@ copyObject(void *from)
case T_AppendPath: case T_AppendPath:
retval = _copyAppendPath(from); retval = _copyAppendPath(from);
break; break;
case T_ResultPath:
retval = _copyResultPath(from);
break;
case T_NestPath: case T_NestPath:
retval = _copyNestPath(from); retval = _copyNestPath(from);
break; break;
......
...@@ -20,7 +20,7 @@ ...@@ -20,7 +20,7 @@
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/nodes/equalfuncs.c,v 1.161 2002/10/14 22:14:34 tgl Exp $ * $Header: /cvsroot/pgsql/src/backend/nodes/equalfuncs.c,v 1.162 2002/11/06 00:00:43 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -463,6 +463,18 @@ _equalAppendPath(AppendPath *a, AppendPath *b) ...@@ -463,6 +463,18 @@ _equalAppendPath(AppendPath *a, AppendPath *b)
return true; return true;
} }
static bool
_equalResultPath(ResultPath *a, ResultPath *b)
{
if (!_equalPath((Path *) a, (Path *) b))
return false;
if (!equal(a->subpath, b->subpath))
return false;
if (!equal(a->constantqual, b->constantqual))
return false;
return true;
}
static bool static bool
_equalJoinPath(JoinPath *a, JoinPath *b) _equalJoinPath(JoinPath *a, JoinPath *b)
{ {
...@@ -2103,6 +2115,9 @@ equal(void *a, void *b) ...@@ -2103,6 +2115,9 @@ equal(void *a, void *b)
case T_AppendPath: case T_AppendPath:
retval = _equalAppendPath(a, b); retval = _equalAppendPath(a, b);
break; break;
case T_ResultPath:
retval = _equalResultPath(a, b);
break;
case T_IndexOptInfo: case T_IndexOptInfo:
retval = _equalIndexOptInfo(a, b); retval = _equalIndexOptInfo(a, b);
break; break;
......
...@@ -5,7 +5,7 @@ ...@@ -5,7 +5,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Header: /cvsroot/pgsql/src/backend/nodes/outfuncs.c,v 1.176 2002/10/14 22:14:34 tgl Exp $ * $Header: /cvsroot/pgsql/src/backend/nodes/outfuncs.c,v 1.177 2002/11/06 00:00:44 tgl Exp $
* *
* NOTES * NOTES
* Every (plan) node in POSTGRES has an associated "out" routine which * Every (plan) node in POSTGRES has an associated "out" routine which
...@@ -597,6 +597,8 @@ _outAgg(StringInfo str, Agg *node) ...@@ -597,6 +597,8 @@ _outAgg(StringInfo str, Agg *node)
{ {
appendStringInfo(str, " AGG "); appendStringInfo(str, " AGG ");
_outPlanInfo(str, (Plan *) node); _outPlanInfo(str, (Plan *) node);
appendStringInfo(str, " :aggstrategy %d :numCols %d ",
(int) node->aggstrategy, node->numCols);
} }
static void static void
...@@ -604,11 +606,7 @@ _outGroup(StringInfo str, Group *node) ...@@ -604,11 +606,7 @@ _outGroup(StringInfo str, Group *node)
{ {
appendStringInfo(str, " GRP "); appendStringInfo(str, " GRP ");
_outPlanInfo(str, (Plan *) node); _outPlanInfo(str, (Plan *) node);
appendStringInfo(str, " :numCols %d ", node->numCols);
/* the actual Group fields */
appendStringInfo(str, " :numCols %d :tuplePerGroup %s ",
node->numCols,
booltostr(node->tuplePerGroup));
} }
static void static void
...@@ -1114,6 +1112,26 @@ _outAppendPath(StringInfo str, AppendPath *node) ...@@ -1114,6 +1112,26 @@ _outAppendPath(StringInfo str, AppendPath *node)
_outNode(str, node->subpaths); _outNode(str, node->subpaths);
} }
/*
* ResultPath is a subclass of Path.
*/
static void
_outResultPath(StringInfo str, ResultPath *node)
{
appendStringInfo(str,
" RESULTPATH :pathtype %d :startup_cost %.2f :total_cost %.2f :pathkeys ",
node->path.pathtype,
node->path.startup_cost,
node->path.total_cost);
_outNode(str, node->path.pathkeys);
appendStringInfo(str, " :subpath ");
_outNode(str, node->subpath);
appendStringInfo(str, " :constantqual ");
_outNode(str, node->constantqual);
}
/* /*
* NestPath is a subclass of Path * NestPath is a subclass of Path
*/ */
...@@ -1717,6 +1735,9 @@ _outNode(StringInfo str, void *obj) ...@@ -1717,6 +1735,9 @@ _outNode(StringInfo str, void *obj)
case T_AppendPath: case T_AppendPath:
_outAppendPath(str, obj); _outAppendPath(str, obj);
break; break;
case T_ResultPath:
_outResultPath(str, obj);
break;
case T_NestPath: case T_NestPath:
_outNestPath(str, obj); _outNestPath(str, obj);
break; break;
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/nodes/readfuncs.c,v 1.135 2002/10/14 22:14:34 tgl Exp $ * $Header: /cvsroot/pgsql/src/backend/nodes/readfuncs.c,v 1.136 2002/11/06 00:00:44 tgl Exp $
* *
* NOTES * NOTES
* Most of the read functions for plan nodes are tested. (In fact, they * Most of the read functions for plan nodes are tested. (In fact, they
...@@ -696,17 +696,6 @@ _readSort(void) ...@@ -696,17 +696,6 @@ _readSort(void)
return local_node; return local_node;
} }
static Agg *
_readAgg(void)
{
Agg *local_node;
local_node = makeNode(Agg);
_getPlan((Plan *) local_node);
return local_node;
}
/* ---------------- /* ----------------
* _readHash * _readHash
* *
...@@ -1880,6 +1869,45 @@ _readAppendPath(void) ...@@ -1880,6 +1869,45 @@ _readAppendPath(void)
return local_node; return local_node;
} }
/* ----------------
* _readResultPath
*
* ResultPath is a subclass of Path.
* ----------------
*/
static ResultPath *
_readResultPath(void)
{
ResultPath *local_node;
char *token;
int length;
local_node = makeNode(ResultPath);
token = pg_strtok(&length); /* get :pathtype */
token = pg_strtok(&length); /* now read it */
local_node->path.pathtype = atoi(token);
token = pg_strtok(&length); /* get :startup_cost */
token = pg_strtok(&length); /* now read it */
local_node->path.startup_cost = (Cost) atof(token);
token = pg_strtok(&length); /* get :total_cost */
token = pg_strtok(&length); /* now read it */
local_node->path.total_cost = (Cost) atof(token);
token = pg_strtok(&length); /* get :pathkeys */
local_node->path.pathkeys = nodeRead(true); /* now read it */
token = pg_strtok(&length); /* get :subpath */
local_node->subpath = nodeRead(true); /* now read it */
token = pg_strtok(&length); /* get :constantqual */
local_node->constantqual = nodeRead(true); /* now read it */
return local_node;
}
/* ---------------- /* ----------------
* _readNestPath * _readNestPath
* *
...@@ -2196,8 +2224,6 @@ parsePlanString(void) ...@@ -2196,8 +2224,6 @@ parsePlanString(void)
return_value = _readFromExpr(); return_value = _readFromExpr();
else if (length == 8 && strncmp(token, "JOINEXPR", length) == 0) else if (length == 8 && strncmp(token, "JOINEXPR", length) == 0)
return_value = _readJoinExpr(); return_value = _readJoinExpr();
else if (length == 3 && strncmp(token, "AGG", length) == 0)
return_value = _readAgg();
else if (length == 4 && strncmp(token, "HASH", length) == 0) else if (length == 4 && strncmp(token, "HASH", length) == 0)
return_value = _readHash(); return_value = _readHash();
else if (length == 6 && strncmp(token, "RESDOM", length) == 0) else if (length == 6 && strncmp(token, "RESDOM", length) == 0)
...@@ -2240,6 +2266,8 @@ parsePlanString(void) ...@@ -2240,6 +2266,8 @@ parsePlanString(void)
return_value = _readTidPath(); return_value = _readTidPath();
else if (length == 10 && strncmp(token, "APPENDPATH", length) == 0) else if (length == 10 && strncmp(token, "APPENDPATH", length) == 0)
return_value = _readAppendPath(); return_value = _readAppendPath();
else if (length == 10 && strncmp(token, "RESULTPATH", length) == 0)
return_value = _readResultPath();
else if (length == 8 && strncmp(token, "NESTPATH", length) == 0) else if (length == 8 && strncmp(token, "NESTPATH", length) == 0)
return_value = _readNestPath(); return_value = _readNestPath();
else if (length == 9 && strncmp(token, "MERGEPATH", length) == 0) else if (length == 9 && strncmp(token, "MERGEPATH", length) == 0)
......
...@@ -219,8 +219,6 @@ planner() ...@@ -219,8 +219,6 @@ planner()
pull out constant quals, which can be used to gate execution of the pull out constant quals, which can be used to gate execution of the
whole plan (if any are found, we make a top-level Result node whole plan (if any are found, we make a top-level Result node
to do the gating) to do the gating)
make a simplified target list that only contains Vars, no expressions
---subplanner()
make list of base relations used in query make list of base relations used in query
split up the qual into restrictions (a=1) and joins (b=c) split up the qual into restrictions (a=1) and joins (b=c)
find qual clauses that enable merge and hash joins find qual clauses that enable merge and hash joins
...@@ -239,7 +237,7 @@ planner() ...@@ -239,7 +237,7 @@ planner()
cheapest path for each newly constructed joinrel. cheapest path for each newly constructed joinrel.
Loop back if this wasn't the top join level. Loop back if this wasn't the top join level.
Back at query_planner: Back at query_planner:
put back constant quals and non-simplified target list put back any constant quals by adding a Result node
Back at grouping_planner: Back at grouping_planner:
do grouping(GROUP) do grouping(GROUP)
do aggregates do aggregates
...@@ -257,8 +255,11 @@ RelOptInfo - a relation or joined relations ...@@ -257,8 +255,11 @@ RelOptInfo - a relation or joined relations
JoinInfo - join clauses, including the relids needed for the join JoinInfo - join clauses, including the relids needed for the join
Path - every way to generate a RelOptInfo(sequential,index,joins) Path - every way to generate a RelOptInfo(sequential,index,joins)
SeqScan - a plain Path node with nodeTag = T_SeqScan SeqScan - a plain Path node with pathtype = T_SeqScan
IndexPath - index scans IndexPath - index scans
TidPath - scan by CTID
AppendPath - append multiple subpaths together
ResultPath - a Result plan (used for variable-free tlist or qual)
NestPath - nested-loop joins NestPath - nested-loop joins
MergePath - merge joins MergePath - merge joins
HashPath - hash joins HashPath - hash joins
...@@ -276,10 +277,10 @@ generated during the optimization process are marked with their sort order ...@@ -276,10 +277,10 @@ generated during the optimization process are marked with their sort order
It is also possible to avoid an explicit sort step to implement a user's It is also possible to avoid an explicit sort step to implement a user's
ORDER BY clause if the final path has the right ordering already, so the ORDER BY clause if the final path has the right ordering already, so the
sort ordering is of interest even at the top level. subplanner() will sort ordering is of interest even at the top level. query_planner() will
look for the cheapest path with a sort order matching the desired order, look for the cheapest path with a sort order matching the desired order,
and will compare its cost to the cost of using the cheapest-overall path and grouping_planner() will compare its cost to the cost of using the
and doing an explicit sort. cheapest-overall path and doing an explicit sort.
When we are generating paths for a particular RelOptInfo, we discard a path When we are generating paths for a particular RelOptInfo, we discard a path
if it is more expensive than another known path that has the same or better if it is more expensive than another known path that has the same or better
......
...@@ -6,7 +6,7 @@ ...@@ -6,7 +6,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: geqo_misc.c,v 1.34 2002/09/04 20:31:20 momjian Exp $ * $Header: /cvsroot/pgsql/src/backend/optimizer/geqo/geqo_misc.c,v 1.35 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -19,19 +19,17 @@ ...@@ -19,19 +19,17 @@
=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*= =*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=*=
*/ */
#include "postgres.h" #include "postgres.h"
#include "optimizer/geqo_misc.h" #include "optimizer/geqo_misc.h"
#include "nodes/print.h" #include "nodes/print.h"
#ifdef GEQO_DEBUG #ifdef GEQO_DEBUG
static float avg_pool(Pool *pool);
/* avg_pool /*
* * avg_pool
*/ */
static float static float
avg_pool(Pool *pool) avg_pool(Pool *pool)
...@@ -81,7 +79,6 @@ print_pool(FILE *fp, Pool *pool, int start, int stop) ...@@ -81,7 +79,6 @@ print_pool(FILE *fp, Pool *pool, int start, int stop)
/* print_gen /* print_gen
* *
* printout for chromosome: best, worst, mean, average * printout for chromosome: best, worst, mean, average
*
*/ */
void void
print_gen(FILE *fp, Pool *pool, int generation) print_gen(FILE *fp, Pool *pool, int generation)
...@@ -121,133 +118,4 @@ print_edge_table(FILE *fp, Edge *edge_table, int num_gene) ...@@ -121,133 +118,4 @@ print_edge_table(FILE *fp, Edge *edge_table, int num_gene)
fprintf(fp, "\n"); fprintf(fp, "\n");
} }
/*************************************************************
Debug output subroutines
*************************************************************/
void
geqo_print_joinclauses(Query *root, List *clauses)
{
List *l;
foreach(l, clauses)
{
RestrictInfo *c = lfirst(l);
print_expr((Node *) c->clause, root->rtable);
if (lnext(l))
printf(" ");
}
}
void
geqo_print_path(Query *root, Path *path, int indent)
{
char *ptype = NULL;
JoinPath *jp;
bool join = false;
int i;
for (i = 0; i < indent; i++)
printf("\t");
switch (nodeTag(path))
{
case T_Path:
ptype = "SeqScan";
join = false;
break;
case T_IndexPath:
ptype = "IdxScan";
join = false;
break;
case T_NestPath:
ptype = "Nestloop";
join = true;
break;
case T_MergePath:
ptype = "MergeJoin";
join = true;
break;
case T_HashPath:
ptype = "HashJoin";
join = true;
break;
default:
break;
}
if (join)
{
jp = (JoinPath *) path;
printf("%s rows=%.0f cost=%.2f..%.2f\n",
ptype, path->parent->rows,
path->startup_cost, path->total_cost);
switch (nodeTag(path))
{
case T_MergePath:
case T_HashPath:
for (i = 0; i < indent + 1; i++)
printf("\t");
printf(" clauses=(");
geqo_print_joinclauses(root, jp->joinrestrictinfo);
printf(")\n");
if (nodeTag(path) == T_MergePath)
{
MergePath *mp = (MergePath *) path;
if (mp->outersortkeys || mp->innersortkeys)
{
for (i = 0; i < indent + 1; i++)
printf("\t");
printf(" sortouter=%d sortinner=%d\n",
((mp->outersortkeys) ? 1 : 0),
((mp->innersortkeys) ? 1 : 0));
}
}
break;
default:
break;
}
geqo_print_path(root, jp->outerjoinpath, indent + 1);
geqo_print_path(root, jp->innerjoinpath, indent + 1);
}
else
{
int relid = lfirsti(path->parent->relids);
printf("%s(%d) rows=%.0f cost=%.2f..%.2f\n",
ptype, relid, path->parent->rows,
path->startup_cost, path->total_cost);
if (IsA(path, IndexPath))
{
printf(" pathkeys=");
print_pathkeys(path->pathkeys, root->rtable);
}
}
}
void
geqo_print_rel(Query *root, RelOptInfo *rel)
{
List *l;
printf("______________________________\n");
printf("(");
foreach(l, rel->relids)
printf("%d ", lfirsti(l));
printf("): rows=%.0f width=%d\n", rel->rows, rel->width);
printf("\tpath list:\n");
foreach(l, rel->pathlist)
geqo_print_path(root, lfirst(l), 1);
printf("\n\tcheapest startup path:\n");
geqo_print_path(root, rel->cheapest_startup_path, 1);
printf("\n\tcheapest total path:\n");
geqo_print_path(root, rel->cheapest_total_path, 1);
}
#endif /* GEQO_DEBUG */ #endif /* GEQO_DEBUG */
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/path/allpaths.c,v 1.88 2002/09/04 20:31:20 momjian Exp $ * $Header: /cvsroot/pgsql/src/backend/optimizer/path/allpaths.c,v 1.89 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -742,6 +742,14 @@ print_path(Query *root, Path *path, int indent) ...@@ -742,6 +742,14 @@ print_path(Query *root, Path *path, int indent)
ptype = "TidScan"; ptype = "TidScan";
join = false; join = false;
break; break;
case T_AppendPath:
ptype = "Append";
join = false;
break;
case T_ResultPath:
ptype = "Result";
join = false;
break;
case T_NestPath: case T_NestPath:
ptype = "Nestloop"; ptype = "Nestloop";
join = true; join = true;
...@@ -762,10 +770,15 @@ print_path(Query *root, Path *path, int indent) ...@@ -762,10 +770,15 @@ print_path(Query *root, Path *path, int indent)
for (i = 0; i < indent; i++) for (i = 0; i < indent; i++)
printf("\t"); printf("\t");
printf("%s(", ptype); printf("%s", ptype);
if (path->parent)
{
printf("(");
print_relids(path->parent->relids); print_relids(path->parent->relids);
printf(") rows=%.0f cost=%.2f..%.2f\n", printf(") rows=%.0f", path->parent->rows);
path->parent->rows, path->startup_cost, path->total_cost); }
printf(" cost=%.2f..%.2f\n", path->startup_cost, path->total_cost);
if (path->pathkeys) if (path->pathkeys)
{ {
...@@ -785,7 +798,7 @@ print_path(Query *root, Path *path, int indent) ...@@ -785,7 +798,7 @@ print_path(Query *root, Path *path, int indent)
print_restrictclauses(root, jp->joinrestrictinfo); print_restrictclauses(root, jp->joinrestrictinfo);
printf("\n"); printf("\n");
if (nodeTag(path) == T_MergePath) if (IsA(path, MergePath))
{ {
MergePath *mp = (MergePath *) path; MergePath *mp = (MergePath *) path;
......
...@@ -10,7 +10,7 @@ ...@@ -10,7 +10,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/createplan.c,v 1.119 2002/09/18 21:35:21 tgl Exp $ * $Header: /cvsroot/pgsql/src/backend/optimizer/plan/createplan.c,v 1.120 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -34,6 +34,7 @@ ...@@ -34,6 +34,7 @@
static Scan *create_scan_plan(Query *root, Path *best_path); static Scan *create_scan_plan(Query *root, Path *best_path);
static Join *create_join_plan(Query *root, JoinPath *best_path); static Join *create_join_plan(Query *root, JoinPath *best_path);
static Append *create_append_plan(Query *root, AppendPath *best_path); static Append *create_append_plan(Query *root, AppendPath *best_path);
static Result *create_result_plan(Query *root, ResultPath *best_path);
static SeqScan *create_seqscan_plan(Path *best_path, List *tlist, static SeqScan *create_seqscan_plan(Path *best_path, List *tlist,
List *scan_clauses); List *scan_clauses);
static IndexScan *create_indexscan_plan(Query *root, IndexPath *best_path, static IndexScan *create_indexscan_plan(Query *root, IndexPath *best_path,
...@@ -135,6 +136,10 @@ create_plan(Query *root, Path *best_path) ...@@ -135,6 +136,10 @@ create_plan(Query *root, Path *best_path)
plan = (Plan *) create_append_plan(root, plan = (Plan *) create_append_plan(root,
(AppendPath *) best_path); (AppendPath *) best_path);
break; break;
case T_Result:
plan = (Plan *) create_result_plan(root,
(ResultPath *) best_path);
break;
default: default:
elog(ERROR, "create_plan: unknown pathtype %d", elog(ERROR, "create_plan: unknown pathtype %d",
best_path->pathtype); best_path->pathtype);
...@@ -342,6 +347,35 @@ create_append_plan(Query *root, AppendPath *best_path) ...@@ -342,6 +347,35 @@ create_append_plan(Query *root, AppendPath *best_path)
return plan; return plan;
} }
/*
* create_result_plan
* Create a Result plan for 'best_path' and (recursively) plans
* for its subpaths.
*
* Returns a Plan node.
*/
static Result *
create_result_plan(Query *root, ResultPath *best_path)
{
Result *plan;
List *tlist;
Plan *subplan;
if (best_path->path.parent)
tlist = best_path->path.parent->targetlist;
else
tlist = NIL; /* will be filled in later */
if (best_path->subpath)
subplan = create_plan(root, best_path->subpath);
else
subplan = NULL;
plan = make_result(tlist, (Node *) best_path->constantqual, subplan);
return plan;
}
/***************************************************************************** /*****************************************************************************
* *
...@@ -1605,11 +1639,16 @@ make_material(List *tlist, Plan *lefttree) ...@@ -1605,11 +1639,16 @@ make_material(List *tlist, Plan *lefttree)
} }
Agg * Agg *
make_agg(List *tlist, List *qual, Plan *lefttree) make_agg(List *tlist, List *qual, AggStrategy aggstrategy,
int ngrp, AttrNumber *grpColIdx, Plan *lefttree)
{ {
Agg *node = makeNode(Agg); Agg *node = makeNode(Agg);
Plan *plan = &node->plan; Plan *plan = &node->plan;
node->aggstrategy = aggstrategy;
node->numCols = ngrp;
node->grpColIdx = grpColIdx;
copy_plan_costsize(plan, lefttree); copy_plan_costsize(plan, lefttree);
/* /*
...@@ -1621,22 +1660,21 @@ make_agg(List *tlist, List *qual, Plan *lefttree) ...@@ -1621,22 +1660,21 @@ make_agg(List *tlist, List *qual, Plan *lefttree)
length(pull_agg_clause((Node *) qual))); length(pull_agg_clause((Node *) qual)));
/* /*
* We will produce a single output tuple if the input is not a Group, * We will produce a single output tuple if not grouping,
* and a tuple per group otherwise. For now, estimate the number of * and a tuple per group otherwise. For now, estimate the number of
* groups as 10% of the number of tuples --- bogus, but how to do * groups as 10% of the number of tuples --- bogus, but how to do
* better? (Note we assume the input Group node is in "tuplePerGroup" * better?
* mode, so it didn't reduce its row count already.)
*/ */
if (IsA(lefttree, Group)) if (aggstrategy == AGG_PLAIN)
{ {
plan->plan_rows *= 0.1;
if (plan->plan_rows < 1)
plan->plan_rows = 1; plan->plan_rows = 1;
plan->startup_cost = plan->total_cost;
} }
else else
{ {
plan->plan_rows *= 0.1;
if (plan->plan_rows < 1)
plan->plan_rows = 1; plan->plan_rows = 1;
plan->startup_cost = plan->total_cost;
} }
plan->state = (EState *) NULL; plan->state = (EState *) NULL;
...@@ -1650,7 +1688,6 @@ make_agg(List *tlist, List *qual, Plan *lefttree) ...@@ -1650,7 +1688,6 @@ make_agg(List *tlist, List *qual, Plan *lefttree)
Group * Group *
make_group(List *tlist, make_group(List *tlist,
bool tuplePerGroup,
int ngrp, int ngrp,
AttrNumber *grpColIdx, AttrNumber *grpColIdx,
Plan *lefttree) Plan *lefttree)
...@@ -1667,25 +1704,18 @@ make_group(List *tlist, ...@@ -1667,25 +1704,18 @@ make_group(List *tlist,
plan->total_cost += cpu_operator_cost * plan->plan_rows * ngrp; plan->total_cost += cpu_operator_cost * plan->plan_rows * ngrp;
/* /*
* If tuplePerGroup (which is named exactly backwards) is true, we * Estimate the number of groups as 10% of the number of tuples
* will return all the input tuples, so the input node's row count is
* OK. Otherwise, we'll return only one tuple from each group. For
* now, estimate the number of groups as 10% of the number of tuples
* --- bogus, but how to do better? * --- bogus, but how to do better?
*/ */
if (!tuplePerGroup)
{
plan->plan_rows *= 0.1; plan->plan_rows *= 0.1;
if (plan->plan_rows < 1) if (plan->plan_rows < 1)
plan->plan_rows = 1; plan->plan_rows = 1;
}
plan->state = (EState *) NULL; plan->state = (EState *) NULL;
plan->qual = NULL; plan->qual = NULL;
plan->targetlist = tlist; plan->targetlist = tlist;
plan->lefttree = lefttree; plan->lefttree = lefttree;
plan->righttree = (Plan *) NULL; plan->righttree = (Plan *) NULL;
node->tuplePerGroup = tuplePerGroup;
node->numCols = ngrp; node->numCols = ngrp;
node->grpColIdx = grpColIdx; node->grpColIdx = grpColIdx;
...@@ -1883,9 +1913,6 @@ make_result(List *tlist, ...@@ -1883,9 +1913,6 @@ make_result(List *tlist,
Result *node = makeNode(Result); Result *node = makeNode(Result);
Plan *plan = &node->plan; Plan *plan = &node->plan;
#ifdef NOT_USED
tlist = generate_fjoin(tlist);
#endif
if (subplan) if (subplan)
copy_plan_costsize(plan, subplan); copy_plan_costsize(plan, subplan);
else else
...@@ -1906,57 +1933,3 @@ make_result(List *tlist, ...@@ -1906,57 +1933,3 @@ make_result(List *tlist,
return node; return node;
} }
#ifdef NOT_USED
List *
generate_fjoin(List *tlist)
{
List tlistP;
List newTlist = NIL;
List fjoinList = NIL;
int nIters = 0;
/*
* Break the target list into elements with Iter nodes, and those
* without them.
*/
foreach(tlistP, tlist)
{
List tlistElem;
tlistElem = lfirst(tlistP);
if (IsA(lsecond(tlistElem), Iter))
{
nIters++;
fjoinList = lappend(fjoinList, tlistElem);
}
else
newTlist = lappend(newTlist, tlistElem);
}
/*
* if we have an Iter node then we need to flatten.
*/
if (nIters > 0)
{
List *inner;
List *tempList;
Fjoin *fjoinNode;
DatumPtr results = (DatumPtr) palloc(nIters * sizeof(Datum));
BoolPtr alwaysDone = (BoolPtr) palloc(nIters * sizeof(bool));
inner = lfirst(fjoinList);
fjoinList = lnext(fjoinList);
fjoinNode = (Fjoin) MakeFjoin(false,
nIters,
inner,
results,
alwaysDone);
tempList = lcons(fjoinNode, fjoinList);
newTlist = lappend(newTlist, tempList);
}
return newTlist;
return tlist; /* do nothing for now - ay 10/94 */
}
#endif
...@@ -14,15 +14,13 @@ ...@@ -14,15 +14,13 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/planmain.c,v 1.70 2002/09/02 02:47:02 momjian Exp $ * $Header: /cvsroot/pgsql/src/backend/optimizer/plan/planmain.c,v 1.71 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
#include "postgres.h" #include "postgres.h"
#include "optimizer/clauses.h" #include "optimizer/clauses.h"
#include "optimizer/cost.h"
#include "optimizer/pathnode.h" #include "optimizer/pathnode.h"
#include "optimizer/paths.h" #include "optimizer/paths.h"
#include "optimizer/planmain.h" #include "optimizer/planmain.h"
...@@ -31,31 +29,37 @@ ...@@ -31,31 +29,37 @@
#include "utils/memutils.h" #include "utils/memutils.h"
static Plan *subplanner(Query *root, List *flat_tlist,
double tuple_fraction);
/*-------------------- /*--------------------
* query_planner * query_planner
* Generate a plan for a basic query, which may involve joins but * Generate a path (that is, a simplified plan) for a basic query,
* not any fancier features. * which may involve joins but not any fancier features.
* *
* Since query_planner does not handle the toplevel processing (grouping,
* sorting, etc) it cannot select the best path by itself. It selects
* two paths: the cheapest path that produces the required tuples, independent
* of any ordering considerations, and the cheapest path that produces the
* required tuples in the required ordering, if there is a path that
* can produce them without an explicit top-level sort step. The caller
* (grouping_planner) will make the final decision about which to use.
*
* Input parameters:
* root is the query to plan
* tlist is the target list the query should produce (NOT root->targetList!) * tlist is the target list the query should produce (NOT root->targetList!)
* tuple_fraction is the fraction of tuples we expect will be retrieved * tuple_fraction is the fraction of tuples we expect will be retrieved
* *
* Note: the Query node now also includes a query_pathkeys field, which * Output parameters:
* is both an input and an output of query_planner(). The input value * *cheapest_path receives the overall-cheapest path for the query
* signals query_planner that the indicated sort order is wanted in the * *sorted_path receives the cheapest presorted path for the query,
* final output plan. The output value is the actual pathkeys of the * if any (it may be NULL, or the same as cheapest_path)
* selected path. This might not be the same as what the caller requested; *
* the caller must do pathkeys_contained_in() to decide whether an * Note: the Query node also includes a query_pathkeys field, which is both
* explicit sort is still needed. (The main reason query_pathkeys is a * an input and an output of query_planner(). The input value signals
* Query field and not a passed parameter is that the low-level routines * query_planner that the indicated sort order is wanted in the final output
* in indxpath.c need to see it.) The pathkeys value passed to query_planner * plan. But this value has not yet been "canonicalized", since the needed
* has not yet been "canonicalized", since the necessary info does not get * info does not get computed until we scan the qual clauses. We canonicalize
* computed until subplanner() scans the qual clauses. We canonicalize it * it as soon as that task is done. (The main reason query_pathkeys is a
* inside subplanner() as soon as that task is done. The output value * Query field and not a passed parameter is that the low-level routines in
* will be in canonical form as well. * indxpath.c need to see it.)
* *
* tuple_fraction is interpreted as follows: * tuple_fraction is interpreted as follows:
* 0 (or less): expect all tuples to be retrieved (normal case) * 0 (or less): expect all tuples to be retrieved (normal case)
...@@ -66,18 +70,14 @@ static Plan *subplanner(Query *root, List *flat_tlist, ...@@ -66,18 +70,14 @@ static Plan *subplanner(Query *root, List *flat_tlist,
* Note that while this routine and its subroutines treat a negative * Note that while this routine and its subroutines treat a negative
* tuple_fraction the same as 0, grouping_planner has a different * tuple_fraction the same as 0, grouping_planner has a different
* interpretation. * interpretation.
*
* Returns a query plan.
*-------------------- *--------------------
*/ */
Plan * void
query_planner(Query *root, query_planner(Query *root, List *tlist, double tuple_fraction,
List *tlist, Path **cheapest_path, Path **sorted_path)
double tuple_fraction)
{ {
List *constant_quals; List *constant_quals;
List *var_only_tlist; RelOptInfo *final_rel;
Plan *subplan;
/* /*
* If the query has an empty join tree, then it's something easy like * If the query has an empty join tree, then it's something easy like
...@@ -85,11 +85,10 @@ query_planner(Query *root, ...@@ -85,11 +85,10 @@ query_planner(Query *root,
*/ */
if (root->jointree->fromlist == NIL) if (root->jointree->fromlist == NIL)
{ {
root->query_pathkeys = NIL; /* signal unordered result */ *cheapest_path = (Path *) create_result_path(NULL, NULL,
(List *) root->jointree->quals);
/* Make childless Result node to evaluate given tlist. */ *sorted_path = NULL;
return (Plan *) make_result(tlist, root->jointree->quals, return;
(Plan *) NULL);
} }
/* /*
...@@ -107,80 +106,8 @@ query_planner(Query *root, ...@@ -107,80 +106,8 @@ query_planner(Query *root,
&constant_quals); &constant_quals);
/* /*
* Create a target list that consists solely of (resdom var) target * init planner lists to empty
* list entries, i.e., contains no arbitrary expressions.
*
* All subplan nodes will have "flat" (var-only) tlists.
*
* This implies that all expression evaluations are done at the root of
* the plan tree. Once upon a time there was code to try to push
* expensive function calls down to lower plan nodes, but that's dead
* code and has been for a long time...
*/
var_only_tlist = flatten_tlist(tlist);
/*
* Choose the best access path and build a plan for it.
*/
subplan = subplanner(root, var_only_tlist, tuple_fraction);
/*
* Build a result node to control the plan if we have constant quals,
* or if the top-level plan node is one that cannot do expression
* evaluation (it won't be able to evaluate the requested tlist).
* Currently, the only plan node we might see here that falls into
* that category is Append.
*
* XXX future improvement: if the given tlist is flat anyway, we don't
* really need a Result node.
*/
if (constant_quals || IsA(subplan, Append))
{
/*
* The result node will also be responsible for evaluating the
* originally requested tlist.
*/
subplan = (Plan *) make_result(tlist,
(Node *) constant_quals,
subplan);
}
else
{
/*
* Replace the toplevel plan node's flattened target list with the
* targetlist given by my caller, so that expressions are
* evaluated.
*/
subplan->targetlist = tlist;
}
return subplan;
}
/*
* subplanner
*
* Subplanner creates an entire plan consisting of joins and scans
* for processing a single level of attributes.
*
* flat_tlist is the flattened target list
* tuple_fraction is the fraction of tuples we expect will be retrieved
*
* See query_planner() comments about the interpretation of tuple_fraction.
*
* Returns a subplan.
*/ */
static Plan *
subplanner(Query *root,
List *flat_tlist,
double tuple_fraction)
{
RelOptInfo *final_rel;
Plan *resultplan;
Path *cheapestpath;
Path *presortedpath;
/* init lists to empty */
root->base_rel_list = NIL; root->base_rel_list = NIL;
root->other_rel_list = NIL; root->other_rel_list = NIL;
root->join_rel_list = NIL; root->join_rel_list = NIL;
...@@ -197,8 +124,14 @@ subplanner(Query *root, ...@@ -197,8 +124,14 @@ subplanner(Query *root,
* clauses are added to appropriate lists belonging to the mentioned * clauses are added to appropriate lists belonging to the mentioned
* relations. We also build lists of equijoined keys for pathkey * relations. We also build lists of equijoined keys for pathkey
* construction. * construction.
*
* Note: all subplan nodes will have "flat" (var-only) tlists.
* This implies that all expression evaluations are done at the root of
* the plan tree. Once upon a time there was code to try to push
* expensive function calls down to lower plan nodes, but that's dead
* code and has been for a long time...
*/ */
build_base_rel_tlists(root, flat_tlist); build_base_rel_tlists(root, tlist);
(void) distribute_quals_to_rels(root, (Node *) root->jointree); (void) distribute_quals_to_rels(root, (Node *) root->jointree);
...@@ -220,31 +153,8 @@ subplanner(Query *root, ...@@ -220,31 +153,8 @@ subplanner(Query *root,
*/ */
final_rel = make_one_rel(root); final_rel = make_one_rel(root);
if (!final_rel) if (!final_rel || !final_rel->cheapest_total_path)
elog(ERROR, "subplanner: failed to construct a relation"); elog(ERROR, "query_planner: failed to construct a relation");
#ifdef NOT_USED /* fix xfunc */
/*
* Perform Predicate Migration on each path, to optimize and correctly
* assess the cost of each before choosing the cheapest one. -- JMH,
* 11/16/92
*
* Needn't do so if the top rel is pruneable: that means there's no
* expensive functions left to pull up. -- JMH, 11/22/92
*/
if (XfuncMode != XFUNC_OFF && XfuncMode != XFUNC_NOPM &&
XfuncMode != XFUNC_NOPULL && !final_rel->pruneable)
{
List *pathnode;
foreach(pathnode, final_rel->pathlist)
{
if (xfunc_do_predmig((Path *) lfirst(pathnode)))
set_cheapest(final_rel);
}
}
#endif
/* /*
* Now that we have an estimate of the final rel's size, we can * Now that we have an estimate of the final rel's size, we can
...@@ -255,75 +165,35 @@ subplanner(Query *root, ...@@ -255,75 +165,35 @@ subplanner(Query *root,
tuple_fraction /= final_rel->rows; tuple_fraction /= final_rel->rows;
/* /*
* Determine the cheapest path, independently of any ordering * Pick out the cheapest-total path and the cheapest presorted path
* considerations. We do, however, take into account whether the * for the requested pathkeys (if there is one). We can take the
* whole plan is expected to be evaluated or not. * tuple fraction into account when selecting the cheapest presorted
*/ * path, but not when selecting the cheapest-total path, since if we
if (tuple_fraction <= 0.0 || tuple_fraction >= 1.0) * have to sort then we'll have to fetch all the tuples. (But there's
cheapestpath = final_rel->cheapest_total_path; * a special case: if query_pathkeys is NIL, meaning order doesn't
else * matter, then the "cheapest presorted" path will be the cheapest
cheapestpath = * overall for the tuple fraction.)
get_cheapest_fractional_path_for_pathkeys(final_rel->pathlist,
NIL,
tuple_fraction);
Assert(cheapestpath != NULL);
/*
* Select the best path and create a subplan to execute it.
*
* If no special sort order is wanted, or if the cheapest path is already
* appropriately ordered, we use the cheapest path found above.
*/ */
if (root->query_pathkeys == NIL || *cheapest_path = final_rel->cheapest_total_path;
pathkeys_contained_in(root->query_pathkeys,
cheapestpath->pathkeys))
{
root->query_pathkeys = cheapestpath->pathkeys;
resultplan = create_plan(root, cheapestpath);
goto plan_built;
}
/* *sorted_path =
* Otherwise, look to see if we have an already-ordered path that is
* cheaper than doing an explicit sort on the cheapest-total-cost
* path.
*/
cheapestpath = final_rel->cheapest_total_path;
presortedpath =
get_cheapest_fractional_path_for_pathkeys(final_rel->pathlist, get_cheapest_fractional_path_for_pathkeys(final_rel->pathlist,
root->query_pathkeys, root->query_pathkeys,
tuple_fraction); tuple_fraction);
if (presortedpath)
{
Path sort_path; /* dummy for result of cost_sort */
cost_sort(&sort_path, root, root->query_pathkeys,
final_rel->rows, final_rel->width);
sort_path.startup_cost += cheapestpath->total_cost;
sort_path.total_cost += cheapestpath->total_cost;
if (compare_fractional_path_costs(presortedpath, &sort_path,
tuple_fraction) <= 0)
{
/* Presorted path is cheaper, use it */
root->query_pathkeys = presortedpath->pathkeys;
resultplan = create_plan(root, presortedpath);
goto plan_built;
}
/* otherwise, doing it the hard way is still cheaper */
}
/* /*
* Nothing for it but to sort the cheapest-total-cost path --- but we * If we have constant quals, add a toplevel Result step to process them.
* let the caller do that. grouping_planner has to be able to add a
* sort node anyway, so no need for extra code here. (Furthermore,
* the given pathkeys might involve something we can't compute here,
* such as an aggregate function...)
*/ */
root->query_pathkeys = cheapestpath->pathkeys; if (constant_quals)
resultplan = create_plan(root, cheapestpath); {
*cheapest_path = (Path *)
plan_built: create_result_path((*cheapest_path)->parent,
*cheapest_path,
return resultplan; constant_quals);
if (*sorted_path)
*sorted_path = (Path *)
create_result_path((*sorted_path)->parent,
*sorted_path,
constant_quals);
}
} }
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/planner.c,v 1.125 2002/09/24 18:38:23 tgl Exp $ * $Header: /cvsroot/pgsql/src/backend/optimizer/plan/planner.c,v 1.126 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -21,6 +21,8 @@ ...@@ -21,6 +21,8 @@
#include "nodes/print.h" #include "nodes/print.h"
#endif #endif
#include "optimizer/clauses.h" #include "optimizer/clauses.h"
#include "optimizer/cost.h"
#include "optimizer/pathnode.h"
#include "optimizer/paths.h" #include "optimizer/paths.h"
#include "optimizer/planmain.h" #include "optimizer/planmain.h"
#include "optimizer/planner.h" #include "optimizer/planner.h"
...@@ -53,10 +55,10 @@ static Plan *inheritance_planner(Query *parse, List *inheritlist); ...@@ -53,10 +55,10 @@ static Plan *inheritance_planner(Query *parse, List *inheritlist);
static Plan *grouping_planner(Query *parse, double tuple_fraction); static Plan *grouping_planner(Query *parse, double tuple_fraction);
static List *make_subplanTargetList(Query *parse, List *tlist, static List *make_subplanTargetList(Query *parse, List *tlist,
AttrNumber **groupColIdx); AttrNumber **groupColIdx);
static Plan *make_groupplan(Query *parse, static Plan *make_groupsortplan(Query *parse,
List *group_tlist, bool tuplePerGroup, List *groupClause,
List *groupClause, AttrNumber *grpColIdx, AttrNumber *grpColIdx,
bool is_presorted, Plan *subplan); Plan *subplan);
static List *postprocess_setop_tlist(List *new_tlist, List *orig_tlist); static List *postprocess_setop_tlist(List *new_tlist, List *orig_tlist);
...@@ -877,9 +879,7 @@ grouping_planner(Query *parse, double tuple_fraction) ...@@ -877,9 +879,7 @@ grouping_planner(Query *parse, double tuple_fraction)
List *tlist = parse->targetList; List *tlist = parse->targetList;
Plan *result_plan; Plan *result_plan;
List *current_pathkeys; List *current_pathkeys;
List *group_pathkeys;
List *sort_pathkeys; List *sort_pathkeys;
AttrNumber *groupColIdx = NULL;
if (parse->setOperations) if (parse->setOperations)
{ {
...@@ -917,17 +917,20 @@ grouping_planner(Query *parse, double tuple_fraction) ...@@ -917,17 +917,20 @@ grouping_planner(Query *parse, double tuple_fraction)
current_pathkeys = NIL; current_pathkeys = NIL;
/* /*
* Calculate pathkeys that represent grouping/ordering * Calculate pathkeys that represent ordering requirements
* requirements (grouping should always be null, but...)
*/ */
group_pathkeys = make_pathkeys_for_sortclauses(parse->groupClause,
tlist);
sort_pathkeys = make_pathkeys_for_sortclauses(parse->sortClause, sort_pathkeys = make_pathkeys_for_sortclauses(parse->sortClause,
tlist); tlist);
sort_pathkeys = canonicalize_pathkeys(parse, sort_pathkeys);
} }
else else
{ {
/* No set operations, do regular planning */
List *sub_tlist; List *sub_tlist;
List *group_pathkeys;
AttrNumber *groupColIdx = NULL;
Path *cheapest_path;
Path *sorted_path;
/* Preprocess targetlist in case we are inside an INSERT/UPDATE. */ /* Preprocess targetlist in case we are inside an INSERT/UPDATE. */
tlist = preprocess_targetlist(tlist, tlist = preprocess_targetlist(tlist,
...@@ -1192,17 +1195,12 @@ grouping_planner(Query *parse, double tuple_fraction) ...@@ -1192,17 +1195,12 @@ grouping_planner(Query *parse, double tuple_fraction)
tuple_fraction = 0.25; tuple_fraction = 0.25;
} }
/* Generate the basic plan for this Query */
result_plan = query_planner(parse,
sub_tlist,
tuple_fraction);
/* /*
* query_planner returns actual sort order (which is not * Generate the best unsorted and presorted paths for this Query
* necessarily what we requested) in query_pathkeys. * (but note there may not be any presorted path).
*/ */
current_pathkeys = parse->query_pathkeys; query_planner(parse, sub_tlist, tuple_fraction,
} &cheapest_path, &sorted_path);
/* /*
* We couldn't canonicalize group_pathkeys and sort_pathkeys before * We couldn't canonicalize group_pathkeys and sort_pathkeys before
...@@ -1212,79 +1210,147 @@ grouping_planner(Query *parse, double tuple_fraction) ...@@ -1212,79 +1210,147 @@ grouping_planner(Query *parse, double tuple_fraction)
sort_pathkeys = canonicalize_pathkeys(parse, sort_pathkeys); sort_pathkeys = canonicalize_pathkeys(parse, sort_pathkeys);
/* /*
* If we have a GROUP BY clause, insert a group node (plus the * Select the best path and create a plan to execute it.
* appropriate sort node, if necessary). *
* If no special sort order is wanted, or if the cheapest path is
* already appropriately ordered, use the cheapest path.
* Otherwise, look to see if we have an already-ordered path that is
* cheaper than doing an explicit sort on the cheapest-total-cost
* path.
*/ */
if (parse->groupClause) if (parse->query_pathkeys == NIL ||
pathkeys_contained_in(parse->query_pathkeys,
cheapest_path->pathkeys))
{ {
bool tuplePerGroup; result_plan = create_plan(parse, cheapest_path);
List *group_tlist; current_pathkeys = cheapest_path->pathkeys;
bool is_sorted; }
else if (sorted_path)
/* {
* Decide whether how many tuples per group the Group node needs Path sort_path; /* dummy for result of cost_sort */
* to return. (Needs only one tuple per group if no aggregate is
* present. Otherwise, need every tuple from the group to do the
* aggregation.) Note tuplePerGroup is named backwards :-(
*/
tuplePerGroup = parse->hasAggs;
cost_sort(&sort_path, parse, parse->query_pathkeys,
sorted_path->parent->rows, sorted_path->parent->width);
sort_path.startup_cost += cheapest_path->total_cost;
sort_path.total_cost += cheapest_path->total_cost;
if (compare_fractional_path_costs(sorted_path, &sort_path,
tuple_fraction) <= 0)
{
/* Presorted path is cheaper, use it */
result_plan = create_plan(parse, sorted_path);
current_pathkeys = sorted_path->pathkeys;
}
else
{
/* otherwise, doing it the hard way is still cheaper */
result_plan = create_plan(parse, cheapest_path);
current_pathkeys = cheapest_path->pathkeys;
}
}
else
{
/* /*
* If there are aggregates then the Group node should just return * No sorted path, so we must use the cheapest-total path.
* the same set of vars as the subplan did. If there are no aggs * The actual sort step will be generated below.
* then the Group node had better compute the final tlist.
*/ */
if (parse->hasAggs) result_plan = create_plan(parse, cheapest_path);
group_tlist = new_unsorted_tlist(result_plan->targetlist); current_pathkeys = cheapest_path->pathkeys;
else }
group_tlist = tlist;
/* /*
* Figure out whether the path result is already ordered the way * create_plan() returns a plan with just a "flat" tlist of required
* we need it --- if so, no need for an explicit sort step. * Vars. We want to insert the sub_tlist as the tlist of the top
* plan node. If the top-level plan node is one that cannot do
* expression evaluation, we must insert a Result node to project the
* desired tlist.
* Currently, the only plan node we might see here that falls into
* that category is Append.
*/ */
if (pathkeys_contained_in(group_pathkeys, current_pathkeys)) if (IsA(result_plan, Append))
{ {
is_sorted = true; /* no sort needed now */ result_plan = (Plan *) make_result(sub_tlist, NULL, result_plan);
/* current_pathkeys remains unchanged */
} }
else else
{ {
/* /*
* We will need to do an explicit sort by the GROUP BY clause. * Otherwise, just replace the flat tlist with the desired tlist.
* make_groupplan will do the work, but set current_pathkeys
* to indicate the resulting order.
*/ */
is_sorted = false; result_plan->targetlist = sub_tlist;
current_pathkeys = group_pathkeys;
}
result_plan = make_groupplan(parse,
group_tlist,
tuplePerGroup,
parse->groupClause,
groupColIdx,
is_sorted,
result_plan);
} }
/* /*
* If aggregate is present, insert the Agg node * If any aggregate is present, insert the Agg node, plus an explicit
* sort if necessary.
* *
* HAVING clause, if any, becomes qual of the Agg node * HAVING clause, if any, becomes qual of the Agg node
*/ */
if (parse->hasAggs) if (parse->hasAggs)
{ {
AggStrategy aggstrategy;
if (parse->groupClause)
{
aggstrategy = AGG_SORTED;
/*
* Add an explicit sort if we couldn't make the path come out
* the way the AGG node needs it.
*/
if (!pathkeys_contained_in(group_pathkeys, current_pathkeys))
{
result_plan = make_groupsortplan(parse,
parse->groupClause,
groupColIdx,
result_plan);
current_pathkeys = group_pathkeys;
}
}
else
aggstrategy = AGG_PLAIN;
result_plan = (Plan *) make_agg(tlist, result_plan = (Plan *) make_agg(tlist,
(List *) parse->havingQual, (List *) parse->havingQual,
aggstrategy,
length(parse->groupClause),
groupColIdx,
result_plan); result_plan);
/* Note: Agg does not affect any existing sort order of the tuples */ /*
* Note: plain or grouped Agg does not affect any existing
* sort order of the tuples
*/
} }
else else
{ {
/* If there are no Aggs, we shouldn't have any HAVING qual anymore */ /*
* If there are no Aggs, we shouldn't have any HAVING qual anymore
*/
Assert(parse->havingQual == NULL); Assert(parse->havingQual == NULL);
/*
* If we have a GROUP BY clause, insert a group node (plus the
* appropriate sort node, if necessary).
*/
if (parse->groupClause)
{
/*
* Add an explicit sort if we couldn't make the path come out
* the way the GROUP node needs it.
*/
if (!pathkeys_contained_in(group_pathkeys, current_pathkeys))
{
result_plan = make_groupsortplan(parse,
parse->groupClause,
groupColIdx,
result_plan);
current_pathkeys = group_pathkeys;
}
result_plan = (Plan *) make_group(tlist,
length(parse->groupClause),
groupColIdx,
result_plan);
}
} }
} /* end of if (setOperations) */
/* /*
* If we were not able to make the plan come out in the right order, * If we were not able to make the plan come out in the right order,
...@@ -1323,7 +1389,7 @@ grouping_planner(Query *parse, double tuple_fraction) ...@@ -1323,7 +1389,7 @@ grouping_planner(Query *parse, double tuple_fraction)
* make_subplanTargetList * make_subplanTargetList
* Generate appropriate target list when grouping is required. * Generate appropriate target list when grouping is required.
* *
* When grouping_planner inserts Aggregate and/or Group plan nodes above * When grouping_planner inserts Aggregate or Group plan nodes above
* the result of query_planner, we typically want to pass a different * the result of query_planner, we typically want to pass a different
* target list to query_planner than the outer plan nodes should have. * target list to query_planner than the outer plan nodes should have.
* This routine generates the correct target list for the subplan. * This routine generates the correct target list for the subplan.
...@@ -1433,32 +1499,22 @@ make_subplanTargetList(Query *parse, ...@@ -1433,32 +1499,22 @@ make_subplanTargetList(Query *parse,
} }
/* /*
* make_groupplan * make_groupsortplan
* Add a Group node for GROUP BY processing. * Add a Sort node to explicitly sort according to the GROUP BY clause.
* If we couldn't make the subplan produce presorted output for grouping, *
* first add an explicit Sort node. * Note: the Sort node always just takes a copy of the subplan's tlist
*/
static Plan *
make_groupplan(Query *parse,
List *group_tlist,
bool tuplePerGroup,
List *groupClause,
AttrNumber *grpColIdx,
bool is_presorted,
Plan *subplan)
{
int numCols = length(groupClause);
if (!is_presorted)
{
/*
* The Sort node always just takes a copy of the subplan's tlist
* plus ordering information. (This might seem inefficient if the * plus ordering information. (This might seem inefficient if the
* subplan contains complex GROUP BY expressions, but in fact Sort * subplan contains complex GROUP BY expressions, but in fact Sort
* does not evaluate its targetlist --- it only outputs the same * does not evaluate its targetlist --- it only outputs the same
* tuples in a new order. So the expressions we might be copying * tuples in a new order. So the expressions we might be copying
* are just dummies with no extra execution cost.) * are just dummies with no extra execution cost.)
*/ */
static Plan *
make_groupsortplan(Query *parse,
List *groupClause,
AttrNumber *grpColIdx,
Plan *subplan)
{
List *sort_tlist = new_unsorted_tlist(subplan->targetlist); List *sort_tlist = new_unsorted_tlist(subplan->targetlist);
int keyno = 0; int keyno = 0;
List *gl; List *gl;
...@@ -1484,11 +1540,7 @@ make_groupplan(Query *parse, ...@@ -1484,11 +1540,7 @@ make_groupplan(Query *parse,
Assert(keyno > 0); Assert(keyno > 0);
subplan = (Plan *) make_sort(parse, sort_tlist, subplan, keyno); return (Plan *) make_sort(parse, sort_tlist, subplan, keyno);
}
return (Plan *) make_group(group_tlist, tuplePerGroup, numCols,
grpColIdx, subplan);
} }
/* /*
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/pathnode.c,v 1.78 2002/06/20 20:29:31 momjian Exp $ * $Header: /cvsroot/pgsql/src/backend/optimizer/util/pathnode.c,v 1.79 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -405,7 +405,6 @@ create_tidscan_path(Query *root, RelOptInfo *rel, List *tideval) ...@@ -405,7 +405,6 @@ create_tidscan_path(Query *root, RelOptInfo *rel, List *tideval)
* create_append_path * create_append_path
* Creates a path corresponding to an Append plan, returning the * Creates a path corresponding to an Append plan, returning the
* pathnode. * pathnode.
*
*/ */
AppendPath * AppendPath *
create_append_path(RelOptInfo *rel, List *subpaths) create_append_path(RelOptInfo *rel, List *subpaths)
...@@ -433,6 +432,41 @@ create_append_path(RelOptInfo *rel, List *subpaths) ...@@ -433,6 +432,41 @@ create_append_path(RelOptInfo *rel, List *subpaths)
return pathnode; return pathnode;
} }
/*
* create_result_path
* Creates a path corresponding to a Result plan, returning the
* pathnode.
*/
ResultPath *
create_result_path(RelOptInfo *rel, Path *subpath, List *constantqual)
{
ResultPath *pathnode = makeNode(ResultPath);
pathnode->path.pathtype = T_Result;
pathnode->path.parent = rel; /* may be NULL */
if (subpath)
pathnode->path.pathkeys = subpath->pathkeys;
else
pathnode->path.pathkeys = NIL;
pathnode->subpath = subpath;
pathnode->constantqual = constantqual;
if (subpath)
{
pathnode->path.startup_cost = subpath->startup_cost;
pathnode->path.total_cost = subpath->total_cost;
}
else
{
pathnode->path.startup_cost = 0;
pathnode->path.total_cost = cpu_tuple_cost;
}
return pathnode;
}
/* /*
* create_subqueryscan_path * create_subqueryscan_path
* Creates a path corresponding to a sequential scan of a subquery, * Creates a path corresponding to a sequential scan of a subquery,
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: execnodes.h,v 1.75 2002/09/04 20:31:42 momjian Exp $ * $Id: execnodes.h,v 1.76 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -673,6 +673,8 @@ typedef struct AggState ...@@ -673,6 +673,8 @@ typedef struct AggState
CommonScanState csstate; /* its first field is NodeTag */ CommonScanState csstate; /* its first field is NodeTag */
List *aggs; /* all Aggref nodes in targetlist & quals */ List *aggs; /* all Aggref nodes in targetlist & quals */
int numaggs; /* length of list (could be zero!) */ int numaggs; /* length of list (could be zero!) */
FmgrInfo *eqfunctions; /* per-grouping-field equality fns */
HeapTuple grp_firstTuple; /* copy of first tuple of current group */
AggStatePerAgg peragg; /* per-Aggref working state */ AggStatePerAgg peragg; /* per-Aggref working state */
MemoryContext tup_cxt; /* context for per-output-tuple MemoryContext tup_cxt; /* context for per-output-tuple
* expressions */ * expressions */
...@@ -691,7 +693,7 @@ typedef struct GroupState ...@@ -691,7 +693,7 @@ typedef struct GroupState
FmgrInfo *eqfunctions; /* per-field lookup data for equality fns */ FmgrInfo *eqfunctions; /* per-field lookup data for equality fns */
bool grp_useFirstTuple; /* first tuple not processed yet */ bool grp_useFirstTuple; /* first tuple not processed yet */
bool grp_done; bool grp_done;
HeapTuple grp_firstTuple; HeapTuple grp_firstTuple; /* copy of first tuple of current group */
} GroupState; } GroupState;
/* ---------------- /* ----------------
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: nodes.h,v 1.120 2002/10/11 04:16:44 momjian Exp $ * $Id: nodes.h,v 1.121 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -82,6 +82,7 @@ typedef enum NodeTag ...@@ -82,6 +82,7 @@ typedef enum NodeTag
T_HashPath, T_HashPath,
T_TidPath, T_TidPath,
T_AppendPath, T_AppendPath,
T_ResultPath,
T_PathKeyItem, T_PathKeyItem,
T_RestrictInfo, T_RestrictInfo,
T_JoinInfo, T_JoinInfo,
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: parsenodes.h,v 1.209 2002/10/14 22:14:35 tgl Exp $ * $Id: parsenodes.h,v 1.210 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -101,7 +101,7 @@ typedef struct Query ...@@ -101,7 +101,7 @@ typedef struct Query
List *join_rel_list; /* list of join-relation RelOptInfos */ List *join_rel_list; /* list of join-relation RelOptInfos */
List *equi_key_list; /* list of lists of equijoined List *equi_key_list; /* list of lists of equijoined
* PathKeyItems */ * PathKeyItems */
List *query_pathkeys; /* pathkeys for query_planner()'s result */ List *query_pathkeys; /* desired pathkeys for query_planner() */
} Query; } Query;
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: plannodes.h,v 1.58 2002/09/04 20:31:44 momjian Exp $ * $Id: plannodes.h,v 1.59 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -140,17 +140,23 @@ typedef struct Plan ...@@ -140,17 +140,23 @@ typedef struct Plan
* =============== * ===============
*/ */
/* all plan nodes "derive" from the Plan structure by having the /*
Plan structure as the first field. This ensures that everything works * all plan nodes "derive" from the Plan structure by having the
when nodes are cast to Plan's. (node pointers are frequently cast to Plan* * Plan structure as the first field. This ensures that everything works
when passed around generically in the executor */ * when nodes are cast to Plan's. (node pointers are frequently cast to Plan*
* when passed around generically in the executor)
*/
/* ---------------- /* ----------------
* Result node - * Result node -
* If no outer plan, evaluate a variable-free targetlist. * If no outer plan, evaluate a variable-free targetlist.
* If outer plan, return tuples from outer plan that satisfy * If outer plan, return tuples from outer plan (after a level of
* given quals (we can also do a level of projection) * projection as shown by targetlist).
*
* If resconstantqual isn't NULL, it represents a one-time qualification
* test (i.e., one that doesn't depend on any variables from the outer plan,
* so needs to be evaluated only once).
* ---------------- * ----------------
*/ */
typedef struct Result typedef struct Result
...@@ -318,30 +324,45 @@ typedef struct HashJoin ...@@ -318,30 +324,45 @@ typedef struct HashJoin
/* --------------- /* ---------------
* aggregate node * aggregate node
*
* An Agg node implements plain or grouped aggregation. For grouped
* aggregation, we can work with presorted input or unsorted input;
* the latter strategy uses an internal hashtable.
*
* Notice the lack of any direct info about the aggregate functions to be
* computed. They are found by scanning the node's tlist and quals during
* executor startup. (It is possible that there are no aggregate functions;
* this could happen if they get optimized away by constant-folding, or if
* we are using the Agg node to implement hash-based grouping.)
* --------------- * ---------------
*/ */
typedef enum AggStrategy
{
AGG_PLAIN, /* simple agg across all input rows */
AGG_SORTED, /* grouped agg, input must be sorted */
AGG_HASHED /* grouped agg, use internal hashtable */
} AggStrategy;
typedef struct Agg typedef struct Agg
{ {
Plan plan; Plan plan;
AggStrategy aggstrategy;
int numCols; /* number of grouping columns */
AttrNumber *grpColIdx; /* their indexes in the target list */
AggState *aggstate; AggState *aggstate;
} Agg; } Agg;
/* --------------- /* ---------------
* group node - * group node -
* use for queries with GROUP BY specified. * Used for queries with GROUP BY (but no aggregates) specified.
* * The input must be presorted according to the grouping columns.
* If tuplePerGroup is true, one tuple (with group columns only) is
* returned for each group and NULL is returned when there are no more
* groups. Otherwise, all the tuples of a group are returned with a
* NULL returned at the end of each group. (see nodeGroup.c for details)
* --------------- * ---------------
*/ */
typedef struct Group typedef struct Group
{ {
Plan plan; Plan plan;
bool tuplePerGroup; /* what tuples to return (see above) */ int numCols; /* number of grouping columns */
int numCols; /* number of group columns */ AttrNumber *grpColIdx; /* their indexes in the target list */
AttrNumber *grpColIdx; /* indexes into the target list */
GroupState *grpstate; GroupState *grpstate;
} Group; } Group;
......
/*------------------------------------------------------------------------- /*-------------------------------------------------------------------------
* *
* relation.h * relation.h
* Definitions for internal planner nodes. * Definitions for planner's internal data structures.
* *
* *
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: relation.h,v 1.67 2002/09/04 20:31:44 momjian Exp $ * $Id: relation.h,v 1.68 2002/11/06 00:00:44 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -402,6 +402,19 @@ typedef struct AppendPath ...@@ -402,6 +402,19 @@ typedef struct AppendPath
List *subpaths; /* list of component Paths */ List *subpaths; /* list of component Paths */
} AppendPath; } AppendPath;
/*
* ResultPath represents use of a Result plan node, either to compute a
* variable-free targetlist or to gate execution of a subplan with a
* one-time (variable-free) qual condition. Note that in the former case
* path.parent will be NULL; in the latter case it is copied from the subpath.
*/
typedef struct ResultPath
{
Path path;
Path *subpath;
List *constantqual;
} ResultPath;
/* /*
* All join-type paths share these fields. * All join-type paths share these fields.
*/ */
......
...@@ -6,7 +6,7 @@ ...@@ -6,7 +6,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: geqo_misc.h,v 1.21 2002/09/04 20:31:45 momjian Exp $ * $Id: geqo_misc.h,v 1.22 2002/11/06 00:00:45 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -32,9 +32,6 @@ extern void print_pool(FILE *fp, Pool *pool, int start, int stop); ...@@ -32,9 +32,6 @@ extern void print_pool(FILE *fp, Pool *pool, int start, int stop);
extern void print_gen(FILE *fp, Pool *pool, int generation); extern void print_gen(FILE *fp, Pool *pool, int generation);
extern void print_edge_table(FILE *fp, Edge *edge_table, int num_gene); extern void print_edge_table(FILE *fp, Edge *edge_table, int num_gene);
extern void geqo_print_rel(Query *root, RelOptInfo *rel);
extern void geqo_print_path(Query *root, Path *path, int indent);
extern void geqo_print_joinclauses(Query *root, List *clauses);
#endif /* GEQO_DEBUG */ #endif /* GEQO_DEBUG */
#endif /* GEQO_MISC_H */ #endif /* GEQO_MISC_H */
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: pathnode.h,v 1.44 2002/06/20 20:29:51 momjian Exp $ * $Id: pathnode.h,v 1.45 2002/11/06 00:00:45 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -35,6 +35,8 @@ extern IndexPath *create_index_path(Query *root, RelOptInfo *rel, ...@@ -35,6 +35,8 @@ extern IndexPath *create_index_path(Query *root, RelOptInfo *rel,
extern TidPath *create_tidscan_path(Query *root, RelOptInfo *rel, extern TidPath *create_tidscan_path(Query *root, RelOptInfo *rel,
List *tideval); List *tideval);
extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths); extern AppendPath *create_append_path(RelOptInfo *rel, List *subpaths);
extern ResultPath *create_result_path(RelOptInfo *rel, Path *subpath,
List *constantqual);
extern Path *create_subqueryscan_path(RelOptInfo *rel); extern Path *create_subqueryscan_path(RelOptInfo *rel);
extern Path *create_functionscan_path(Query *root, RelOptInfo *rel); extern Path *create_functionscan_path(Query *root, RelOptInfo *rel);
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $Id: planmain.h,v 1.60 2002/09/04 20:31:45 momjian Exp $ * $Id: planmain.h,v 1.61 2002/11/06 00:00:45 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -20,7 +20,8 @@ ...@@ -20,7 +20,8 @@
/* /*
* prototypes for plan/planmain.c * prototypes for plan/planmain.c
*/ */
extern Plan *query_planner(Query *root, List *tlist, double tuple_fraction); extern void query_planner(Query *root, List *tlist, double tuple_fraction,
Path **cheapest_path, Path **sorted_path);
/* /*
* prototypes for plan/createplan.c * prototypes for plan/createplan.c
...@@ -33,9 +34,10 @@ extern Sort *make_sort(Query *root, List *tlist, ...@@ -33,9 +34,10 @@ extern Sort *make_sort(Query *root, List *tlist,
Plan *lefttree, int keycount); Plan *lefttree, int keycount);
extern Sort *make_sort_from_pathkeys(Query *root, List *tlist, extern Sort *make_sort_from_pathkeys(Query *root, List *tlist,
Plan *lefttree, List *pathkeys); Plan *lefttree, List *pathkeys);
extern Agg *make_agg(List *tlist, List *qual, Plan *lefttree); extern Agg *make_agg(List *tlist, List *qual, AggStrategy aggstrategy,
extern Group *make_group(List *tlist, bool tuplePerGroup, int ngrp, int ngrp, AttrNumber *grpColIdx, Plan *lefttree);
AttrNumber *grpColIdx, Plan *lefttree); extern Group *make_group(List *tlist, int ngrp, AttrNumber *grpColIdx,
Plan *lefttree);
extern Material *make_material(List *tlist, Plan *lefttree); extern Material *make_material(List *tlist, Plan *lefttree);
extern Unique *make_unique(List *tlist, Plan *lefttree, List *distinctList); extern Unique *make_unique(List *tlist, Plan *lefttree, List *distinctList);
extern Limit *make_limit(List *tlist, Plan *lefttree, extern Limit *make_limit(List *tlist, Plan *lefttree,
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment