]> granicus.if.org Git - postgresql/commitdiff
Support multi-stage aggregation.
authorRobert Haas <rhaas@postgresql.org>
Wed, 20 Jan 2016 18:46:50 +0000 (13:46 -0500)
committerRobert Haas <rhaas@postgresql.org>
Wed, 20 Jan 2016 18:46:50 +0000 (13:46 -0500)
Aggregate nodes now have two new modes: a "partial" mode where they
output the unfinalized transition state, and a "finalize" mode where
they accept unfinalized transition states rather than individual
values as input.

These new modes are not used anywhere yet, but they will be necessary
for parallel aggregation.  The infrastructure also figures to be
useful for cases where we want to aggregate local data and remote
data via the FDW interface, and want to bring back partial aggregates
from the remote side that can then be combined with locally generated
partial aggregates to produce the final value.  It may also be useful
even when neither FDWs nor parallelism are in play, as explained in
the comments in nodeAgg.c.

David Rowley and Simon Riggs, reviewed by KaiGai Kohei, Heikki
Linnakangas, Haribabu Kommi, and me.

21 files changed:
doc/src/sgml/ref/create_aggregate.sgml
src/backend/catalog/pg_aggregate.c
src/backend/commands/aggregatecmds.c
src/backend/commands/explain.c
src/backend/executor/nodeAgg.c
src/backend/nodes/copyfuncs.c
src/backend/nodes/outfuncs.c
src/backend/nodes/readfuncs.c
src/backend/optimizer/plan/createplan.c
src/backend/optimizer/plan/planner.c
src/backend/optimizer/prep/prepunion.c
src/backend/parser/parse_agg.c
src/bin/pg_dump/pg_dump.c
src/include/catalog/catversion.h
src/include/catalog/pg_aggregate.h
src/include/nodes/execnodes.h
src/include/nodes/plannodes.h
src/include/optimizer/planmain.h
src/include/parser/parse_agg.h
src/test/regress/expected/create_aggregate.out
src/test/regress/sql/create_aggregate.sql

index eaa410bc94cf181da4b126286f62d36906232c43..4bda23ada6951ad05b1303cb32cc3f8dba3cb038 100644 (file)
@@ -27,6 +27,7 @@ CREATE AGGREGATE <replaceable class="parameter">name</replaceable> ( [ <replacea
     [ , SSPACE = <replaceable class="PARAMETER">state_data_size</replaceable> ]
     [ , FINALFUNC = <replaceable class="PARAMETER">ffunc</replaceable> ]
     [ , FINALFUNC_EXTRA ]
+    [ , COMBINEFUNC = <replaceable class="PARAMETER">combinefunc</replaceable> ]
     [ , INITCOND = <replaceable class="PARAMETER">initial_condition</replaceable> ]
     [ , MSFUNC = <replaceable class="PARAMETER">msfunc</replaceable> ]
     [ , MINVFUNC = <replaceable class="PARAMETER">minvfunc</replaceable> ]
@@ -45,6 +46,7 @@ CREATE AGGREGATE <replaceable class="parameter">name</replaceable> ( [ [ <replac
     [ , SSPACE = <replaceable class="PARAMETER">state_data_size</replaceable> ]
     [ , FINALFUNC = <replaceable class="PARAMETER">ffunc</replaceable> ]
     [ , FINALFUNC_EXTRA ]
+    [ , COMBINEFUNC = <replaceable class="PARAMETER">combinefunc</replaceable> ]
     [ , INITCOND = <replaceable class="PARAMETER">initial_condition</replaceable> ]
     [ , HYPOTHETICAL ]
 )
@@ -58,6 +60,7 @@ CREATE AGGREGATE <replaceable class="PARAMETER">name</replaceable> (
     [ , SSPACE = <replaceable class="PARAMETER">state_data_size</replaceable> ]
     [ , FINALFUNC = <replaceable class="PARAMETER">ffunc</replaceable> ]
     [ , FINALFUNC_EXTRA ]
+    [ , COMBINEFUNC = <replaceable class="PARAMETER">combinefunc</replaceable> ]
     [ , INITCOND = <replaceable class="PARAMETER">initial_condition</replaceable> ]
     [ , MSFUNC = <replaceable class="PARAMETER">msfunc</replaceable> ]
     [ , MINVFUNC = <replaceable class="PARAMETER">minvfunc</replaceable> ]
@@ -105,12 +108,15 @@ CREATE AGGREGATE <replaceable class="PARAMETER">name</replaceable> (
    functions:
    a state transition function
    <replaceable class="PARAMETER">sfunc</replaceable>,
-   and an optional final calculation function
-   <replaceable class="PARAMETER">ffunc</replaceable>.
+   an optional final calculation function
+   <replaceable class="PARAMETER">ffunc</replaceable>,
+   and an optional combine function
+   <replaceable class="PARAMETER">combinefunc</replaceable>.
    These are used as follows:
 <programlisting>
 <replaceable class="PARAMETER">sfunc</replaceable>( internal-state, next-data-values ) ---> next-internal-state
 <replaceable class="PARAMETER">ffunc</replaceable>( internal-state ) ---> aggregate-value
+<replaceable class="PARAMETER">combinefunc</replaceable>( internal-state, internal-state ) ---> next-internal-state
 </programlisting>
   </para>
 
@@ -127,6 +133,12 @@ CREATE AGGREGATE <replaceable class="PARAMETER">name</replaceable> (
    is returned as-is.
   </para>
 
+  <para>
+   An aggregate function may also supply a combining function, which allows
+   the aggregation process to be broken down into multiple steps.  This
+   facilitates query optimization techniques such as parallel query.
+  </para>
+
   <para>
    An aggregate function can provide an initial condition,
    that is, an initial value for the internal state value.
index 1d845ec824c857ff96f7651162e31a0cc2f8dd94..c612ab9809ec126de459b3327af7d429506f1588 100644 (file)
@@ -57,6 +57,7 @@ AggregateCreate(const char *aggName,
                                Oid variadicArgType,
                                List *aggtransfnName,
                                List *aggfinalfnName,
+                               List *aggcombinefnName,
                                List *aggmtransfnName,
                                List *aggminvtransfnName,
                                List *aggmfinalfnName,
@@ -77,6 +78,7 @@ AggregateCreate(const char *aggName,
        Form_pg_proc proc;
        Oid                     transfn;
        Oid                     finalfn = InvalidOid;   /* can be omitted */
+       Oid                     combinefn = InvalidOid; /* can be omitted */
        Oid                     mtransfn = InvalidOid;  /* can be omitted */
        Oid                     minvtransfn = InvalidOid;               /* can be omitted */
        Oid                     mfinalfn = InvalidOid;  /* can be omitted */
@@ -396,6 +398,30 @@ AggregateCreate(const char *aggName,
        }
        Assert(OidIsValid(finaltype));
 
+       /* handle the combinefn, if supplied */
+       if (aggcombinefnName)
+       {
+               Oid combineType;
+
+               /*
+                * Combine function must have 2 argument, each of which is the
+                * trans type
+                */
+               fnArgs[0] = aggTransType;
+               fnArgs[1] = aggTransType;
+
+               combinefn = lookup_agg_function(aggcombinefnName, 2, fnArgs,
+                                                                               variadicArgType, &combineType);
+
+               /* Ensure the return type matches the aggregates trans type */
+               if (combineType != aggTransType)
+                       ereport(ERROR,
+                                       (errcode(ERRCODE_DATATYPE_MISMATCH),
+                       errmsg("return type of combine function %s is not %s",
+                                  NameListToString(aggcombinefnName),
+                                  format_type_be(aggTransType))));
+       }
+
        /*
         * If finaltype (i.e. aggregate return type) is polymorphic, inputs must
         * be polymorphic also, else parser will fail to deduce result type.
@@ -567,6 +593,7 @@ AggregateCreate(const char *aggName,
        values[Anum_pg_aggregate_aggnumdirectargs - 1] = Int16GetDatum(numDirectArgs);
        values[Anum_pg_aggregate_aggtransfn - 1] = ObjectIdGetDatum(transfn);
        values[Anum_pg_aggregate_aggfinalfn - 1] = ObjectIdGetDatum(finalfn);
+       values[Anum_pg_aggregate_aggcombinefn - 1] = ObjectIdGetDatum(combinefn);
        values[Anum_pg_aggregate_aggmtransfn - 1] = ObjectIdGetDatum(mtransfn);
        values[Anum_pg_aggregate_aggminvtransfn - 1] = ObjectIdGetDatum(minvtransfn);
        values[Anum_pg_aggregate_aggmfinalfn - 1] = ObjectIdGetDatum(mfinalfn);
@@ -618,6 +645,15 @@ AggregateCreate(const char *aggName,
                recordDependencyOn(&myself, &referenced, DEPENDENCY_NORMAL);
        }
 
+       /* Depends on combine function, if any */
+       if (OidIsValid(combinefn))
+       {
+               referenced.classId = ProcedureRelationId;
+               referenced.objectId = combinefn;
+               referenced.objectSubId = 0;
+               recordDependencyOn(&myself, &referenced, DEPENDENCY_NORMAL);
+       }
+
        /* Depends on forward transition function, if any */
        if (OidIsValid(mtransfn))
        {
@@ -659,7 +695,7 @@ AggregateCreate(const char *aggName,
 
 /*
  * lookup_agg_function
- * common code for finding transfn, invtransfn and finalfn
+ * common code for finding transfn, invtransfn, finalfn, and combinefn
  *
  * Returns OID of function, and stores its return type into *rettype
  *
index 441b3aa9e55e1666ec004d3e6e4633f8dc303562..59bc6e6fd8f7741cb9c66cc034b64e9b35f63606 100644 (file)
@@ -61,6 +61,7 @@ DefineAggregate(List *name, List *args, bool oldstyle, List *parameters,
        char            aggKind = AGGKIND_NORMAL;
        List       *transfuncName = NIL;
        List       *finalfuncName = NIL;
+       List       *combinefuncName = NIL;
        List       *mtransfuncName = NIL;
        List       *minvtransfuncName = NIL;
        List       *mfinalfuncName = NIL;
@@ -124,6 +125,8 @@ DefineAggregate(List *name, List *args, bool oldstyle, List *parameters,
                        transfuncName = defGetQualifiedName(defel);
                else if (pg_strcasecmp(defel->defname, "finalfunc") == 0)
                        finalfuncName = defGetQualifiedName(defel);
+               else if (pg_strcasecmp(defel->defname, "combinefunc") == 0)
+                       combinefuncName = defGetQualifiedName(defel);
                else if (pg_strcasecmp(defel->defname, "msfunc") == 0)
                        mtransfuncName = defGetQualifiedName(defel);
                else if (pg_strcasecmp(defel->defname, "minvfunc") == 0)
@@ -383,6 +386,7 @@ DefineAggregate(List *name, List *args, bool oldstyle, List *parameters,
                                                   variadicArgType,
                                                   transfuncName,               /* step function name */
                                                   finalfuncName,               /* final function name */
+                                                  combinefuncName,             /* combine function name */
                                                   mtransfuncName,              /* fwd trans function name */
                                                   minvtransfuncName,   /* inv trans function name */
                                                   mfinalfuncName,              /* final function name */
index 9827c39e09de09dfb7ea4ab63694f321e2b88937..25d8ca075d4e938404533b48326b0bcf546a606b 100644 (file)
@@ -909,24 +909,36 @@ ExplainNode(PlanState *planstate, List *ancestors,
                        break;
                case T_Agg:
                        sname = "Aggregate";
-                       switch (((Agg *) plan)->aggstrategy)
                        {
-                               case AGG_PLAIN:
-                                       pname = "Aggregate";
-                                       strategy = "Plain";
-                                       break;
-                               case AGG_SORTED:
-                                       pname = "GroupAggregate";
-                                       strategy = "Sorted";
-                                       break;
-                               case AGG_HASHED:
-                                       pname = "HashAggregate";
-                                       strategy = "Hashed";
-                                       break;
-                               default:
-                                       pname = "Aggregate ???";
-                                       strategy = "???";
-                                       break;
+                               Agg                *agg = (Agg *) plan;
+
+                               if (agg->finalizeAggs == false)
+                                       operation = "Partial";
+                               else if (agg->combineStates == true)
+                                       operation = "Finalize";
+
+                               switch (agg->aggstrategy)
+                               {
+                                       case AGG_PLAIN:
+                                               pname = "Aggregate";
+                                               strategy = "Plain";
+                                               break;
+                                       case AGG_SORTED:
+                                               pname = "GroupAggregate";
+                                               strategy = "Sorted";
+                                               break;
+                                       case AGG_HASHED:
+                                               pname = "HashAggregate";
+                                               strategy = "Hashed";
+                                               break;
+                                       default:
+                                               pname = "Aggregate ???";
+                                               strategy = "???";
+                                               break;
+                               }
+
+                               if (operation != NULL)
+                                       pname = psprintf("%s %s", operation, pname);
                        }
                        break;
                case T_WindowAgg:
index f49114abe3b5217a2102c46a91ffe85081d97eef..b5aac67489d7127cb0d92222fe8dfe32e35916f7 100644 (file)
@@ -3,15 +3,46 @@
  * nodeAgg.c
  *       Routines to handle aggregate nodes.
  *
- *       ExecAgg evaluates each aggregate in the following steps:
+ *       ExecAgg normally evaluates each aggregate in the following steps:
  *
  *              transvalue = initcond
  *              foreach input_tuple do
  *                     transvalue = transfunc(transvalue, input_value(s))
  *              result = finalfunc(transvalue, direct_argument(s))
  *
- *       If a finalfunc is not supplied then the result is just the ending
- *       value of transvalue.
+ *       If a finalfunc is not supplied or finalizeAggs is false, then the result
+ *       is just the ending value of transvalue.
+ *
+ *       Other behavior is also supported and is controlled by the 'combineStates'
+ *       and 'finalizeAggs'. 'combineStates' controls whether the trans func or
+ *       the combine func is used during aggregation.  When 'combineStates' is
+ *       true we expect other (previously) aggregated states as input rather than
+ *       input tuples. This mode facilitates multiple aggregate stages which
+ *       allows us to support pushing aggregation down deeper into the plan rather
+ *       than leaving it for the final stage. For example with a query such as:
+ *
+ *       SELECT count(*) FROM (SELECT * FROM a UNION ALL SELECT * FROM b);
+ *
+ *       with this functionality the planner has the flexibility to generate a
+ *       plan which performs count(*) on table a and table b separately and then
+ *       add a combine phase to combine both results. In this case the combine
+ *       function would simply add both counts together.
+ *
+ *       When multiple aggregate stages exist the planner should have set the
+ *       'finalizeAggs' to true only for the final aggregtion state, and each
+ *       stage, apart from the very first one should have 'combineStates' set to
+ *       true. This permits plans such as:
+ *
+ *             Finalize Aggregate
+ *                     ->      Partial Aggregate
+ *                             ->      Partial Aggregate
+ *
+ *       Combine functions which use pass-by-ref states should be careful to
+ *       always update the 1st state parameter by adding the 2nd parameter to it,
+ *       rather than the other way around. If the 1st state is NULL, then it's not
+ *       sufficient to simply return the 2nd state, as the memory context is
+ *       incorrect. Instead a new state should be created in the correct aggregate
+ *       memory context and the 2nd state should be copied over.
  *
  *       If a normal aggregate call specifies DISTINCT or ORDER BY, we sort the
  *       input tuples and eliminate duplicates (if required) before performing
 #include "catalog/objectaccess.h"
 #include "catalog/pg_aggregate.h"
 #include "catalog/pg_proc.h"
+#include "catalog/pg_type.h"
 #include "executor/executor.h"
 #include "executor/nodeAgg.h"
 #include "miscadmin.h"
@@ -197,7 +229,7 @@ typedef struct AggStatePerTransData
         */
        int                     numTransInputs;
 
-       /* Oid of the state transition function */
+       /* Oid of the state transition or combine function */
        Oid                     transfn_oid;
 
        /* Oid of state value's datatype */
@@ -209,8 +241,8 @@ typedef struct AggStatePerTransData
        List       *aggdirectargs;      /* states of direct-argument expressions */
 
        /*
-        * fmgr lookup data for transition function.  Note in particular that the
-        * fn_strict flag is kept here.
+        * fmgr lookup data for transition function or combine function.  Note in
+        * particular that the fn_strict flag is kept here.
         */
        FmgrInfo        transfn;
 
@@ -421,6 +453,10 @@ static void advance_transition_function(AggState *aggstate,
                                                        AggStatePerTrans pertrans,
                                                        AggStatePerGroup pergroupstate);
 static void advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup);
+static void advance_combine_function(AggState *aggstate,
+                                                AggStatePerTrans pertrans,
+                                                AggStatePerGroup pergroupstate);
+static void combine_aggregates(AggState *aggstate, AggStatePerGroup pergroup);
 static void process_ordered_aggregate_single(AggState *aggstate,
                                                                 AggStatePerTrans pertrans,
                                                                 AggStatePerGroup pergroupstate);
@@ -458,7 +494,7 @@ static int find_compatible_peragg(Aggref *newagg, AggState *aggstate,
 static int find_compatible_pertrans(AggState *aggstate, Aggref *newagg,
                                                 Oid aggtransfn, Oid aggtranstype,
                                                 Datum initValue, bool initValueIsNull,
-                                                List *possible_matches);
+                                                List *transnos);
 
 
 /*
@@ -796,6 +832,8 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup)
        int                     numGroupingSets = Max(aggstate->phase->numsets, 1);
        int                     numTrans = aggstate->numtrans;
 
+       Assert(!aggstate->combineStates);
+
        for (transno = 0; transno < numTrans; transno++)
        {
                AggStatePerTrans pertrans = &aggstate->pertrans[transno];
@@ -879,6 +917,131 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup)
        }
 }
 
+/*
+ * combine_aggregates is used when running in 'combineState' mode. This
+ * advances each aggregate transition state by adding another transition state
+ * to it.
+ */
+static void
+combine_aggregates(AggState *aggstate, AggStatePerGroup pergroup)
+{
+       int                     transno;
+       int                     numTrans = aggstate->numtrans;
+
+       /* combine not supported with grouping sets */
+       Assert(aggstate->phase->numsets == 0);
+       Assert(aggstate->combineStates);
+
+       for (transno = 0; transno < numTrans; transno++)
+       {
+               AggStatePerTrans pertrans = &aggstate->pertrans[transno];
+               TupleTableSlot *slot;
+               FunctionCallInfo fcinfo = &pertrans->transfn_fcinfo;
+               AggStatePerGroup pergroupstate = &pergroup[transno];
+
+               /* Evaluate the current input expressions for this aggregate */
+               slot = ExecProject(pertrans->evalproj, NULL);
+               Assert(slot->tts_nvalid >= 1);
+
+               fcinfo->arg[1] = slot->tts_values[0];
+               fcinfo->argnull[1] = slot->tts_isnull[0];
+
+               advance_combine_function(aggstate, pertrans, pergroupstate);
+       }
+}
+
+/*
+ * Perform combination of states between 2 aggregate states. Effectively this
+ * 'adds' two states together by whichever logic is defined in the aggregate
+ * function's combine function.
+ *
+ * Note that in this case transfn is set to the combination function. This
+ * perhaps should be changed to avoid confusion, but one field is ok for now
+ * as they'll never be needed at the same time.
+ */
+static void
+advance_combine_function(AggState *aggstate,
+                                                AggStatePerTrans pertrans,
+                                                AggStatePerGroup pergroupstate)
+{
+       FunctionCallInfo fcinfo = &pertrans->transfn_fcinfo;
+       MemoryContext oldContext;
+       Datum           newVal;
+
+       if (pertrans->transfn.fn_strict)
+       {
+               /* if we're asked to merge to a NULL state, then do nothing */
+               if (fcinfo->argnull[1])
+                       return;
+
+               if (pergroupstate->noTransValue)
+               {
+                       /*
+                        * transValue has not yet been initialized.  If pass-by-ref
+                        * datatype we must copy the combining state value into
+                        * aggcontext.
+                        */
+                       if (!pertrans->transtypeByVal)
+                       {
+                               oldContext = MemoryContextSwitchTo(
+                                                                                                  aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory);
+                               pergroupstate->transValue = datumCopy(fcinfo->arg[1],
+                                                                                                       pertrans->transtypeByVal,
+                                                                                                         pertrans->transtypeLen);
+                               MemoryContextSwitchTo(oldContext);
+                       }
+                       else
+                               pergroupstate->transValue = fcinfo->arg[1];
+
+                       pergroupstate->transValueIsNull = false;
+                       pergroupstate->noTransValue = false;
+                       return;
+               }
+       }
+
+       /* We run the combine functions in per-input-tuple memory context */
+       oldContext = MemoryContextSwitchTo(aggstate->tmpcontext->ecxt_per_tuple_memory);
+
+       /* set up aggstate->curpertrans for AggGetAggref() */
+       aggstate->curpertrans = pertrans;
+
+       /*
+        * OK to call the combine function
+        */
+       fcinfo->arg[0] = pergroupstate->transValue;
+       fcinfo->argnull[0] = pergroupstate->transValueIsNull;
+       fcinfo->isnull = false;         /* just in case combine func doesn't set it */
+
+       newVal = FunctionCallInvoke(fcinfo);
+
+       aggstate->curpertrans = NULL;
+
+       /*
+        * If pass-by-ref datatype, must copy the new value into aggcontext and
+        * pfree the prior transValue.  But if the combine function returned a
+        * pointer to its first input, we don't need to do anything.
+        */
+       if (!pertrans->transtypeByVal &&
+               DatumGetPointer(newVal) != DatumGetPointer(pergroupstate->transValue))
+       {
+               if (!fcinfo->isnull)
+               {
+                       MemoryContextSwitchTo(aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory);
+                       newVal = datumCopy(newVal,
+                                                          pertrans->transtypeByVal,
+                                                          pertrans->transtypeLen);
+               }
+               if (!pergroupstate->transValueIsNull)
+                       pfree(DatumGetPointer(pergroupstate->transValue));
+       }
+
+       pergroupstate->transValue = newVal;
+       pergroupstate->transValueIsNull = fcinfo->isnull;
+
+       MemoryContextSwitchTo(oldContext);
+
+}
+
 
 /*
  * Run the transition function for a DISTINCT or ORDER BY aggregate
@@ -1278,8 +1441,14 @@ finalize_aggregates(AggState *aggstate,
                                                                                                pergroupstate);
                }
 
-               finalize_aggregate(aggstate, peragg, pergroupstate,
-                                                  &aggvalues[aggno], &aggnulls[aggno]);
+               if (aggstate->finalizeAggs)
+                       finalize_aggregate(aggstate, peragg, pergroupstate,
+                                                          &aggvalues[aggno], &aggnulls[aggno]);
+               else
+               {
+                       aggvalues[aggno] = pergroupstate->transValue;
+                       aggnulls[aggno] = pergroupstate->transValueIsNull;
+               }
        }
 }
 
@@ -1811,7 +1980,10 @@ agg_retrieve_direct(AggState *aggstate)
                                 */
                                for (;;)
                                {
-                                       advance_aggregates(aggstate, pergroup);
+                                       if (!aggstate->combineStates)
+                                               advance_aggregates(aggstate, pergroup);
+                                       else
+                                               combine_aggregates(aggstate, pergroup);
 
                                        /* Reset per-input-tuple context after each tuple */
                                        ResetExprContext(tmpcontext);
@@ -1919,7 +2091,10 @@ agg_fill_hash_table(AggState *aggstate)
                entry = lookup_hash_entry(aggstate, outerslot);
 
                /* Advance the aggregates */
-               advance_aggregates(aggstate, entry->pergroup);
+               if (!aggstate->combineStates)
+                       advance_aggregates(aggstate, entry->pergroup);
+               else
+                       combine_aggregates(aggstate, entry->pergroup);
 
                /* Reset per-input-tuple context after each tuple */
                ResetExprContext(tmpcontext);
@@ -2051,6 +2226,8 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
        aggstate->pertrans = NULL;
        aggstate->curpertrans = NULL;
        aggstate->agg_done = false;
+       aggstate->combineStates = node->combineStates;
+       aggstate->finalizeAggs = node->finalizeAggs;
        aggstate->input_done = false;
        aggstate->pergroup = NULL;
        aggstate->grp_firstTuple = NULL;
@@ -2402,8 +2579,26 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
                                                   get_func_name(aggref->aggfnoid));
                InvokeFunctionExecuteHook(aggref->aggfnoid);
 
-               transfn_oid = aggform->aggtransfn;
-               peragg->finalfn_oid = finalfn_oid = aggform->aggfinalfn;
+               /*
+                * If this aggregation is performing state combines, then instead of
+                * using the transition function, we'll use the combine function
+                */
+               if (aggstate->combineStates)
+               {
+                       transfn_oid = aggform->aggcombinefn;
+
+                       /* If not set then the planner messed up */
+                       if (!OidIsValid(transfn_oid))
+                               elog(ERROR, "combinefn not set for aggregate function");
+               }
+               else
+                       transfn_oid = aggform->aggtransfn;
+
+               /* Final function only required if we're finalizing the aggregates */
+               if (aggstate->finalizeAggs)
+                       peragg->finalfn_oid = finalfn_oid = aggform->aggfinalfn;
+               else
+                       peragg->finalfn_oid = finalfn_oid = InvalidOid;
 
                /* Check that aggregate owner has permission to call component fns */
                {
@@ -2459,7 +2654,7 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
 
                /*
                 * build expression trees using actual argument & result types for the
-                * finalfn, if it exists
+                * finalfn, if it exists and is required.
                 */
                if (OidIsValid(finalfn_oid))
                {
@@ -2474,10 +2669,11 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
                        fmgr_info_set_expr((Node *) finalfnexpr, &peragg->finalfn);
                }
 
-               /* get info about the result type's datatype */
-               get_typlenbyval(aggref->aggtype,
-                                               &peragg->resulttypeLen,
-                                               &peragg->resulttypeByVal);
+               /* when finalizing we get info about the final result's datatype */
+               if (aggstate->finalizeAggs)
+                       get_typlenbyval(aggref->aggtype,
+                                                       &peragg->resulttypeLen,
+                                                       &peragg->resulttypeByVal);
 
                /*
                 * initval is potentially null, so don't try to access it as a struct
@@ -2551,7 +2747,6 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans,
                                                  Oid *inputTypes, int numArguments)
 {
        int                     numGroupingSets = Max(aggstate->maxsets, 1);
-       Expr       *transfnexpr;
        ListCell   *lc;
        int                     numInputs;
        int                     numDirectArgs;
@@ -2583,44 +2778,72 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans,
                pertrans->numTransInputs = numArguments;
 
        /*
-        * Set up infrastructure for calling the transfn
-        */
-       build_aggregate_transfn_expr(inputTypes,
-                                                                numArguments,
-                                                                numDirectArgs,
-                                                                aggref->aggvariadic,
-                                                                aggtranstype,
-                                                                aggref->inputcollid,
-                                                                aggtransfn,
-                                                                InvalidOid,    /* invtrans is not needed here */
-                                                                &transfnexpr,
-                                                                NULL);
-       fmgr_info(aggtransfn, &pertrans->transfn);
-       fmgr_info_set_expr((Node *) transfnexpr, &pertrans->transfn);
-
-       InitFunctionCallInfoData(pertrans->transfn_fcinfo,
-                                                        &pertrans->transfn,
-                                                        pertrans->numTransInputs + 1,
-                                                        pertrans->aggCollation,
-                                                        (void *) aggstate, NULL);
-
-       /*
-        * If the transfn is strict and the initval is NULL, make sure input type
-        * and transtype are the same (or at least binary-compatible), so that
-        * it's OK to use the first aggregated input value as the initial
-        * transValue.  This should have been checked at agg definition time, but
-        * we must check again in case the transfn's strictness property has been
-        * changed.
+        * When combining states, we have no use at all for the aggregate
+        * function's transfn. Instead we use the combinefn.  In this case, the
+        * transfn and transfn_oid fields of pertrans refer to the combine
+        * function rather than the transition function.
         */
-       if (pertrans->transfn.fn_strict && pertrans->initValueIsNull)
+       if (aggstate->combineStates)
        {
-               if (numArguments <= numDirectArgs ||
-                       !IsBinaryCoercible(inputTypes[numDirectArgs],
-                                                          aggtranstype))
-                       ereport(ERROR,
-                                       (errcode(ERRCODE_INVALID_FUNCTION_DEFINITION),
-                                        errmsg("aggregate %u needs to have compatible input type and transition type",
-                                                       aggref->aggfnoid)));
+               Expr       *combinefnexpr;
+
+               build_aggregate_combinefn_expr(aggtranstype,
+                                                                          aggref->inputcollid,
+                                                                          aggtransfn,
+                                                                          &combinefnexpr);
+               fmgr_info(aggtransfn, &pertrans->transfn);
+               fmgr_info_set_expr((Node *) combinefnexpr, &pertrans->transfn);
+
+               InitFunctionCallInfoData(pertrans->transfn_fcinfo,
+                                                                &pertrans->transfn,
+                                                                2,
+                                                                pertrans->aggCollation,
+                                                                (void *) aggstate, NULL);
+       }
+       else
+       {
+               Expr       *transfnexpr;
+
+               /*
+                * Set up infrastructure for calling the transfn
+                */
+               build_aggregate_transfn_expr(inputTypes,
+                                                                        numArguments,
+                                                                        numDirectArgs,
+                                                                        aggref->aggvariadic,
+                                                                        aggtranstype,
+                                                                        aggref->inputcollid,
+                                                                        aggtransfn,
+                                                                        InvalidOid,    /* invtrans is not needed here */
+                                                                        &transfnexpr,
+                                                                        NULL);
+               fmgr_info(aggtransfn, &pertrans->transfn);
+               fmgr_info_set_expr((Node *) transfnexpr, &pertrans->transfn);
+
+               InitFunctionCallInfoData(pertrans->transfn_fcinfo,
+                                                                &pertrans->transfn,
+                                                                pertrans->numTransInputs + 1,
+                                                                pertrans->aggCollation,
+                                                                (void *) aggstate, NULL);
+
+               /*
+                * If the transfn is strict and the initval is NULL, make sure input
+                * type and transtype are the same (or at least binary-compatible), so
+                * that it's OK to use the first aggregated input value as the initial
+                * transValue.  This should have been checked at agg definition time,
+                * but we must check again in case the transfn's strictness property
+                * has been changed.
+                */
+               if (pertrans->transfn.fn_strict && pertrans->initValueIsNull)
+               {
+                       if (numArguments <= numDirectArgs ||
+                               !IsBinaryCoercible(inputTypes[numDirectArgs],
+                                                                  aggtranstype))
+                               ereport(ERROR,
+                                               (errcode(ERRCODE_INVALID_FUNCTION_DEFINITION),
+                                                errmsg("aggregate %u needs to have compatible input type and transition type",
+                                                               aggref->aggfnoid)));
+               }
        }
 
        /* get info about the state value's datatype */
index f47e0dad20110ff8133ef6b8b847a9aa9fc5cfd0..5877037df4c6a94fbe595663a1b8cd53e73f1ce3 100644 (file)
@@ -865,6 +865,8 @@ _copyAgg(const Agg *from)
 
        COPY_SCALAR_FIELD(aggstrategy);
        COPY_SCALAR_FIELD(numCols);
+       COPY_SCALAR_FIELD(combineStates);
+       COPY_SCALAR_FIELD(finalizeAggs);
        if (from->numCols > 0)
        {
                COPY_POINTER_FIELD(grpColIdx, from->numCols * sizeof(AttrNumber));
index f1e22e5fb94deadcb3898e66d68bcc2dfdc9853e..8817b56df97a1ddc27c77d2a2c3def281f5cad29 100644 (file)
@@ -695,6 +695,9 @@ _outAgg(StringInfo str, const Agg *node)
        for (i = 0; i < node->numCols; i++)
                appendStringInfo(str, " %d", node->grpColIdx[i]);
 
+       WRITE_BOOL_FIELD(combineStates);
+       WRITE_BOOL_FIELD(finalizeAggs);
+
        appendStringInfoString(str, " :grpOperators");
        for (i = 0; i < node->numCols; i++)
                appendStringInfo(str, " %u", node->grpOperators[i]);
index 719a52cc19f80134e2c942322ca4269d4e7c26d1..a67b3370da06c81668c0527b34db9a7b369dc8a3 100644 (file)
@@ -1989,6 +1989,8 @@ _readAgg(void)
        READ_ENUM_FIELD(aggstrategy, AggStrategy);
        READ_INT_FIELD(numCols);
        READ_ATTRNUMBER_ARRAY(grpColIdx, local_node->numCols);
+       READ_BOOL_FIELD(combineStates);
+       READ_BOOL_FIELD(finalizeAggs);
        READ_OID_ARRAY(grpOperators, local_node->numCols);
        READ_LONG_FIELD(numGroups);
        READ_NODE_FIELD(groupingSets);
index 953aa6265fb7514177c70bcdbe83e579f8d6bc0b..01bd7e746b50e0c77ef75e6ba49aaddbcb1ad3eb 100644 (file)
@@ -1054,6 +1054,8 @@ create_unique_plan(PlannerInfo *root, UniquePath *best_path)
                                                                 groupOperators,
                                                                 NIL,
                                                                 numGroups,
+                                                                false,
+                                                                true,
                                                                 subplan);
        }
        else
@@ -4557,9 +4559,8 @@ Agg *
 make_agg(PlannerInfo *root, List *tlist, List *qual,
                 AggStrategy aggstrategy, const AggClauseCosts *aggcosts,
                 int numGroupCols, AttrNumber *grpColIdx, Oid *grpOperators,
-                List *groupingSets,
-                long numGroups,
-                Plan *lefttree)
+                List *groupingSets, long numGroups, bool combineStates,
+                bool finalizeAggs, Plan *lefttree)
 {
        Agg                *node = makeNode(Agg);
        Plan       *plan = &node->plan;
@@ -4568,6 +4569,8 @@ make_agg(PlannerInfo *root, List *tlist, List *qual,
 
        node->aggstrategy = aggstrategy;
        node->numCols = numGroupCols;
+       node->combineStates = combineStates;
+       node->finalizeAggs = finalizeAggs;
        node->grpColIdx = grpColIdx;
        node->grpOperators = grpOperators;
        node->numGroups = numGroups;
index 131dc8a7b1aa1854f9110b072343af5fc33afebe..c0ec905eb3f4687d521558a95df712fe31504072 100644 (file)
@@ -2005,6 +2005,8 @@ grouping_planner(PlannerInfo *root, double tuple_fraction)
                                                                        extract_grouping_ops(parse->groupClause),
                                                                                                NIL,
                                                                                                numGroups,
+                                                                                               false,
+                                                                                               true,
                                                                                                result_plan);
                                /* Hashed aggregation produces randomly-ordered results */
                                current_pathkeys = NIL;
@@ -2312,6 +2314,8 @@ grouping_planner(PlannerInfo *root, double tuple_fraction)
                                                                 extract_grouping_ops(parse->distinctClause),
                                                                                        NIL,
                                                                                        numDistinctRows,
+                                                                                       false,
+                                                                                       true,
                                                                                        result_plan);
                        /* Hashed aggregation produces randomly-ordered results */
                        current_pathkeys = NIL;
@@ -2549,6 +2553,8 @@ build_grouping_chain(PlannerInfo *root,
                                                                                 extract_grouping_ops(groupClause),
                                                                                 gsets,
                                                                                 numGroups,
+                                                                                false,
+                                                                                true,
                                                                                 sort_plan);
 
                        /*
@@ -2588,6 +2594,8 @@ build_grouping_chain(PlannerInfo *root,
                                                                                extract_grouping_ops(groupClause),
                                                                                gsets,
                                                                                numGroups,
+                                                                               false,
+                                                                               true,
                                                                                result_plan);
 
                ((Agg *) result_plan)->chain = chain;
index 694e9ed0830180175161d920d4049637276faeea..e509a1aa1f878d181343e821f6d9abae8362219e 100644 (file)
@@ -775,6 +775,8 @@ make_union_unique(SetOperationStmt *op, Plan *plan,
                                                                 extract_grouping_ops(groupList),
                                                                 NIL,
                                                                 numGroups,
+                                                                false,
+                                                                true,
                                                                 plan);
                /* Hashed aggregation produces randomly-ordered results */
                *sortClauses = NIL;
index b718169dffbb862cb503036eb3c72b3e5712894b..b790bb27c5dcd398ad981ce1de3070cbb9bc30ef 100644 (file)
@@ -1927,6 +1927,42 @@ build_aggregate_transfn_expr(Oid *agg_input_types,
        }
 }
 
+/*
+ * Like build_aggregate_transfn_expr, but creates an expression tree for the
+ * combine function of an aggregate, rather than the transition function.
+ */
+void
+build_aggregate_combinefn_expr(Oid agg_state_type,
+                                                          Oid agg_input_collation,
+                                                          Oid combinefn_oid,
+                                                          Expr **combinefnexpr)
+{
+       Param      *argp;
+       List       *args;
+       FuncExpr   *fexpr;
+
+       /* Build arg list to use in the combinefn FuncExpr node. */
+       argp = makeNode(Param);
+       argp->paramkind = PARAM_EXEC;
+       argp->paramid = -1;
+       argp->paramtype = agg_state_type;
+       argp->paramtypmod = -1;
+       argp->paramcollid = agg_input_collation;
+       argp->location = -1;
+
+       /* transition state type is arg 1 and 2 */
+       args = list_make2(argp, argp);
+
+       fexpr = makeFuncExpr(combinefn_oid,
+                                                agg_state_type,
+                                                args,
+                                                InvalidOid,
+                                                agg_input_collation,
+                                                COERCE_EXPLICIT_CALL);
+       fexpr->funcvariadic = false;
+       *combinefnexpr = (Expr *) fexpr;
+}
+
 /*
  * Like build_aggregate_transfn_expr, but creates an expression tree for the
  * final function of an aggregate, rather than the transition function.
index 59542aa1d3d442aa7556b3ca15c1b9c3c0aeb403..9c6f88540e335587027a20f92a7d43e9a106dd16 100644 (file)
@@ -12383,6 +12383,7 @@ dumpAgg(Archive *fout, AggInfo *agginfo)
        PGresult   *res;
        int                     i_aggtransfn;
        int                     i_aggfinalfn;
+       int                     i_aggcombinefn;
        int                     i_aggmtransfn;
        int                     i_aggminvtransfn;
        int                     i_aggmfinalfn;
@@ -12399,6 +12400,7 @@ dumpAgg(Archive *fout, AggInfo *agginfo)
        int                     i_convertok;
        const char *aggtransfn;
        const char *aggfinalfn;
+       const char *aggcombinefn;
        const char *aggmtransfn;
        const char *aggminvtransfn;
        const char *aggmfinalfn;
@@ -12429,7 +12431,26 @@ dumpAgg(Archive *fout, AggInfo *agginfo)
        selectSourceSchema(fout, agginfo->aggfn.dobj.namespace->dobj.name);
 
        /* Get aggregate-specific details */
-       if (fout->remoteVersion >= 90400)
+       if (fout->remoteVersion >= 90600)
+       {
+               appendPQExpBuffer(query, "SELECT aggtransfn, "
+                       "aggfinalfn, aggtranstype::pg_catalog.regtype, "
+                       "aggcombinefn, aggmtransfn, "
+                       "aggminvtransfn, aggmfinalfn, aggmtranstype::pg_catalog.regtype, "
+                       "aggfinalextra, aggmfinalextra, "
+                       "aggsortop::pg_catalog.regoperator, "
+                       "(aggkind = 'h') AS hypothetical, "
+                       "aggtransspace, agginitval, "
+                       "aggmtransspace, aggminitval, "
+                       "true AS convertok, "
+                       "pg_catalog.pg_get_function_arguments(p.oid) AS funcargs, "
+                       "pg_catalog.pg_get_function_identity_arguments(p.oid) AS funciargs "
+                       "FROM pg_catalog.pg_aggregate a, pg_catalog.pg_proc p "
+                       "WHERE a.aggfnoid = p.oid "
+                       "AND p.oid = '%u'::pg_catalog.oid",
+                       agginfo->aggfn.dobj.catId.oid);
+       }
+       else if (fout->remoteVersion >= 90400)
        {
                appendPQExpBuffer(query, "SELECT aggtransfn, "
                                                  "aggfinalfn, aggtranstype::pg_catalog.regtype, "
@@ -12539,6 +12560,7 @@ dumpAgg(Archive *fout, AggInfo *agginfo)
 
        i_aggtransfn = PQfnumber(res, "aggtransfn");
        i_aggfinalfn = PQfnumber(res, "aggfinalfn");
+       i_aggcombinefn = PQfnumber(res, "aggcombinefn");
        i_aggmtransfn = PQfnumber(res, "aggmtransfn");
        i_aggminvtransfn = PQfnumber(res, "aggminvtransfn");
        i_aggmfinalfn = PQfnumber(res, "aggmfinalfn");
@@ -12556,6 +12578,7 @@ dumpAgg(Archive *fout, AggInfo *agginfo)
 
        aggtransfn = PQgetvalue(res, 0, i_aggtransfn);
        aggfinalfn = PQgetvalue(res, 0, i_aggfinalfn);
+       aggcombinefn = PQgetvalue(res, 0, i_aggcombinefn);
        aggmtransfn = PQgetvalue(res, 0, i_aggmtransfn);
        aggminvtransfn = PQgetvalue(res, 0, i_aggminvtransfn);
        aggmfinalfn = PQgetvalue(res, 0, i_aggmfinalfn);
@@ -12644,6 +12667,11 @@ dumpAgg(Archive *fout, AggInfo *agginfo)
                        appendPQExpBufferStr(details, ",\n    FINALFUNC_EXTRA");
        }
 
+       if (strcmp(aggcombinefn, "-") != 0)
+       {
+               appendPQExpBuffer(details, ",\n    COMBINEFUNC = %s",   aggcombinefn);
+       }
+
        if (strcmp(aggmtransfn, "-") != 0)
        {
                appendPQExpBuffer(details, ",\n    MSFUNC = %s,\n    MINVFUNC = %s,\n    MSTYPE = %s",
index 548d49440aa3f5a24e2809dbfd57b4b0d396ee3f..58709f56ac50e0984de6f4062ba8d17d1aa3c333 100644 (file)
@@ -53,6 +53,6 @@
  */
 
 /*                                                     yyyymmddN */
-#define CATALOG_VERSION_NO     201601192
+#define CATALOG_VERSION_NO     201601201
 
 #endif
index 28b0669b462429e2831d8b5102f6066d1daebc11..441db304dfb1b3918615b1944dac95712e1aaae7 100644 (file)
@@ -33,6 +33,7 @@
  *     aggnumdirectargs        number of arguments that are "direct" arguments
  *     aggtransfn                      transition function
  *     aggfinalfn                      final function (0 if none)
+ *     aggcombinefn            combine function (0 if none)
  *     aggmtransfn                     forward function for moving-aggregate mode (0 if none)
  *     aggminvtransfn          inverse function for moving-aggregate mode (0 if none)
  *     aggmfinalfn                     final function for moving-aggregate mode (0 if none)
@@ -56,6 +57,7 @@ CATALOG(pg_aggregate,2600) BKI_WITHOUT_OIDS
        int16           aggnumdirectargs;
        regproc         aggtransfn;
        regproc         aggfinalfn;
+       regproc         aggcombinefn;
        regproc         aggmtransfn;
        regproc         aggminvtransfn;
        regproc         aggmfinalfn;
@@ -85,24 +87,25 @@ typedef FormData_pg_aggregate *Form_pg_aggregate;
  * ----------------
  */
 
-#define Natts_pg_aggregate                                     17
+#define Natts_pg_aggregate                                     18
 #define Anum_pg_aggregate_aggfnoid                     1
 #define Anum_pg_aggregate_aggkind                      2
 #define Anum_pg_aggregate_aggnumdirectargs     3
 #define Anum_pg_aggregate_aggtransfn           4
 #define Anum_pg_aggregate_aggfinalfn           5
-#define Anum_pg_aggregate_aggmtransfn          6
-#define Anum_pg_aggregate_aggminvtransfn       7
-#define Anum_pg_aggregate_aggmfinalfn          8
-#define Anum_pg_aggregate_aggfinalextra                9
-#define Anum_pg_aggregate_aggmfinalextra       10
-#define Anum_pg_aggregate_aggsortop                    11
-#define Anum_pg_aggregate_aggtranstype         12
-#define Anum_pg_aggregate_aggtransspace                13
-#define Anum_pg_aggregate_aggmtranstype                14
-#define Anum_pg_aggregate_aggmtransspace       15
-#define Anum_pg_aggregate_agginitval           16
-#define Anum_pg_aggregate_aggminitval          17
+#define Anum_pg_aggregate_aggcombinefn         6
+#define Anum_pg_aggregate_aggmtransfn          7
+#define Anum_pg_aggregate_aggminvtransfn       8
+#define Anum_pg_aggregate_aggmfinalfn          9
+#define Anum_pg_aggregate_aggfinalextra                10
+#define Anum_pg_aggregate_aggmfinalextra       11
+#define Anum_pg_aggregate_aggsortop                    12
+#define Anum_pg_aggregate_aggtranstype         13
+#define Anum_pg_aggregate_aggtransspace                14
+#define Anum_pg_aggregate_aggmtranstype                15
+#define Anum_pg_aggregate_aggmtransspace       16
+#define Anum_pg_aggregate_agginitval           17
+#define Anum_pg_aggregate_aggminitval          18
 
 /*
  * Symbolic values for aggkind column.  We distinguish normal aggregates
@@ -126,184 +129,184 @@ typedef FormData_pg_aggregate *Form_pg_aggregate;
  */
 
 /* avg */
-DATA(insert ( 2100     n 0 int8_avg_accum      numeric_poly_avg                int8_avg_accum  int8_avg_accum_inv      numeric_poly_avg        f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2101     n 0 int4_avg_accum      int8_avg                int4_avg_accum  int4_avg_accum_inv      int8_avg                                        f f 0   1016    0       1016    0       "{0,0}" "{0,0}" ));
-DATA(insert ( 2102     n 0 int2_avg_accum      int8_avg                int2_avg_accum  int2_avg_accum_inv      int8_avg                                        f f 0   1016    0       1016    0       "{0,0}" "{0,0}" ));
-DATA(insert ( 2103     n 0 numeric_avg_accum numeric_avg       numeric_avg_accum numeric_accum_inv numeric_avg                                 f f 0   2281    128 2281        128 _null_ _null_ ));
-DATA(insert ( 2104     n 0 float4_accum        float8_avg              -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2105     n 0 float8_accum        float8_avg              -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2106     n 0 interval_accum      interval_avg    interval_accum  interval_accum_inv interval_avg                                 f f 0   1187    0       1187    0       "{0 second,0 second}" "{0 second,0 second}" ));
+DATA(insert ( 2100     n 0 int8_avg_accum      numeric_poly_avg        -       int8_avg_accum  int8_avg_accum_inv      numeric_poly_avg        f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2101     n 0 int4_avg_accum      int8_avg                        -       int4_avg_accum  int4_avg_accum_inv      int8_avg                        f f 0   1016    0       1016    0       "{0,0}" "{0,0}" ));
+DATA(insert ( 2102     n 0 int2_avg_accum      int8_avg                        -       int2_avg_accum  int2_avg_accum_inv      int8_avg                        f f 0   1016    0       1016    0       "{0,0}" "{0,0}" ));
+DATA(insert ( 2103     n 0 numeric_avg_accum numeric_avg               -       numeric_avg_accum numeric_accum_inv numeric_avg                 f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2104     n 0 float4_accum        float8_avg                      -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2105     n 0 float8_accum        float8_avg                      -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2106     n 0 interval_accum      interval_avg            -       interval_accum  interval_accum_inv interval_avg                 f f 0   1187    0       1187    0       "{0 second,0 second}" "{0 second,0 second}" ));
 
 /* sum */
-DATA(insert ( 2107     n 0 int8_avg_accum      numeric_poly_sum                int8_avg_accum  int8_avg_accum_inv numeric_poly_sum f f 0       2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2108     n 0 int4_sum            -                               int4_avg_accum  int4_avg_accum_inv int2int4_sum                                 f f 0   20              0       1016    0       _null_ "{0,0}" ));
-DATA(insert ( 2109     n 0 int2_sum            -                               int2_avg_accum  int2_avg_accum_inv int2int4_sum                                 f f 0   20              0       1016    0       _null_ "{0,0}" ));
-DATA(insert ( 2110     n 0 float4pl            -                               -                               -                               -                                                               f f 0   700             0       0               0       _null_ _null_ ));
-DATA(insert ( 2111     n 0 float8pl            -                               -                               -                               -                                                               f f 0   701             0       0               0       _null_ _null_ ));
-DATA(insert ( 2112     n 0 cash_pl                     -                               cash_pl                 cash_mi                 -                                                               f f 0   790             0       790             0       _null_ _null_ ));
-DATA(insert ( 2113     n 0 interval_pl         -                               interval_pl             interval_mi             -                                                               f f 0   1186    0       1186    0       _null_ _null_ ));
-DATA(insert ( 2114     n 0 numeric_avg_accum   numeric_sum numeric_avg_accum numeric_accum_inv numeric_sum                                     f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2107     n 0 int8_avg_accum      numeric_poly_sum        -                                       int8_avg_accum  int8_avg_accum_inv      numeric_poly_sum        f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2108     n 0 int4_sum            -                                       int8pl                          int4_avg_accum  int4_avg_accum_inv      int2int4_sum            f f 0   20              0       1016    0       _null_ "{0,0}" ));
+DATA(insert ( 2109     n 0 int2_sum            -                                       int8pl                          int2_avg_accum  int2_avg_accum_inv      int2int4_sum            f f 0   20              0       1016    0       _null_ "{0,0}" ));
+DATA(insert ( 2110     n 0 float4pl            -                                       float4pl                        -                               -                                       -                                       f f 0   700             0       0               0       _null_ _null_ ));
+DATA(insert ( 2111     n 0 float8pl            -                                       float8pl                        -                               -                                       -                                       f f 0   701             0       0               0       _null_ _null_ ));
+DATA(insert ( 2112     n 0 cash_pl                     -                                       cash_pl                         cash_pl                 cash_mi                         -                                       f f 0   790             0       790             0       _null_ _null_ ));
+DATA(insert ( 2113     n 0 interval_pl         -                                       interval_pl                     interval_pl             interval_mi                     -                                       f f 0   1186    0       1186    0       _null_ _null_ ));
+DATA(insert ( 2114     n 0 numeric_avg_accum   numeric_sum             -                                       numeric_avg_accum       numeric_accum_inv       numeric_sum             f f 0   2281    128 2281        128 _null_ _null_ ));
 
 /* max */
-DATA(insert ( 2115     n 0 int8larger          -                               -                               -                               -                               f f 413         20              0       0               0       _null_ _null_ ));
-DATA(insert ( 2116     n 0 int4larger          -                               -                               -                               -                               f f 521         23              0       0               0       _null_ _null_ ));
-DATA(insert ( 2117     n 0 int2larger          -                               -                               -                               -                               f f 520         21              0       0               0       _null_ _null_ ));
-DATA(insert ( 2118     n 0 oidlarger           -                               -                               -                               -                               f f 610         26              0       0               0       _null_ _null_ ));
-DATA(insert ( 2119     n 0 float4larger        -                               -                               -                               -                               f f 623         700             0       0               0       _null_ _null_ ));
-DATA(insert ( 2120     n 0 float8larger        -                               -                               -                               -                               f f 674         701             0       0               0       _null_ _null_ ));
-DATA(insert ( 2121     n 0 int4larger          -                               -                               -                               -                               f f 563         702             0       0               0       _null_ _null_ ));
-DATA(insert ( 2122     n 0 date_larger         -                               -                               -                               -                               f f 1097        1082    0       0               0       _null_ _null_ ));
-DATA(insert ( 2123     n 0 time_larger         -                               -                               -                               -                               f f 1112        1083    0       0               0       _null_ _null_ ));
-DATA(insert ( 2124     n 0 timetz_larger       -                               -                               -                               -                               f f 1554        1266    0       0               0       _null_ _null_ ));
-DATA(insert ( 2125     n 0 cashlarger          -                               -                               -                               -                               f f 903         790             0       0               0       _null_ _null_ ));
-DATA(insert ( 2126     n 0 timestamp_larger    -                       -                               -                               -                               f f 2064        1114    0       0               0       _null_ _null_ ));
-DATA(insert ( 2127     n 0 timestamptz_larger  -                       -                               -                               -                               f f 1324        1184    0       0               0       _null_ _null_ ));
-DATA(insert ( 2128     n 0 interval_larger -                           -                               -                               -                               f f 1334        1186    0       0               0       _null_ _null_ ));
-DATA(insert ( 2129     n 0 text_larger         -                               -                               -                               -                               f f 666         25              0       0               0       _null_ _null_ ));
-DATA(insert ( 2130     n 0 numeric_larger      -                               -                               -                               -                               f f 1756        1700    0       0               0       _null_ _null_ ));
-DATA(insert ( 2050     n 0 array_larger        -                               -                               -                               -                               f f 1073        2277    0       0               0       _null_ _null_ ));
-DATA(insert ( 2244     n 0 bpchar_larger       -                               -                               -                               -                               f f 1060        1042    0       0               0       _null_ _null_ ));
-DATA(insert ( 2797     n 0 tidlarger           -                               -                               -                               -                               f f 2800        27              0       0               0       _null_ _null_ ));
-DATA(insert ( 3526     n 0 enum_larger         -                               -                               -                               -                               f f 3519        3500    0       0               0       _null_ _null_ ));
-DATA(insert ( 3564     n 0 network_larger      -                               -                               -                               -                               f f 1205        869             0       0               0       _null_ _null_ ));
+DATA(insert ( 2115     n 0 int8larger          -                               int8larger                      -                               -                               -                               f f 413         20              0       0               0       _null_ _null_ ));
+DATA(insert ( 2116     n 0 int4larger          -                               int4larger                      -                               -                               -                               f f 521         23              0       0               0       _null_ _null_ ));
+DATA(insert ( 2117     n 0 int2larger          -                               int2larger                      -                               -                               -                               f f 520         21              0       0               0       _null_ _null_ ));
+DATA(insert ( 2118     n 0 oidlarger           -                               oidlarger                       -                               -                               -                               f f 610         26              0       0               0       _null_ _null_ ));
+DATA(insert ( 2119     n 0 float4larger        -                               float4larger            -                               -                               -                               f f 623         700             0       0               0       _null_ _null_ ));
+DATA(insert ( 2120     n 0 float8larger        -                               float8larger            -                               -                               -                               f f 674         701             0       0               0       _null_ _null_ ));
+DATA(insert ( 2121     n 0 int4larger          -                               int4larger                      -                               -                               -                               f f 563         702             0       0               0       _null_ _null_ ));
+DATA(insert ( 2122     n 0 date_larger         -                               date_larger                     -                               -                               -                               f f 1097        1082    0       0               0       _null_ _null_ ));
+DATA(insert ( 2123     n 0 time_larger         -                               time_larger                     -                               -                               -                               f f 1112        1083    0       0               0       _null_ _null_ ));
+DATA(insert ( 2124     n 0 timetz_larger       -                               timetz_larger           -                               -                               -                               f f 1554        1266    0       0               0       _null_ _null_ ));
+DATA(insert ( 2125     n 0 cashlarger          -                               cashlarger                      -                               -                               -                               f f 903         790             0       0               0       _null_ _null_ ));
+DATA(insert ( 2126     n 0 timestamp_larger    -                       timestamp_larger        -                               -                               -                               f f 2064        1114    0       0               0       _null_ _null_ ));
+DATA(insert ( 2127     n 0 timestamptz_larger  -                       timestamptz_larger      -                               -                               -                               f f 1324        1184    0       0               0       _null_ _null_ ));
+DATA(insert ( 2128     n 0 interval_larger -                           interval_larger         -                               -                               -                               f f 1334        1186    0       0               0       _null_ _null_ ));
+DATA(insert ( 2129     n 0 text_larger         -                               text_larger                     -                               -                               -                               f f 666         25              0       0               0       _null_ _null_ ));
+DATA(insert ( 2130     n 0 numeric_larger      -                               numeric_larger          -                               -                               -                               f f 1756        1700    0       0               0       _null_ _null_ ));
+DATA(insert ( 2050     n 0 array_larger        -                               array_larger            -                               -                               -                               f f 1073        2277    0       0               0       _null_ _null_ ));
+DATA(insert ( 2244     n 0 bpchar_larger       -                               bpchar_larger           -                               -                               -                               f f 1060        1042    0       0               0       _null_ _null_ ));
+DATA(insert ( 2797     n 0 tidlarger           -                               tidlarger                       -                               -                               -                               f f 2800        27              0       0               0       _null_ _null_ ));
+DATA(insert ( 3526     n 0 enum_larger         -                               enum_larger                     -                               -                               -                               f f 3519        3500    0       0               0       _null_ _null_ ));
+DATA(insert ( 3564     n 0 network_larger      -                               network_larger          -                               -                               -                               f f 1205        869             0       0               0       _null_ _null_ ));
 
 /* min */
-DATA(insert ( 2131     n 0 int8smaller         -                               -                               -                               -                               f f 412         20              0       0               0       _null_ _null_ ));
-DATA(insert ( 2132     n 0 int4smaller         -                               -                               -                               -                               f f 97          23              0       0               0       _null_ _null_ ));
-DATA(insert ( 2133     n 0 int2smaller         -                               -                               -                               -                               f f 95          21              0       0               0       _null_ _null_ ));
-DATA(insert ( 2134     n 0 oidsmaller          -                               -                               -                               -                               f f 609         26              0       0               0       _null_ _null_ ));
-DATA(insert ( 2135     n 0 float4smaller       -                               -                               -                               -                               f f 622         700             0       0               0       _null_ _null_ ));
-DATA(insert ( 2136     n 0 float8smaller       -                               -                               -                               -                               f f 672         701             0       0               0       _null_ _null_ ));
-DATA(insert ( 2137     n 0 int4smaller         -                               -                               -                               -                               f f 562         702             0       0               0       _null_ _null_ ));
-DATA(insert ( 2138     n 0 date_smaller        -                               -                               -                               -                               f f 1095        1082    0       0               0       _null_ _null_ ));
-DATA(insert ( 2139     n 0 time_smaller        -                               -                               -                               -                               f f 1110        1083    0       0               0       _null_ _null_ ));
-DATA(insert ( 2140     n 0 timetz_smaller      -                               -                               -                               -                               f f 1552        1266    0       0               0       _null_ _null_ ));
-DATA(insert ( 2141     n 0 cashsmaller         -                               -                               -                               -                               f f 902         790             0       0               0       _null_ _null_ ));
-DATA(insert ( 2142     n 0 timestamp_smaller   -                       -                               -                               -                               f f 2062        1114    0       0               0       _null_ _null_ ));
-DATA(insert ( 2143     n 0 timestamptz_smaller -                       -                               -                               -                               f f 1322        1184    0       0               0       _null_ _null_ ));
-DATA(insert ( 2144     n 0 interval_smaller    -                       -                               -                               -                               f f 1332        1186    0       0               0       _null_ _null_ ));
-DATA(insert ( 2145     n 0 text_smaller        -                               -                               -                               -                               f f 664         25              0       0               0       _null_ _null_ ));
-DATA(insert ( 2146     n 0 numeric_smaller -                           -                               -                               -                               f f 1754        1700    0       0               0       _null_ _null_ ));
-DATA(insert ( 2051     n 0 array_smaller       -                               -                               -                               -                               f f 1072        2277    0       0               0       _null_ _null_ ));
-DATA(insert ( 2245     n 0 bpchar_smaller      -                               -                               -                               -                               f f 1058        1042    0       0               0       _null_ _null_ ));
-DATA(insert ( 2798     n 0 tidsmaller          -                               -                               -                               -                               f f 2799        27              0       0               0       _null_ _null_ ));
-DATA(insert ( 3527     n 0 enum_smaller        -                               -                               -                               -                               f f 3518        3500    0       0               0       _null_ _null_ ));
-DATA(insert ( 3565     n 0 network_smaller -                           -                               -                               -                               f f 1203        869             0       0               0       _null_ _null_ ));
+DATA(insert ( 2131     n 0 int8smaller         -                               int8smaller                     -                               -                               -                               f f 412         20              0       0               0       _null_ _null_ ));
+DATA(insert ( 2132     n 0 int4smaller         -                               int4smaller                     -                               -                               -                               f f 97          23              0       0               0       _null_ _null_ ));
+DATA(insert ( 2133     n 0 int2smaller         -                               int2smaller                     -                               -                               -                               f f 95          21              0       0               0       _null_ _null_ ));
+DATA(insert ( 2134     n 0 oidsmaller          -                               oidsmaller                      -                               -                               -                               f f 609         26              0       0               0       _null_ _null_ ));
+DATA(insert ( 2135     n 0 float4smaller       -                               float4smaller           -                               -                               -                               f f 622         700             0       0               0       _null_ _null_ ));
+DATA(insert ( 2136     n 0 float8smaller       -                               float8smaller           -                               -                               -                               f f 672         701             0       0               0       _null_ _null_ ));
+DATA(insert ( 2137     n 0 int4smaller         -                               int4smaller                     -                               -                               -                               f f 562         702             0       0               0       _null_ _null_ ));
+DATA(insert ( 2138     n 0 date_smaller        -                               date_smaller            -                               -                               -                               f f 1095        1082    0       0               0       _null_ _null_ ));
+DATA(insert ( 2139     n 0 time_smaller        -                               time_smaller            -                               -                               -                               f f 1110        1083    0       0               0       _null_ _null_ ));
+DATA(insert ( 2140     n 0 timetz_smaller      -                               timetz_smaller          -                               -                               -                               f f 1552        1266    0       0               0       _null_ _null_ ));
+DATA(insert ( 2141     n 0 cashsmaller         -                               cashsmaller                     -                               -                               -                               f f 902         790             0       0               0       _null_ _null_ ));
+DATA(insert ( 2142     n 0 timestamp_smaller   -                       timestamp_smaller       -                               -                               -                               f f 2062        1114    0       0               0       _null_ _null_ ));
+DATA(insert ( 2143     n 0 timestamptz_smaller -                       timestamptz_smaller     -                               -                               -                               f f 1322        1184    0       0               0       _null_ _null_ ));
+DATA(insert ( 2144     n 0 interval_smaller    -                       interval_smaller        -                               -                               -                               f f 1332        1186    0       0               0       _null_ _null_ ));
+DATA(insert ( 2145     n 0 text_smaller        -                               text_smaller            -                               -                               -                               f f 664         25              0       0               0       _null_ _null_ ));
+DATA(insert ( 2146     n 0 numeric_smaller -                           numeric_smaller         -                               -                               -                               f f 1754        1700    0       0               0       _null_ _null_ ));
+DATA(insert ( 2051     n 0 array_smaller       -                               array_smaller           -                               -                               -                               f f 1072        2277    0       0               0       _null_ _null_ ));
+DATA(insert ( 2245     n 0 bpchar_smaller      -                               bpchar_smaller          -                               -                               -                               f f 1058        1042    0       0               0       _null_ _null_ ));
+DATA(insert ( 2798     n 0 tidsmaller          -                               tidsmaller                      -                               -                               -                               f f 2799        27              0       0               0       _null_ _null_ ));
+DATA(insert ( 3527     n 0 enum_smaller        -                               enum_smaller            -                               -                               -                               f f 3518        3500    0       0               0       _null_ _null_ ));
+DATA(insert ( 3565     n 0 network_smaller -                           network_smaller         -                               -                               -                               f f 1203        869             0       0               0       _null_ _null_ ));
 
 /* count */
-DATA(insert ( 2147     n 0 int8inc_any         -                               int8inc_any             int8dec_any             -                               f f 0           20              0       20              0       "0" "0" ));
-DATA(insert ( 2803     n 0 int8inc                     -                               int8inc                 int8dec                 -                               f f 0           20              0       20              0       "0" "0" ));
+DATA(insert ( 2147     n 0 int8inc_any         -                               int8pl  int8inc_any             int8dec_any             -                               f f 0           20              0       20              0       "0" "0" ));
+DATA(insert ( 2803     n 0 int8inc                     -                               int8pl  int8inc                 int8dec                 -                               f f 0           20              0       20              0       "0" "0" ));
 
 /* var_pop */
-DATA(insert ( 2718     n 0 int8_accum  numeric_var_pop         int8_accum              int8_accum_inv  numeric_var_pop                                 f f 0   2281    128 2281        128 _null_ _null_ ));
-DATA(insert ( 2719     n 0 int4_accum  numeric_poly_var_pop            int4_accum              int4_accum_inv  numeric_poly_var_pop    f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2720     n 0 int2_accum  numeric_poly_var_pop            int2_accum              int2_accum_inv  numeric_poly_var_pop    f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2721     n 0 float4_accum        float8_var_pop  -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2722     n 0 float8_accum        float8_var_pop  -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2723     n 0 numeric_accum       numeric_var_pop numeric_accum numeric_accum_inv numeric_var_pop                                 f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2718     n 0 int8_accum  numeric_var_pop                 -       int8_accum              int8_accum_inv  numeric_var_pop                 f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2719     n 0 int4_accum  numeric_poly_var_pop    -       int4_accum              int4_accum_inv  numeric_poly_var_pop    f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2720     n 0 int2_accum  numeric_poly_var_pop    -       int2_accum              int2_accum_inv  numeric_poly_var_pop    f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2721     n 0 float4_accum        float8_var_pop          -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2722     n 0 float8_accum        float8_var_pop          -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2723     n 0 numeric_accum       numeric_var_pop         -       numeric_accum numeric_accum_inv numeric_var_pop                 f f 0   2281    128 2281        128 _null_ _null_ ));
 
 /* var_samp */
-DATA(insert ( 2641     n 0 int8_accum  numeric_var_samp        int8_accum              int8_accum_inv  numeric_var_samp                                f f 0   2281    128 2281        128 _null_ _null_ ));
-DATA(insert ( 2642     n 0 int4_accum  numeric_poly_var_samp           int4_accum              int4_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2643     n 0 int2_accum  numeric_poly_var_samp           int2_accum              int2_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2644     n 0 float4_accum        float8_var_samp -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2645     n 0 float8_accum        float8_var_samp -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2646     n 0 numeric_accum       numeric_var_samp numeric_accum numeric_accum_inv numeric_var_samp                               f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2641     n 0 int8_accum  numeric_var_samp                -       int8_accum              int8_accum_inv  numeric_var_samp                f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2642     n 0 int4_accum  numeric_poly_var_samp   -       int4_accum              int4_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2643     n 0 int2_accum  numeric_poly_var_samp   -       int2_accum              int2_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2644     n 0 float4_accum        float8_var_samp         -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2645     n 0 float8_accum        float8_var_samp         -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2646     n 0 numeric_accum       numeric_var_samp        -       numeric_accum   numeric_accum_inv numeric_var_samp              f f 0   2281    128 2281        128 _null_ _null_ ));
 
 /* variance: historical Postgres syntax for var_samp */
-DATA(insert ( 2148     n 0 int8_accum  numeric_var_samp        int8_accum              int8_accum_inv  numeric_var_samp                                f f 0   2281    128 2281        128 _null_ _null_ ));
-DATA(insert ( 2149     n 0 int4_accum  numeric_poly_var_samp           int4_accum              int4_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2150     n 0 int2_accum  numeric_poly_var_samp           int2_accum              int2_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2151     n 0 float4_accum        float8_var_samp -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2152     n 0 float8_accum        float8_var_samp -                               -                               -                                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2153     n 0 numeric_accum       numeric_var_samp numeric_accum numeric_accum_inv numeric_var_samp                               f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2148     n 0 int8_accum  numeric_var_samp                -       int8_accum              int8_accum_inv  numeric_var_samp                f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2149     n 0 int4_accum  numeric_poly_var_samp   -       int4_accum              int4_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2150     n 0 int2_accum  numeric_poly_var_samp   -       int2_accum              int2_accum_inv  numeric_poly_var_samp   f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2151     n 0 float4_accum        float8_var_samp         -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2152     n 0 float8_accum        float8_var_samp         -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2153     n 0 numeric_accum       numeric_var_samp        -       numeric_accum numeric_accum_inv numeric_var_samp                f f 0   2281    128 2281        128 _null_ _null_ ));
 
 /* stddev_pop */
-DATA(insert ( 2724     n 0 int8_accum  numeric_stddev_pop      int8_accum      int8_accum_inv  numeric_stddev_pop                                      f f 0   2281    128 2281        128 _null_ _null_ ));
-DATA(insert ( 2725     n 0 int4_accum  numeric_poly_stddev_pop int4_accum      int4_accum_inv  numeric_poly_stddev_pop f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2726     n 0 int2_accum  numeric_poly_stddev_pop int2_accum      int2_accum_inv  numeric_poly_stddev_pop f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2727     n 0 float4_accum        float8_stddev_pop       -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2728     n 0 float8_accum        float8_stddev_pop       -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2729     n 0 numeric_accum       numeric_stddev_pop numeric_accum numeric_accum_inv numeric_stddev_pop                   f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2724     n 0 int8_accum  numeric_stddev_pop              -       int8_accum      int8_accum_inv  numeric_stddev_pop              f f 0   2281    128     2281    128 _null_ _null_ ));
+DATA(insert ( 2725     n 0 int4_accum  numeric_poly_stddev_pop -       int4_accum      int4_accum_inv  numeric_poly_stddev_pop f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2726     n 0 int2_accum  numeric_poly_stddev_pop -       int2_accum      int2_accum_inv  numeric_poly_stddev_pop f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2727     n 0 float4_accum        float8_stddev_pop       -       -                               -                               -                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2728     n 0 float8_accum        float8_stddev_pop       -       -                               -                               -                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2729     n 0 numeric_accum       numeric_stddev_pop      -       numeric_accum numeric_accum_inv numeric_stddev_pop      f f 0   2281    128 2281        128 _null_ _null_ ));
 
 /* stddev_samp */
-DATA(insert ( 2712     n 0 int8_accum  numeric_stddev_samp             int8_accum      int8_accum_inv  numeric_stddev_samp                             f f 0   2281    128 2281        128 _null_ _null_ ));
-DATA(insert ( 2713     n 0 int4_accum  numeric_poly_stddev_samp        int4_accum      int4_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2714     n 0 int2_accum  numeric_poly_stddev_samp        int2_accum      int2_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2715     n 0 float4_accum        float8_stddev_samp      -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2716     n 0 float8_accum        float8_stddev_samp      -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2717     n 0 numeric_accum       numeric_stddev_samp numeric_accum numeric_accum_inv numeric_stddev_samp                 f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2712     n 0 int8_accum  numeric_stddev_samp                     -       int8_accum      int8_accum_inv  numeric_stddev_samp                     f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2713     n 0 int4_accum  numeric_poly_stddev_samp        -       int4_accum      int4_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2714     n 0 int2_accum  numeric_poly_stddev_samp        -       int2_accum      int2_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2715     n 0 float4_accum        float8_stddev_samp              -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2716     n 0 float8_accum        float8_stddev_samp              -       -                               -                               -                                               f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2717     n 0 numeric_accum       numeric_stddev_samp             -       numeric_accum numeric_accum_inv numeric_stddev_samp             f f 0   2281    128 2281        128 _null_ _null_ ));
 
 /* stddev: historical Postgres syntax for stddev_samp */
-DATA(insert ( 2154     n 0 int8_accum  numeric_stddev_samp             int8_accum      int8_accum_inv  numeric_stddev_samp                             f f 0   2281    128 2281        128 _null_ _null_ ));
-DATA(insert ( 2155     n 0 int4_accum  numeric_poly_stddev_samp        int4_accum      int4_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2156     n 0 int2_accum  numeric_poly_stddev_samp        int2_accum      int2_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
-DATA(insert ( 2157     n 0 float4_accum        float8_stddev_samp      -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2158     n 0 float8_accum        float8_stddev_samp      -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
-DATA(insert ( 2159     n 0 numeric_accum       numeric_stddev_samp numeric_accum numeric_accum_inv numeric_stddev_samp                 f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2154     n 0 int8_accum  numeric_stddev_samp                     -       int8_accum              int8_accum_inv  numeric_stddev_samp                     f f 0   2281    128 2281        128 _null_ _null_ ));
+DATA(insert ( 2155     n 0 int4_accum  numeric_poly_stddev_samp        -       int4_accum              int4_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2156     n 0 int2_accum  numeric_poly_stddev_samp        -       int2_accum              int2_accum_inv  numeric_poly_stddev_samp        f f 0   2281    48      2281    48      _null_ _null_ ));
+DATA(insert ( 2157     n 0 float4_accum        float8_stddev_samp              -       -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2158     n 0 float8_accum        float8_stddev_samp              -       -                               -                               -                                                       f f 0   1022    0       0               0       "{0,0,0}" _null_ ));
+DATA(insert ( 2159     n 0 numeric_accum       numeric_stddev_samp             -       numeric_accum   numeric_accum_inv numeric_stddev_samp           f f 0   2281    128 2281        128 _null_ _null_ ));
 
 /* SQL2003 binary regression aggregates */
-DATA(insert ( 2818     n 0 int8inc_float8_float8       -                                       -                               -                               -                               f f 0   20              0       0               0       "0" _null_ ));
-DATA(insert ( 2819     n 0 float8_regr_accum   float8_regr_sxx                 -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2820     n 0 float8_regr_accum   float8_regr_syy                 -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2821     n 0 float8_regr_accum   float8_regr_sxy                 -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2822     n 0 float8_regr_accum   float8_regr_avgx                -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2823     n 0 float8_regr_accum   float8_regr_avgy                -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2824     n 0 float8_regr_accum   float8_regr_r2                  -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2825     n 0 float8_regr_accum   float8_regr_slope               -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2826     n 0 float8_regr_accum   float8_regr_intercept   -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2827     n 0 float8_regr_accum   float8_covar_pop                -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2828     n 0 float8_regr_accum   float8_covar_samp               -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2829     n 0 float8_regr_accum   float8_corr                             -                               -                               -                               f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2818     n 0 int8inc_float8_float8       -                                       -       -                               -                               -                       f f 0   20              0       0               0       "0" _null_ ));
+DATA(insert ( 2819     n 0 float8_regr_accum   float8_regr_sxx                 -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2820     n 0 float8_regr_accum   float8_regr_syy                 -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2821     n 0 float8_regr_accum   float8_regr_sxy                 -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2822     n 0 float8_regr_accum   float8_regr_avgx                -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2823     n 0 float8_regr_accum   float8_regr_avgy                -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2824     n 0 float8_regr_accum   float8_regr_r2                  -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2825     n 0 float8_regr_accum   float8_regr_slope               -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2826     n 0 float8_regr_accum   float8_regr_intercept   -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2827     n 0 float8_regr_accum   float8_covar_pop                -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2828     n 0 float8_regr_accum   float8_covar_samp               -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2829     n 0 float8_regr_accum   float8_corr                             -       -                               -                               -                       f f 0   1022    0       0               0       "{0,0,0,0,0,0}" _null_ ));
 
 /* boolean-and and boolean-or */
-DATA(insert ( 2517     n 0 booland_statefunc   -                       bool_accum              bool_accum_inv  bool_alltrue    f f 58  16              0       2281    16      _null_ _null_ ));
-DATA(insert ( 2518     n 0 boolor_statefunc    -                       bool_accum              bool_accum_inv  bool_anytrue    f f 59  16              0       2281    16      _null_ _null_ ));
-DATA(insert ( 2519     n 0 booland_statefunc   -                       bool_accum              bool_accum_inv  bool_alltrue    f f 58  16              0       2281    16      _null_ _null_ ));
+DATA(insert ( 2517     n 0 booland_statefunc   -       -       bool_accum      bool_accum_inv  bool_alltrue    f f 58  16              0       2281    16      _null_ _null_ ));
+DATA(insert ( 2518     n 0 boolor_statefunc    -       -       bool_accum      bool_accum_inv  bool_anytrue    f f 59  16              0       2281    16      _null_ _null_ ));
+DATA(insert ( 2519     n 0 booland_statefunc   -       -       bool_accum      bool_accum_inv  bool_alltrue    f f 58  16              0       2281    16      _null_ _null_ ));
 
 /* bitwise integer */
-DATA(insert ( 2236     n 0 int2and             -                                       -                               -                               -                               f f 0   21              0       0               0       _null_ _null_ ));
-DATA(insert ( 2237     n 0 int2or              -                                       -                               -                               -                               f f 0   21              0       0               0       _null_ _null_ ));
-DATA(insert ( 2238     n 0 int4and             -                                       -                               -                               -                               f f 0   23              0       0               0       _null_ _null_ ));
-DATA(insert ( 2239     n 0 int4or              -                                       -                               -                               -                               f f 0   23              0       0               0       _null_ _null_ ));
-DATA(insert ( 2240     n 0 int8and             -                                       -                               -                               -                               f f 0   20              0       0               0       _null_ _null_ ));
-DATA(insert ( 2241     n 0 int8or              -                                       -                               -                               -                               f f 0   20              0       0               0       _null_ _null_ ));
-DATA(insert ( 2242     n 0 bitand              -                                       -                               -                               -                               f f 0   1560    0       0               0       _null_ _null_ ));
-DATA(insert ( 2243     n 0 bitor               -                                       -                               -                               -                               f f 0   1560    0       0               0       _null_ _null_ ));
+DATA(insert ( 2236     n 0 int2and             -                               int2and -                               -                               -                               f f 0   21              0       0               0       _null_ _null_ ));
+DATA(insert ( 2237     n 0 int2or              -                               int2or  -                               -                               -                               f f 0   21              0       0               0       _null_ _null_ ));
+DATA(insert ( 2238     n 0 int4and             -                               int4and -                               -                               -                               f f 0   23              0       0               0       _null_ _null_ ));
+DATA(insert ( 2239     n 0 int4or              -                               int4or  -                               -                               -                               f f 0   23              0       0               0       _null_ _null_ ));
+DATA(insert ( 2240     n 0 int8and             -                               int8and -                               -                               -                               f f 0   20              0       0               0       _null_ _null_ ));
+DATA(insert ( 2241     n 0 int8or              -                               int8or  -                               -                               -                               f f 0   20              0       0               0       _null_ _null_ ));
+DATA(insert ( 2242     n 0 bitand              -                               bitand  -                               -                               -                               f f 0   1560    0       0               0       _null_ _null_ ));
+DATA(insert ( 2243     n 0 bitor               -                               bitor   -                               -                               -                               f f 0   1560    0       0               0       _null_ _null_ ));
 
 /* xml */
-DATA(insert ( 2901     n 0 xmlconcat2  -                                       -                               -                               -                               f f 0   142             0       0               0       _null_ _null_ ));
+DATA(insert ( 2901     n 0 xmlconcat2  -                               -               -                               -                               -                               f f 0   142             0       0               0       _null_ _null_ ));
 
 /* array */
-DATA(insert ( 2335     n 0 array_agg_transfn   array_agg_finalfn       -                               -                               -                               t f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 4053     n 0 array_agg_array_transfn array_agg_array_finalfn -           -                               -                               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 2335     n 0 array_agg_transfn           array_agg_finalfn               -       -               -                               -                               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 4053     n 0 array_agg_array_transfn array_agg_array_finalfn     -       -               -                               -                               t f 0   2281    0       0               0       _null_ _null_ ));
 
 /* text */
-DATA(insert ( 3538     n 0 string_agg_transfn  string_agg_finalfn      -                               -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3538     n 0 string_agg_transfn  string_agg_finalfn      -       -                               -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
 
 /* bytea */
-DATA(insert ( 3545     n 0 bytea_string_agg_transfn    bytea_string_agg_finalfn        -                               -                               -               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3545     n 0 bytea_string_agg_transfn    bytea_string_agg_finalfn        -       -                               -                               -               f f 0   2281    0       0               0       _null_ _null_ ));
 
 /* json */
-DATA(insert ( 3175     n 0 json_agg_transfn    json_agg_finalfn                        -                               -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3197     n 0 json_object_agg_transfn json_object_agg_finalfn -                           -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3175     n 0 json_agg_transfn    json_agg_finalfn                        -       -                               -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3197     n 0 json_object_agg_transfn json_object_agg_finalfn -   -                               -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
 
 /* jsonb */
-DATA(insert ( 3267     n 0 jsonb_agg_transfn   jsonb_agg_finalfn                       -                               -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3270     n 0 jsonb_object_agg_transfn jsonb_object_agg_finalfn -                         -                               -                               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3267     n 0 jsonb_agg_transfn   jsonb_agg_finalfn                               -       -                               -                               -                       f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3270     n 0 jsonb_object_agg_transfn jsonb_object_agg_finalfn   -       -                               -                               -                       f f 0   2281    0       0               0       _null_ _null_ ));
 
 /* ordered-set and hypothetical-set aggregates */
-DATA(insert ( 3972     o 1 ordered_set_transition                      percentile_disc_final                                   -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3974     o 1 ordered_set_transition                      percentile_cont_float8_final                    -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3976     o 1 ordered_set_transition                      percentile_cont_interval_final                  -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3978     o 1 ordered_set_transition                      percentile_disc_multi_final                             -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3980     o 1 ordered_set_transition                      percentile_cont_float8_multi_final              -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3982     o 1 ordered_set_transition                      percentile_cont_interval_multi_final    -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3984     o 0 ordered_set_transition                      mode_final                                                              -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3986     h 1 ordered_set_transition_multi        rank_final                                                              -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3988     h 1 ordered_set_transition_multi        percent_rank_final                                              -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3990     h 1 ordered_set_transition_multi        cume_dist_final                                                 -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
-DATA(insert ( 3992     h 1 ordered_set_transition_multi        dense_rank_final                                                -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3972     o 1 ordered_set_transition                      percentile_disc_final                                   -       -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3974     o 1 ordered_set_transition                      percentile_cont_float8_final                    -       -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3976     o 1 ordered_set_transition                      percentile_cont_interval_final                  -       -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3978     o 1 ordered_set_transition                      percentile_disc_multi_final                             -       -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3980     o 1 ordered_set_transition                      percentile_cont_float8_multi_final              -       -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3982     o 1 ordered_set_transition                      percentile_cont_interval_multi_final    -       -               -               -               f f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3984     o 0 ordered_set_transition                      mode_final                                                              -       -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3986     h 1 ordered_set_transition_multi        rank_final                                                              -       -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3988     h 1 ordered_set_transition_multi        percent_rank_final                                              -       -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3990     h 1 ordered_set_transition_multi        cume_dist_final                                                 -       -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
+DATA(insert ( 3992     h 1 ordered_set_transition_multi        dense_rank_final                                                -       -               -               -               t f 0   2281    0       0               0       _null_ _null_ ));
 
 
 /*
@@ -322,6 +325,7 @@ extern ObjectAddress AggregateCreate(const char *aggName,
                                Oid variadicArgType,
                                List *aggtransfnName,
                                List *aggfinalfnName,
+                               List *aggcombinefnName,
                                List *aggmtransfnName,
                                List *aggminvtransfnName,
                                List *aggmfinalfnName,
index bfa512568168b6d4cc79ecf7065a1df86b46e403..07cd20ac504b27f3fc74cf683ae16b7744fc83af 100644 (file)
@@ -1851,6 +1851,8 @@ typedef struct AggState
        AggStatePerTrans curpertrans;   /* currently active trans state */
        bool            input_done;             /* indicates end of input */
        bool            agg_done;               /* indicates completion of Agg scan */
+       bool            combineStates;  /* input tuples contain transition states */
+       bool            finalizeAggs;   /* should we call the finalfn on agg states? */
        int                     projected_set;  /* The last projected grouping set */
        int                     current_set;    /* The current grouping set being evaluated */
        Bitmapset  *grouped_cols;       /* grouped cols in current projection */
index c92579b2b9e730621bb405a54a05c111f30abce7..e823c830115b942db42b986513082e9d93526866 100644 (file)
@@ -726,6 +726,8 @@ typedef struct Agg
        AggStrategy aggstrategy;
        int                     numCols;                /* number of grouping columns */
        AttrNumber *grpColIdx;          /* their indexes in the target list */
+       bool            combineStates;  /* input tuples contain transition states */
+       bool            finalizeAggs;   /* should we call the finalfn on agg states? */
        Oid                *grpOperators;       /* equality operators to compare with */
        long            numGroups;              /* estimated number of groups in input */
        List       *groupingSets;       /* grouping sets to use */
index 275054f3bd5bfbcc4a4101af969f1767d63dbdf7..7ae73676e8e87dde6232ef386dc46eb528e304b8 100644 (file)
@@ -60,9 +60,8 @@ extern Sort *make_sort_from_groupcols(PlannerInfo *root, List *groupcls,
 extern Agg *make_agg(PlannerInfo *root, List *tlist, List *qual,
                 AggStrategy aggstrategy, const AggClauseCosts *aggcosts,
                 int numGroupCols, AttrNumber *grpColIdx, Oid *grpOperators,
-                List *groupingSets,
-                long numGroups,
-                Plan *lefttree);
+                List *groupingSets, long numGroups, bool combineStates,
+                bool finalizeAggs, Plan *lefttree);
 extern WindowAgg *make_windowagg(PlannerInfo *root, List *tlist,
                           List *windowFuncs, Index winref,
                           int partNumCols, AttrNumber *partColIdx, Oid *partOperators,
index 3e336b93729720119d1521f0627526ea0b9caf98..699b61c528e4b6bf12ae72d330314b2b2045c6ae 100644 (file)
@@ -46,6 +46,11 @@ extern void build_aggregate_transfn_expr(Oid *agg_input_types,
                                                Expr **transfnexpr,
                                                Expr **invtransfnexpr);
 
+extern void build_aggregate_combinefn_expr(Oid agg_state_type,
+                                                                                  Oid agg_input_collation,
+                                                                                  Oid combinefn_oid,
+                                                                                  Expr **combinefnexpr);
+
 extern void build_aggregate_finalfn_expr(Oid *agg_input_types,
                                                int num_finalfn_inputs,
                                                Oid agg_state_type,
index 82a34fbcd47b4bf9e43af3a3971adbdf1925b00a..66e073d2b7b7b93ba4cd0f4c3de4eb2ce32e9069 100644 (file)
@@ -101,6 +101,24 @@ CREATE AGGREGATE sumdouble (float8)
     msfunc = float8pl,
     minvfunc = float8mi
 );
+-- Test aggregate combine function
+-- ensure create aggregate works.
+CREATE AGGREGATE mysum (int)
+(
+       stype = int,
+       sfunc = int4pl,
+       combinefunc = int4pl
+);
+-- Ensure all these functions made it into the catalog
+SELECT aggfnoid,aggtransfn,aggcombinefn,aggtranstype
+FROM pg_aggregate
+WHERE aggfnoid = 'mysum'::REGPROC;
+ aggfnoid | aggtransfn | aggcombinefn | aggtranstype 
+----------+------------+--------------+--------------
+ mysum    | int4pl     | int4pl       |           23
+(1 row)
+
+DROP AGGREGATE mysum (int);
 -- invalid: nonstrict inverse with strict forward function
 CREATE FUNCTION float8mi_n(float8, float8) RETURNS float8 AS
 $$ SELECT $1 - $2; $$
index 0ec1572dff609454bd03434fa821094c8e0bf156..dfcbc5a0e8b71c2e167a77c1976abfe322001b7b 100644 (file)
@@ -115,6 +115,23 @@ CREATE AGGREGATE sumdouble (float8)
     minvfunc = float8mi
 );
 
+-- Test aggregate combine function
+
+-- ensure create aggregate works.
+CREATE AGGREGATE mysum (int)
+(
+       stype = int,
+       sfunc = int4pl,
+       combinefunc = int4pl
+);
+
+-- Ensure all these functions made it into the catalog
+SELECT aggfnoid,aggtransfn,aggcombinefn,aggtranstype
+FROM pg_aggregate
+WHERE aggfnoid = 'mysum'::REGPROC;
+
+DROP AGGREGATE mysum (int);
+
 -- invalid: nonstrict inverse with strict forward function
 
 CREATE FUNCTION float8mi_n(float8, float8) RETURNS float8 AS