aboutsummaryrefslogtreecommitdiff
path: root/src/backend/executor
diff options
context:
space:
mode:
Diffstat (limited to 'src/backend/executor')
-rw-r--r--src/backend/executor/nodeAgg.c336
-rw-r--r--src/backend/executor/nodeGroup.c169
2 files changed, 216 insertions, 289 deletions
diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c
index bf4a9bbbdaa..7714a680909 100644
--- a/src/backend/executor/nodeAgg.c
+++ b/src/backend/executor/nodeAgg.c
@@ -46,7 +46,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* 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 @@
#include "catalog/pg_operator.h"
#include "executor/executor.h"
#include "executor/nodeAgg.h"
+#include "executor/nodeGroup.h"
#include "miscadmin.h"
#include "optimizer/clauses.h"
#include "parser/parse_coerce.h"
@@ -159,6 +160,7 @@ typedef struct AggStatePerAggData
static void initialize_aggregate(AggStatePerAgg peraggstate);
static void advance_transition_function(AggStatePerAgg peraggstate,
Datum newVal, bool isNull);
+static void advance_aggregates(AggState *aggstate, ExprContext *econtext);
static void process_sorted_aggregate(AggState *aggstate,
AggStatePerAgg peraggstate);
static void finalize_aggregate(AggStatePerAgg peraggstate,
@@ -314,6 +316,62 @@ advance_transition_function(AggStatePerAgg peraggstate,
}
/*
+ * 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
* after we have completed entering all the input values into the sort
* object. We complete the sort, read out the values in sorted order,
@@ -448,23 +506,18 @@ finalize_aggregate(AggStatePerAgg peraggstate,
}
-/* ---------------------------------------
- *
+/*
* ExecAgg -
*
* ExecAgg receives tuples from its outer subplan and aggregates over
* the appropriate attribute for each aggregate function use (Aggref
* 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
- * present. We can produce an aggregate result row per group, or just
- * one for the whole query. The value of each aggregate is 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.
- *
- * ------------------------------------------
+ * of tuples to aggregate over depends on whether grouped or plain
+ * aggregation is selected. In grouped aggregation, we produce a result
+ * row for each group; in plain aggregation there's a single result row
+ * for the whole query. In either case, the value of each aggregate is
+ * stored in the expression context to be used when ExecProject evaluates
+ * the result tuple.
*/
TupleTableSlot *
ExecAgg(Agg *node)
@@ -478,10 +531,10 @@ ExecAgg(Agg *node)
bool *aggnulls;
AggStatePerAgg peragg;
MemoryContext oldContext;
+ TupleTableSlot *outerslot;
+ TupleTableSlot *firstSlot;
TupleTableSlot *resultSlot;
- HeapTuple inputTuple;
int aggno;
- bool isNull;
/*
* get state info from node
@@ -494,6 +547,7 @@ ExecAgg(Agg *node)
aggnulls = econtext->ecxt_aggnulls;
projInfo = aggstate->csstate.cstate.cs_ProjInfo;
peragg = aggstate->peragg;
+ firstSlot = aggstate->csstate.css_ScanTupleSlot;
/*
* We loop retrieving groups until we find one matching
@@ -505,6 +559,31 @@ ExecAgg(Agg *node)
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
*/
MemoryContextReset(aggstate->tup_cxt);
@@ -519,74 +598,61 @@ ExecAgg(Agg *node)
initialize_aggregate(peraggstate);
}
- inputTuple = NULL; /* no saved input tuple yet */
-
- /*
- * for each tuple from the outer plan, update all the aggregates
- */
- for (;;)
+ if (aggstate->grp_firstTuple != NULL)
{
- TupleTableSlot *outerslot;
+ /*
+ * 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 */
- outerslot = ExecProcNode(outerPlan, (Plan *) node);
- if (TupIsNull(outerslot))
- break;
- econtext->ecxt_scantuple = outerslot;
+ /* set up for first advance_aggregates call */
+ econtext->ecxt_scantuple = firstSlot;
/*
- * Clear and select the current working context for evaluation
- * of the input expressions and transition functions at this
- * input tuple.
+ * Process each outer-plan tuple, and then fetch the next one,
+ * until we exhaust the outer plan or cross a group boundary.
*/
- 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++)
+ for (;;)
{
- AggStatePerAgg peraggstate = &peragg[aggno];
- Aggref *aggref = peraggstate->aggref;
- Datum newVal;
-
- newVal = ExecEvalExpr(aggref->target, econtext,
- &isNull, NULL);
+ advance_aggregates(aggstate, econtext);
- if (aggref->aggdistinct)
+ outerslot = ExecProcNode(outerPlan, (Plan *) node);
+ if (TupIsNull(outerslot))
{
- /* 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);
+ /* no more outer-plan tuples available */
+ aggstate->agg_done = true;
+ break;
}
- else
+ /* set up for next advance_aggregates call */
+ econtext->ecxt_scantuple = outerslot;
+
+ /*
+ * If we are grouping, check whether we've crossed a group
+ * boundary.
+ */
+ if (node->aggstrategy == AGG_SORTED)
{
- advance_transition_function(peraggstate,
- newVal, isNull);
+ if (!execTuplesMatch(firstSlot->val,
+ outerslot->val,
+ firstSlot->ttc_tupleDescriptor,
+ node->numCols, node->grpColIdx,
+ aggstate->eqfunctions,
+ aggstate->agg_cxt[aggstate->which_cxt]))
+ {
+ /*
+ * Save the first input tuple of the next group.
+ */
+ aggstate->grp_firstTuple = heap_copytuple(outerslot->val);
+ break;
+ }
}
}
-
- /*
- * 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,78 +692,51 @@ ExecAgg(Agg *node)
}
/*
- * If the outerPlan is a Group node, we will reach here after each
- * 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 we have no first tuple (ie, the outerPlan didn't return
+ * anything), create a dummy all-nulls input tuple for use by
+ * ExecProject. 99.44% of the time this is a waste of cycles,
+ * because ordinarily the projected output tuple's targetlist
+ * cannot contain any direct (non-aggregated) references to
+ * input columns, so the dummy tuple will not be referenced.
+ * However there are special cases where this isn't so --- in
+ * particular an UPDATE involving an aggregate will have a
+ * targetlist reference to ctid. We need to return a null for
+ * ctid in that situation, not coredump.
*
- * 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.
+ * The values returned for the aggregates will be the initial
+ * values of the transition functions.
*/
- if (IsA(outerPlan, Group))
+ if (TupIsNull(firstSlot))
{
- /* 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;
+ TupleDesc tupType;
- /*
- * If inputtuple==NULL (ie, the outerPlan didn't return
- * anything), create a dummy all-nulls input tuple for use by
- * ExecProject. 99.44% of the time this is a waste of cycles,
- * because ordinarily the projected output tuple's targetlist
- * cannot contain any direct (non-aggregated) references to
- * input columns, so the dummy tuple will not be referenced.
- * However there are special cases where this isn't so --- in
- * particular an UPDATE involving an aggregate will have a
- * targetlist reference to ctid. We need to return a null for
- * ctid in that situation, not coredump.
- *
- * The values returned for the aggregates will be the initial
- * values of the transition functions.
- */
- if (inputTuple == NULL)
+ /* Should only happen in non-grouped mode */
+ 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)
{
- TupleDesc tupType;
+ HeapTuple nullsTuple;
Datum *dvalues;
char *dnulls;
- tupType = aggstate->csstate.css_ScanTupleSlot->ttc_tupleDescriptor;
- /* watch out for null input tuples, though... */
- if (tupType && tupType->natts > 0)
- {
- dvalues = (Datum *) palloc(sizeof(Datum) * tupType->natts);
- dnulls = (char *) palloc(sizeof(char) * tupType->natts);
- MemSet(dvalues, 0, sizeof(Datum) * tupType->natts);
- MemSet(dnulls, 'n', sizeof(char) * tupType->natts);
- inputTuple = heap_formtuple(tupType, dvalues, dnulls);
- pfree(dvalues);
- pfree(dnulls);
- }
+ dvalues = (Datum *) palloc(sizeof(Datum) * tupType->natts);
+ dnulls = (char *) palloc(sizeof(char) * tupType->natts);
+ MemSet(dvalues, 0, sizeof(Datum) * tupType->natts);
+ MemSet(dnulls, 'n', sizeof(char) * tupType->natts);
+ nullsTuple = heap_formtuple(tupType, dvalues, dnulls);
+ ExecStoreTuple(nullsTuple,
+ firstSlot,
+ InvalidBuffer,
+ true);
+ pfree(dvalues);
+ 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.
*/
econtext->ecxt_per_tuple_memory = aggstate->tup_cxt;
@@ -707,6 +746,7 @@ ExecAgg(Agg *node)
* representative input tuple. Store it in the result tuple slot.
* Note we do not support aggregates returning sets ...
*/
+ econtext->ecxt_scantuple = firstSlot;
resultSlot = ExecProject(projInfo, NULL);
/*
@@ -748,6 +788,8 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent)
*/
aggstate = makeNode(AggState);
node->aggstate = aggstate;
+ aggstate->eqfunctions = NULL;
+ aggstate->grp_firstTuple = NULL;
aggstate->agg_done = false;
/*
@@ -765,13 +807,12 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent)
if (numaggs <= 0)
{
/*
- * This used to be treated as an error, but we can't do that
- * anymore because constant-expression simplification could
- * optimize away all of the Aggrefs in the targetlist and qual.
- * So, just make a debug note, and force numaggs positive so that
- * palloc()s below don't choke.
+ * This is not an error condition: we might be using the Agg node just
+ * to do hash-based grouping. Even in the regular case,
+ * constant-expression simplification could optimize away all of the
+ * Aggrefs in the targetlist and qual. So keep going, but force local
+ * copy of numaggs positive so that palloc()s below don't choke.
*/
- elog(DEBUG1, "ExecInitAgg: could not find any aggregate functions");
numaggs = 1;
}
@@ -844,6 +885,17 @@ ExecInitAgg(Agg *node, EState *estate, Plan *parent)
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
* unchanging fields of the per-agg data
*/
@@ -1024,6 +1076,11 @@ ExecEndAgg(Agg *node)
/* clean up tuple table */
ExecClearTuple(aggstate->csstate.css_ScanTupleSlot);
+ if (aggstate->grp_firstTuple != NULL)
+ {
+ heap_freetuple(aggstate->grp_firstTuple);
+ aggstate->grp_firstTuple = NULL;
+ }
}
void
@@ -1033,6 +1090,11 @@ ExecReScanAgg(Agg *node, ExprContext *exprCtxt, Plan *parent)
ExprContext *econtext = aggstate->csstate.cstate.cs_ExprContext;
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_aggnulls, 0, sizeof(bool) * aggstate->numaggs);
diff --git a/src/backend/executor/nodeGroup.c b/src/backend/executor/nodeGroup.c
index 5d7f6a69924..662c3d4798c 100644
--- a/src/backend/executor/nodeGroup.c
+++ b/src/backend/executor/nodeGroup.c
@@ -15,7 +15,7 @@
* locate group boundaries.
*
* 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,148 +31,15 @@
#include "utils/lsyscache.h"
#include "utils/syscache.h"
-static TupleTableSlot *ExecGroupEveryTuple(Group *node);
-static TupleTableSlot *ExecGroupOneTuple(Group *node);
-/* ---------------------------------------
+/*
* ExecGroup -
*
- * There are two modes in which tuples are returned by ExecGroup. If
- * 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)
- * ------------------------------------------
+ * Return one tuple for each group of matching input tuples.
*/
TupleTableSlot *
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;
EState *estate;
ExprContext *econtext;
@@ -198,10 +65,11 @@ ExecGroupOneTuple(Group *node)
* 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;
if (firsttuple == NULL)
{
- /* this should occur on the first call only */
outerslot = ExecProcNode(outerPlan(node), (Plan *) node);
if (TupIsNull(outerslot))
{
@@ -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 (;;)
{
@@ -239,22 +107,18 @@ ExecGroupOneTuple(Group *node)
}
/*
- * form a projection tuple, store it in the result tuple slot and
- * return it.
- */
- 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.
+ * form a projection tuple based on the (copied) first tuple of the
+ * group, and store it in the result tuple slot.
*/
ExecStoreTuple(firsttuple,
grpstate->csstate.css_ScanTupleSlot,
- InvalidBuffer, false);
+ InvalidBuffer,
+ false);
econtext->ecxt_scantuple = grpstate->csstate.css_ScanTupleSlot;
+ projInfo = grpstate->csstate.cstate.cs_ProjInfo;
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)
{
heap_freetuple(firsttuple);
@@ -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
* Return true if two tuples match in all the indicated fields.
- * This is used to detect group boundaries in nodeGroup, and to
- * decide whether two tuples are distinct or not in nodeUnique.
+ * This is used to detect group boundaries in nodeGroup and nodeAgg,
+ * and to decide whether two tuples are distinct or not in nodeUnique.
*
* tuple1, tuple2: the tuples to compare
* tupdesc: tuple descriptor applying to both tuples
@@ -425,7 +289,8 @@ execTuplesMatch(HeapTuple tuple1,
* 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,
* 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;