Tweak planner and executor to avoid doing ExecProject() in table scan

nodes where it's not really necessary.  In many cases where the scan node
is not the topmost plan node (eg, joins, aggregation), it's possible to
just return the table tuple directly instead of generating an intermediate
projection tuple.  In preliminary testing, this reduced the CPU time
needed for 'SELECT COUNT(*) FROM foo' by about 10%.
This commit is contained in:
Tom Lane 2003-02-03 15:07:08 +00:00
parent 0d3e36b668
commit 4cff59d8d5
12 changed files with 476 additions and 230 deletions

View File

@ -26,7 +26,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/execMain.c,v 1.199 2003/01/23 05:10:37 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/execMain.c,v 1.200 2003/02/03 15:07:06 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -612,9 +612,11 @@ InitPlan(QueryDesc *queryDesc)
tupType = ExecGetTupType(planstate);
/*
* Initialize the junk filter if needed. SELECT and INSERT queries
* need a filter if there are any junk attrs in the tlist. UPDATE and
* DELETE always need one, since there's always a junk 'ctid'
* Initialize the junk filter if needed. SELECT and INSERT queries need a
* filter if there are any junk attrs in the tlist. INSERT and SELECT
* INTO also need a filter if the top plan node is a scan node that's not
* doing projection (else we'll be scribbling on the scan tuple!) UPDATE
* and DELETE always need a filter, since there's always a junk 'ctid'
* attribute present --- no need to look first.
*/
{
@ -635,6 +637,19 @@ InitPlan(QueryDesc *queryDesc)
break;
}
}
if (!junk_filter_needed &&
(operation == CMD_INSERT || do_select_into))
{
if (IsA(planstate, SeqScanState) ||
IsA(planstate, IndexScanState) ||
IsA(planstate, TidScanState) ||
IsA(planstate, SubqueryScanState) ||
IsA(planstate, FunctionScanState))
{
if (planstate->ps_ProjInfo == NULL)
junk_filter_needed = true;
}
}
break;
case CMD_UPDATE:
case CMD_DELETE:

View File

@ -12,19 +12,20 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/execScan.c,v 1.22 2002/12/05 15:50:32 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/execScan.c,v 1.23 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include <sys/file.h>
#include "executor/executor.h"
#include "miscadmin.h"
#include "utils/memutils.h"
static bool tlist_matches_tupdesc(List *tlist, Index varno, TupleDesc tupdesc);
/* ----------------------------------------------------------------
* ExecScan
*
@ -50,6 +51,7 @@ ExecScan(ScanState *node,
EState *estate;
ExprContext *econtext;
List *qual;
ProjectionInfo *projInfo;
ExprDoneCond isDone;
TupleTableSlot *resultSlot;
@ -59,6 +61,7 @@ ExecScan(ScanState *node,
estate = node->ps.state;
econtext = node->ps.ps_ExprContext;
qual = node->ps.qual;
projInfo = node->ps.ps_ProjInfo;
/*
* Check to see if we're still projecting out tuples from a previous
@ -67,7 +70,8 @@ ExecScan(ScanState *node,
*/
if (node->ps.ps_TupFromTlist)
{
resultSlot = ExecProject(node->ps.ps_ProjInfo, &isDone);
Assert(projInfo); /* can't get here if not projecting */
resultSlot = ExecProject(projInfo, &isDone);
if (isDone == ExprMultipleResult)
return resultSlot;
/* Done with that source tuple... */
@ -101,10 +105,13 @@ ExecScan(ScanState *node,
*/
if (TupIsNull(slot))
{
return ExecStoreTuple(NULL,
node->ps.ps_ProjInfo->pi_slot,
InvalidBuffer,
true);
if (projInfo)
return ExecStoreTuple(NULL,
projInfo->pi_slot,
InvalidBuffer,
true);
else
return slot;
}
/*
@ -123,16 +130,27 @@ ExecScan(ScanState *node,
{
/*
* Found a satisfactory scan tuple.
*
* Form a projection tuple, store it in the result tuple slot and
* return it --- unless we find we can project no tuples from
* this scan tuple, in which case continue scan.
*/
resultSlot = ExecProject(node->ps.ps_ProjInfo, &isDone);
if (isDone != ExprEndResult)
if (projInfo)
{
node->ps.ps_TupFromTlist = (isDone == ExprMultipleResult);
return resultSlot;
/*
* Form a projection tuple, store it in the result tuple slot
* and return it --- unless we find we can project no tuples
* from this scan tuple, in which case continue scan.
*/
resultSlot = ExecProject(projInfo, &isDone);
if (isDone != ExprEndResult)
{
node->ps.ps_TupFromTlist = (isDone == ExprMultipleResult);
return resultSlot;
}
}
else
{
/*
* Here, we aren't projecting, so just return scan tuple.
*/
return slot;
}
}
@ -142,3 +160,61 @@ ExecScan(ScanState *node,
ResetExprContext(econtext);
}
}
/*
* ExecAssignScanProjectionInfo
* Set up projection info for a scan node, if necessary.
*
* We can avoid a projection step if the requested tlist exactly matches
* the underlying tuple type. If so, we just set ps_ProjInfo to NULL.
* Note that this case occurs not only for simple "SELECT * FROM ...", but
* also in most cases where there are joins or other processing nodes above
* the scan node, because the planner will preferentially generate a matching
* tlist.
*
* ExecAssignScanType must have been called already.
*/
void
ExecAssignScanProjectionInfo(ScanState *node)
{
Scan *scan = (Scan *) node->ps.plan;
if (tlist_matches_tupdesc(scan->plan.targetlist,
scan->scanrelid,
node->ss_ScanTupleSlot->ttc_tupleDescriptor))
node->ps.ps_ProjInfo = NULL;
else
ExecAssignProjectionInfo(&node->ps);
}
static bool
tlist_matches_tupdesc(List *tlist, Index varno, TupleDesc tupdesc)
{
int numattrs = tupdesc->natts;
int attrno;
for (attrno = 1; attrno <= numattrs; attrno++)
{
Form_pg_attribute att_tup = tupdesc->attrs[attrno - 1];
Var *var;
if (tlist == NIL)
return false; /* tlist too short */
var = (Var *) ((TargetEntry *) lfirst(tlist))->expr;
if (!var || !IsA(var, Var))
return false; /* tlist item not a Var */
Assert(var->varno == varno);
if (var->varattno != attrno)
return false; /* out of order */
Assert(var->vartype == att_tup->atttypid);
Assert(var->vartypmod == att_tup->atttypmod);
Assert(var->varlevelsup == 0);
tlist = lnext(tlist);
}
if (tlist)
return false; /* tlist too long */
return true;
}

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeIndexscan.c,v 1.77 2003/01/12 22:01:38 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeIndexscan.c,v 1.78 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -582,12 +582,6 @@ ExecInitIndexScan(IndexScan *node, EState *estate)
ExecInitResultTupleSlot(estate, &indexstate->ss.ps);
ExecInitScanTupleSlot(estate, &indexstate->ss);
/*
* Initialize result tuple type and projection info.
*/
ExecAssignResultTypeFromTL(&indexstate->ss.ps);
ExecAssignProjectionInfo(&indexstate->ss.ps);
/*
* Initialize index-specific scan state
*/
@ -917,6 +911,12 @@ ExecInitIndexScan(IndexScan *node, EState *estate)
indexstate->iss_RelationDescs = indexDescs;
indexstate->iss_ScanDescs = scanDescs;
/*
* Initialize result tuple type and projection info.
*/
ExecAssignResultTypeFromTL(&indexstate->ss.ps);
ExecAssignScanProjectionInfo(&indexstate->ss);
/*
* all done.
*/

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSeqscan.c,v 1.42 2003/01/12 22:01:38 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeSeqscan.c,v 1.43 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -232,7 +232,7 @@ ExecInitSeqScan(SeqScan *node, EState *estate)
* Initialize result tuple type and projection info.
*/
ExecAssignResultTypeFromTL(&scanstate->ps);
ExecAssignProjectionInfo(&scanstate->ps);
ExecAssignScanProjectionInfo(scanstate);
return scanstate;
}

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/executor/nodeTidscan.c,v 1.31 2003/01/12 22:01:38 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/executor/nodeTidscan.c,v 1.32 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -383,12 +383,6 @@ ExecInitTidScan(TidScan *node, EState *estate)
ExecInitResultTupleSlot(estate, &tidstate->ss.ps);
ExecInitScanTupleSlot(estate, &tidstate->ss);
/*
* Initialize result tuple type and projection info.
*/
ExecAssignResultTypeFromTL(&tidstate->ss.ps);
ExecAssignProjectionInfo(&tidstate->ss.ps);
/*
* get the tid node information
*/
@ -438,6 +432,12 @@ ExecInitTidScan(TidScan *node, EState *estate)
*/
tidstate->ss.ps.chgParam = execParam;
/*
* Initialize result tuple type and projection info.
*/
ExecAssignResultTypeFromTL(&tidstate->ss.ps);
ExecAssignScanProjectionInfo(&tidstate->ss);
/*
* all done.
*/

View File

@ -10,7 +10,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/createplan.c,v 1.133 2003/01/22 00:07:00 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/createplan.c,v 1.134 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -34,6 +34,7 @@
static Scan *create_scan_plan(Query *root, Path *best_path);
static bool use_physical_tlist(RelOptInfo *rel);
static Join *create_join_plan(Query *root, JoinPath *best_path);
static Append *create_append_plan(Query *root, AppendPath *best_path);
static Result *create_result_plan(Query *root, ResultPath *best_path);
@ -185,15 +186,41 @@ create_plan(Query *root, Path *best_path)
static Scan *
create_scan_plan(Query *root, Path *best_path)
{
Scan *plan;
List *tlist = best_path->parent->targetlist;
RelOptInfo *rel = best_path->parent;
List *tlist;
List *scan_clauses;
Scan *plan;
/*
* For table scans, rather than using the relation targetlist (which is
* only those Vars actually needed by the query), we prefer to generate a
* tlist containing all Vars in order. This will allow the executor to
* optimize away projection of the table tuples, if possible. (Note that
* planner.c may replace the tlist we generate here, forcing projection to
* occur.)
*/
if (use_physical_tlist(rel))
{
int resdomno = 1;
List *v;
tlist = NIL;
foreach(v, rel->varlist)
{
Var *var = (Var *) lfirst(v);
tlist = lappend(tlist, create_tl_element(var, resdomno));
resdomno++;
}
}
else
tlist = rel->targetlist;
/*
* Extract the relevant restriction clauses from the parent relation;
* the executor must apply all these restrictions during the scan.
*/
scan_clauses = get_actual_clauses(best_path->parent->baserestrictinfo);
scan_clauses = get_actual_clauses(rel->baserestrictinfo);
/* Sort clauses into best execution order */
scan_clauses = order_qual_clauses(root, scan_clauses);
@ -241,6 +268,47 @@ create_scan_plan(Query *root, Path *best_path)
return plan;
}
/*
* use_physical_tlist
* Decide whether to use a tlist matching relation structure,
* rather than only those Vars actually referenced.
*/
static bool
use_physical_tlist(RelOptInfo *rel)
{
List *t;
/*
* Currently, can't do this for subquery or function scans. (This
* is mainly because we don't set up the necessary info when creating
* their RelOptInfo nodes.)
*/
if (rel->rtekind != RTE_RELATION)
return false;
/*
* Can't do it with inheritance cases either (mainly because Append
* doesn't project).
*/
if (rel->reloptkind != RELOPT_BASEREL)
return false;
/*
* Can't do it if any system columns are requested, either. (This could
* possibly be fixed but would take some fragile assumptions in setrefs.c,
* I think.)
*/
foreach(t, rel->targetlist)
{
TargetEntry *tle = (TargetEntry *) lfirst(t);
Var *var = (Var *) tle->expr;
if (!var || !IsA(var, Var))
return false; /* probably can't happen */
if (var->varattno <= 0)
return false; /* system column! */
}
return true;
}
/*
* create_join_plan
* Create a join plan for 'best_path' and (recursively) plans for its
@ -399,7 +467,7 @@ create_material_plan(Query *root, MaterialPath *best_path)
subplan = create_plan(root, best_path->subpath);
plan = make_material(best_path->path.parent->targetlist, subplan);
plan = make_material(subplan->targetlist, subplan);
copy_path_costsize(&plan->plan, (Path *) best_path);

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/planner.c,v 1.142 2003/01/25 23:10:27 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/plan/planner.c,v 1.143 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -55,7 +55,11 @@ static Plan *inheritance_planner(Query *parse, List *inheritlist);
static Plan *grouping_planner(Query *parse, double tuple_fraction);
static bool hash_safe_grouping(Query *parse);
static List *make_subplanTargetList(Query *parse, List *tlist,
AttrNumber **groupColIdx);
AttrNumber **groupColIdx, bool *need_tlist_eval);
static void locate_grouping_columns(Query *parse,
List *tlist,
List *sub_tlist,
AttrNumber *groupColIdx);
static Plan *make_groupsortplan(Query *parse,
List *groupClause,
AttrNumber *grpColIdx,
@ -530,6 +534,7 @@ grouping_planner(Query *parse, double tuple_fraction)
List *sub_tlist;
List *group_pathkeys;
AttrNumber *groupColIdx = NULL;
bool need_tlist_eval = true;
QualCost tlist_cost;
double sub_tuple_fraction;
Path *cheapest_path;
@ -602,7 +607,8 @@ grouping_planner(Query *parse, double tuple_fraction)
* Generate appropriate target list for subplan; may be different
* from tlist if grouping or aggregation is needed.
*/
sub_tlist = make_subplanTargetList(parse, tlist, &groupColIdx);
sub_tlist = make_subplanTargetList(parse, tlist,
&groupColIdx, &need_tlist_eval);
/*
* Calculate pathkeys that represent grouping/ordering
@ -1003,45 +1009,65 @@ grouping_planner(Query *parse, double tuple_fraction)
/*
* create_plan() returns a plan with just a "flat" tlist of required
* 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.
* Vars. Usually we need to insert the sub_tlist as the tlist of the
* top plan node. However, we can skip that if we determined that
* whatever query_planner chose to return will be good enough.
*/
if (IsA(result_plan, Append))
if (need_tlist_eval)
{
result_plan = (Plan *) make_result(sub_tlist, NULL, result_plan);
/*
* 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 (IsA(result_plan, Append))
{
result_plan = (Plan *) make_result(sub_tlist, NULL,
result_plan);
}
else
{
/*
* Otherwise, just replace the subplan's flat tlist with
* the desired tlist.
*/
result_plan->targetlist = sub_tlist;
}
/*
* Also, account for the cost of evaluation of the sub_tlist.
*
* Up to now, we have only been dealing with "flat" tlists,
* containing just Vars. So their evaluation cost is zero
* according to the model used by cost_qual_eval() (or if you
* prefer, the cost is factored into cpu_tuple_cost). Thus we can
* avoid accounting for tlist cost throughout query_planner() and
* subroutines. But now we've inserted a tlist that might contain
* actual operators, sub-selects, etc --- so we'd better account
* for its cost.
*
* Below this point, any tlist eval cost for added-on nodes should
* be accounted for as we create those nodes. Presently, of the
* node types we can add on, only Agg and Group project new tlists
* (the rest just copy their input tuples) --- so make_agg() and
* make_group() are responsible for computing the added cost.
*/
cost_qual_eval(&tlist_cost, sub_tlist);
result_plan->startup_cost += tlist_cost.startup;
result_plan->total_cost += tlist_cost.startup +
tlist_cost.per_tuple * result_plan->plan_rows;
}
else
{
/*
* Otherwise, just replace the flat tlist with the desired tlist.
* Since we're using query_planner's tlist and not the one
* make_subplanTargetList calculated, we have to refigure
* any grouping-column indexes make_subplanTargetList computed.
*/
result_plan->targetlist = sub_tlist;
locate_grouping_columns(parse, tlist, result_plan->targetlist,
groupColIdx);
}
/*
* Also, account for the cost of evaluation of the sub_tlist.
*
* Up to now, we have only been dealing with "flat" tlists, containing
* just Vars. So their evaluation cost is zero according to the
* model used by cost_qual_eval() (or if you prefer, the cost is
* factored into cpu_tuple_cost). Thus we can avoid accounting for
* tlist cost throughout query_planner() and subroutines.
* But now we've inserted a tlist that might contain actual operators,
* sub-selects, etc --- so we'd better account for its cost.
*
* Below this point, any tlist eval cost for added-on nodes should
* be accounted for as we create those nodes. Presently, of the
* node types we can add on, only Agg and Group project new tlists
* (the rest just copy their input tuples) --- so make_agg() and
* make_group() are responsible for computing the added cost.
*/
cost_qual_eval(&tlist_cost, sub_tlist);
result_plan->startup_cost += tlist_cost.startup;
result_plan->total_cost += tlist_cost.startup +
tlist_cost.per_tuple * result_plan->plan_rows;
/*
* Insert AGG or GROUP node if needed, plus an explicit sort step
@ -1245,10 +1271,17 @@ hash_safe_grouping(Query *parse)
* the extra computation to recompute a+b at the outer level; see
* replace_vars_with_subplan_refs() in setrefs.c.)
*
* If we are grouping or aggregating, *and* there are no non-Var grouping
* expressions, then the returned tlist is effectively dummy; we do not
* need to force it to be evaluated, because all the Vars it contains
* should be present in the output of query_planner anyway.
*
* 'parse' is the query being processed.
* 'tlist' is the query's target list.
* 'groupColIdx' receives an array of column numbers for the GROUP BY
* expressions (if there are any) in the subplan's target list.
* expressions (if there are any) in the subplan's target list.
* 'need_tlist_eval' is set true if we really need to evaluate the
* result tlist.
*
* The result is the targetlist to be passed to the subplan.
*---------------
@ -1256,7 +1289,8 @@ hash_safe_grouping(Query *parse)
static List *
make_subplanTargetList(Query *parse,
List *tlist,
AttrNumber **groupColIdx)
AttrNumber **groupColIdx,
bool *need_tlist_eval)
{
List *sub_tlist;
List *extravars;
@ -1269,7 +1303,10 @@ make_subplanTargetList(Query *parse,
* query_planner should receive the unmodified target list.
*/
if (!parse->hasAggs && !parse->groupClause && !parse->havingQual)
{
*need_tlist_eval = true;
return tlist;
}
/*
* Otherwise, start with a "flattened" tlist (having just the vars
@ -1280,6 +1317,7 @@ make_subplanTargetList(Query *parse,
extravars = pull_var_clause(parse->havingQual, false);
sub_tlist = add_to_flat_tlist(sub_tlist, extravars);
freeList(extravars);
*need_tlist_eval = false; /* only eval if not flat tlist */
/*
* If grouping, create sub_tlist entries for all GROUP BY expressions
@ -1320,6 +1358,7 @@ make_subplanTargetList(Query *parse,
false),
(Expr *) groupexpr);
sub_tlist = lappend(sub_tlist, te);
*need_tlist_eval = true; /* it's not flat anymore */
}
/* and save its resno */
@ -1330,6 +1369,53 @@ make_subplanTargetList(Query *parse,
return sub_tlist;
}
/*
* locate_grouping_columns
* Locate grouping columns in the tlist chosen by query_planner.
*
* This is only needed if we don't use the sub_tlist chosen by
* make_subplanTargetList. We have to forget the column indexes found
* by that routine and re-locate the grouping vars in the real sub_tlist.
*/
static void
locate_grouping_columns(Query *parse,
List *tlist,
List *sub_tlist,
AttrNumber *groupColIdx)
{
int keyno = 0;
List *gl;
/*
* No work unless grouping.
*/
if (!parse->groupClause)
{
Assert(groupColIdx == NULL);
return;
}
Assert(groupColIdx != NULL);
foreach(gl, parse->groupClause)
{
GroupClause *grpcl = (GroupClause *) lfirst(gl);
Node *groupexpr = get_sortgroupclause_expr(grpcl, tlist);
TargetEntry *te = NULL;
List *sl;
foreach(sl, sub_tlist)
{
te = (TargetEntry *) lfirst(sl);
if (equal(groupexpr, te->expr))
break;
}
if (!sl)
elog(ERROR, "locate_grouping_columns: failed");
groupColIdx[keyno++] = te->resdom->resno;
}
}
/*
* make_groupsortplan
* Add a Sort node to explicitly sort according to the GROUP BY clause.

View File

@ -9,7 +9,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/plancat.c,v 1.76 2003/01/28 22:13:35 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/plancat.c,v 1.77 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -23,6 +23,7 @@
#include "catalog/pg_amop.h"
#include "catalog/pg_inherits.h"
#include "catalog/pg_index.h"
#include "nodes/makefuncs.h"
#include "optimizer/clauses.h"
#include "optimizer/plancat.h"
#include "parser/parsetree.h"
@ -38,155 +39,160 @@
/*
* get_relation_info -
* Retrieves catalog information for a given relation.
* Given the Oid of the relation, return the following info:
* whether the relation has secondary indices
* number of pages
* number of tuples
*
* Given the Oid of the relation, return the following info into fields
* of the RelOptInfo struct:
*
* varlist list of physical columns (expressed as Vars)
* indexlist list of IndexOptInfos for relation's indexes
* pages number of pages
* tuples number of tuples
*/
void
get_relation_info(Oid relationObjectId,
bool *hasindex, long *pages, double *tuples)
get_relation_info(Oid relationObjectId, RelOptInfo *rel)
{
HeapTuple relationTuple;
Form_pg_class relation;
relationTuple = SearchSysCache(RELOID,
ObjectIdGetDatum(relationObjectId),
0, 0, 0);
if (!HeapTupleIsValid(relationTuple))
elog(ERROR, "get_relation_info: Relation %u not found",
relationObjectId);
relation = (Form_pg_class) GETSTRUCT(relationTuple);
if (IsIgnoringSystemIndexes() && IsSystemClass(relation))
*hasindex = false;
else
*hasindex = relation->relhasindex;
*pages = relation->relpages;
*tuples = relation->reltuples;
ReleaseSysCache(relationTuple);
}
/*
* find_secondary_indexes
* Creates a list of IndexOptInfo nodes containing information for each
* secondary index defined on the specified relation.
*
* 'relationObjectId' is the OID of the relation for which indices are wanted
*
* Returns a list of new IndexOptInfo nodes.
*/
List *
find_secondary_indexes(Oid relationObjectId)
{
List *indexinfos = NIL;
List *indexoidlist,
*indexoidscan;
Relation relation;
Index varno = lfirsti(rel->relids);
bool hasindex;
List *varlist = NIL;
List *indexinfos = NIL;
int attrno,
numattrs;
relation = heap_open(relationObjectId, AccessShareLock);
/*
* We used to scan pg_index directly, but now the relcache offers a
* cached list of OID indexes for each relation. So, get that list
* and then use the syscache to obtain pg_index entries.
* Make list of physical Vars. Note we do NOT ignore dropped columns;
* the intent is to model the physical tuples of the relation.
*/
relation = heap_open(relationObjectId, AccessShareLock);
indexoidlist = RelationGetIndexList(relation);
numattrs = RelationGetNumberOfAttributes(relation);
foreach(indexoidscan, indexoidlist)
for (attrno = 1; attrno <= numattrs; attrno++)
{
Oid indexoid = lfirsti(indexoidscan);
Relation indexRelation;
Form_pg_index index;
IndexOptInfo *info;
int i;
int16 amorderstrategy;
Form_pg_attribute att_tup = relation->rd_att->attrs[attrno - 1];
/* Extract info from the relation descriptor for the index */
indexRelation = index_open(indexoid);
info = makeNode(IndexOptInfo);
/*
* Need to make these arrays large enough to be sure there is room
* for a terminating 0 at the end of each one.
*/
info->classlist = (Oid *) palloc(sizeof(Oid) * (INDEX_MAX_KEYS + 1));
info->indexkeys = (int *) palloc(sizeof(int) * (INDEX_MAX_KEYS + 1));
info->ordering = (Oid *) palloc(sizeof(Oid) * (INDEX_MAX_KEYS + 1));
/* Extract info from the pg_index tuple */
index = indexRelation->rd_index;
info->indexoid = index->indexrelid;
info->indproc = index->indproc; /* functional index ?? */
if (VARSIZE(&index->indpred) > VARHDRSZ) /* partial index ?? */
{
char *predString;
predString = DatumGetCString(DirectFunctionCall1(textout,
PointerGetDatum(&index->indpred)));
info->indpred = (List *) stringToNode(predString);
pfree(predString);
}
else
info->indpred = NIL;
info->unique = index->indisunique;
for (i = 0; i < INDEX_MAX_KEYS; i++)
{
if (index->indclass[i] == (Oid) 0)
break;
info->classlist[i] = index->indclass[i];
}
info->classlist[i] = (Oid) 0;
info->ncolumns = i;
for (i = 0; i < INDEX_MAX_KEYS; i++)
{
if (index->indkey[i] == 0)
break;
info->indexkeys[i] = index->indkey[i];
}
info->indexkeys[i] = 0;
info->nkeys = i;
info->relam = indexRelation->rd_rel->relam;
info->pages = indexRelation->rd_rel->relpages;
info->tuples = indexRelation->rd_rel->reltuples;
info->amcostestimate = index_cost_estimator(indexRelation);
amorderstrategy = indexRelation->rd_am->amorderstrategy;
/*
* Fetch the ordering operators associated with the index, if any.
*/
MemSet(info->ordering, 0, sizeof(Oid) * (INDEX_MAX_KEYS + 1));
if (amorderstrategy != 0)
{
int oprindex = amorderstrategy - 1;
for (i = 0; i < info->ncolumns; i++)
{
info->ordering[i] = indexRelation->rd_operator[oprindex];
oprindex += indexRelation->rd_am->amstrategies;
}
}
/* initialize cached join info to empty */
info->outer_relids = NIL;
info->inner_paths = NIL;
index_close(indexRelation);
indexinfos = lcons(info, indexinfos);
varlist = lappend(varlist,
makeVar(varno,
attrno,
att_tup->atttypid,
att_tup->atttypmod,
0));
}
freeList(indexoidlist);
rel->varlist = varlist;
/*
* Make list of indexes. Ignore indexes on system catalogs if told to.
*/
if (IsIgnoringSystemIndexes() && IsSystemClass(relation->rd_rel))
hasindex = false;
else
hasindex = relation->rd_rel->relhasindex;
if (hasindex)
{
List *indexoidlist,
*indexoidscan;
indexoidlist = RelationGetIndexList(relation);
foreach(indexoidscan, indexoidlist)
{
Oid indexoid = lfirsti(indexoidscan);
Relation indexRelation;
Form_pg_index index;
IndexOptInfo *info;
int i;
int16 amorderstrategy;
/* Extract info from the relation descriptor for the index */
indexRelation = index_open(indexoid);
info = makeNode(IndexOptInfo);
/*
* Need to make these arrays large enough to be sure there is room
* for a terminating 0 at the end of each one.
*/
info->classlist = (Oid *) palloc(sizeof(Oid) * (INDEX_MAX_KEYS + 1));
info->indexkeys = (int *) palloc(sizeof(int) * (INDEX_MAX_KEYS + 1));
info->ordering = (Oid *) palloc(sizeof(Oid) * (INDEX_MAX_KEYS + 1));
/* Extract info from the pg_index tuple */
index = indexRelation->rd_index;
info->indexoid = index->indexrelid;
info->indproc = index->indproc; /* functional index ?? */
if (VARSIZE(&index->indpred) > VARHDRSZ) /* partial index ?? */
{
char *predString;
predString = DatumGetCString(DirectFunctionCall1(textout,
PointerGetDatum(&index->indpred)));
info->indpred = (List *) stringToNode(predString);
pfree(predString);
}
else
info->indpred = NIL;
info->unique = index->indisunique;
for (i = 0; i < INDEX_MAX_KEYS; i++)
{
if (index->indclass[i] == (Oid) 0)
break;
info->classlist[i] = index->indclass[i];
}
info->classlist[i] = (Oid) 0;
info->ncolumns = i;
for (i = 0; i < INDEX_MAX_KEYS; i++)
{
if (index->indkey[i] == 0)
break;
info->indexkeys[i] = index->indkey[i];
}
info->indexkeys[i] = 0;
info->nkeys = i;
info->relam = indexRelation->rd_rel->relam;
info->pages = indexRelation->rd_rel->relpages;
info->tuples = indexRelation->rd_rel->reltuples;
info->amcostestimate = index_cost_estimator(indexRelation);
amorderstrategy = indexRelation->rd_am->amorderstrategy;
/*
* Fetch the ordering operators associated with the index, if any.
*/
MemSet(info->ordering, 0, sizeof(Oid) * (INDEX_MAX_KEYS + 1));
if (amorderstrategy != 0)
{
int oprindex = amorderstrategy - 1;
for (i = 0; i < info->ncolumns; i++)
{
info->ordering[i] = indexRelation->rd_operator[oprindex];
oprindex += indexRelation->rd_am->amstrategies;
}
}
/* initialize cached join info to empty */
info->outer_relids = NIL;
info->inner_paths = NIL;
index_close(indexRelation);
indexinfos = lcons(info, indexinfos);
}
freeList(indexoidlist);
}
rel->indexlist = indexinfos;
rel->pages = relation->rd_rel->relpages;
rel->tuples = relation->rd_rel->reltuples;
/* XXX keep the lock here? */
heap_close(relation, AccessShareLock);
return indexinfos;
}
/*

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/relnode.c,v 1.45 2003/01/24 03:58:43 tgl Exp $
* $Header: /cvsroot/pgsql/src/backend/optimizer/util/relnode.c,v 1.46 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -143,6 +143,7 @@ make_base_rel(Query *root, int relid)
rel->cheapest_unique_path = NULL;
rel->pruneable = true;
rel->rtekind = rte->rtekind;
rel->varlist = NIL;
rel->indexlist = NIL;
rel->pages = 0;
rel->tuples = 0;
@ -159,16 +160,9 @@ make_base_rel(Query *root, int relid)
switch (rte->rtekind)
{
case RTE_RELATION:
{
/* Table --- retrieve statistics from the system catalogs */
bool indexed;
get_relation_info(rte->relid,
&indexed, &rel->pages, &rel->tuples);
if (indexed)
rel->indexlist = find_secondary_indexes(rte->relid);
break;
}
/* Table --- retrieve statistics from the system catalogs */
get_relation_info(rte->relid, rel);
break;
case RTE_SUBQUERY:
case RTE_FUNCTION:
/* Subquery or function --- nothing to do here */
@ -304,6 +298,7 @@ build_join_rel(Query *root,
joinrel->cheapest_unique_path = NULL;
joinrel->pruneable = true;
joinrel->rtekind = RTE_JOIN;
joinrel->varlist = NIL;
joinrel->indexlist = NIL;
joinrel->pages = 0;
joinrel->tuples = 0;

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: executor.h,v 1.87 2003/01/12 04:03:34 tgl Exp $
* $Id: executor.h,v 1.88 2003/02/03 15:07:07 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -137,6 +137,7 @@ extern TupleTableSlot *ExecProject(ProjectionInfo *projInfo,
typedef TupleTableSlot *(*ExecScanAccessMtd) (ScanState *node);
extern TupleTableSlot *ExecScan(ScanState *node, ExecScanAccessMtd accessMtd);
extern void ExecAssignScanProjectionInfo(ScanState *node);
/*
* prototypes from functions in execTuples.c

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: relation.h,v 1.77 2003/01/20 18:55:04 tgl Exp $
* $Id: relation.h,v 1.78 2003/02/03 15:07:08 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -105,6 +105,7 @@ typedef struct QualCost
* If the relation is a base relation it will have these fields set:
*
* rtekind - distinguishes plain relation, subquery, or function RTE
* varlist - list of Vars for physical columns (only if table)
* indexlist - list of IndexOptInfo nodes for relation's indexes
* (always NIL if it's not a table)
* pages - number of disk pages in relation (zero if not a table)
@ -190,6 +191,7 @@ typedef struct RelOptInfo
/* information about a base rel (not set for join rels!) */
RTEKind rtekind; /* RELATION, SUBQUERY, or FUNCTION */
List *varlist;
List *indexlist;
long pages;
double tuples;

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2002, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $Id: plancat.h,v 1.28 2003/01/28 22:13:41 tgl Exp $
* $Id: plancat.h,v 1.29 2003/02/03 15:07:08 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -17,10 +17,7 @@
#include "nodes/relation.h"
extern void get_relation_info(Oid relationObjectId,
bool *hasindex, long *pages, double *tuples);
extern List *find_secondary_indexes(Oid relationObjectId);
extern void get_relation_info(Oid relationObjectId, RelOptInfo *rel);
extern List *find_inheritance_children(Oid inhparent);