Support hashed aggregation with grouping sets.
authorAndrew Gierth <rhodiumtoad@postgresql.org>
Mon, 27 Mar 2017 03:20:54 +0000 (04:20 +0100)
committerAndrew Gierth <rhodiumtoad@postgresql.org>
Mon, 27 Mar 2017 03:20:54 +0000 (04:20 +0100)
This extends the Aggregate node with two new features: HashAggregate
can now run multiple hashtables concurrently, and a new strategy
MixedAggregate populates hashtables while doing sorted grouping.

The planner will now attempt to save as many sorts as possible when
planning grouping sets queries, while not exceeding work_mem for the
estimated combined sizes of all hashtables used.  No SQL-level changes
are required.  There should be no user-visible impact other than the
new EXPLAIN output and possible changes to result ordering when ORDER
BY was not used (which affected a few regression tests).  The
enable_hashagg option is respected.

Author: Andrew Gierth
Reviewers: Mark Dilger, Andres Freund
Discussion: https://postgr.es/m/87vatszyhj.fsf@news-spur.riddles.org.uk

22 files changed:
contrib/postgres_fdw/expected/postgres_fdw.out
src/backend/commands/explain.c
src/backend/executor/nodeAgg.c
src/backend/lib/Makefile
src/backend/lib/knapsack.c [new file with mode: 0644]
src/backend/nodes/bitmapset.c
src/backend/nodes/outfuncs.c
src/backend/optimizer/path/costsize.c
src/backend/optimizer/plan/createplan.c
src/backend/optimizer/plan/planner.c
src/backend/optimizer/util/pathnode.c
src/include/lib/knapsack.h [new file with mode: 0644]
src/include/nodes/bitmapset.h
src/include/nodes/execnodes.h
src/include/nodes/nodes.h
src/include/nodes/plannodes.h
src/include/nodes/relation.h
src/include/optimizer/pathnode.h
src/test/regress/expected/groupingsets.out
src/test/regress/expected/tsrf.out
src/test/regress/sql/groupingsets.sql
src/test/regress/sql/tsrf.sql

index 059c5c309a8a0d21a5b09db5f268906136727938..a466bf2079a482efdec6d1192a82cb4efd496521 100644 (file)
@@ -3276,16 +3276,19 @@ select sum(q.a), count(q.b) from ft4 left join (select 13, avg(ft1.c1), sum(ft2.
 -- Grouping sets
 explain (verbose, costs off)
 select c2, sum(c1) from ft1 where c2 < 3 group by rollup(c2) order by 1 nulls last;
-                                            QUERY PLAN                                             
----------------------------------------------------------------------------------------------------
- GroupAggregate
-   Output: c2, sum(c1)
-   Group Key: ft1.c2
-   Group Key: ()
-   ->  Foreign Scan on public.ft1
-         Output: c2, c1
-         Remote SQL: SELECT "C 1", c2 FROM "S 1"."T 1" WHERE ((c2 < 3)) ORDER BY c2 ASC NULLS LAST
-(7 rows)
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: c2, (sum(c1))
+   Sort Key: ft1.c2
+   ->  MixedAggregate
+         Output: c2, sum(c1)
+         Hash Key: ft1.c2
+         Group Key: ()
+         ->  Foreign Scan on public.ft1
+               Output: c2, c1
+               Remote SQL: SELECT "C 1", c2 FROM "S 1"."T 1" WHERE ((c2 < 3))
+(10 rows)
 
 select c2, sum(c1) from ft1 where c2 < 3 group by rollup(c2) order by 1 nulls last;
  c2 |  sum   
@@ -3298,16 +3301,19 @@ select c2, sum(c1) from ft1 where c2 < 3 group by rollup(c2) order by 1 nulls la
 
 explain (verbose, costs off)
 select c2, sum(c1) from ft1 where c2 < 3 group by cube(c2) order by 1 nulls last;
-                                            QUERY PLAN                                             
----------------------------------------------------------------------------------------------------
- GroupAggregate
-   Output: c2, sum(c1)
-   Group Key: ft1.c2
-   Group Key: ()
-   ->  Foreign Scan on public.ft1
-         Output: c2, c1
-         Remote SQL: SELECT "C 1", c2 FROM "S 1"."T 1" WHERE ((c2 < 3)) ORDER BY c2 ASC NULLS LAST
-(7 rows)
+                                  QUERY PLAN                                  
+------------------------------------------------------------------------------
+ Sort
+   Output: c2, (sum(c1))
+   Sort Key: ft1.c2
+   ->  MixedAggregate
+         Output: c2, sum(c1)
+         Hash Key: ft1.c2
+         Group Key: ()
+         ->  Foreign Scan on public.ft1
+               Output: c2, c1
+               Remote SQL: SELECT "C 1", c2 FROM "S 1"."T 1" WHERE ((c2 < 3))
+(10 rows)
 
 select c2, sum(c1) from ft1 where c2 < 3 group by cube(c2) order by 1 nulls last;
  c2 |  sum   
@@ -3320,20 +3326,19 @@ select c2, sum(c1) from ft1 where c2 < 3 group by cube(c2) order by 1 nulls last
 
 explain (verbose, costs off)
 select c2, c6, sum(c1) from ft1 where c2 < 3 group by grouping sets(c2, c6) order by 1 nulls last, 2 nulls last;
-                                                 QUERY PLAN                                                  
--------------------------------------------------------------------------------------------------------------
+                                    QUERY PLAN                                    
+----------------------------------------------------------------------------------
  Sort
    Output: c2, c6, (sum(c1))
    Sort Key: ft1.c2, ft1.c6
-   ->  GroupAggregate
+   ->  HashAggregate
          Output: c2, c6, sum(c1)
-         Group Key: ft1.c2
-         Sort Key: ft1.c6
-           Group Key: ft1.c6
+         Hash Key: ft1.c2
+         Hash Key: ft1.c6
          ->  Foreign Scan on public.ft1
                Output: c2, c6, c1
-               Remote SQL: SELECT "C 1", c2, c6 FROM "S 1"."T 1" WHERE ((c2 < 3)) ORDER BY c2 ASC NULLS LAST
-(11 rows)
+               Remote SQL: SELECT "C 1", c2, c6 FROM "S 1"."T 1" WHERE ((c2 < 3))
+(10 rows)
 
 select c2, c6, sum(c1) from ft1 where c2 < 3 group by grouping sets(c2, c6) order by 1 nulls last, 2 nulls last;
  c2 | c6 |  sum  
index 1036b96aaea00f686c18ebb4b6a102ad4b487069..ea19ba60c59efb33854d017d06d634b5bc48f6d0 100644 (file)
@@ -1015,6 +1015,10 @@ ExplainNode(PlanState *planstate, List *ancestors,
                        pname = "HashAggregate";
                        strategy = "Hashed";
                        break;
+                   case AGG_MIXED:
+                       pname = "MixedAggregate";
+                       strategy = "Mixed";
+                       break;
                    default:
                        pname = "Aggregate ???";
                        strategy = "???";
@@ -1978,6 +1982,19 @@ show_grouping_set_keys(PlanState *planstate,
    ListCell   *lc;
    List       *gsets = aggnode->groupingSets;
    AttrNumber *keycols = aggnode->grpColIdx;
+   const char *keyname;
+   const char *keysetname;
+
+   if (aggnode->aggstrategy == AGG_HASHED || aggnode->aggstrategy == AGG_MIXED)
+   {
+       keyname = "Hash Key";
+       keysetname = "Hash Keys";
+   }
+   else
+   {
+       keyname = "Group Key";
+       keysetname = "Group Keys";
+   }
 
    ExplainOpenGroup("Grouping Set", NULL, true, es);
 
@@ -1992,7 +2009,7 @@ show_grouping_set_keys(PlanState *planstate,
            es->indent++;
    }
 
-   ExplainOpenGroup("Group Keys", "Group Keys", false, es);
+   ExplainOpenGroup(keysetname, keysetname, false, es);
 
    foreach(lc, gsets)
    {
@@ -2016,12 +2033,12 @@ show_grouping_set_keys(PlanState *planstate,
        }
 
        if (!result && es->format == EXPLAIN_FORMAT_TEXT)
-           ExplainPropertyText("Group Key", "()", es);
+           ExplainPropertyText(keyname, "()", es);
        else
-           ExplainPropertyListNested("Group Key", result, es);
+           ExplainPropertyListNested(keyname, result, es);
    }
 
-   ExplainCloseGroup("Group Keys", "Group Keys", false, es);
+   ExplainCloseGroup(keysetname, keysetname, false, es);
 
    if (sortnode && es->format == EXPLAIN_FORMAT_TEXT)
        es->indent--;
index 471acc4b3ec91db7a4c5a6670dbed526cc244560..ef35da6ade6ac051dd448d9954b0724730a7d490 100644 (file)
  *   specific).
  *
  *   Where more complex grouping sets are used, we break them down into
- *   "phases", where each phase has a different sort order.  During each
- *   phase but the last, the input tuples are additionally stored in a
- *   tuplesort which is keyed to the next phase's sort order; during each
- *   phase but the first, the input tuples are drawn from the previously
- *   sorted data.  (The sorting of the data for the first phase is handled by
- *   the planner, as it might be satisfied by underlying nodes.)
+ *   "phases", where each phase has a different sort order (except phase 0
+ *   which is reserved for hashing).  During each phase but the last, the
+ *   input tuples are additionally stored in a tuplesort which is keyed to the
+ *   next phase's sort order; during each phase but the first, the input
+ *   tuples are drawn from the previously sorted data.  (The sorting of the
+ *   data for the first phase is handled by the planner, as it might be
+ *   satisfied by underlying nodes.)
+ *
+ *   Hashing can be mixed with sorted grouping.  To do this, we have an
+ *   AGG_MIXED strategy that populates the hashtables during the first sorted
+ *   phase, and switches to reading them out after completing all sort phases.
+ *   We can also support AGG_HASHED with multiple hash tables and no sorting
+ *   at all.
  *
  *   From the perspective of aggregate transition and final functions, the
  *   only issue regarding grouping sets is this: a single call site (flinfo)
  *   sensitive to the grouping set for which the aggregate function is
  *   currently being called.
  *
- *   TODO: AGG_HASHED doesn't support multiple grouping sets yet.
+ *   Plan structure:
+ *
+ *   What we get from the planner is actually one "real" Agg node which is
+ *   part of the plan tree proper, but which optionally has an additional list
+ *   of Agg nodes hung off the side via the "chain" field.  This is because an
+ *   Agg node happens to be a convenient representation of all the data we
+ *   need for grouping sets.
+ *
+ *   For many purposes, we treat the "real" node as if it were just the first
+ *   node in the chain.  The chain must be ordered such that hashed entries
+ *   come before sorted/plain entries; the real node is marked AGG_MIXED if
+ *   there are both types present (in which case the real node describes one
+ *   of the hashed groupings, other AGG_HASHED nodes may optionally follow in
+ *   the chain, followed in turn by AGG_SORTED or (one) AGG_PLAIN node).  If
+ *   the real node is marked AGG_HASHED or AGG_SORTED, then all the chained
+ *   nodes must be of the same type; if it is AGG_PLAIN, there can be no
+ *   chained nodes.
+ *
+ *   We collect all hashed nodes into a single "phase", numbered 0, and create
+ *   a sorted phase (numbered 1..n) for each AGG_SORTED or AGG_PLAIN node.
+ *   Phase 0 is allocated even if there are no hashes, but remains unused in
+ *   that case.
+ *
+ *   AGG_HASHED nodes actually refer to only a single grouping set each,
+ *   because for each hashed grouping we need a separate grpColIdx and
+ *   numGroups estimate.  AGG_SORTED nodes represent a "rollup", a list of
+ *   grouping sets that share a sort order.  Each AGG_SORTED node other than
+ *   the first one has an associated Sort node which describes the sort order
+ *   to be used; the first sorted node takes its input from the outer subtree,
+ *   which the planner has already arranged to provide ordered data.
+ *
+ *   Memory and ExprContext usage:
+ *
+ *   Because we're accumulating aggregate values across input rows, we need to
+ *   use more memory contexts than just simple input/output tuple contexts.
+ *   In fact, for a rollup, we need a separate context for each grouping set
+ *   so that we can reset the inner (finer-grained) aggregates on their group
+ *   boundaries while continuing to accumulate values for outer
+ *   (coarser-grained) groupings.  On top of this, we might be simultaneously
+ *   populating hashtables; however, we only need one context for all the
+ *   hashtables.
+ *
+ *   So we create an array, aggcontexts, with an ExprContext for each grouping
+ *   set in the largest rollup that we're going to process, and use the
+ *   per-tuple memory context of those ExprContexts to store the aggregate
+ *   transition values.  hashcontext is the single context created to support
+ *   all hash tables.
+ *
  *
  * Portions Copyright (c) 1996-2017, PostgreSQL Global Development Group
  * Portions Copyright (c) 1994, Regents of the University of California
@@ -432,6 +486,7 @@ typedef struct AggStatePerGroupData
  */
 typedef struct AggStatePerPhaseData
 {
+   AggStrategy aggstrategy;    /* strategy for this phase */
    int         numsets;        /* number of grouping sets (or 0) */
    int        *gset_lengths;   /* lengths of grouping sets */
    Bitmapset **grouped_cols;   /* column groupings for rollup */
@@ -440,7 +495,30 @@ typedef struct AggStatePerPhaseData
    Sort       *sortnode;       /* Sort node for input ordering for phase */
 }  AggStatePerPhaseData;
 
+/*
+ * AggStatePerHashData - per-hashtable state
+ *
+ * When doing grouping sets with hashing, we have one of these for each
+ * grouping set. (When doing hashing without grouping sets, we have just one of
+ * them.)
+ */
+typedef struct AggStatePerHashData
+{
+   TupleHashTable hashtable;   /* hash table with one entry per group */
+   TupleHashIterator hashiter; /* for iterating through hash table */
+   TupleTableSlot *hashslot;   /* slot for loading hash table */
+   FmgrInfo   *hashfunctions;  /* per-grouping-field hash fns */
+   FmgrInfo   *eqfunctions;    /* per-grouping-field equality fns */
+   int         numCols;        /* number of hash key columns */
+   int         numhashGrpCols; /* number of columns in hash table */
+   int         largestGrpColIdx;       /* largest col required for hashing */
+   AttrNumber *hashGrpColIdxInput;     /* hash col indices in input slot */
+   AttrNumber *hashGrpColIdxHash;      /* indices in hashtbl tuples */
+   Agg        *aggnode;        /* original Agg node, for numGroups etc. */
+} AggStatePerHashData;
+
 
+static void select_current_set(AggState *aggstate, int setno, bool is_hash);
 static void initialize_phase(AggState *aggstate, int newphase);
 static TupleTableSlot *fetch_input_tuple(AggState *aggstate);
 static void initialize_aggregates(AggState *aggstate,
@@ -449,7 +527,8 @@ static void initialize_aggregates(AggState *aggstate,
 static void advance_transition_function(AggState *aggstate,
                            AggStatePerTrans pertrans,
                            AggStatePerGroup pergroupstate);
-static void advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup);
+static void advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup,
+                  AggStatePerGroup *pergroups);
 static void advance_combine_function(AggState *aggstate,
                         AggStatePerTrans pertrans,
                         AggStatePerGroup pergroupstate);
@@ -473,14 +552,13 @@ static void prepare_projection_slot(AggState *aggstate,
                        int currentSet);
 static void finalize_aggregates(AggState *aggstate,
                    AggStatePerAgg peragg,
-                   AggStatePerGroup pergroup,
-                   int currentSet);
+                   AggStatePerGroup pergroup);
 static TupleTableSlot *project_aggregates(AggState *aggstate);
 static Bitmapset *find_unaggregated_cols(AggState *aggstate);
 static bool find_unaggregated_cols_walker(Node *node, Bitmapset **colnos);
 static void build_hash_table(AggState *aggstate);
-static TupleHashEntryData *lookup_hash_entry(AggState *aggstate,
-                 TupleTableSlot *inputslot);
+static TupleHashEntryData *lookup_hash_entry(AggState *aggstate);
+static AggStatePerGroup *lookup_hash_entries(AggState *aggstate);
 static TupleTableSlot *agg_retrieve_direct(AggState *aggstate);
 static void agg_fill_hash_table(AggState *aggstate);
 static TupleTableSlot *agg_retrieve_hash_table(AggState *aggstate);
@@ -501,13 +579,31 @@ static int find_compatible_pertrans(AggState *aggstate, Aggref *newagg,
 
 
 /*
- * Switch to phase "newphase", which must either be 0 (to reset) or
+ * Select the current grouping set; affects current_set and
+ * curaggcontext.
+ */
+static void
+select_current_set(AggState *aggstate, int setno, bool is_hash)
+{
+   if (is_hash)
+       aggstate->curaggcontext = aggstate->hashcontext;
+   else
+       aggstate->curaggcontext = aggstate->aggcontexts[setno];
+
+   aggstate->current_set = setno;
+}
+
+/*
+ * Switch to phase "newphase", which must either be 0 or 1 (to reset) or
  * current_phase + 1. Juggle the tuplesorts accordingly.
+ *
+ * Phase 0 is for hashing, which we currently handle last in the AGG_MIXED
+ * case, so when entering phase 0, all we need to do is drop open sorts.
  */
 static void
 initialize_phase(AggState *aggstate, int newphase)
 {
-   Assert(newphase == 0 || newphase == aggstate->current_phase + 1);
+   Assert(newphase <= 1 || newphase == aggstate->current_phase + 1);
 
    /*
     * Whatever the previous state, we're now done with whatever input
@@ -519,7 +615,7 @@ initialize_phase(AggState *aggstate, int newphase)
        aggstate->sort_in = NULL;
    }
 
-   if (newphase == 0)
+   if (newphase <= 1)
    {
        /*
         * Discard any existing output tuplesort.
@@ -546,7 +642,7 @@ initialize_phase(AggState *aggstate, int newphase)
     * If this isn't the last phase, we need to sort appropriately for the
     * next phase in sequence.
     */
-   if (newphase < aggstate->numphases - 1)
+   if (newphase > 0 && newphase < aggstate->numphases - 1)
    {
        Sort       *sortnode = aggstate->phases[newphase + 1].sortnode;
        PlanState  *outerNode = outerPlanState(aggstate);
@@ -567,7 +663,7 @@ initialize_phase(AggState *aggstate, int newphase)
 }
 
 /*
- * Fetch a tuple from either the outer plan (for phase 0) or from the sorter
+ * Fetch a tuple from either the outer plan (for phase 1) or from the sorter
  * populated by the previous phase.  Copy it to the sorter for the next phase
  * if any.
  */
@@ -595,8 +691,8 @@ fetch_input_tuple(AggState *aggstate)
 /*
  * (Re)Initialize an individual aggregate.
  *
- * This function handles only one grouping set (already set in
- * aggstate->current_set).
+ * This function handles only one grouping setalready set in
+ * aggstate->current_set.
  *
  * When called, CurrentMemoryContext should be the per-query context.
  */
@@ -653,7 +749,7 @@ initialize_aggregate(AggState *aggstate, AggStatePerTrans pertrans,
        MemoryContext oldContext;
 
        oldContext = MemoryContextSwitchTo(
-       aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory);
+                            aggstate->curaggcontext->ecxt_per_tuple_memory);
        pergroupstate->transValue = datumCopy(pertrans->initValue,
                                              pertrans->transtypeByVal,
                                              pertrans->transtypeLen);
@@ -676,8 +772,9 @@ initialize_aggregate(AggState *aggstate, AggStatePerTrans pertrans,
  *
  * If there are multiple grouping sets, we initialize only the first numReset
  * of them (the grouping sets are ordered so that the most specific one, which
- * is reset most often, is first). As a convenience, if numReset is < 1, we
- * reinitialize all sets.
+ * is reset most often, is first). As a convenience, if numReset is 0, we
+ * reinitialize all sets. numReset is -1 to initialize a hashtable entry, in
+ * which case the caller must have used select_current_set appropriately.
  *
  * When called, CurrentMemoryContext should be the per-query context.
  */
@@ -689,25 +786,37 @@ initialize_aggregates(AggState *aggstate,
    int         transno;
    int         numGroupingSets = Max(aggstate->phase->numsets, 1);
    int         setno = 0;
+   int         numTrans = aggstate->numtrans;
    AggStatePerTrans transstates = aggstate->pertrans;
 
-   if (numReset < 1)
+   if (numReset == 0)
        numReset = numGroupingSets;
 
-   for (transno = 0; transno < aggstate->numtrans; transno++)
+   for (transno = 0; transno < numTrans; transno++)
    {
        AggStatePerTrans pertrans = &transstates[transno];
 
-       for (setno = 0; setno < numReset; setno++)
+       if (numReset < 0)
        {
            AggStatePerGroup pergroupstate;
 
-           pergroupstate = &pergroup[transno + (setno * (aggstate->numtrans))];
-
-           aggstate->current_set = setno;
+           pergroupstate = &pergroup[transno];
 
            initialize_aggregate(aggstate, pertrans, pergroupstate);
        }
+       else
+       {
+           for (setno = 0; setno < numReset; setno++)
+           {
+               AggStatePerGroup pergroupstate;
+
+               pergroupstate = &pergroup[transno + (setno * numTrans)];
+
+               select_current_set(aggstate, setno, false);
+
+               initialize_aggregate(aggstate, pertrans, pergroupstate);
+           }
+       }
    }
 }
 
@@ -757,7 +866,7 @@ advance_transition_function(AggState *aggstate,
             * do not need to pfree the old transValue, since it's NULL.
             */
            oldContext = MemoryContextSwitchTo(
-                                              aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory);
+                            aggstate->curaggcontext->ecxt_per_tuple_memory);
            pergroupstate->transValue = datumCopy(fcinfo->arg[1],
                                                  pertrans->transtypeByVal,
                                                  pertrans->transtypeLen);
@@ -807,7 +916,7 @@ advance_transition_function(AggState *aggstate,
    {
        if (!fcinfo->isnull)
        {
-           MemoryContextSwitchTo(aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory);
+           MemoryContextSwitchTo(aggstate->curaggcontext->ecxt_per_tuple_memory);
            if (DatumIsReadWriteExpandedObject(newVal,
                                               false,
                                               pertrans->transtypeLen) &&
@@ -838,17 +947,21 @@ advance_transition_function(AggState *aggstate,
 /*
  * Advance each aggregate transition state for one input tuple.  The input
  * tuple has been stored in tmpcontext->ecxt_outertuple, so that it is
- * accessible to ExecEvalExpr.  pergroup is the array of per-group structs to
- * use (this might be in a hashtable entry).
+ * accessible to ExecEvalExpr.
+ *
+ * We have two sets of transition states to handle: one for sorted aggregation
+ * and one for hashed; we do them both here, to avoid multiple evaluation of
+ * the inputs.
  *
  * When called, CurrentMemoryContext should be the per-query context.
  */
 static void
-advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup)
+advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup, AggStatePerGroup *pergroups)
 {
    int         transno;
    int         setno = 0;
    int         numGroupingSets = Max(aggstate->phase->numsets, 1);
+   int         numHashes = aggstate->num_hashes;
    int         numTrans = aggstate->numtrans;
    TupleTableSlot *slot = aggstate->evalslot;
 
@@ -880,6 +993,7 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup)
        {
            /* DISTINCT and/or ORDER BY case */
            Assert(slot->tts_nvalid >= (pertrans->numInputs + inputoff));
+           Assert(!pergroups);
 
            /*
             * If the transfn is strict, we want to check for nullity before
@@ -940,13 +1054,36 @@ advance_aggregates(AggState *aggstate, AggStatePerGroup pergroup)
                fcinfo->argnull[i + 1] = slot->tts_isnull[i + inputoff];
            }
 
-           for (setno = 0; setno < numGroupingSets; setno++)
+           if (pergroup)
            {
-               AggStatePerGroup pergroupstate = &pergroup[transno + (setno * numTrans)];
+               /* advance transition states for ordered grouping */
+
+               for (setno = 0; setno < numGroupingSets; setno++)
+               {
+                   AggStatePerGroup pergroupstate;
 
-               aggstate->current_set = setno;
+                   select_current_set(aggstate, setno, false);
 
-               advance_transition_function(aggstate, pertrans, pergroupstate);
+                   pergroupstate = &pergroup[transno + (setno * numTrans)];
+
+                   advance_transition_function(aggstate, pertrans, pergroupstate);
+               }
+           }
+
+           if (pergroups)
+           {
+               /* advance transition states for hashed grouping */
+
+               for (setno = 0; setno < numHashes; setno++)
+               {
+                   AggStatePerGroup pergroupstate;
+
+                   select_current_set(aggstate, setno, true);
+
+                   pergroupstate = &pergroups[setno][transno];
+
+                   advance_transition_function(aggstate, pertrans, pergroupstate);
+               }
            }
        }
    }
@@ -967,7 +1104,7 @@ combine_aggregates(AggState *aggstate, AggStatePerGroup pergroup)
    TupleTableSlot *slot;
 
    /* combine not supported with grouping sets */
-   Assert(aggstate->phase->numsets == 0);
+   Assert(aggstate->phase->numsets <= 1);
 
    /* compute input for all aggregates */
    slot = ExecProject(aggstate->evalproj);
@@ -1060,7 +1197,7 @@ advance_combine_function(AggState *aggstate,
            if (!pertrans->transtypeByVal)
            {
                oldContext = MemoryContextSwitchTo(
-                                                  aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory);
+                            aggstate->curaggcontext->ecxt_per_tuple_memory);
                pergroupstate->transValue = datumCopy(fcinfo->arg[1],
                                                    pertrans->transtypeByVal,
                                                      pertrans->transtypeLen);
@@ -1105,7 +1242,7 @@ advance_combine_function(AggState *aggstate,
    {
        if (!fcinfo->isnull)
        {
-           MemoryContextSwitchTo(aggstate->aggcontexts[aggstate->current_set]->ecxt_per_tuple_memory);
+           MemoryContextSwitchTo(aggstate->curaggcontext->ecxt_per_tuple_memory);
            if (DatumIsReadWriteExpandedObject(newVal,
                                               false,
                                               pertrans->transtypeLen) &&
@@ -1559,15 +1696,16 @@ prepare_projection_slot(AggState *aggstate, TupleTableSlot *slot, int currentSet
 /*
  * Compute the final value of all aggregates for one group.
  *
- * This function handles only one grouping set at a time.
+ * This function handles only one grouping set at a time, which the caller must
+ * have selected.  It's also the caller's responsibility to adjust the supplied
+ * pergroup parameter to point to the current set's transvalues.
  *
  * Results are stored in the output econtext aggvalues/aggnulls.
  */
 static void
 finalize_aggregates(AggState *aggstate,
                    AggStatePerAgg peraggs,
-                   AggStatePerGroup pergroup,
-                   int currentSet)
+                   AggStatePerGroup pergroup)
 {
    ExprContext *econtext = aggstate->ss.ps.ps_ExprContext;
    Datum      *aggvalues = econtext->ecxt_aggvalues;
@@ -1575,11 +1713,6 @@ finalize_aggregates(AggState *aggstate,
    int         aggno;
    int         transno;
 
-   Assert(currentSet == 0 ||
-          ((Agg *) aggstate->ss.ps.plan)->aggstrategy != AGG_HASHED);
-
-   aggstate->current_set = currentSet;
-
    /*
     * If there were any DISTINCT and/or ORDER BY aggregates, sort their
     * inputs and run the transition functions.
@@ -1589,11 +1722,12 @@ finalize_aggregates(AggState *aggstate,
        AggStatePerTrans pertrans = &aggstate->pertrans[transno];
        AggStatePerGroup pergroupstate;
 
-       pergroupstate = &pergroup[transno + (currentSet * (aggstate->numtrans))];
+       pergroupstate = &pergroup[transno];
 
        if (pertrans->numSortCols > 0)
        {
-           Assert(((Agg *) aggstate->ss.ps.plan)->aggstrategy != AGG_HASHED);
+           Assert(aggstate->aggstrategy != AGG_HASHED &&
+                  aggstate->aggstrategy != AGG_MIXED);
 
            if (pertrans->numInputs == 1)
                process_ordered_aggregate_single(aggstate,
@@ -1615,7 +1749,7 @@ finalize_aggregates(AggState *aggstate,
        int         transno = peragg->transno;
        AggStatePerGroup pergroupstate;
 
-       pergroupstate = &pergroup[transno + (currentSet * (aggstate->numtrans))];
+       pergroupstate = &pergroup[transno];
 
        if (DO_AGGSPLIT_SKIPFINAL(aggstate->aggsplit))
            finalize_partialaggregate(aggstate, peragg, pergroupstate,
@@ -1697,7 +1831,7 @@ find_unaggregated_cols_walker(Node *node, Bitmapset **colnos)
 }
 
 /*
- * Initialize the hash table to empty.
+ * Initialize the hash table(s) to empty.
  *
  * To implement hashed aggregation, we need a hashtable that stores a
  * representative tuple and an array of AggStatePerGroup structs for each
@@ -1705,29 +1839,40 @@ find_unaggregated_cols_walker(Node *node, Bitmapset **colnos)
  * GROUP BY columns.  The per-group data is allocated in lookup_hash_entry(),
  * for each entry.
  *
- * The hash table always lives in the aggcontext memory context.
+ * We have a separate hashtable and associated perhash data structure for each
+ * grouping set for which we're doing hashing.
+ *
+ * The hash tables always live in the hashcontext's per-tuple memory context
+ * (there is only one of these for all tables together, since they are all
+ * reset at the same time).
  */
 static void
 build_hash_table(AggState *aggstate)
 {
-   Agg        *node = (Agg *) aggstate->ss.ps.plan;
    MemoryContext tmpmem = aggstate->tmpcontext->ecxt_per_tuple_memory;
    Size        additionalsize;
+   int         i;
 
-   Assert(node->aggstrategy == AGG_HASHED);
-   Assert(node->numGroups > 0);
+   Assert(aggstate->aggstrategy == AGG_HASHED || aggstate->aggstrategy == AGG_MIXED);
 
-   additionalsize = aggstate->numaggs * sizeof(AggStatePerGroupData);
+   additionalsize = aggstate->numtrans * sizeof(AggStatePerGroupData);
 
-   aggstate->hashtable = BuildTupleHashTable(node->numCols,
-                                             aggstate->hashGrpColIdxHash,
-                                             aggstate->phase->eqfunctions,
-                                             aggstate->hashfunctions,
-                                             node->numGroups,
-                                             additionalsize,
-                            aggstate->aggcontexts[0]->ecxt_per_tuple_memory,
-                                             tmpmem,
+   for (i = 0; i < aggstate->num_hashes; ++i)
+   {
+       AggStatePerHash perhash = &aggstate->perhash[i];
+
+       Assert(perhash->aggnode->numGroups > 0);
+
+       perhash->hashtable = BuildTupleHashTable(perhash->numCols,
+                                                perhash->hashGrpColIdxHash,
+                                                perhash->eqfunctions,
+                                                perhash->hashfunctions,
+                                                perhash->aggnode->numGroups,
+                                                additionalsize,
+                               aggstate->hashcontext->ecxt_per_tuple_memory,
+                                                tmpmem,
                                  DO_AGGSPLIT_SKIPFINAL(aggstate->aggsplit));
+   }
 }
 
 /*
@@ -1750,72 +1895,98 @@ build_hash_table(AggState *aggstate)
  * the array is preserved over ExecReScanAgg, so we allocate it in the
  * per-query context (unlike the hash table itself).
  */
-static List *
+static void
 find_hash_columns(AggState *aggstate)
 {
-   Agg        *node = (Agg *) aggstate->ss.ps.plan;
-   Bitmapset  *colnos;
-   List       *collist;
-   TupleDesc   hashDesc;
+   Bitmapset  *base_colnos;
    List       *outerTlist = outerPlanState(aggstate)->plan->targetlist;
-   List        *hashTlist = NIL;
-   int         i;
-
-   aggstate->largestGrpColIdx = 0;
+   int         numHashes = aggstate->num_hashes;
+   int         j;
 
    /* Find Vars that will be needed in tlist and qual */
-   colnos = find_unaggregated_cols(aggstate);
-   /* Add in all the grouping columns */
-   for (i = 0; i < node->numCols; i++)
-       colnos = bms_add_member(colnos, node->grpColIdx[i]);
-   /* Convert to list, using lcons so largest element ends up first */
-   collist = NIL;
+   base_colnos = find_unaggregated_cols(aggstate);
 
-   aggstate->hashGrpColIdxInput =
-       palloc(bms_num_members(colnos) * sizeof(AttrNumber));
-   aggstate->hashGrpColIdxHash =
-       palloc(node->numCols * sizeof(AttrNumber));
-
-   /*
-    * First build mapping for columns directly hashed. These are the first,
-    * because they'll be accessed when computing hash values and comparing
-    * tuples for exact matches. We also build simple mapping for
-    * execGrouping, so it knows where to find the to-be-hashed / compared
-    * columns in the input.
-    */
-   for (i = 0; i < node->numCols; i++)
+   for (j = 0; j < numHashes; ++j)
    {
-       aggstate->hashGrpColIdxInput[i] = node->grpColIdx[i];
-       aggstate->hashGrpColIdxHash[i] = i + 1;
-       aggstate->numhashGrpCols++;
-       /* delete already mapped columns */
-       bms_del_member(colnos, node->grpColIdx[i]);
-   }
+       AggStatePerHash perhash = &aggstate->perhash[j];
+       Bitmapset  *colnos = bms_copy(base_colnos);
+       AttrNumber *grpColIdx = perhash->aggnode->grpColIdx;
+       List       *hashTlist = NIL;
+       TupleDesc   hashDesc;
+       int         i;
 
-   /* and add the remaining columns */
-   while ((i = bms_first_member(colnos)) >= 0)
-   {
-       aggstate->hashGrpColIdxInput[aggstate->numhashGrpCols] = i;
-       aggstate->numhashGrpCols++;
-   }
+       perhash->largestGrpColIdx = 0;
 
-   /* and build a tuple descriptor for the hashtable */
-   for (i = 0; i < aggstate->numhashGrpCols; i++)
-   {
-       int         varNumber = aggstate->hashGrpColIdxInput[i] - 1;
+       /*
+        * If we're doing grouping sets, then some Vars might be referenced in
+        * tlist/qual for the benefit of other grouping sets, but not needed
+        * when hashing; i.e. prepare_projection_slot will null them out, so
+        * there'd be no point storing them.  Use prepare_projection_slot's
+        * logic to determine which.
+        */
+       if (aggstate->phases[0].grouped_cols)
+       {
+           Bitmapset  *grouped_cols = aggstate->phases[0].grouped_cols[j];
+           ListCell   *lc;
 
-       hashTlist = lappend(hashTlist, list_nth(outerTlist, varNumber));
-       aggstate->largestGrpColIdx =
-           Max(varNumber + 1, aggstate->largestGrpColIdx);
-   }
+           foreach(lc, aggstate->all_grouped_cols)
+           {
+               int         attnum = lfirst_int(lc);
 
-   hashDesc = ExecTypeFromTL(hashTlist, false);
-   ExecSetSlotDescriptor(aggstate->hashslot, hashDesc);
+               if (!bms_is_member(attnum, grouped_cols))
+                   colnos = bms_del_member(colnos, attnum);
+           }
+       }
+       /* Add in all the grouping columns */
+       for (i = 0; i < perhash->numCols; i++)
+           colnos = bms_add_member(colnos, grpColIdx[i]);
 
-   list_free(hashTlist);
-   bms_free(colnos);
+       perhash->hashGrpColIdxInput =
+           palloc(bms_num_members(colnos) * sizeof(AttrNumber));
+       perhash->hashGrpColIdxHash =
+           palloc(perhash->numCols * sizeof(AttrNumber));
+
+       /*
+        * First build mapping for columns directly hashed. These are the
+        * first, because they'll be accessed when computing hash values and
+        * comparing tuples for exact matches. We also build simple mapping
+        * for execGrouping, so it knows where to find the to-be-hashed /
+        * compared columns in the input.
+        */
+       for (i = 0; i < perhash->numCols; i++)
+       {
+           perhash->hashGrpColIdxInput[i] = grpColIdx[i];
+           perhash->hashGrpColIdxHash[i] = i + 1;
+           perhash->numhashGrpCols++;
+           /* delete already mapped columns */
+           bms_del_member(colnos, grpColIdx[i]);
+       }
 
-   return collist;
+       /* and add the remaining columns */
+       while ((i = bms_first_member(colnos)) >= 0)
+       {
+           perhash->hashGrpColIdxInput[perhash->numhashGrpCols] = i;
+           perhash->numhashGrpCols++;
+       }
+
+       /* and build a tuple descriptor for the hashtable */
+       for (i = 0; i < perhash->numhashGrpCols; i++)
+       {
+           int         varNumber = perhash->hashGrpColIdxInput[i] - 1;
+
+           hashTlist = lappend(hashTlist, list_nth(outerTlist, varNumber));
+           perhash->largestGrpColIdx =
+               Max(varNumber + 1, perhash->largestGrpColIdx);
+       }
+
+       hashDesc = ExecTypeFromTL(hashTlist, false);
+       ExecSetSlotDescriptor(perhash->hashslot, hashDesc);
+
+       list_free(hashTlist);
+       bms_free(colnos);
+   }
+
+   bms_free(base_colnos);
 }
 
 /*
@@ -1840,26 +2011,30 @@ hash_agg_entry_size(int numAggs)
 }
 
 /*
- * Find or create a hashtable entry for the tuple group containing the
- * given tuple.
+ * Find or create a hashtable entry for the tuple group containing the current
+ * tuple (already set in tmpcontext's outertuple slot), in the current grouping
+ * set (which the caller must have selected - note that initialize_aggregate
+ * depends on this).
  *
  * When called, CurrentMemoryContext should be the per-query context.
  */
 static TupleHashEntryData *
-lookup_hash_entry(AggState *aggstate, TupleTableSlot *inputslot)
+lookup_hash_entry(AggState *aggstate)
 {
-   TupleTableSlot *hashslot = aggstate->hashslot;
+   TupleTableSlot *inputslot = aggstate->tmpcontext->ecxt_outertuple;
+   AggStatePerHash perhash = &aggstate->perhash[aggstate->current_set];
+   TupleTableSlot *hashslot = perhash->hashslot;
    TupleHashEntryData *entry;
    bool        isnew;
-   int i;
+   int         i;
 
    /* transfer just the needed columns into hashslot */
-   slot_getsomeattrs(inputslot, aggstate->largestGrpColIdx);
+   slot_getsomeattrs(inputslot, perhash->largestGrpColIdx);
    ExecClearTuple(hashslot);
 
-   for (i = 0; i < aggstate->numhashGrpCols; i++)
+   for (i = 0; i < perhash->numhashGrpCols; i++)
    {
-       int         varNumber = aggstate->hashGrpColIdxInput[i] - 1;
+       int         varNumber = perhash->hashGrpColIdxInput[i] - 1;
 
        hashslot->tts_values[i] = inputslot->tts_values[varNumber];
        hashslot->tts_isnull[i] = inputslot->tts_isnull[varNumber];
@@ -1867,21 +2042,43 @@ lookup_hash_entry(AggState *aggstate, TupleTableSlot *inputslot)
    ExecStoreVirtualTuple(hashslot);
 
    /* find or create the hashtable entry using the filtered tuple */
-   entry = LookupTupleHashEntry(aggstate->hashtable, hashslot, &isnew);
+   entry = LookupTupleHashEntry(perhash->hashtable, hashslot, &isnew);
 
    if (isnew)
    {
        entry->additional = (AggStatePerGroup)
-           MemoryContextAlloc(aggstate->hashtable->tablecxt,
+           MemoryContextAlloc(perhash->hashtable->tablecxt,
                          sizeof(AggStatePerGroupData) * aggstate->numtrans);
        /* initialize aggregates for new tuple group */
        initialize_aggregates(aggstate, (AggStatePerGroup) entry->additional,
-                             0);
+                             -1);
    }
 
    return entry;
 }
 
+/*
+ * Look up hash entries for the current tuple in all hashed grouping sets,
+ * returning an array of pergroup pointers suitable for advance_aggregates.
+ *
+ * Be aware that lookup_hash_entry can reset the tmpcontext.
+ */
+static AggStatePerGroup *
+lookup_hash_entries(AggState *aggstate)
+{
+   int         numHashes = aggstate->num_hashes;
+   AggStatePerGroup *pergroup = aggstate->hash_pergroup;
+   int         setno;
+
+   for (setno = 0; setno < numHashes; setno++)
+   {
+       select_current_set(aggstate, setno, true);
+       pergroup[setno] = lookup_hash_entry(aggstate)->additional;
+   }
+
+   return pergroup;
+}
+
 /*
  * ExecAgg -
  *
@@ -1898,19 +2095,22 @@ lookup_hash_entry(AggState *aggstate, TupleTableSlot *inputslot)
 TupleTableSlot *
 ExecAgg(AggState *node)
 {
-   TupleTableSlot *result;
+   TupleTableSlot *result = NULL;
 
    if (!node->agg_done)
    {
        /* Dispatch based on strategy */
-       switch (node->phase->aggnode->aggstrategy)
+       switch (node->phase->aggstrategy)
        {
            case AGG_HASHED:
                if (!node->table_filled)
                    agg_fill_hash_table(node);
+               /* FALLTHROUGH */
+           case AGG_MIXED:
                result = agg_retrieve_hash_table(node);
                break;
-           default:
+           case AGG_PLAIN:
+           case AGG_SORTED:
                result = agg_retrieve_direct(node);
                break;
        }
@@ -1933,6 +2133,7 @@ agg_retrieve_direct(AggState *aggstate)
    ExprContext *tmpcontext;
    AggStatePerAgg peragg;
    AggStatePerGroup pergroup;
+   AggStatePerGroup *hash_pergroups = NULL;
    TupleTableSlot *outerslot;
    TupleTableSlot *firstSlot;
    TupleTableSlot *result;
@@ -2019,6 +2220,19 @@ agg_retrieve_direct(AggState *aggstate)
                node = aggstate->phase->aggnode;
                numReset = numGroupingSets;
            }
+           else if (aggstate->aggstrategy == AGG_MIXED)
+           {
+               /*
+                * Mixed mode; we've output all the grouped stuff and have
+                * full hashtables, so switch to outputting those.
+                */
+               initialize_phase(aggstate, 0);
+               aggstate->table_filled = true;
+               ResetTupleHashIterator(aggstate->perhash[0].hashtable,
+                                      &aggstate->perhash[0].hashiter);
+               select_current_set(aggstate, 0, true);
+               return agg_retrieve_hash_table(aggstate);
+           }
            else
            {
                aggstate->agg_done = true;
@@ -2055,7 +2269,7 @@ agg_retrieve_direct(AggState *aggstate)
         *----------
         */
        if (aggstate->input_done ||
-           (node->aggstrategy == AGG_SORTED &&
+           (node->aggstrategy != AGG_PLAIN &&
             aggstate->projected_set != -1 &&
             aggstate->projected_set < (numGroupingSets - 1) &&
             nextSetSize > 0 &&
@@ -2168,10 +2382,22 @@ agg_retrieve_direct(AggState *aggstate)
                 */
                for (;;)
                {
+                   /*
+                    * During phase 1 only of a mixed agg, we need to update
+                    * hashtables as well in advance_aggregates.
+                    */
+                   if (aggstate->aggstrategy == AGG_MIXED &&
+                       aggstate->current_phase == 1)
+                   {
+                       hash_pergroups = lookup_hash_entries(aggstate);
+                   }
+                   else
+                       hash_pergroups = NULL;
+
                    if (DO_AGGSPLIT_COMBINE(aggstate->aggsplit))
                        combine_aggregates(aggstate, pergroup);
                    else
-                       advance_aggregates(aggstate, pergroup);
+                       advance_aggregates(aggstate, pergroup, hash_pergroups);
 
                    /* Reset per-input-tuple context after each tuple */
                    ResetExprContext(tmpcontext);
@@ -2198,7 +2424,7 @@ agg_retrieve_direct(AggState *aggstate)
                     * If we are grouping, check whether we've crossed a group
                     * boundary.
                     */
-                   if (node->aggstrategy == AGG_SORTED)
+                   if (node->aggstrategy != AGG_PLAIN)
                    {
                        if (!execTuplesMatch(firstSlot,
                                             outerslot,
@@ -2231,7 +2457,11 @@ agg_retrieve_direct(AggState *aggstate)
 
        prepare_projection_slot(aggstate, econtext->ecxt_outertuple, currentSet);
 
-       finalize_aggregates(aggstate, peragg, pergroup, currentSet);
+       select_current_set(aggstate, currentSet, false);
+
+       finalize_aggregates(aggstate,
+                           peragg,
+                           pergroup + (currentSet * aggstate->numtrans));
 
        /*
         * If there's no row to project right now, we must continue rather
@@ -2247,21 +2477,13 @@ agg_retrieve_direct(AggState *aggstate)
 }
 
 /*
- * ExecAgg for hashed case: phase 1, read input and build hash table
+ * ExecAgg for hashed case: read input and build hash table
  */
 static void
 agg_fill_hash_table(AggState *aggstate)
 {
-   ExprContext *tmpcontext;
-   TupleHashEntryData *entry;
    TupleTableSlot *outerslot;
-
-   /*
-    * get state info from node
-    *
-    * tmpcontext is the per-input-tuple expression context
-    */
-   tmpcontext = aggstate->tmpcontext;
+   ExprContext *tmpcontext = aggstate->tmpcontext;
 
    /*
     * Process each outer-plan tuple, and then fetch the next one, until we
@@ -2269,32 +2491,40 @@ agg_fill_hash_table(AggState *aggstate)
     */
    for (;;)
    {
+       AggStatePerGroup *pergroups;
+
        outerslot = fetch_input_tuple(aggstate);
        if (TupIsNull(outerslot))
            break;
-       /* set up for advance_aggregates call */
+
+       /* set up for lookup_hash_entries and advance_aggregates */
        tmpcontext->ecxt_outertuple = outerslot;
 
-       /* Find or build hashtable entry for this tuple's group */
-       entry = lookup_hash_entry(aggstate, outerslot);
+       /* Find or build hashtable entries */
+       pergroups = lookup_hash_entries(aggstate);
 
        /* Advance the aggregates */
        if (DO_AGGSPLIT_COMBINE(aggstate->aggsplit))
-           combine_aggregates(aggstate, (AggStatePerGroup) entry->additional);
+           combine_aggregates(aggstate, pergroups[0]);
        else
-           advance_aggregates(aggstate, (AggStatePerGroup) entry->additional);
+           advance_aggregates(aggstate, NULL, pergroups);
 
-       /* Reset per-input-tuple context after each tuple */
-       ResetExprContext(tmpcontext);
+       /*
+        * Reset per-input-tuple context after each tuple, but note that the
+        * hash lookups do this too
+        */
+       ResetExprContext(aggstate->tmpcontext);
    }
 
    aggstate->table_filled = true;
-   /* Initialize to walk the hash table */
-   ResetTupleHashIterator(aggstate->hashtable, &aggstate->hashiter);
+   /* Initialize to walk the first hash table */
+   select_current_set(aggstate, 0, true);
+   ResetTupleHashIterator(aggstate->perhash[0].hashtable,
+                          &aggstate->perhash[0].hashiter);
 }
 
 /*
- * ExecAgg for hashed case: phase 2, retrieving groups from hash table
+ * ExecAgg for hashed case: retrieving groups from hash table
  */
 static TupleTableSlot *
 agg_retrieve_hash_table(AggState *aggstate)
@@ -2305,17 +2535,22 @@ agg_retrieve_hash_table(AggState *aggstate)
    TupleHashEntryData *entry;
    TupleTableSlot *firstSlot;
    TupleTableSlot *result;
-   TupleTableSlot *hashslot;
+   AggStatePerHash perhash;
 
    /*
-    * get state info from node
+    * get state info from node.
+    *
+    * econtext is the per-output-tuple expression context.
     */
-   /* econtext is the per-output-tuple expression context */
    econtext = aggstate->ss.ps.ps_ExprContext;
    peragg = aggstate->peragg;
    firstSlot = aggstate->ss.ss_ScanTupleSlot;
-   hashslot = aggstate->hashslot;
 
+   /*
+    * Note that perhash (and therefore anything accessed through it) can
+    * change inside the loop, as we change between grouping sets.
+    */
+   perhash = &aggstate->perhash[aggstate->current_set];
 
    /*
     * We loop retrieving groups until we find one satisfying
@@ -2323,17 +2558,37 @@ agg_retrieve_hash_table(AggState *aggstate)
     */
    while (!aggstate->agg_done)
    {
-       int i;
+       TupleTableSlot *hashslot = perhash->hashslot;
+       int         i;
 
        /*
         * Find the next entry in the hash table
         */
-       entry = ScanTupleHashTable(aggstate->hashtable, &aggstate->hashiter);
+       entry = ScanTupleHashTable(perhash->hashtable, &perhash->hashiter);
        if (entry == NULL)
        {
-           /* No more entries in hashtable, so done */
-           aggstate->agg_done = TRUE;
-           return NULL;
+           int         nextset = aggstate->current_set + 1;
+
+           if (nextset < aggstate->num_hashes)
+           {
+               /*
+                * Switch to next grouping set, reinitialize, and restart the
+                * loop.
+                */
+               select_current_set(aggstate, nextset, true);
+
+               perhash = &aggstate->perhash[aggstate->current_set];
+
+               ResetTupleHashIterator(perhash->hashtable, &perhash->hashiter);
+
+               continue;
+           }
+           else
+           {
+               /* No more hashtables, so done */
+               aggstate->agg_done = TRUE;
+               return NULL;
+           }
        }
 
        /*
@@ -2356,9 +2611,9 @@ agg_retrieve_hash_table(AggState *aggstate)
        memset(firstSlot->tts_isnull, true,
               firstSlot->tts_tupleDescriptor->natts * sizeof(bool));
 
-       for (i = 0; i < aggstate->numhashGrpCols; i++)
+       for (i = 0; i < perhash->numhashGrpCols; i++)
        {
-           int         varNumber = aggstate->hashGrpColIdxInput[i] - 1;
+           int         varNumber = perhash->hashGrpColIdxInput[i] - 1;
 
            firstSlot->tts_values[varNumber] = hashslot->tts_values[i];
            firstSlot->tts_isnull[varNumber] = hashslot->tts_isnull[i];
@@ -2367,14 +2622,18 @@ agg_retrieve_hash_table(AggState *aggstate)
 
        pergroup = (AggStatePerGroup) entry->additional;
 
-       finalize_aggregates(aggstate, peragg, pergroup, 0);
-
        /*
         * Use the representative input tuple for any references to
         * non-aggregated input columns in the qual and tlist.
         */
        econtext->ecxt_outertuple = firstSlot;
 
+       prepare_projection_slot(aggstate,
+                               econtext->ecxt_outertuple,
+                               aggstate->current_set);
+
+       finalize_aggregates(aggstate, peragg, pergroup);
+
        result = project_aggregates(aggstate);
        if (result)
            return result;
@@ -2388,7 +2647,8 @@ agg_retrieve_hash_table(AggState *aggstate)
  * ExecInitAgg
  *
  * Creates the run-time information for the agg node produced by the
- * planner and initializes its outer subtree
+ * planner and initializes its outer subtree.
+ *
  * -----------------
  */
 AggState *
@@ -2403,14 +2663,18 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
                transno,
                aggno;
    int         phase;
+   int         phaseidx;
    List       *combined_inputeval;
    ListCell   *l;
    Bitmapset  *all_grouped_cols = NULL;
    int         numGroupingSets = 1;
    int         numPhases;
+   int         numHashes;
    int         column_offset;
    int         i = 0;
    int         j = 0;
+   bool        use_hashing = (node->aggstrategy == AGG_HASHED ||
+                              node->aggstrategy == AGG_MIXED);
 
    /* check for unsupported flags */
    Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK)));
@@ -2425,9 +2689,9 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
    aggstate->aggs = NIL;
    aggstate->numaggs = 0;
    aggstate->numtrans = 0;
+   aggstate->aggstrategy = node->aggstrategy;
    aggstate->aggsplit = node->aggsplit;
    aggstate->maxsets = 0;
-   aggstate->hashfunctions = NULL;
    aggstate->projected_set = -1;
    aggstate->current_set = 0;
    aggstate->peragg = NULL;
@@ -2437,18 +2701,22 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
    aggstate->agg_done = false;
    aggstate->pergroup = NULL;
    aggstate->grp_firstTuple = NULL;
-   aggstate->hashtable = NULL;
    aggstate->sort_in = NULL;
    aggstate->sort_out = NULL;
 
+   /*
+    * phases[0] always exists, but is dummy in sorted/plain mode
+    */
+   numPhases = (use_hashing ? 1 : 2);
+   numHashes = (use_hashing ? 1 : 0);
+
    /*
     * Calculate the maximum number of grouping sets in any phase; this
-    * determines the size of some allocations.
+    * determines the size of some allocations.  Also calculate the number of
+    * phases, since all hashed/mixed nodes contribute to only a single phase.
     */
    if (node->groupingSets)
    {
-       Assert(node->aggstrategy != AGG_HASHED);
-
        numGroupingSets = list_length(node->groupingSets);
 
        foreach(l, node->chain)
@@ -2457,22 +2725,32 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
 
            numGroupingSets = Max(numGroupingSets,
                                  list_length(agg->groupingSets));
+
+           /*
+            * additional AGG_HASHED aggs become part of phase 0, but all
+            * others add an extra phase.
+            */
+           if (agg->aggstrategy != AGG_HASHED)
+               ++numPhases;
+           else
+               ++numHashes;
        }
    }
 
    aggstate->maxsets = numGroupingSets;
-   aggstate->numphases = numPhases = 1 + list_length(node->chain);
+   aggstate->numphases = numPhases;
 
    aggstate->aggcontexts = (ExprContext **)
        palloc0(sizeof(ExprContext *) * numGroupingSets);
 
    /*
     * Create expression contexts.  We need three or more, one for
-    * per-input-tuple processing, one for per-output-tuple processing, and
-    * one for each grouping set.  The per-tuple memory context of the
-    * per-grouping-set ExprContexts (aggcontexts) replaces the standalone
-    * memory context formerly used to hold transition values.  We cheat a
-    * little by using ExecAssignExprContext() to build all of them.
+    * per-input-tuple processing, one for per-output-tuple processing, one
+    * for all the hashtables, and one for each grouping set.  The per-tuple
+    * memory context of the per-grouping-set ExprContexts (aggcontexts)
+    * replaces the standalone memory context formerly used to hold transition
+    * values.  We cheat a little by using ExecAssignExprContext() to build
+    * all of them.
     *
     * NOTE: the details of what is stored in aggcontexts and what is stored
     * in the regular per-query memory context are driven by a simple
@@ -2488,14 +2766,21 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
        aggstate->aggcontexts[i] = aggstate->ss.ps.ps_ExprContext;
    }
 
+   if (use_hashing)
+   {
+       ExecAssignExprContext(estate, &aggstate->ss.ps);
+       aggstate->hashcontext = aggstate->ss.ps.ps_ExprContext;
+   }
+
    ExecAssignExprContext(estate, &aggstate->ss.ps);
 
    /*
-    * tuple table initialization
+    * tuple table initialization.
+    *
+    * For hashtables, we create some additional slots below.
     */
    ExecInitScanTupleSlot(estate, &aggstate->ss);
    ExecInitResultTupleSlot(estate, &aggstate->ss.ps);
-   aggstate->hashslot = ExecInitExtraTupleSlot(estate);
    aggstate->sort_slot = ExecInitExtraTupleSlot(estate);
 
    /*
@@ -2559,19 +2844,26 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
     * For each phase, prepare grouping set data and fmgr lookup data for
     * compare functions.  Accumulate all_grouped_cols in passing.
     */
-
    aggstate->phases = palloc0(numPhases * sizeof(AggStatePerPhaseData));
 
-   for (phase = 0; phase < numPhases; ++phase)
+   aggstate->num_hashes = numHashes;
+   if (numHashes)
+   {
+       aggstate->perhash = palloc0(sizeof(AggStatePerHashData) * numHashes);
+       aggstate->phases[0].numsets = 0;
+       aggstate->phases[0].gset_lengths = palloc(numHashes * sizeof(int));
+       aggstate->phases[0].grouped_cols = palloc(numHashes * sizeof(Bitmapset *));
+   }
+
+   phase = 0;
+   for (phaseidx = 0; phaseidx <= list_length(node->chain); ++phaseidx)
    {
-       AggStatePerPhase phasedata = &aggstate->phases[phase];
        Agg        *aggnode;
        Sort       *sortnode;
-       int         num_sets;
 
-       if (phase > 0)
+       if (phaseidx > 0)
        {
-           aggnode = castNode(Agg, list_nth(node->chain, phase - 1));
+           aggnode = castNode(Agg, list_nth(node->chain, phaseidx - 1));
            sortnode = castNode(Sort, aggnode->plan.lefttree);
        }
        else
@@ -2580,53 +2872,91 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
            sortnode = NULL;
        }
 
-       phasedata->numsets = num_sets = list_length(aggnode->groupingSets);
+       Assert(phase <= 1 || sortnode);
 
-       if (num_sets)
+       if (aggnode->aggstrategy == AGG_HASHED
+           || aggnode->aggstrategy == AGG_MIXED)
        {
-           phasedata->gset_lengths = palloc(num_sets * sizeof(int));
-           phasedata->grouped_cols = palloc(num_sets * sizeof(Bitmapset *));
+           AggStatePerPhase phasedata = &aggstate->phases[0];
+           AggStatePerHash perhash;
+           Bitmapset  *cols = NULL;
 
-           i = 0;
-           foreach(l, aggnode->groupingSets)
-           {
-               int         current_length = list_length(lfirst(l));
-               Bitmapset  *cols = NULL;
+           Assert(phase == 0);
+           i = phasedata->numsets++;
+           perhash = &aggstate->perhash[i];
 
-               /* planner forces this to be correct */
-               for (j = 0; j < current_length; ++j)
-                   cols = bms_add_member(cols, aggnode->grpColIdx[j]);
+           /* phase 0 always points to the "real" Agg in the hash case */
+           phasedata->aggnode = node;
+           phasedata->aggstrategy = node->aggstrategy;
 
-               phasedata->grouped_cols[i] = cols;
-               phasedata->gset_lengths[i] = current_length;
-               ++i;
-           }
+           /* but the actual Agg node representing this hash is saved here */
+           perhash->aggnode = aggnode;
+
+           phasedata->gset_lengths[i] = perhash->numCols = aggnode->numCols;
+
+           for (j = 0; j < aggnode->numCols; ++j)
+               cols = bms_add_member(cols, aggnode->grpColIdx[j]);
+
+           phasedata->grouped_cols[i] = cols;
 
-           all_grouped_cols = bms_add_members(all_grouped_cols,
-                                              phasedata->grouped_cols[0]);
+           all_grouped_cols = bms_add_members(all_grouped_cols, cols);
+           continue;
        }
        else
        {
-           Assert(phase == 0);
+           AggStatePerPhase phasedata = &aggstate->phases[++phase];
+           int         num_sets;
 
-           phasedata->gset_lengths = NULL;
-           phasedata->grouped_cols = NULL;
-       }
+           phasedata->numsets = num_sets = list_length(aggnode->groupingSets);
 
-       /*
-        * If we are grouping, precompute fmgr lookup data for inner loop.
-        */
-       if (aggnode->aggstrategy == AGG_SORTED)
-       {
-           Assert(aggnode->numCols > 0);
+           if (num_sets)
+           {
+               phasedata->gset_lengths = palloc(num_sets * sizeof(int));
+               phasedata->grouped_cols = palloc(num_sets * sizeof(Bitmapset *));
 
-           phasedata->eqfunctions =
-               execTuplesMatchPrepare(aggnode->numCols,
-                                      aggnode->grpOperators);
-       }
+               i = 0;
+               foreach(l, aggnode->groupingSets)
+               {
+                   int         current_length = list_length(lfirst(l));
+                   Bitmapset  *cols = NULL;
+
+                   /* planner forces this to be correct */
+                   for (j = 0; j < current_length; ++j)
+                       cols = bms_add_member(cols, aggnode->grpColIdx[j]);
+
+                   phasedata->grouped_cols[i] = cols;
+                   phasedata->gset_lengths[i] = current_length;
+
+                   ++i;
+               }
+
+               all_grouped_cols = bms_add_members(all_grouped_cols,
+                                                phasedata->grouped_cols[0]);
+           }
+           else
+           {
+               Assert(phaseidx == 0);
+
+               phasedata->gset_lengths = NULL;
+               phasedata->grouped_cols = NULL;
+           }
+
+           /*
+            * If we are grouping, precompute fmgr lookup data for inner loop.
+            */
+           if (aggnode->aggstrategy == AGG_SORTED)
+           {
+               Assert(aggnode->numCols > 0);
+
+               phasedata->eqfunctions =
+                   execTuplesMatchPrepare(aggnode->numCols,
+                                          aggnode->grpOperators);
+           }
 
-       phasedata->aggnode = aggnode;
-       phasedata->sortnode = sortnode;
+           phasedata->aggnode = aggnode;
+           phasedata->aggstrategy = aggnode->aggstrategy;
+           phasedata->sortnode = sortnode;
+       }
    }
 
    /*
@@ -2636,13 +2966,6 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
    while ((i = bms_next_member(all_grouped_cols, i)) >= 0)
        aggstate->all_grouped_cols = lcons_int(i, aggstate->all_grouped_cols);
 
-   /*
-    * Initialize current phase-dependent values to initial phase
-    */
-
-   aggstate->current_phase = 0;
-   initialize_phase(aggstate, 0);
-
    /*
     * Set up aggregate-result storage in the output expr context, and also
     * allocate my private per-agg working storage
@@ -2657,23 +2980,30 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
    aggstate->peragg = peraggs;
    aggstate->pertrans = pertransstates;
 
-
    /*
     * Hashing can only appear in the initial phase.
     */
-   if (node->aggstrategy == AGG_HASHED)
+   if (use_hashing)
    {
-       find_hash_columns(aggstate);
+       for (i = 0; i < numHashes; ++i)
+       {
+           aggstate->perhash[i].hashslot = ExecInitExtraTupleSlot(estate);
+
+           execTuplesHashPrepare(aggstate->perhash[i].numCols,
+                                 aggstate->perhash[i].aggnode->grpOperators,
+                                 &aggstate->perhash[i].eqfunctions,
+                                 &aggstate->perhash[i].hashfunctions);
+       }
 
-       execTuplesHashPrepare(node->numCols,
-                             node->grpOperators,
-                             &aggstate->phases[0].eqfunctions,
-                             &aggstate->hashfunctions);
+       /* this is an array of pointers, not structures */
+       aggstate->hash_pergroup = palloc0(sizeof(AggStatePerGroup) * numHashes);
 
+       find_hash_columns(aggstate);
        build_hash_table(aggstate);
        aggstate->table_filled = false;
    }
-   else
+
+   if (node->aggstrategy != AGG_HASHED)
    {
        AggStatePerGroup pergroup;
 
@@ -2684,6 +3014,25 @@ ExecInitAgg(Agg *node, EState *estate, int eflags)
        aggstate->pergroup = pergroup;
    }
 
+   /*
+    * Initialize current phase-dependent values to initial phase. The initial
+    * phase is 1 (first sort pass) for all strategies that use sorting (if
+    * hashing is being done too, then phase 0 is processed last); but if only
+    * hashing is being done, then phase 0 is all there is.
+    */
+   if (node->aggstrategy == AGG_HASHED)
+   {
+       aggstate->current_phase = 0;
+       initialize_phase(aggstate, 0);
+       select_current_set(aggstate, 0, true);
+   }
+   else
+   {
+       aggstate->current_phase = 1;
+       initialize_phase(aggstate, 1);
+       select_current_set(aggstate, 0, false);
+   }
+
    /* -----------------
     * Perform lookups of aggregate function info, and initialize the
     * unchanging fields of the per-agg and per-trans data.
@@ -3261,7 +3610,7 @@ build_pertrans_for_aggref(AggStatePerTrans pertrans,
         * We don't implement DISTINCT or ORDER BY aggs in the HASHED case
         * (yet)
         */
-       Assert(((Agg *) aggstate->ss.ps.plan)->aggstrategy != AGG_HASHED);
+       Assert(aggstate->aggstrategy != AGG_HASHED && aggstate->aggstrategy != AGG_MIXED);
 
        /* If we have only one input, we need its len/byval info. */
        if (numInputs == 1)
@@ -3510,6 +3859,8 @@ ExecEndAgg(AggState *node)
    /* And ensure any agg shutdown callbacks have been called */
    for (setno = 0; setno < numGroupingSets; setno++)
        ReScanExprContext(node->aggcontexts[setno]);
+   if (node->hashcontext)
+       ReScanExprContext(node->hashcontext);
 
    /*
     * We don't actually free any ExprContexts here (see comment in
@@ -3537,7 +3888,7 @@ ExecReScanAgg(AggState *node)
 
    node->agg_done = false;
 
-   if (aggnode->aggstrategy == AGG_HASHED)
+   if (node->aggstrategy == AGG_HASHED)
    {
        /*
         * In the hashed case, if we haven't yet built the hash table then we
@@ -3557,7 +3908,9 @@ ExecReScanAgg(AggState *node)
        if (outerPlan->chgParam == NULL &&
            !bms_overlap(node->ss.ps.chgParam, aggnode->aggParams))
        {
-           ResetTupleHashIterator(node->hashtable, &node->hashiter);
+           ResetTupleHashIterator(node->perhash[0].hashtable,
+                                  &node->perhash[0].hashiter);
+           select_current_set(node, 0, true);
            return;
        }
    }
@@ -3582,11 +3935,7 @@ ExecReScanAgg(AggState *node)
     * ExecReScan already did it. But we do need to reset our per-grouping-set
     * contexts, which may have transvalues stored in them. (We use rescan
     * rather than just reset because transfns may have registered callbacks
-    * that need to be run now.)
-    *
-    * Note that with AGG_HASHED, the hash table is allocated in a sub-context
-    * of the aggcontext. This used to be an issue, but now, resetting a
-    * context automatically deletes sub-contexts too.
+    * that need to be run now.) For the AGG_HASHED case, see below.
     */
 
    for (setno = 0; setno < numGroupingSets; setno++)
@@ -3606,13 +3955,21 @@ ExecReScanAgg(AggState *node)
    MemSet(econtext->ecxt_aggvalues, 0, sizeof(Datum) * node->numaggs);
    MemSet(econtext->ecxt_aggnulls, 0, sizeof(bool) * node->numaggs);
 
-   if (aggnode->aggstrategy == AGG_HASHED)
+   /*
+    * With AGG_HASHED/MIXED, the hash table is allocated in a sub-context of
+    * the hashcontext. This used to be an issue, but now, resetting a context
+    * automatically deletes sub-contexts too.
+    */
+   if (node->aggstrategy == AGG_HASHED || node->aggstrategy == AGG_MIXED)
    {
+       ReScanExprContext(node->hashcontext);
        /* Rebuild an empty hash table */
        build_hash_table(node);
        node->table_filled = false;
+       /* iterator will be reset when the table is filled */
    }
-   else
+
+   if (node->aggstrategy != AGG_HASHED)
    {
        /*
         * Reset the per-group state (in particular, mark transvalues null)
@@ -3620,8 +3977,8 @@ ExecReScanAgg(AggState *node)
        MemSet(node->pergroup, 0,
             sizeof(AggStatePerGroupData) * node->numaggs * numGroupingSets);
 
-       /* reset to phase 0 */
-       initialize_phase(node, 0);
+       /* reset to phase 1 */
+       initialize_phase(node, 1);
 
        node->input_done = false;
        node->projected_set = -1;
@@ -3662,7 +4019,7 @@ AggCheckCallContext(FunctionCallInfo fcinfo, MemoryContext *aggcontext)
        if (aggcontext)
        {
            AggState   *aggstate = ((AggState *) fcinfo->context);
-           ExprContext *cxt = aggstate->aggcontexts[aggstate->current_set];
+           ExprContext *cxt = aggstate->curaggcontext;
 
            *aggcontext = cxt->ecxt_per_tuple_memory;
        }
@@ -3751,7 +4108,7 @@ AggRegisterCallback(FunctionCallInfo fcinfo,
    if (fcinfo->context && IsA(fcinfo->context, AggState))
    {
        AggState   *aggstate = (AggState *) fcinfo->context;
-       ExprContext *cxt = aggstate->aggcontexts[aggstate->current_set];
+       ExprContext *cxt = aggstate->curaggcontext;
 
        RegisterExprContextCallback(cxt, func, arg);
 
index 2d2ba84fe9d1202a5f0c51307ccaca8529b57245..f222c6c20d19be5d84ac980c6420340ad7792217 100644 (file)
@@ -12,7 +12,7 @@ subdir = src/backend/lib
 top_builddir = ../../..
 include $(top_builddir)/src/Makefile.global
 
-OBJS = binaryheap.o bipartite_match.o hyperloglog.o ilist.o pairingheap.o \
-       rbtree.o stringinfo.o
+OBJS = binaryheap.o bipartite_match.o hyperloglog.o ilist.o knapsack.o \
+       pairingheap.o rbtree.o stringinfo.o
 
 include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/lib/knapsack.c b/src/backend/lib/knapsack.c
new file mode 100644 (file)
index 0000000..ddf2b9a
--- /dev/null
@@ -0,0 +1,114 @@
+/*-------------------------------------------------------------------------
+ *
+ * knapsack.c
+ *   Knapsack problem solver
+ *
+ * Given input vectors of integral item weights (must be >= 0) and values
+ * (double >= 0), compute the set of items which produces the greatest total
+ * value without exceeding a specified total weight; each item is included at
+ * most once (this is the 0/1 knapsack problem).  Weight 0 items will always be
+ * included.
+ *
+ * The performance of this algorithm is pseudo-polynomial, O(nW) where W is the
+ * weight limit.  To use with non-integral weights or approximate solutions,
+ * the caller should pre-scale the input weights to a suitable range.  This
+ * allows approximate solutions in polynomial time (the general case of the
+ * exact problem is NP-hard).
+ *
+ * Copyright (c) 2017, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ *   src/backend/lib/knapsack.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include <math.h>
+#include <limits.h>
+
+#include "lib/knapsack.h"
+#include "miscadmin.h"
+#include "nodes/bitmapset.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/palloc.h"
+
+/*
+ * DiscreteKnapsack
+ *
+ * The item_values input is optional; if omitted, all the items are assumed to
+ * have value 1.
+ *
+ * Returns a Bitmapset of the 0..(n-1) indexes of the items chosen for
+ * inclusion in the solution.
+ *
+ * This uses the usual dynamic-programming algorithm, adapted to reuse the
+ * memory on each pass (by working from larger weights to smaller).  At the
+ * start of pass number i, the values[w] array contains the largest value
+ * computed with total weight <= w, using only items with indices < i; and
+ * sets[w] contains the bitmap of items actually used for that value.  (The
+ * bitmapsets are all pre-initialized with an unused high bit so that memory
+ * allocation is done only once.)
+ */
+Bitmapset *
+DiscreteKnapsack(int max_weight, int num_items,
+                int *item_weights, double *item_values)
+{
+   MemoryContext local_ctx = AllocSetContextCreate(CurrentMemoryContext,
+                                                   "Knapsack",
+                                                   ALLOCSET_SMALL_MINSIZE,
+                                                   ALLOCSET_SMALL_INITSIZE,
+                                                   ALLOCSET_SMALL_MAXSIZE);
+   MemoryContext oldctx = MemoryContextSwitchTo(local_ctx);
+   double     *values;
+   Bitmapset **sets;
+   Bitmapset  *result;
+   int         i,
+               j;
+
+   Assert(max_weight >= 0);
+   Assert(num_items > 0 && item_weights);
+
+   values = palloc((1 + max_weight) * sizeof(double));
+   sets = palloc((1 + max_weight) * sizeof(Bitmapset *));
+
+   for (i = 0; i <= max_weight; ++i)
+   {
+       values[i] = 0;
+       sets[i] = bms_make_singleton(num_items);
+   }
+
+   for (i = 0; i < num_items; ++i)
+   {
+       int         iw = item_weights[i];
+       double      iv = item_values ? item_values[i] : 1;
+
+       for (j = max_weight; j >= iw; --j)
+       {
+           int         ow = j - iw;
+
+           if (values[j] <= values[ow] + iv)
+           {
+               /* copy sets[ow] to sets[j] without realloc */
+               if (j != ow)
+               {
+                   sets[j] = bms_del_members(sets[j], sets[j]);
+                   sets[j] = bms_add_members(sets[j], sets[ow]);
+               }
+
+               sets[j] = bms_add_member(sets[j], i);
+
+               values[j] = values[ow] + iv;
+           }
+       }
+   }
+
+   MemoryContextSwitchTo(oldctx);
+
+   result = bms_del_member(bms_copy(sets[max_weight]), num_items);
+
+   MemoryContextDelete(local_ctx);
+
+   return result;
+}
index 252af5c870972e233190ecbaaefb7af657474f47..bf8545d43782e24f9f2bd9f94e5f979f6dd97f1e 100644 (file)
@@ -21,6 +21,7 @@
 #include "postgres.h"
 
 #include "access/hash.h"
+#include "nodes/pg_list.h"
 
 
 #define WORDNUM(x) ((x) / BITS_PER_BITMAPWORD)
@@ -457,6 +458,35 @@ bms_overlap(const Bitmapset *a, const Bitmapset *b)
    return false;
 }
 
+/*
+ * bms_overlap_list - does a set overlap an integer list?
+ */
+bool
+bms_overlap_list(const Bitmapset *a, const List *b)
+{
+   ListCell   *lc;
+   int         wordnum,
+               bitnum;
+
+   if (a == NULL || b == NIL)
+       return false;
+
+   foreach(lc, b)
+   {
+       int         x = lfirst_int(lc);
+
+       if (x < 0)
+           elog(ERROR, "negative bitmapset member not allowed");
+       wordnum = WORDNUM(x);
+       bitnum = BITNUM(x);
+       if (wordnum < a->nwords)
+           if ((a->words[wordnum] & ((bitmapword) 1 << bitnum)) != 0)
+               return true;
+   }
+
+   return false;
+}
+
 /*
  * bms_nonempty_difference - do sets have a nonempty difference?
  */
index 541af029353382d50b19884a178fbe74ad76dcc2..bbb63a4bfae9092dbc85bc52ab0363a4ca62a771 100644 (file)
@@ -1941,6 +1941,28 @@ _outAggPath(StringInfo str, const AggPath *node)
    WRITE_NODE_FIELD(qual);
 }
 
+static void
+_outRollupData(StringInfo str, const RollupData *node)
+{
+   WRITE_NODE_TYPE("ROLLUP");
+
+   WRITE_NODE_FIELD(groupClause);
+   WRITE_NODE_FIELD(gsets);
+   WRITE_NODE_FIELD(gsets_data);
+   WRITE_FLOAT_FIELD(numGroups, "%.0f");
+   WRITE_BOOL_FIELD(hashable);
+   WRITE_BOOL_FIELD(is_hashed);
+}
+
+static void
+_outGroupingSetData(StringInfo str, const GroupingSetData *node)
+{
+   WRITE_NODE_TYPE("GSDATA");
+
+   WRITE_NODE_FIELD(set);
+   WRITE_FLOAT_FIELD(numGroups, "%.0f");
+}
+
 static void
 _outGroupingSetsPath(StringInfo str, const GroupingSetsPath *node)
 {
@@ -1949,8 +1971,8 @@ _outGroupingSetsPath(StringInfo str, const GroupingSetsPath *node)
    _outPathInfo(str, (const Path *) node);
 
    WRITE_NODE_FIELD(subpath);
-   WRITE_NODE_FIELD(rollup_groupclauses);
-   WRITE_NODE_FIELD(rollup_lists);
+   WRITE_ENUM_FIELD(aggstrategy, AggStrategy);
+   WRITE_NODE_FIELD(rollups);
    WRITE_NODE_FIELD(qual);
 }
 
@@ -3961,14 +3983,18 @@ outNode(StringInfo str, const void *obj)
            case T_PlannerParamItem:
                _outPlannerParamItem(str, obj);
                break;
+           case T_RollupData:
+               _outRollupData(str, obj);
+               break;
+           case T_GroupingSetData:
+               _outGroupingSetData(str, obj);
+               break;
            case T_StatisticExtInfo:
                _outStatisticExtInfo(str, obj);
                break;
-
            case T_ExtensibleNode:
                _outExtensibleNode(str, obj);
                break;
-
            case T_CreateStmt:
                _outCreateStmt(str, obj);
                break;
index 57229059bda29b53bce5bffd06e5724009c9ec54..92de2b7d4808e145e0b44dbea9d2a5cad204bfcb 100644 (file)
@@ -1884,11 +1884,16 @@ cost_agg(Path *path, PlannerInfo *root,
        total_cost = startup_cost + cpu_tuple_cost;
        output_tuples = 1;
    }
-   else if (aggstrategy == AGG_SORTED)
+   else if (aggstrategy == AGG_SORTED || aggstrategy == AGG_MIXED)
    {
        /* Here we are able to deliver output on-the-fly */
        startup_cost = input_startup_cost;
        total_cost = input_total_cost;
+       if (aggstrategy == AGG_MIXED && !enable_hashagg)
+       {
+           startup_cost += disable_cost;
+           total_cost += disable_cost;
+       }
        /* calcs phrased this way to match HASHED case, see note above */
        total_cost += aggcosts->transCost.startup;
        total_cost += aggcosts->transCost.per_tuple * input_tuples;
index c80c9992c9f78605f91f786abfaa712e96d81628..aafec58281b6caf32c2c3327baeb0545f8275fae 100644 (file)
@@ -1783,18 +1783,15 @@ create_groupingsets_plan(PlannerInfo *root, GroupingSetsPath *best_path)
 {
    Agg        *plan;
    Plan       *subplan;
-   List       *rollup_groupclauses = best_path->rollup_groupclauses;
-   List       *rollup_lists = best_path->rollup_lists;
+   List       *rollups = best_path->rollups;
    AttrNumber *grouping_map;
    int         maxref;
    List       *chain;
-   ListCell   *lc,
-              *lc2;
+   ListCell   *lc;
 
    /* Shouldn't get here without grouping sets */
    Assert(root->parse->groupingSets);
-   Assert(rollup_lists != NIL);
-   Assert(list_length(rollup_lists) == list_length(rollup_groupclauses));
+   Assert(rollups != NIL);
 
    /*
     * Agg can project, so no need to be terribly picky about child tlist, but
@@ -1846,72 +1843,86 @@ create_groupingsets_plan(PlannerInfo *root, GroupingSetsPath *best_path)
     * costs will be shown by EXPLAIN.
     */
    chain = NIL;
-   if (list_length(rollup_groupclauses) > 1)
+   if (list_length(rollups) > 1)
    {
-       forboth(lc, rollup_groupclauses, lc2, rollup_lists)
+       ListCell   *lc2 = lnext(list_head(rollups));
+       bool        is_first_sort = ((RollupData *) linitial(rollups))->is_hashed;
+
+       for_each_cell(lc, lc2)
        {
-           List       *groupClause = (List *) lfirst(lc);
-           List       *gsets = (List *) lfirst(lc2);
+           RollupData *rollup = lfirst(lc);
            AttrNumber *new_grpColIdx;
-           Plan       *sort_plan;
+           Plan       *sort_plan = NULL;
            Plan       *agg_plan;
+           AggStrategy strat;
 
-           /* We want to iterate over all but the last rollup list elements */
-           if (lnext(lc) == NULL)
-               break;
+           new_grpColIdx = remap_groupColIdx(root, rollup->groupClause);
+
+           if (!rollup->is_hashed && !is_first_sort)
+           {
+               sort_plan = (Plan *)
+                   make_sort_from_groupcols(rollup->groupClause,
+                                            new_grpColIdx,
+                                            subplan);
+           }
 
-           new_grpColIdx = remap_groupColIdx(root, groupClause);
+           if (!rollup->is_hashed)
+               is_first_sort = false;
 
-           sort_plan = (Plan *)
-               make_sort_from_groupcols(groupClause,
-                                        new_grpColIdx,
-                                        subplan);
+           if (rollup->is_hashed)
+               strat = AGG_HASHED;
+           else if (list_length(linitial(rollup->gsets)) == 0)
+               strat = AGG_PLAIN;
+           else
+               strat = AGG_SORTED;
 
            agg_plan = (Plan *) make_agg(NIL,
                                         NIL,
-                                        AGG_SORTED,
+                                        strat,
                                         AGGSPLIT_SIMPLE,
-                                      list_length((List *) linitial(gsets)),
+                              list_length((List *) linitial(rollup->gsets)),
                                         new_grpColIdx,
-                                        extract_grouping_ops(groupClause),
-                                        gsets,
+                                  extract_grouping_ops(rollup->groupClause),
+                                        rollup->gsets,
                                         NIL,
-                                        0,     /* numGroups not needed */
+                                        rollup->numGroups,
                                         sort_plan);
 
            /*
-            * Nuke stuff we don't need to avoid bloating debug output.
+            * Remove stuff we don't need to avoid bloating debug output.
             */
-           sort_plan->targetlist = NIL;
-           sort_plan->lefttree = NULL;
+           if (sort_plan)
+           {
+               sort_plan->targetlist = NIL;
+               sort_plan->lefttree = NULL;
+           }
 
            chain = lappend(chain, agg_plan);
        }
    }
 
    /*
-    * Now make the final Agg node
+    * Now make the real Agg node
     */
    {
-       List       *groupClause = (List *) llast(rollup_groupclauses);
-       List       *gsets = (List *) llast(rollup_lists);
+       RollupData *rollup = linitial(rollups);
        AttrNumber *top_grpColIdx;
        int         numGroupCols;
 
-       top_grpColIdx = remap_groupColIdx(root, groupClause);
+       top_grpColIdx = remap_groupColIdx(root, rollup->groupClause);
 
-       numGroupCols = list_length((List *) linitial(gsets));
+       numGroupCols = list_length((List *) linitial(rollup->gsets));
 
        plan = make_agg(build_path_tlist(root, &best_path->path),
                        best_path->qual,
-                       (numGroupCols > 0) ? AGG_SORTED : AGG_PLAIN,
+                       best_path->aggstrategy,
                        AGGSPLIT_SIMPLE,
                        numGroupCols,
                        top_grpColIdx,
-                       extract_grouping_ops(groupClause),
-                       gsets,
+                       extract_grouping_ops(rollup->groupClause),
+                       rollup->gsets,
                        chain,
-                       0,      /* numGroups not needed */
+                       rollup->numGroups,
                        subplan);
 
        /* Copy cost data from Path to Plan */
index 90619509a2aa7d320a390929ea282a1e9b03fdf0..fa7a5f8427744a8002d8f10e8ed5001fdf2dcf76 100644 (file)
@@ -30,6 +30,7 @@
 #include "foreign/fdwapi.h"
 #include "miscadmin.h"
 #include "lib/bipartite_match.h"
+#include "lib/knapsack.h"
 #include "nodes/makefuncs.h"
 #include "nodes/nodeFuncs.h"
 #ifdef OPTIMIZER_DEBUG
@@ -91,12 +92,31 @@ typedef struct
    List       *groupClause;    /* overrides parse->groupClause */
 } standard_qp_extra;
 
+/*
+ * Data specific to grouping sets
+ */
+
+typedef struct
+{
+   List       *rollups;
+   List       *hash_sets_idx;
+   double      dNumHashGroups;
+   bool        any_hashable;
+   Bitmapset  *unsortable_refs;
+   Bitmapset  *unhashable_refs;
+   List       *unsortable_sets;
+   int        *tleref_to_colnum_map;
+} grouping_sets_data;
+
 /* Local functions */
 static Node *preprocess_expression(PlannerInfo *root, Node *expr, int kind);
 static void preprocess_qual_conditions(PlannerInfo *root, Node *jtnode);
 static void inheritance_planner(PlannerInfo *root);
 static void grouping_planner(PlannerInfo *root, bool inheritance_update,
                 double tuple_fraction);
+static grouping_sets_data *preprocess_grouping_sets(PlannerInfo *root);
+static List *remap_to_groupclause_idx(List *groupClause, List *gsets,
+                        int *tleref_to_colnum_map);
 static void preprocess_rowmarks(PlannerInfo *root);
 static double preprocess_limit(PlannerInfo *root,
                 double tuple_fraction,
@@ -109,8 +129,7 @@ static List *reorder_grouping_sets(List *groupingSets, List *sortclause);
 static void standard_qp_callback(PlannerInfo *root, void *extra);
 static double get_number_of_groups(PlannerInfo *root,
                     double path_rows,
-                    List *rollup_lists,
-                    List *rollup_groupclauses);
+                    grouping_sets_data *gd);
 static Size estimate_hashagg_tablesize(Path *path,
                           const AggClauseCosts *agg_costs,
                           double dNumGroups);
@@ -118,8 +137,16 @@ static RelOptInfo *create_grouping_paths(PlannerInfo *root,
                      RelOptInfo *input_rel,
                      PathTarget *target,
                      const AggClauseCosts *agg_costs,
-                     List *rollup_lists,
-                     List *rollup_groupclauses);
+                     grouping_sets_data *gd);
+static void consider_groupingsets_paths(PlannerInfo *root,
+                           RelOptInfo *grouped_rel,
+                           Path *path,
+                           bool is_sorted,
+                           bool can_hash,
+                           PathTarget *target,
+                           grouping_sets_data *gd,
+                           const AggClauseCosts *agg_costs,
+                           double dNumGroups);
 static RelOptInfo *create_window_paths(PlannerInfo *root,
                    RelOptInfo *input_rel,
                    PathTarget *input_target,
@@ -1540,8 +1567,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
        AggClauseCosts agg_costs;
        WindowFuncLists *wflists = NULL;
        List       *activeWindows = NIL;
-       List       *rollup_lists = NIL;
-       List       *rollup_groupclauses = NIL;
+       grouping_sets_data *gset_data = NULL;
        standard_qp_extra qp_extra;
 
        /* A recursive query should always have setOperations */
@@ -1550,84 +1576,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
        /* Preprocess grouping sets and GROUP BY clause, if any */
        if (parse->groupingSets)
        {
-           int        *tleref_to_colnum_map;
-           List       *sets;
-           int         maxref;
-           ListCell   *lc;
-           ListCell   *lc2;
-           ListCell   *lc_set;
-
-           parse->groupingSets = expand_grouping_sets(parse->groupingSets, -1);
-
-           /* Identify max SortGroupRef in groupClause, for array sizing */
-           maxref = 0;
-           foreach(lc, parse->groupClause)
-           {
-               SortGroupClause *gc = lfirst(lc);
-
-               if (gc->tleSortGroupRef > maxref)
-                   maxref = gc->tleSortGroupRef;
-           }
-
-           /* Allocate workspace array for remapping */
-           tleref_to_colnum_map = (int *) palloc((maxref + 1) * sizeof(int));
-
-           /* Examine the rollup sets */
-           sets = extract_rollup_sets(parse->groupingSets);
-
-           foreach(lc_set, sets)
-           {
-               List       *current_sets = (List *) lfirst(lc_set);
-               List       *groupclause;
-               int         ref;
-
-               /*
-                * Reorder the current list of grouping sets into correct
-                * prefix order.  If only one aggregation pass is needed, try
-                * to make the list match the ORDER BY clause; if more than
-                * one pass is needed, we don't bother with that.
-                */
-               current_sets = reorder_grouping_sets(current_sets,
-                                                    (list_length(sets) == 1
-                                                     ? parse->sortClause
-                                                     : NIL));
-
-               /*
-                * Order the groupClause appropriately.  If the first grouping
-                * set is empty, this can match regular GROUP BY
-                * preprocessing, otherwise we have to force the groupClause
-                * to match that grouping set's order.
-                */
-               groupclause = preprocess_groupclause(root,
-                                                    linitial(current_sets));
-
-               /*
-                * Now that we've pinned down an order for the groupClause for
-                * this list of grouping sets, we need to remap the entries in
-                * the grouping sets from sortgrouprefs to plain indices
-                * (0-based) into the groupClause for this collection of
-                * grouping sets.
-                */
-               ref = 0;
-               foreach(lc, groupclause)
-               {
-                   SortGroupClause *gc = lfirst(lc);
-
-                   tleref_to_colnum_map[gc->tleSortGroupRef] = ref++;
-               }
-
-               foreach(lc, current_sets)
-               {
-                   foreach(lc2, (List *) lfirst(lc))
-                   {
-                       lfirst_int(lc2) = tleref_to_colnum_map[lfirst_int(lc2)];
-                   }
-               }
-
-               /* Save the reordered sets and corresponding groupclauses */
-               rollup_lists = lcons(current_sets, rollup_lists);
-               rollup_groupclauses = lcons(groupclause, rollup_groupclauses);
-           }
+           gset_data = preprocess_grouping_sets(root);
        }
        else
        {
@@ -1721,8 +1670,9 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
        /* Set up data needed by standard_qp_callback */
        qp_extra.tlist = tlist;
        qp_extra.activeWindows = activeWindows;
-       qp_extra.groupClause =
-           parse->groupingSets ? llast(rollup_groupclauses) : parse->groupClause;
+       qp_extra.groupClause = (gset_data
+                               ? (gset_data->rollups ? ((RollupData *) linitial(gset_data->rollups))->groupClause : NIL)
+                               : parse->groupClause);
 
        /*
         * Generate the best unsorted and presorted paths for the scan/join
@@ -1922,8 +1872,7 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
                                                current_rel,
                                                grouping_target,
                                                &agg_costs,
-                                               rollup_lists,
-                                               rollup_groupclauses);
+                                               gset_data);
            /* Fix things up if grouping_target contains SRFs */
            if (parse->hasTargetSRFs)
                adjust_paths_for_srfs(root, current_rel,
@@ -1960,7 +1909,6 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
            current_rel = create_distinct_paths(root,
                                                current_rel);
        }
-
    }                           /* end of if (setOperations) */
 
    /*
@@ -2113,6 +2061,221 @@ grouping_planner(PlannerInfo *root, bool inheritance_update,
    /* Note: currently, we leave it to callers to do set_cheapest() */
 }
 
+/*
+ * Do preprocessing for groupingSets clause and related data.  This handles the
+ * preliminary steps of expanding the grouping sets, organizing them into lists
+ * of rollups, and preparing annotations which will later be filled in with
+ * size estimates.
+ */
+static grouping_sets_data *
+preprocess_grouping_sets(PlannerInfo *root)
+{
+   Query      *parse = root->parse;
+   List       *sets;
+   int         maxref = 0;
+   ListCell   *lc;
+   ListCell   *lc_set;
+   grouping_sets_data *gd = palloc0(sizeof(grouping_sets_data));
+
+   parse->groupingSets = expand_grouping_sets(parse->groupingSets, -1);
+
+   gd->any_hashable = false;
+   gd->unhashable_refs = NULL;
+   gd->unsortable_refs = NULL;
+   gd->unsortable_sets = NIL;
+
+   if (parse->groupClause)
+   {
+       ListCell   *lc;
+
+       foreach(lc, parse->groupClause)
+       {
+           SortGroupClause *gc = lfirst(lc);
+           Index       ref = gc->tleSortGroupRef;
+
+           if (ref > maxref)
+               maxref = ref;
+
+           if (!gc->hashable)
+               gd->unhashable_refs = bms_add_member(gd->unhashable_refs, ref);
+
+           if (!OidIsValid(gc->sortop))
+               gd->unsortable_refs = bms_add_member(gd->unsortable_refs, ref);
+       }
+   }
+
+   /* Allocate workspace array for remapping */
+   gd->tleref_to_colnum_map = (int *) palloc((maxref + 1) * sizeof(int));
+
+   /*
+    * If we have any unsortable sets, we must extract them before trying to
+    * prepare rollups. Unsortable sets don't go through
+    * reorder_grouping_sets, so we must apply the GroupingSetData annotation
+    * here.
+    */
+   if (!bms_is_empty(gd->unsortable_refs))
+   {
+       List       *sortable_sets = NIL;
+
+       foreach(lc, parse->groupingSets)
+       {
+           List       *gset = lfirst(lc);
+
+           if (bms_overlap_list(gd->unsortable_refs, gset))
+           {
+               GroupingSetData *gs = makeNode(GroupingSetData);
+
+               gs->set = gset;
+               gd->unsortable_sets = lappend(gd->unsortable_sets, gs);
+
+               /*
+                * We must enforce here that an unsortable set is hashable;
+                * later code assumes this.  Parse analysis only checks that
+                * every individual column is either hashable or sortable.
+                *
+                * Note that passing this test doesn't guarantee we can
+                * generate a plan; there might be other showstoppers.
+                */
+               if (bms_overlap_list(gd->unhashable_refs, gset))
+                   ereport(ERROR,
+                           (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+                            errmsg("could not implement GROUP BY"),
+                            errdetail("Some of the datatypes only support hashing, while others only support sorting.")));
+           }
+           else
+               sortable_sets = lappend(sortable_sets, gset);
+       }
+
+       if (sortable_sets)
+           sets = extract_rollup_sets(sortable_sets);
+       else
+           sets = NIL;
+   }
+   else
+       sets = extract_rollup_sets(parse->groupingSets);
+
+   foreach(lc_set, sets)
+   {
+       List       *current_sets = (List *) lfirst(lc_set);
+       RollupData *rollup = makeNode(RollupData);
+       GroupingSetData *gs;
+
+       /*
+        * Reorder the current list of grouping sets into correct prefix
+        * order.  If only one aggregation pass is needed, try to make the
+        * list match the ORDER BY clause; if more than one pass is needed, we
+        * don't bother with that.
+        *
+        * Note that this reorders the sets from smallest-member-first to
+        * largest-member-first, and applies the GroupingSetData annotations,
+        * though the data will be filled in later.
+        */
+       current_sets = reorder_grouping_sets(current_sets,
+                                            (list_length(sets) == 1
+                                             ? parse->sortClause
+                                             : NIL));
+
+       /*
+        * Get the initial (and therefore largest) grouping set.
+        */
+       gs = linitial(current_sets);
+
+       /*
+        * Order the groupClause appropriately.  If the first grouping set is
+        * empty, then the groupClause must also be empty; otherwise we have
+        * to force the groupClause to match that grouping set's order.
+        *
+        * (The first grouping set can be empty even though parse->groupClause
+        * is not empty only if all non-empty grouping sets are unsortable.
+        * The groupClauses for hashed grouping sets are built later on.)
+        */
+       if (gs->set)
+           rollup->groupClause = preprocess_groupclause(root, gs->set);
+       else
+           rollup->groupClause = NIL;
+
+       /*
+        * Is it hashable? We pretend empty sets are hashable even though we
+        * actually force them not to be hashed later. But don't bother if
+        * there's nothing but empty sets (since in that case we can't hash
+        * anything).
+        */
+       if (gs->set &&
+           !bms_overlap_list(gd->unhashable_refs, gs->set))
+       {
+           rollup->hashable = true;
+           gd->any_hashable = true;
+       }
+
+       /*
+        * Now that we've pinned down an order for the groupClause for this
+        * list of grouping sets, we need to remap the entries in the grouping
+        * sets from sortgrouprefs to plain indices (0-based) into the
+        * groupClause for this collection of grouping sets. We keep the
+        * original form for later use, though.
+        */
+       rollup->gsets = remap_to_groupclause_idx(rollup->groupClause,
+                                                current_sets,
+                                                gd->tleref_to_colnum_map);
+       rollup->gsets_data = current_sets;
+
+       gd->rollups = lappend(gd->rollups, rollup);
+   }
+
+   if (gd->unsortable_sets)
+   {
+       /*
+        * We have not yet pinned down a groupclause for this, but we will
+        * need index-based lists for estimation purposes. Construct
+        * hash_sets_idx based on the entire original groupclause for now.
+        */
+       gd->hash_sets_idx = remap_to_groupclause_idx(parse->groupClause,
+                                                    gd->unsortable_sets,
+                                                  gd->tleref_to_colnum_map);
+       gd->any_hashable = true;
+   }
+
+   return gd;
+}
+
+/*
+ * Given a groupclause and a list of GroupingSetData, return equivalent sets
+ * (without annotation) mapped to indexes into the given groupclause.
+ */
+static List *
+remap_to_groupclause_idx(List *groupClause,
+                        List *gsets,
+                        int *tleref_to_colnum_map)
+{
+   int         ref = 0;
+   List       *result = NIL;
+   ListCell   *lc;
+
+   foreach(lc, groupClause)
+   {
+       SortGroupClause *gc = lfirst(lc);
+
+       tleref_to_colnum_map[gc->tleSortGroupRef] = ref++;
+   }
+
+   foreach(lc, gsets)
+   {
+       List       *set = NIL;
+       ListCell   *lc2;
+       GroupingSetData *gs = lfirst(lc);
+
+       foreach(lc2, gs->set)
+       {
+           set = lappend_int(set, tleref_to_colnum_map[lfirst_int(lc2)]);
+       }
+
+       result = lappend(result, set);
+   }
+
+   return result;
+}
+
+
 
 /*
  * Detect whether a plan node is a "dummy" plan created when a relation
@@ -3028,7 +3191,7 @@ extract_rollup_sets(List *groupingSets)
 
 /*
  * Reorder the elements of a list of grouping sets such that they have correct
- * prefix relationships.
+ * prefix relationships. Also inserts the GroupingSetData annotations.
  *
  * The input must be ordered with smallest sets first; the result is returned
  * with largest sets first.  Note that the result shares no list substructure
@@ -3051,6 +3214,7 @@ reorder_grouping_sets(List *groupingsets, List *sortclause)
    {
        List       *candidate = lfirst(lc);
        List       *new_elems = list_difference_int(candidate, previous);
+       GroupingSetData *gs = makeNode(GroupingSetData);
 
        if (list_length(new_elems) > 0)
        {
@@ -3078,7 +3242,8 @@ reorder_grouping_sets(List *groupingsets, List *sortclause)
            }
        }
 
-       result = lcons(list_copy(previous), result);
+       gs->set = list_copy(previous);
+       result = lcons(gs, result);
        list_free(new_elems);
    }
 
@@ -3173,15 +3338,16 @@ standard_qp_callback(PlannerInfo *root, void *extra)
  * Estimate number of groups produced by grouping clauses (1 if not grouping)
  *
  * path_rows: number of output rows from scan/join step
- * rollup_lists: list of grouping sets, or NIL if not doing grouping sets
- * rollup_groupclauses: list of grouping clauses for grouping sets,
- *     or NIL if not doing grouping sets
+ * gsets: grouping set data, or NULL if not doing grouping sets
+ *
+ * If doing grouping sets, we also annotate the gsets data with the estimates
+ * for each set and each individual rollup list, with a view to later
+ * determining whether some combination of them could be hashed instead.
  */
 static double
 get_number_of_groups(PlannerInfo *root,
                     double path_rows,
-                    List *rollup_lists,
-                    List *rollup_groupclauses)
+                    grouping_sets_data *gd)
 {
    Query      *parse = root->parse;
    double      dNumGroups;
@@ -3193,28 +3359,60 @@ get_number_of_groups(PlannerInfo *root,
        if (parse->groupingSets)
        {
            /* Add up the estimates for each grouping set */
-           ListCell   *lc,
-                      *lc2;
+           ListCell   *lc;
+           ListCell   *lc2;
 
            dNumGroups = 0;
-           forboth(lc, rollup_groupclauses, lc2, rollup_lists)
+
+           foreach(lc, gd->rollups)
            {
-               List       *groupClause = (List *) lfirst(lc);
-               List       *gsets = (List *) lfirst(lc2);
-               ListCell   *lc3;
+               RollupData *rollup = lfirst(lc);
+               ListCell   *lc;
 
-               groupExprs = get_sortgrouplist_exprs(groupClause,
+               groupExprs = get_sortgrouplist_exprs(rollup->groupClause,
                                                     parse->targetList);
 
-               foreach(lc3, gsets)
+               rollup->numGroups = 0.0;
+
+               forboth(lc, rollup->gsets, lc2, rollup->gsets_data)
                {
-                   List       *gset = (List *) lfirst(lc3);
+                   List       *gset = (List *) lfirst(lc);
+                   GroupingSetData *gs = lfirst(lc2);
+                   double      numGroups = estimate_num_groups(root,
+                                                               groupExprs,
+                                                               path_rows,
+                                                               &gset);
+
+                   gs->numGroups = numGroups;
+                   rollup->numGroups += numGroups;
+               }
+
+               dNumGroups += rollup->numGroups;
+           }
+
+           if (gd->hash_sets_idx)
+           {
+               ListCell   *lc;
+
+               gd->dNumHashGroups = 0;
 
-                   dNumGroups += estimate_num_groups(root,
-                                                     groupExprs,
-                                                     path_rows,
-                                                     &gset);
+               groupExprs = get_sortgrouplist_exprs(parse->groupClause,
+                                                    parse->targetList);
+
+               forboth(lc, gd->hash_sets_idx, lc2, gd->unsortable_sets)
+               {
+                   List       *gset = (List *) lfirst(lc);
+                   GroupingSetData *gs = lfirst(lc2);
+                   double      numGroups = estimate_num_groups(root,
+                                                               groupExprs,
+                                                               path_rows,
+                                                               &gset);
+
+                   gs->numGroups = numGroups;
+                   gd->dNumHashGroups += numGroups;
                }
+
+               dNumGroups += gd->dNumHashGroups;
            }
        }
        else
@@ -3250,6 +3448,11 @@ get_number_of_groups(PlannerInfo *root,
  * estimate_hashagg_tablesize
  *   estimate the number of bytes that a hash aggregate hashtable will
  *   require based on the agg_costs, path width and dNumGroups.
+ *
+ * XXX this may be over-estimating the size now that hashagg knows to omit
+ * unneeded columns from the hashtable. Also for mixed-mode grouping sets,
+ * grouping columns not in the hashed set are counted here even though hashagg
+ * won't store them. Is this a problem?
  */
 static Size
 estimate_hashagg_tablesize(Path *path, const AggClauseCosts *agg_costs,
@@ -3300,8 +3503,7 @@ create_grouping_paths(PlannerInfo *root,
                      RelOptInfo *input_rel,
                      PathTarget *target,
                      const AggClauseCosts *agg_costs,
-                     List *rollup_lists,
-                     List *rollup_groupclauses)
+                     grouping_sets_data *gd)
 {
    Query      *parse = root->parse;
    Path       *cheapest_path = input_rel->cheapest_total_path;
@@ -3410,8 +3612,7 @@ create_grouping_paths(PlannerInfo *root,
     */
    dNumGroups = get_number_of_groups(root,
                                      cheapest_path->rows,
-                                     rollup_lists,
-                                     rollup_groupclauses);
+                                     gd);
 
    /*
     * Determine whether it's possible to perform sort-based implementations
@@ -3419,15 +3620,22 @@ create_grouping_paths(PlannerInfo *root,
     * grouping_is_sortable() is trivially true, and all the
     * pathkeys_contained_in() tests will succeed too, so that we'll consider
     * every surviving input path.)
+    *
+    * If we have grouping sets, we might be able to sort some but not all of
+    * them; in this case, we need can_sort to be true as long as we must
+    * consider any sorted-input plan.
     */
-   can_sort = grouping_is_sortable(parse->groupClause);
+   can_sort = (gd && gd->rollups != NIL)
+       || grouping_is_sortable(parse->groupClause);
 
    /*
     * Determine whether we should consider hash-based implementations of
     * grouping.
     *
-    * Hashed aggregation only applies if we're grouping.  We currently can't
-    * hash if there are grouping sets, though.
+    * Hashed aggregation only applies if we're grouping. If we have grouping
+    * sets, some groups might be hashable but others not; in this case we set
+    * can_hash true as long as there is nothing globally preventing us from
+    * hashing (and we should therefore consider plans with hashes).
     *
     * Executor doesn't support hashed aggregation with DISTINCT or ORDER BY
     * aggregates.  (Doing so would imply storing *all* the input values in
@@ -3440,9 +3648,8 @@ create_grouping_paths(PlannerInfo *root,
     * other gating conditions, so we want to do it last.
     */
    can_hash = (parse->groupClause != NIL &&
-               parse->groupingSets == NIL &&
                agg_costs->numOrderedAggs == 0 &&
-               grouping_is_hashable(parse->groupClause));
+        (gd ? gd->any_hashable : grouping_is_hashable(parse->groupClause)));
 
    /*
     * If grouped_rel->consider_parallel is true, then paths that we generate
@@ -3508,8 +3715,7 @@ create_grouping_paths(PlannerInfo *root,
        /* Estimate number of partial groups. */
        dNumPartialGroups = get_number_of_groups(root,
                                                 cheapest_partial_path->rows,
-                                                NIL,
-                                                NIL);
+                                                gd);
 
        /*
         * Collect statistics about aggregates for estimating costs of
@@ -3642,20 +3848,9 @@ create_grouping_paths(PlannerInfo *root,
                /* Now decide what to stick atop it */
                if (parse->groupingSets)
                {
-                   /*
-                    * We have grouping sets, possibly with aggregation.  Make
-                    * a GroupingSetsPath.
-                    */
-                   add_path(grouped_rel, (Path *)
-                            create_groupingsets_path(root,
-                                                     grouped_rel,
-                                                     path,
-                                                     target,
-                                                 (List *) parse->havingQual,
-                                                     rollup_lists,
-                                                     rollup_groupclauses,
-                                                     agg_costs,
-                                                     dNumGroups));
+                   consider_groupingsets_paths(root, grouped_rel,
+                                               path, true, can_hash, target,
+                                               gd, agg_costs, dNumGroups);
                }
                else if (parse->hasAggs)
                {
@@ -3816,33 +4011,45 @@ create_grouping_paths(PlannerInfo *root,
 
    if (can_hash)
    {
-       hashaggtablesize = estimate_hashagg_tablesize(cheapest_path,
-                                                     agg_costs,
-                                                     dNumGroups);
-
-       /*
-        * Provided that the estimated size of the hashtable does not exceed
-        * work_mem, we'll generate a HashAgg Path, although if we were unable
-        * to sort above, then we'd better generate a Path, so that we at
-        * least have one.
-        */
-       if (hashaggtablesize < work_mem * 1024L ||
-           grouped_rel->pathlist == NIL)
+       if (parse->groupingSets)
        {
            /*
-            * We just need an Agg over the cheapest-total input path, since
-            * input order won't matter.
+            * Try for a hash-only groupingsets path over unsorted input.
             */
-           add_path(grouped_rel, (Path *)
-                    create_agg_path(root, grouped_rel,
-                                    cheapest_path,
-                                    target,
-                                    AGG_HASHED,
-                                    AGGSPLIT_SIMPLE,
-                                    parse->groupClause,
-                                    (List *) parse->havingQual,
-                                    agg_costs,
-                                    dNumGroups));
+           consider_groupingsets_paths(root, grouped_rel,
+                                       cheapest_path, false, true, target,
+                                       gd, agg_costs, dNumGroups);
+       }
+       else
+       {
+           hashaggtablesize = estimate_hashagg_tablesize(cheapest_path,
+                                                         agg_costs,
+                                                         dNumGroups);
+
+           /*
+            * Provided that the estimated size of the hashtable does not
+            * exceed work_mem, we'll generate a HashAgg Path, although if we
+            * were unable to sort above, then we'd better generate a Path, so
+            * that we at least have one.
+            */
+           if (hashaggtablesize < work_mem * 1024L ||
+               grouped_rel->pathlist == NIL)
+           {
+               /*
+                * We just need an Agg over the cheapest-total input path,
+                * since input order won't matter.
+                */
+               add_path(grouped_rel, (Path *)
+                        create_agg_path(root, grouped_rel,
+                                        cheapest_path,
+                                        target,
+                                        AGG_HASHED,
+                                        AGGSPLIT_SIMPLE,
+                                        parse->groupClause,
+                                        (List *) parse->havingQual,
+                                        agg_costs,
+                                        dNumGroups));
+           }
        }
 
        /*
@@ -3921,6 +4128,344 @@ create_grouping_paths(PlannerInfo *root,
    return grouped_rel;
 }
 
+
+/*
+ * For a given input path, consider the possible ways of doing grouping sets on
+ * it, by combinations of hashing and sorting.  This can be called multiple
+ * times, so it's important that it not scribble on input.  No result is
+ * returned, but any generated paths are added to grouped_rel.
+ */
+static void
+consider_groupingsets_paths(PlannerInfo *root,
+                           RelOptInfo *grouped_rel,
+                           Path *path,
+                           bool is_sorted,
+                           bool can_hash,
+                           PathTarget *target,
+                           grouping_sets_data *gd,
+                           const AggClauseCosts *agg_costs,
+                           double dNumGroups)
+{
+   Query      *parse = root->parse;
+
+   /*
+    * If we're not being offered sorted input, then only consider plans that
+    * can be done entirely by hashing.
+    *
+    * We can hash everything if it looks like it'll fit in work_mem. But if
+    * the input is actually sorted despite not being advertised as such, we
+    * prefer to make use of that in order to use less memory.
+    *
+    * If none of the grouping sets are sortable, then ignore the work_mem
+    * limit and generate a path anyway, since otherwise we'll just fail.
+    */
+   if (!is_sorted)
+   {
+       List       *new_rollups = NIL;
+       RollupData *unhashed_rollup = NULL;
+       List       *sets_data;
+       List       *empty_sets_data = NIL;
+       List       *empty_sets = NIL;
+       ListCell   *lc;
+       ListCell   *l_start = list_head(gd->rollups);
+       AggStrategy strat = AGG_HASHED;
+       Size        hashsize;
+       double      exclude_groups = 0.0;
+
+       Assert(can_hash);
+
+       if (pathkeys_contained_in(root->group_pathkeys, path->pathkeys))
+       {
+           unhashed_rollup = lfirst(l_start);
+           exclude_groups = unhashed_rollup->numGroups;
+           l_start = lnext(l_start);
+       }
+
+       hashsize = estimate_hashagg_tablesize(path,
+                                             agg_costs,
+                                             dNumGroups - exclude_groups);
+
+       /*
+        * gd->rollups is empty if we have only unsortable columns to work
+        * with.  Override work_mem in that case; otherwise, we'll rely on the
+        * sorted-input case to generate usable mixed paths.
+        */
+       if (hashsize > work_mem * 1024L && gd->rollups)
+           return;             /* nope, won't fit */
+
+       /*
+        * We need to burst the existing rollups list into individual grouping
+        * sets and recompute a groupClause for each set.
+        */
+       sets_data = list_copy(gd->unsortable_sets);
+
+       for_each_cell(lc, l_start)
+       {
+           RollupData *rollup = lfirst(lc);
+
+           /*
+            * If we find an unhashable rollup that's not been skipped by the
+            * "actually sorted" check above, we can't cope; we'd need sorted
+            * input (with a different sort order) but we can't get that here.
+            * So bail out; we'll get a valid path from the is_sorted case
+            * instead.
+            *
+            * The mere presence of empty grouping sets doesn't make a rollup
+            * unhashable (see preprocess_grouping_sets), we handle those
+            * specially below.
+            */
+           if (!rollup->hashable)
+               return;
+           else
+               sets_data = list_concat(sets_data, list_copy(rollup->gsets_data));
+       }
+       foreach(lc, sets_data)
+       {
+           GroupingSetData *gs = lfirst(lc);
+           List       *gset = gs->set;
+           RollupData *rollup;
+
+           if (gset == NIL)
+           {
+               /* Empty grouping sets can't be hashed. */
+               empty_sets_data = lappend(empty_sets_data, gs);
+               empty_sets = lappend(empty_sets, NIL);
+           }
+           else
+           {
+               rollup = makeNode(RollupData);
+
+               rollup->groupClause = preprocess_groupclause(root, gset);
+               rollup->gsets_data = list_make1(gs);
+               rollup->gsets = remap_to_groupclause_idx(rollup->groupClause,
+                                                        rollup->gsets_data,
+                                                  gd->tleref_to_colnum_map);
+               rollup->numGroups = gs->numGroups;
+               rollup->hashable = true;
+               rollup->is_hashed = true;
+               new_rollups = lappend(new_rollups, rollup);
+           }
+       }
+
+       /*
+        * If we didn't find anything nonempty to hash, then bail.  We'll
+        * generate a path from the is_sorted case.
+        */
+       if (new_rollups == NIL)
+           return;
+
+       /*
+        * If there were empty grouping sets they should have been in the
+        * first rollup.
+        */
+       Assert(!unhashed_rollup || !empty_sets);
+
+       if (unhashed_rollup)
+       {
+           new_rollups = lappend(new_rollups, unhashed_rollup);
+           strat = AGG_MIXED;
+       }
+       else if (empty_sets)
+       {
+           RollupData *rollup = makeNode(RollupData);
+
+           rollup->groupClause = NIL;
+           rollup->gsets_data = empty_sets_data;
+           rollup->gsets = empty_sets;
+           rollup->numGroups = list_length(empty_sets);
+           rollup->hashable = false;
+           rollup->is_hashed = false;
+           new_rollups = lappend(new_rollups, rollup);
+           strat = AGG_MIXED;
+       }
+
+       add_path(grouped_rel, (Path *)
+                create_groupingsets_path(root,
+                                         grouped_rel,
+                                         path,
+                                         target,
+                                         (List *) parse->havingQual,
+                                         strat,
+                                         new_rollups,
+                                         agg_costs,
+                                         dNumGroups));
+       return;
+   }
+
+   /*
+    * If we have sorted input but nothing we can do with it, bail.
+    */
+   if (list_length(gd->rollups) == 0)
+       return;
+
+   /*
+    * Given sorted input, we try and make two paths: one sorted and one mixed
+    * sort/hash. (We need to try both because hashagg might be disabled, or
+    * some columns might not be sortable.)
+    *
+    * can_hash is passed in as false if some obstacle elsewhere (such as
+    * ordered aggs) means that we shouldn't consider hashing at all.
+    */
+   if (can_hash && gd->any_hashable)
+   {
+       List       *rollups = NIL;
+       List       *hash_sets = list_copy(gd->unsortable_sets);
+       double      availspace = (work_mem * 1024.0);
+       ListCell   *lc;
+
+       /*
+        * Account first for space needed for groups we can't sort at all.
+        */
+       availspace -= (double) estimate_hashagg_tablesize(path,
+                                                         agg_costs,
+                                                         gd->dNumHashGroups);
+
+       if (availspace > 0 && list_length(gd->rollups) > 1)
+       {
+           double      scale;
+           int         num_rollups = list_length(gd->rollups);
+           int         k_capacity;
+           int        *k_weights = palloc(num_rollups * sizeof(int));
+           Bitmapset  *hash_items = NULL;
+           int         i;
+
+           /*
+            * We treat this as a knapsack problem: the knapsack capacity
+            * represents work_mem, the item weights are the estimated memory
+            * usage of the hashtables needed to implement a single rollup, and
+            * we really ought to use the cost saving as the item value;
+            * however, currently the costs assigned to sort nodes don't
+            * reflect the comparison costs well, and so we treat all items as
+            * of equal value (each rollup we hash instead saves us one sort).
+            *
+            * To use the discrete knapsack, we need to scale the values to a
+            * reasonably small bounded range.  We choose to allow a 5% error
+            * margin; we have no more than 4096 rollups in the worst possible
+            * case, which with a 5% error margin will require a bit over 42MB
+            * of workspace. (Anyone wanting to plan queries that complex had
+            * better have the memory for it.  In more reasonable cases, with
+            * no more than a couple of dozen rollups, the memory usage will
+            * be negligible.)
+            *
+            * k_capacity is naturally bounded, but we clamp the values for
+            * scale and weight (below) to avoid overflows or underflows (or
+            * uselessly trying to use a scale factor less than 1 byte).
+            */
+           scale = Max(availspace / (20.0 * num_rollups), 1.0);
+           k_capacity = (int) floor(availspace / scale);
+
+           /*
+            * We leave the first rollup out of consideration since it's the
+            * one that matches the input sort order.  We assign indexes "i"
+            * to only those entries considered for hashing; the second loop,
+            * below, must use the same condition.
+            */
+           i = 0;
+           for_each_cell(lc, lnext(list_head(gd->rollups)))
+           {
+               RollupData *rollup = lfirst(lc);
+
+               if (rollup->hashable)
+               {
+                   double      sz = estimate_hashagg_tablesize(path,
+                                                               agg_costs,
+                                                         rollup->numGroups);
+
+                   /*
+                    * If sz is enormous, but work_mem (and hence scale) is
+                    * small, avoid integer overflow here.
+                    */
+                   k_weights[i] = (int) Min(floor(sz / scale),
+                                            k_capacity + 1.0);
+                   ++i;
+               }
+           }
+
+           /*
+            * Apply knapsack algorithm; compute the set of items which
+            * maximizes the value stored (in this case the number of sorts
+            * saved) while keeping the total size (approximately) within
+            * capacity.
+            */
+           if (i > 0)
+               hash_items = DiscreteKnapsack(k_capacity, i, k_weights, NULL);
+
+           if (!bms_is_empty(hash_items))
+           {
+               rollups = list_make1(linitial(gd->rollups));
+
+               i = 0;
+               for_each_cell(lc, lnext(list_head(gd->rollups)))
+               {
+                   RollupData *rollup = lfirst(lc);
+
+                   if (rollup->hashable)
+                   {
+                       if (bms_is_member(i, hash_items))
+                           hash_sets = list_concat(hash_sets,
+                                             list_copy(rollup->gsets_data));
+                       else
+                           rollups = lappend(rollups, rollup);
+                       ++i;
+                   }
+                   else
+                       rollups = lappend(rollups, rollup);
+               }
+           }
+       }
+
+       if (!rollups && hash_sets)
+           rollups = list_copy(gd->rollups);
+
+       foreach(lc, hash_sets)
+       {
+           GroupingSetData *gs = lfirst(lc);
+           RollupData *rollup = makeNode(RollupData);
+
+           Assert(gs->set != NIL);
+
+           rollup->groupClause = preprocess_groupclause(root, gs->set);
+           rollup->gsets_data = list_make1(gs);
+           rollup->gsets = remap_to_groupclause_idx(rollup->groupClause,
+                                                    rollup->gsets_data,
+                                                  gd->tleref_to_colnum_map);
+           rollup->numGroups = gs->numGroups;
+           rollup->hashable = true;
+           rollup->is_hashed = true;
+           rollups = lcons(rollup, rollups);
+       }
+
+       if (rollups)
+       {
+           add_path(grouped_rel, (Path *)
+                    create_groupingsets_path(root,
+                                             grouped_rel,
+                                             path,
+                                             target,
+                                             (List *) parse->havingQual,
+                                             AGG_MIXED,
+                                             rollups,
+                                             agg_costs,
+                                             dNumGroups));
+       }
+   }
+
+   /*
+    * Now try the simple sorted case.
+    */
+   if (!gd->unsortable_sets)
+       add_path(grouped_rel, (Path *)
+                create_groupingsets_path(root,
+                                         grouped_rel,
+                                         path,
+                                         target,
+                                         (List *) parse->havingQual,
+                                         AGG_SORTED,
+                                         gd->rollups,
+                                         agg_costs,
+                                         dNumGroups));
+}
+
 /*
  * create_window_paths
  *
index fca96eb0010e6fa929c86fe44cf18575944a4b08..999ebcee704f84749729876635b28a9b5692ef75 100644 (file)
@@ -2697,10 +2697,9 @@ create_agg_path(PlannerInfo *root,
  * 'subpath' is the path representing the source of data
  * 'target' is the PathTarget to be computed
  * 'having_qual' is the HAVING quals if any
- * 'rollup_lists' is a list of grouping sets
- * 'rollup_groupclauses' is a list of grouping clauses for grouping sets
+ * 'rollups' is a list of RollupData nodes
  * 'agg_costs' contains cost info about the aggregate functions to be computed
- * 'numGroups' is the estimated number of groups
+ * 'numGroups' is the estimated total number of groups
  */
 GroupingSetsPath *
 create_groupingsets_path(PlannerInfo *root,
@@ -2708,13 +2707,15 @@ create_groupingsets_path(PlannerInfo *root,
                         Path *subpath,
                         PathTarget *target,
                         List *having_qual,
-                        List *rollup_lists,
-                        List *rollup_groupclauses,
+                        AggStrategy aggstrategy,
+                        List *rollups,
                         const AggClauseCosts *agg_costs,
                         double numGroups)
 {
    GroupingSetsPath *pathnode = makeNode(GroupingSetsPath);
-   int         numGroupCols;
+   ListCell   *lc;
+   bool        is_first = true;
+   bool        is_first_sort = true;
 
    /* The topmost generated Plan node will be an Agg */
    pathnode->path.pathtype = T_Agg;
@@ -2727,75 +2728,110 @@ create_groupingsets_path(PlannerInfo *root,
    pathnode->path.parallel_workers = subpath->parallel_workers;
    pathnode->subpath = subpath;
 
+   /*
+    * Simplify callers by downgrading AGG_SORTED to AGG_PLAIN, and AGG_MIXED
+    * to AGG_HASHED, here if possible.
+    */
+   if (aggstrategy == AGG_SORTED &&
+       list_length(rollups) == 1 &&
+       ((RollupData *) linitial(rollups))->groupClause == NIL)
+       aggstrategy = AGG_PLAIN;
+
+   if (aggstrategy == AGG_MIXED &&
+       list_length(rollups) == 1)
+       aggstrategy = AGG_HASHED;
+
    /*
     * Output will be in sorted order by group_pathkeys if, and only if, there
     * is a single rollup operation on a non-empty list of grouping
     * expressions.
     */
-   if (list_length(rollup_groupclauses) == 1 &&
-       ((List *) linitial(rollup_groupclauses)) != NIL)
+   if (aggstrategy == AGG_SORTED && list_length(rollups) == 1)
        pathnode->path.pathkeys = root->group_pathkeys;
    else
        pathnode->path.pathkeys = NIL;
 
-   pathnode->rollup_groupclauses = rollup_groupclauses;
-   pathnode->rollup_lists = rollup_lists;
+   pathnode->aggstrategy = aggstrategy;
+   pathnode->rollups = rollups;
    pathnode->qual = having_qual;
 
-   Assert(rollup_lists != NIL);
-   Assert(list_length(rollup_lists) == list_length(rollup_groupclauses));
-
-   /* Account for cost of the topmost Agg node */
-   numGroupCols = list_length((List *) linitial((List *) llast(rollup_lists)));
-
-   cost_agg(&pathnode->path, root,
-            (numGroupCols > 0) ? AGG_SORTED : AGG_PLAIN,
-            agg_costs,
-            numGroupCols,
-            numGroups,
-            subpath->startup_cost,
-            subpath->total_cost,
-            subpath->rows);
+   Assert(rollups != NIL);
+   Assert(aggstrategy != AGG_PLAIN || list_length(rollups) == 1);
+   Assert(aggstrategy != AGG_MIXED || list_length(rollups) > 1);
 
-   /*
-    * Add in the costs and output rows of the additional sorting/aggregation
-    * steps, if any.  Only total costs count, since the extra sorts aren't
-    * run on startup.
-    */
-   if (list_length(rollup_lists) > 1)
+   foreach(lc, rollups)
    {
-       ListCell   *lc;
+       RollupData *rollup = lfirst(lc);
+       List       *gsets = rollup->gsets;
+       int         numGroupCols = list_length(linitial(gsets));
 
-       foreach(lc, rollup_lists)
+       /*
+        * In AGG_SORTED or AGG_PLAIN mode, the first rollup takes the
+        * (already-sorted) input, and following ones do their own sort.
+        *
+        * In AGG_HASHED mode, there is one rollup for each grouping set.
+        *
+        * In AGG_MIXED mode, the first rollups are hashed, the first
+        * non-hashed one takes the (already-sorted) input, and following ones
+        * do their own sort.
+        */
+       if (is_first)
+       {
+           cost_agg(&pathnode->path, root,
+                    aggstrategy,
+                    agg_costs,
+                    numGroupCols,
+                    rollup->numGroups,
+                    subpath->startup_cost,
+                    subpath->total_cost,
+                    subpath->rows);
+           is_first = false;
+           if (!rollup->is_hashed)
+               is_first_sort = false;
+       }
+       else
        {
-           List       *gsets = (List *) lfirst(lc);
            Path        sort_path;      /* dummy for result of cost_sort */
            Path        agg_path;       /* dummy for result of cost_agg */
 
-           /* We must iterate over all but the last rollup_lists element */
-           if (lnext(lc) == NULL)
-               break;
-
-           /* Account for cost of sort, but don't charge input cost again */
-           cost_sort(&sort_path, root, NIL,
-                     0.0,
-                     subpath->rows,
-                     subpath->pathtarget->width,
-                     0.0,
-                     work_mem,
-                     -1.0);
-
-           /* Account for cost of aggregation */
-           numGroupCols = list_length((List *) linitial(gsets));
-
-           cost_agg(&agg_path, root,
-                    AGG_SORTED,
-                    agg_costs,
-                    numGroupCols,
-                    numGroups, /* XXX surely not right for all steps? */
-                    sort_path.startup_cost,
-                    sort_path.total_cost,
-                    sort_path.rows);
+           if (rollup->is_hashed || is_first_sort)
+           {
+               /*
+                * Account for cost of aggregation, but don't charge input
+                * cost again
+                */
+               cost_agg(&agg_path, root,
+                        rollup->is_hashed ? AGG_HASHED : AGG_SORTED,
+                        agg_costs,
+                        numGroupCols,
+                        rollup->numGroups,
+                        0.0, 0.0,
+                        subpath->rows);
+               if (!rollup->is_hashed)
+                   is_first_sort = false;
+           }
+           else
+           {
+               /* Account for cost of sort, but don't charge input cost again */
+               cost_sort(&sort_path, root, NIL,
+                         0.0,
+                         subpath->rows,
+                         subpath->pathtarget->width,
+                         0.0,
+                         work_mem,
+                         -1.0);
+
+               /* Account for cost of aggregation */
+
+               cost_agg(&agg_path, root,
+                        AGG_SORTED,
+                        agg_costs,
+                        numGroupCols,
+                        rollup->numGroups,
+                        sort_path.startup_cost,
+                        sort_path.total_cost,
+                        sort_path.rows);
+           }
 
            pathnode->path.total_cost += agg_path.total_cost;
            pathnode->path.rows += agg_path.rows;
diff --git a/src/include/lib/knapsack.h b/src/include/lib/knapsack.h
new file mode 100644 (file)
index 0000000..8d1e6d0
--- /dev/null
@@ -0,0 +1,17 @@
+/*
+ * knapsack.h
+ *
+ * Copyright (c) 2017, PostgreSQL Global Development Group
+ *
+ * src/include/lib/knapsack.h
+ */
+#ifndef KNAPSACK_H
+#define KNAPSACK_H
+
+#include "postgres.h"
+#include "nodes/bitmapset.h"
+
+extern Bitmapset *DiscreteKnapsack(int max_weight, int num_items,
+                int *item_weights, double *item_values);
+
+#endif   /* KNAPSACK_H */
index 4f1910e5a98b96cb4c5f116eb9a867caca31bcee..109f7b0c1482fe930511adf2f4259a00dee12aec 100644 (file)
 #ifndef BITMAPSET_H
 #define BITMAPSET_H
 
+/*
+ * Forward decl to save including pg_list.h
+ */
+struct List;
+
 /*
  * Data representation
  */
@@ -70,6 +75,7 @@ extern bool bms_is_subset(const Bitmapset *a, const Bitmapset *b);
 extern BMS_Comparison bms_subset_compare(const Bitmapset *a, const Bitmapset *b);
 extern bool bms_is_member(int x, const Bitmapset *a);
 extern bool bms_overlap(const Bitmapset *a, const Bitmapset *b);
+extern bool bms_overlap_list(const Bitmapset *a, const struct List *b);
 extern bool bms_nonempty_difference(const Bitmapset *a, const Bitmapset *b);
 extern int bms_singleton_member(const Bitmapset *a);
 extern bool bms_get_singleton_member(const Bitmapset *a, int *member);
index ff428951186007083f9bb35d4be6a57bdfb8e760..11a68500eeaa6891775d714d5b5280118dcf2340 100644 (file)
@@ -1699,6 +1699,7 @@ typedef struct AggStatePerAggData *AggStatePerAgg;
 typedef struct AggStatePerTransData *AggStatePerTrans;
 typedef struct AggStatePerGroupData *AggStatePerGroup;
 typedef struct AggStatePerPhaseData *AggStatePerPhase;
+typedef struct AggStatePerHashData *AggStatePerHash;
 
 typedef struct AggState
 {
@@ -1706,15 +1707,17 @@ typedef struct AggState
    List       *aggs;           /* all Aggref nodes in targetlist & quals */
    int         numaggs;        /* length of list (could be zero!) */
    int         numtrans;       /* number of pertrans items */
+   AggStrategy aggstrategy;    /* strategy mode */
    AggSplit    aggsplit;       /* agg-splitting mode, see nodes.h */
    AggStatePerPhase phase;     /* pointer to current phase data */
-   int         numphases;      /* number of phases */
+   int         numphases;      /* number of phases (including phase 0) */
    int         current_phase;  /* current phase number */
-   FmgrInfo   *hashfunctions;  /* per-grouping-field hash fns */
    AggStatePerAgg peragg;      /* per-Aggref information */
    AggStatePerTrans pertrans;  /* per-Trans state information */
+   ExprContext *hashcontext;   /* econtexts for long-lived data (hashtable) */
    ExprContext **aggcontexts;  /* econtexts for long-lived data (per GS) */
    ExprContext *tmpcontext;    /* econtext for input expressions */
+   ExprContext *curaggcontext; /* currently active aggcontext */
    AggStatePerTrans curpertrans;       /* currently active trans state */
    bool        input_done;     /* indicates end of input */
    bool        agg_done;       /* indicates completion of Agg scan */
@@ -1726,21 +1729,17 @@ typedef struct AggState
    /* These fields are for grouping set phase data */
    int         maxsets;        /* The max number of sets in any phase */
    AggStatePerPhase phases;    /* array of all phases */
-   Tuplesortstate *sort_in;    /* sorted input to phases > 0 */
+   Tuplesortstate *sort_in;    /* sorted input to phases > 1 */
    Tuplesortstate *sort_out;   /* input is copied here for next phase */
    TupleTableSlot *sort_slot;  /* slot for sort results */
    /* these fields are used in AGG_PLAIN and AGG_SORTED modes: */
    AggStatePerGroup pergroup;  /* per-Aggref-per-group working state */
    HeapTuple   grp_firstTuple; /* copy of first tuple of current group */
-   /* these fields are used in AGG_HASHED mode: */
-   TupleHashTable hashtable;   /* hash table with one entry per group */
-   TupleTableSlot *hashslot;   /* slot for loading hash table */
-   int         numhashGrpCols; /* number of columns in hash table */
-   int         largestGrpColIdx; /* largest column required for hashing */
-   AttrNumber *hashGrpColIdxInput; /* and their indices in input slot */
-   AttrNumber *hashGrpColIdxHash;  /* indices for execGrouping in hashtbl */
+   /* these fields are used in AGG_HASHED and AGG_MIXED modes: */
    bool        table_filled;   /* hash table filled yet? */
-   TupleHashIterator hashiter; /* for iterating through hash table */
+   int         num_hashes;
+   AggStatePerHash perhash;
+   AggStatePerGroup *hash_pergroup;    /* array of per-group pointers */
    /* support for evaluation of agg inputs */
    TupleTableSlot *evalslot;   /* slot for agg inputs */
    ProjectionInfo *evalproj;   /* projection machinery */
index c83216943c1ab65554fd55106bf39a4c26e4e9fd..b9369ac275426a08698e524284574d2d5f1274bf 100644 (file)
@@ -261,6 +261,8 @@ typedef enum NodeTag
    T_PlaceHolderInfo,
    T_MinMaxAggInfo,
    T_PlannerParamItem,
+   T_RollupData,
+   T_GroupingSetData,
    T_StatisticExtInfo,
 
    /*
@@ -724,7 +726,8 @@ typedef enum AggStrategy
 {
    AGG_PLAIN,                  /* simple agg across all input rows */
    AGG_SORTED,                 /* grouped agg, input must be sorted */
-   AGG_HASHED                  /* grouped agg, use internal hashtable */
+   AGG_HASHED,                 /* grouped agg, use internal hashtable */
+   AGG_MIXED                   /* grouped agg, hash and sort both used */
 } AggStrategy;
 
 /*
index 4a95e16b695c1d33490952fb3980a83f1e638132..6e531b6238651f301a16f87d0cb538d5cbb3ab3d 100644 (file)
@@ -758,7 +758,7 @@ typedef struct Agg
    Oid        *grpOperators;   /* equality operators to compare with */
    long        numGroups;      /* estimated number of groups in input */
    Bitmapset  *aggParams;      /* IDs of Params used in Aggref inputs */
-   /* Note: planner provides numGroups & aggParams only in AGG_HASHED case */
+   /* Note: planner provides numGroups & aggParams only in HASHED/MIXED case */
    List       *groupingSets;   /* grouping sets to use */
    List       *chain;          /* chained Agg/Sort nodes */
 } Agg;
index 0a5187cef3ba823ec3b67fb58cc2040ea2ca846a..8930edf8264ee466c25a13cbe083c670c55f5b27 100644 (file)
@@ -1417,18 +1417,38 @@ typedef struct AggPath
    List       *qual;           /* quals (HAVING quals), if any */
 } AggPath;
 
+/*
+ * Various annotations used for grouping sets in the planner.
+ */
+
+typedef struct GroupingSetData
+{
+   NodeTag     type;
+   List       *set;            /* grouping set as list of sortgrouprefs */
+   double      numGroups;      /* est. number of result groups */
+} GroupingSetData;
+
+typedef struct RollupData
+{
+   NodeTag     type;
+   List       *groupClause;    /* applicable subset of parse->groupClause */
+   List       *gsets;          /* lists of integer indexes into groupClause */
+   List       *gsets_data;     /* list of GroupingSetData */
+   double      numGroups;      /* est. number of result groups */
+   bool        hashable;       /* can be hashed */
+   bool        is_hashed;      /* to be implemented as a hashagg */
+} RollupData;
+
 /*
  * GroupingSetsPath represents a GROUPING SETS aggregation
- *
- * Currently we only support this in sorted not hashed form, so the input
- * must always be appropriately presorted.
  */
+
 typedef struct GroupingSetsPath
 {
    Path        path;
    Path       *subpath;        /* path representing input source */
-   List       *rollup_groupclauses;    /* list of lists of SortGroupClause's */
-   List       *rollup_lists;   /* parallel list of lists of grouping sets */
+   AggStrategy aggstrategy;    /* basic strategy */
+   List       *rollups;        /* list of RollupData */
    List       *qual;           /* quals (HAVING quals), if any */
 } GroupingSetsPath;
 
index 81640de7ab7f680f8c3eb559340241c3f6cf2204..c72c7e02cbbc35ffc6b9d358caf365eb81c2dc48 100644 (file)
@@ -195,8 +195,8 @@ extern GroupingSetsPath *create_groupingsets_path(PlannerInfo *root,
                         Path *subpath,
                         PathTarget *target,
                         List *having_qual,
-                        List *rollup_lists,
-                        List *rollup_groupclauses,
+                        AggStrategy aggstrategy,
+                        List *rollups,
                         const AggClauseCosts *agg_costs,
                         double numGroups);
 extern MinMaxAggPath *create_minmaxagg_path(PlannerInfo *root,
index b0886da8d707c9cc9d60e955915c530d2997f6ba..1374710087bbbbb4d0c2d4bd447c6db3300a1fe6 100644 (file)
@@ -13,6 +13,13 @@ copy gstest2 from stdin;
 create temp table gstest3 (a integer, b integer, c integer, d integer);
 copy gstest3 from stdin;
 alter table gstest3 add primary key (a);
+create temp table gstest4(id integer, v integer,
+                          unhashable_col bit(4), unsortable_col xid);
+insert into gstest4
+values (1,1,b'0000','1'), (2,2,b'0001','1'),
+       (3,4,b'0010','2'), (4,8,b'0011','2'),
+       (5,16,b'0000','2'), (6,32,b'0001','2'),
+       (7,64,b'0010','1'), (8,128,b'0011','1');
 create temp table gstest_empty (a integer, b integer, v integer);
 create function gstest_data(v integer, out a integer, out b integer)
   returns setof record
@@ -22,6 +29,7 @@ create function gstest_data(v integer, out a integer, out b integer)
     end;
   $f$ language plpgsql;
 -- basic functionality
+set enable_hashagg = false;  -- test hashing explicitly later
 -- simple rollup with multiple plain aggregates, with and without ordering
 -- (and with ordering differing from grouping)
 select a, b, grouping(a,b), sum(v), count(*), max(v)
@@ -462,7 +470,7 @@ select a, b from (values (1,2),(2,3)) v(a,b) group by a,b, grouping sets(a);
 
 -- Tests for chained aggregates
 select a, b, grouping(a,b), sum(v), count(*), max(v)
-  from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2));
+  from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6;
  a | b | grouping | sum | count | max 
 ---+---+----------+-----+-------+-----
  1 | 1 |        0 |  21 |     2 |  11
@@ -473,19 +481,19 @@ select a, b, grouping(a,b), sum(v), count(*), max(v)
  3 | 4 |        0 |  17 |     1 |  17
  4 | 1 |        0 |  37 |     2 |  19
    |   |        3 |  21 |     2 |  11
-   |   |        3 |  25 |     2 |  13
-   |   |        3 |  14 |     1 |  14
-   |   |        3 |  15 |     1 |  15
-   |   |        3 |  16 |     1 |  16
-   |   |        3 |  17 |     1 |  17
-   |   |        3 |  37 |     2 |  19
    |   |        3 |  21 |     2 |  11
    |   |        3 |  25 |     2 |  13
+   |   |        3 |  25 |     2 |  13
+   |   |        3 |  14 |     1 |  14
    |   |        3 |  14 |     1 |  14
    |   |        3 |  15 |     1 |  15
+   |   |        3 |  15 |     1 |  15
    |   |        3 |  16 |     1 |  16
+   |   |        3 |  16 |     1 |  16
+   |   |        3 |  17 |     1 |  17
    |   |        3 |  17 |     1 |  17
    |   |        3 |  37 |     2 |  19
+   |   |        3 |  37 |     2 |  19
 (21 rows)
 
 select(select (select grouping(a,b) from (values (1)) v2(c)) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY ROLLUP((e+1),(f+1));
@@ -847,4 +855,598 @@ select sum(ten) from onek group by rollup(four::text), two order by 1;
  2500
 (6 rows)
 
+-- hashing support
+set enable_hashagg = true;
+-- failure cases
+select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col);
+ERROR:  could not implement GROUP BY
+DETAIL:  Some of the datatypes only support hashing, while others only support sorting.
+select array_agg(v order by v) from gstest4 group by grouping sets ((id,unsortable_col),(id));
+ERROR:  could not implement GROUP BY
+DETAIL:  Some of the datatypes only support hashing, while others only support sorting.
+-- simple cases
+select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by grouping sets ((a),(b)) order by 3,1,2;
+ a | b | grouping | sum | count | max 
+---+---+----------+-----+-------+-----
+ 1 |   |        1 |  60 |     5 |  14
+ 2 |   |        1 |  15 |     1 |  15
+ 3 |   |        1 |  33 |     2 |  17
+ 4 |   |        1 |  37 |     2 |  19
+   | 1 |        2 |  58 |     4 |  19
+   | 2 |        2 |  25 |     2 |  13
+   | 3 |        2 |  45 |     3 |  16
+   | 4 |        2 |  17 |     1 |  17
+(8 rows)
+
+explain (costs off) select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by grouping sets ((a),(b)) order by 3,1,2;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: (GROUPING("*VALUES*".column1, "*VALUES*".column2)), "*VALUES*".column1, "*VALUES*".column2
+   ->  HashAggregate
+         Hash Key: "*VALUES*".column1
+         Hash Key: "*VALUES*".column2
+         ->  Values Scan on "*VALUES*"
+(6 rows)
+
+select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by cube(a,b) order by 3,1,2;
+ a | b | grouping | sum | count | max 
+---+---+----------+-----+-------+-----
+ 1 | 1 |        0 |  21 |     2 |  11
+ 1 | 2 |        0 |  25 |     2 |  13
+ 1 | 3 |        0 |  14 |     1 |  14
+ 2 | 3 |        0 |  15 |     1 |  15
+ 3 | 3 |        0 |  16 |     1 |  16
+ 3 | 4 |        0 |  17 |     1 |  17
+ 4 | 1 |        0 |  37 |     2 |  19
+ 1 |   |        1 |  60 |     5 |  14
+ 2 |   |        1 |  15 |     1 |  15
+ 3 |   |        1 |  33 |     2 |  17
+ 4 |   |        1 |  37 |     2 |  19
+   | 1 |        2 |  58 |     4 |  19
+   | 2 |        2 |  25 |     2 |  13
+   | 3 |        2 |  45 |     3 |  16
+   | 4 |        2 |  17 |     1 |  17
+   |   |        3 | 145 |    10 |  19
+(16 rows)
+
+explain (costs off) select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by cube(a,b) order by 3,1,2;
+                                               QUERY PLAN                                               
+--------------------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: (GROUPING("*VALUES*".column1, "*VALUES*".column2)), "*VALUES*".column1, "*VALUES*".column2
+   ->  MixedAggregate
+         Hash Key: "*VALUES*".column1, "*VALUES*".column2
+         Hash Key: "*VALUES*".column1
+         Hash Key: "*VALUES*".column2
+         Group Key: ()
+         ->  Values Scan on "*VALUES*"
+(8 rows)
+
+-- shouldn't try and hash
+explain (costs off)
+  select a, b, grouping(a,b), array_agg(v order by v)
+    from gstest1 group by cube(a,b);
+                        QUERY PLAN                        
+----------------------------------------------------------
+ GroupAggregate
+   Group Key: "*VALUES*".column1, "*VALUES*".column2
+   Group Key: "*VALUES*".column1
+   Group Key: ()
+   Sort Key: "*VALUES*".column2
+     Group Key: "*VALUES*".column2
+   ->  Sort
+         Sort Key: "*VALUES*".column1, "*VALUES*".column2
+         ->  Values Scan on "*VALUES*"
+(9 rows)
+
+-- mixed hashable/sortable cases
+select unhashable_col, unsortable_col,
+       grouping(unhashable_col, unsortable_col),
+       count(*), sum(v)
+  from gstest4 group by grouping sets ((unhashable_col),(unsortable_col))
+ order by 3, 5;
+ unhashable_col | unsortable_col | grouping | count | sum 
+----------------+----------------+----------+-------+-----
+ 0000           |                |        1 |     2 |  17
+ 0001           |                |        1 |     2 |  34
+ 0010           |                |        1 |     2 |  68
+ 0011           |                |        1 |     2 | 136
+                |              2 |        2 |     4 |  60
+                |              1 |        2 |     4 | 195
+(6 rows)
+
+explain (costs off)
+  select unhashable_col, unsortable_col,
+         grouping(unhashable_col, unsortable_col),
+         count(*), sum(v)
+    from gstest4 group by grouping sets ((unhashable_col),(unsortable_col))
+   order by 3,5;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: (GROUPING(unhashable_col, unsortable_col)), (sum(v))
+   ->  MixedAggregate
+         Hash Key: unsortable_col
+         Group Key: unhashable_col
+         ->  Sort
+               Sort Key: unhashable_col
+               ->  Seq Scan on gstest4
+(8 rows)
+
+select unhashable_col, unsortable_col,
+       grouping(unhashable_col, unsortable_col),
+       count(*), sum(v)
+  from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col))
+ order by 3,5;
+ unhashable_col | unsortable_col | grouping | count | sum 
+----------------+----------------+----------+-------+-----
+ 0000           |                |        1 |     1 |   1
+ 0001           |                |        1 |     1 |   2
+ 0010           |                |        1 |     1 |   4
+ 0011           |                |        1 |     1 |   8
+ 0000           |                |        1 |     1 |  16
+ 0001           |                |        1 |     1 |  32
+ 0010           |                |        1 |     1 |  64
+ 0011           |                |        1 |     1 | 128
+                |              1 |        2 |     1 |   1
+                |              1 |        2 |     1 |   2
+                |              2 |        2 |     1 |   4
+                |              2 |        2 |     1 |   8
+                |              2 |        2 |     1 |  16
+                |              2 |        2 |     1 |  32
+                |              1 |        2 |     1 |  64
+                |              1 |        2 |     1 | 128
+(16 rows)
+
+explain (costs off)
+  select unhashable_col, unsortable_col,
+         grouping(unhashable_col, unsortable_col),
+         count(*), sum(v)
+    from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col))
+   order by 3,5;
+                            QUERY PLAN                            
+------------------------------------------------------------------
+ Sort
+   Sort Key: (GROUPING(unhashable_col, unsortable_col)), (sum(v))
+   ->  MixedAggregate
+         Hash Key: v, unsortable_col
+         Group Key: v, unhashable_col
+         ->  Sort
+               Sort Key: v, unhashable_col
+               ->  Seq Scan on gstest4
+(8 rows)
+
+-- empty input: first is 0 rows, second 1, third 3 etc.
+select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a);
+ a | b | sum | count 
+---+---+-----+-------
+(0 rows)
+
+explain (costs off)
+  select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a);
+           QUERY PLAN           
+--------------------------------
+ HashAggregate
+   Hash Key: a, b
+   Hash Key: a
+   ->  Seq Scan on gstest_empty
+(4 rows)
+
+select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),());
+ a | b | sum | count 
+---+---+-----+-------
+   |   |     |     0
+(1 row)
+
+select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),());
+ a | b | sum | count 
+---+---+-----+-------
+   |   |     |     0
+   |   |     |     0
+   |   |     |     0
+(3 rows)
+
+explain (costs off)
+  select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),());
+           QUERY PLAN           
+--------------------------------
+ MixedAggregate
+   Hash Key: a, b
+   Group Key: ()
+   Group Key: ()
+   Group Key: ()
+   ->  Seq Scan on gstest_empty
+(6 rows)
+
+select sum(v), count(*) from gstest_empty group by grouping sets ((),(),());
+ sum | count 
+-----+-------
+     |     0
+     |     0
+     |     0
+(3 rows)
+
+explain (costs off)
+  select sum(v), count(*) from gstest_empty group by grouping sets ((),(),());
+           QUERY PLAN           
+--------------------------------
+ Aggregate
+   Group Key: ()
+   Group Key: ()
+   Group Key: ()
+   ->  Seq Scan on gstest_empty
+(5 rows)
+
+-- check that functionally dependent cols are not nulled
+select a, d, grouping(a,b,c)
+  from gstest3
+ group by grouping sets ((a,b), (a,c));
+ a | d | grouping 
+---+---+----------
+ 1 | 1 |        1
+ 2 | 2 |        1
+ 1 | 1 |        2
+ 2 | 2 |        2
+(4 rows)
+
+explain (costs off)
+  select a, d, grouping(a,b,c)
+    from gstest3
+   group by grouping sets ((a,b), (a,c));
+        QUERY PLAN         
+---------------------------
+ HashAggregate
+   Hash Key: a, b
+   Hash Key: a, c
+   ->  Seq Scan on gstest3
+(4 rows)
+
+-- simple rescan tests
+select a, b, sum(v.x)
+  from (values (1),(2)) v(x), gstest_data(v.x)
+ group by grouping sets (a,b);
+ a | b | sum 
+---+---+-----
+ 2 |   |   6
+ 1 |   |   3
+   | 2 |   3
+   | 3 |   3
+   | 1 |   3
+(5 rows)
+
+explain (costs off)
+  select a, b, sum(v.x)
+    from (values (1),(2)) v(x), gstest_data(v.x)
+   group by grouping sets (a,b);
+                QUERY PLAN                
+------------------------------------------
+ HashAggregate
+   Hash Key: gstest_data.a
+   Hash Key: gstest_data.b
+   ->  Nested Loop
+         ->  Values Scan on "*VALUES*"
+         ->  Function Scan on gstest_data
+(6 rows)
+
+select *
+  from (values (1),(2)) v(x),
+       lateral (select a, b, sum(v.x) from gstest_data(v.x) group by grouping sets (a,b)) s;
+ERROR:  aggregate functions are not allowed in FROM clause of their own query level
+LINE 3:        lateral (select a, b, sum(v.x) from gstest_data(v.x) ...
+                                     ^
+explain (costs off)
+  select *
+    from (values (1),(2)) v(x),
+         lateral (select a, b, sum(v.x) from gstest_data(v.x) group by grouping sets (a,b)) s;
+ERROR:  aggregate functions are not allowed in FROM clause of their own query level
+LINE 4:          lateral (select a, b, sum(v.x) from gstest_data(v.x...
+                                       ^
+-- Tests for chained aggregates
+select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6;
+ a | b | grouping | sum | count | max 
+---+---+----------+-----+-------+-----
+ 1 | 1 |        0 |  21 |     2 |  11
+ 1 | 2 |        0 |  25 |     2 |  13
+ 1 | 3 |        0 |  14 |     1 |  14
+ 2 | 3 |        0 |  15 |     1 |  15
+ 3 | 3 |        0 |  16 |     1 |  16
+ 3 | 4 |        0 |  17 |     1 |  17
+ 4 | 1 |        0 |  37 |     2 |  19
+   |   |        3 |  21 |     2 |  11
+   |   |        3 |  21 |     2 |  11
+   |   |        3 |  25 |     2 |  13
+   |   |        3 |  25 |     2 |  13
+   |   |        3 |  14 |     1 |  14
+   |   |        3 |  14 |     1 |  14
+   |   |        3 |  15 |     1 |  15
+   |   |        3 |  15 |     1 |  15
+   |   |        3 |  16 |     1 |  16
+   |   |        3 |  16 |     1 |  16
+   |   |        3 |  17 |     1 |  17
+   |   |        3 |  17 |     1 |  17
+   |   |        3 |  37 |     2 |  19
+   |   |        3 |  37 |     2 |  19
+(21 rows)
+
+explain (costs off)
+  select a, b, grouping(a,b), sum(v), count(*), max(v)
+    from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6;
+                                        QUERY PLAN                                         
+-------------------------------------------------------------------------------------------
+ Sort
+   Sort Key: (GROUPING("*VALUES*".column1, "*VALUES*".column2)), (max("*VALUES*".column3))
+   ->  HashAggregate
+         Hash Key: "*VALUES*".column1, "*VALUES*".column2
+         Hash Key: ("*VALUES*".column1 + 1), ("*VALUES*".column2 + 1)
+         Hash Key: ("*VALUES*".column1 + 2), ("*VALUES*".column2 + 2)
+         ->  Values Scan on "*VALUES*"
+(7 rows)
+
+select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum
+  from gstest2 group by cube (a,b) order by rsum, a, b;
+ a | b | sum | rsum 
+---+---+-----+------
+ 1 | 1 |   8 |    8
+ 1 | 2 |   2 |   10
+ 1 |   |  10 |   20
+ 2 | 2 |   2 |   22
+ 2 |   |   2 |   24
+   | 1 |   8 |   32
+   | 2 |   4 |   36
+   |   |  12 |   48
+(8 rows)
+
+explain (costs off)
+  select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum
+    from gstest2 group by cube (a,b) order by rsum, a, b;
+                 QUERY PLAN                  
+---------------------------------------------
+ Sort
+   Sort Key: (sum((sum(c))) OVER (?)), a, b
+   ->  WindowAgg
+         ->  Sort
+               Sort Key: a, b
+               ->  MixedAggregate
+                     Hash Key: a, b
+                     Hash Key: a
+                     Hash Key: b
+                     Group Key: ()
+                     ->  Seq Scan on gstest2
+(11 rows)
+
+select a, b, sum(v.x)
+  from (values (1),(2)) v(x), gstest_data(v.x)
+ group by cube (a,b) order by a,b;
+ a | b | sum 
+---+---+-----
+ 1 | 1 |   1
+ 1 | 2 |   1
+ 1 | 3 |   1
+ 1 |   |   3
+ 2 | 1 |   2
+ 2 | 2 |   2
+ 2 | 3 |   2
+ 2 |   |   6
+   | 1 |   3
+   | 2 |   3
+   | 3 |   3
+   |   |   9
+(12 rows)
+
+explain (costs off)
+  select a, b, sum(v.x)
+    from (values (1),(2)) v(x), gstest_data(v.x)
+   group by cube (a,b) order by a,b;
+                   QUERY PLAN                   
+------------------------------------------------
+ Sort
+   Sort Key: gstest_data.a, gstest_data.b
+   ->  MixedAggregate
+         Hash Key: gstest_data.a, gstest_data.b
+         Hash Key: gstest_data.a
+         Hash Key: gstest_data.b
+         Group Key: ()
+         ->  Nested Loop
+               ->  Values Scan on "*VALUES*"
+               ->  Function Scan on gstest_data
+(10 rows)
+
+-- More rescan tests
+select * from (values (1),(2)) v(a) left join lateral (select v.a, four, ten, count(*) from onek group by cube(four,ten)) s on true order by v.a,four,ten;
+ a | a | four | ten | count 
+---+---+------+-----+-------
+ 1 | 1 |    0 |   0 |    50
+ 1 | 1 |    0 |   2 |    50
+ 1 | 1 |    0 |   4 |    50
+ 1 | 1 |    0 |   6 |    50
+ 1 | 1 |    0 |   8 |    50
+ 1 | 1 |    0 |     |   250
+ 1 | 1 |    1 |   1 |    50
+ 1 | 1 |    1 |   3 |    50
+ 1 | 1 |    1 |   5 |    50
+ 1 | 1 |    1 |   7 |    50
+ 1 | 1 |    1 |   9 |    50
+ 1 | 1 |    1 |     |   250
+ 1 | 1 |    2 |   0 |    50
+ 1 | 1 |    2 |   2 |    50
+ 1 | 1 |    2 |   4 |    50
+ 1 | 1 |    2 |   6 |    50
+ 1 | 1 |    2 |   8 |    50
+ 1 | 1 |    2 |     |   250
+ 1 | 1 |    3 |   1 |    50
+ 1 | 1 |    3 |   3 |    50
+ 1 | 1 |    3 |   5 |    50
+ 1 | 1 |    3 |   7 |    50
+ 1 | 1 |    3 |   9 |    50
+ 1 | 1 |    3 |     |   250
+ 1 | 1 |      |   0 |   100
+ 1 | 1 |      |   1 |   100
+ 1 | 1 |      |   2 |   100
+ 1 | 1 |      |   3 |   100
+ 1 | 1 |      |   4 |   100
+ 1 | 1 |      |   5 |   100
+ 1 | 1 |      |   6 |   100
+ 1 | 1 |      |   7 |   100
+ 1 | 1 |      |   8 |   100
+ 1 | 1 |      |   9 |   100
+ 1 | 1 |      |     |  1000
+ 2 | 2 |    0 |   0 |    50
+ 2 | 2 |    0 |   2 |    50
+ 2 | 2 |    0 |   4 |    50
+ 2 | 2 |    0 |   6 |    50
+ 2 | 2 |    0 |   8 |    50
+ 2 | 2 |    0 |     |   250
+ 2 | 2 |    1 |   1 |    50
+ 2 | 2 |    1 |   3 |    50
+ 2 | 2 |    1 |   5 |    50
+ 2 | 2 |    1 |   7 |    50
+ 2 | 2 |    1 |   9 |    50
+ 2 | 2 |    1 |     |   250
+ 2 | 2 |    2 |   0 |    50
+ 2 | 2 |    2 |   2 |    50
+ 2 | 2 |    2 |   4 |    50
+ 2 | 2 |    2 |   6 |    50
+ 2 | 2 |    2 |   8 |    50
+ 2 | 2 |    2 |     |   250
+ 2 | 2 |    3 |   1 |    50
+ 2 | 2 |    3 |   3 |    50
+ 2 | 2 |    3 |   5 |    50
+ 2 | 2 |    3 |   7 |    50
+ 2 | 2 |    3 |   9 |    50
+ 2 | 2 |    3 |     |   250
+ 2 | 2 |      |   0 |   100
+ 2 | 2 |      |   1 |   100
+ 2 | 2 |      |   2 |   100
+ 2 | 2 |      |   3 |   100
+ 2 | 2 |      |   4 |   100
+ 2 | 2 |      |   5 |   100
+ 2 | 2 |      |   6 |   100
+ 2 | 2 |      |   7 |   100
+ 2 | 2 |      |   8 |   100
+ 2 | 2 |      |   9 |   100
+ 2 | 2 |      |     |  1000
+(70 rows)
+
+select array(select row(v.a,s1.*) from (select two,four, count(*) from onek group by cube(two,four) order by two,four) s1) from (values (1),(2)) v(a);
+                                                                        array                                                                         
+------------------------------------------------------------------------------------------------------------------------------------------------------
+ {"(1,0,0,250)","(1,0,2,250)","(1,0,,500)","(1,1,1,250)","(1,1,3,250)","(1,1,,500)","(1,,0,250)","(1,,1,250)","(1,,2,250)","(1,,3,250)","(1,,,1000)"}
+ {"(2,0,0,250)","(2,0,2,250)","(2,0,,500)","(2,1,1,250)","(2,1,3,250)","(2,1,,500)","(2,,0,250)","(2,,1,250)","(2,,2,250)","(2,,3,250)","(2,,,1000)"}
+(2 rows)
+
+-- Rescan logic changes when there are no empty grouping sets, so test
+-- that too:
+select * from (values (1),(2)) v(a) left join lateral (select v.a, four, ten, count(*) from onek group by grouping sets(four,ten)) s on true order by v.a,four,ten;
+ a | a | four | ten | count 
+---+---+------+-----+-------
+ 1 | 1 |    0 |     |   250
+ 1 | 1 |    1 |     |   250
+ 1 | 1 |    2 |     |   250
+ 1 | 1 |    3 |     |   250
+ 1 | 1 |      |   0 |   100
+ 1 | 1 |      |   1 |   100
+ 1 | 1 |      |   2 |   100
+ 1 | 1 |      |   3 |   100
+ 1 | 1 |      |   4 |   100
+ 1 | 1 |      |   5 |   100
+ 1 | 1 |      |   6 |   100
+ 1 | 1 |      |   7 |   100
+ 1 | 1 |      |   8 |   100
+ 1 | 1 |      |   9 |   100
+ 2 | 2 |    0 |     |   250
+ 2 | 2 |    1 |     |   250
+ 2 | 2 |    2 |     |   250
+ 2 | 2 |    3 |     |   250
+ 2 | 2 |      |   0 |   100
+ 2 | 2 |      |   1 |   100
+ 2 | 2 |      |   2 |   100
+ 2 | 2 |      |   3 |   100
+ 2 | 2 |      |   4 |   100
+ 2 | 2 |      |   5 |   100
+ 2 | 2 |      |   6 |   100
+ 2 | 2 |      |   7 |   100
+ 2 | 2 |      |   8 |   100
+ 2 | 2 |      |   9 |   100
+(28 rows)
+
+select array(select row(v.a,s1.*) from (select two,four, count(*) from onek group by grouping sets(two,four) order by two,four) s1) from (values (1),(2)) v(a);
+                                      array                                      
+---------------------------------------------------------------------------------
+ {"(1,0,,500)","(1,1,,500)","(1,,0,250)","(1,,1,250)","(1,,2,250)","(1,,3,250)"}
+ {"(2,0,,500)","(2,1,,500)","(2,,0,250)","(2,,1,250)","(2,,2,250)","(2,,3,250)"}
+(2 rows)
+
+-- test the knapsack
+set work_mem = '64kB';
+explain (costs off)
+  select unique1,
+         count(two), count(four), count(ten),
+         count(hundred), count(thousand), count(twothousand),
+         count(*)
+    from tenk1 group by grouping sets (unique1,twothousand,thousand,hundred,ten,four,two);
+          QUERY PLAN           
+-------------------------------
+ MixedAggregate
+   Hash Key: two
+   Hash Key: four
+   Hash Key: ten
+   Hash Key: hundred
+   Group Key: unique1
+   Sort Key: twothousand
+     Group Key: twothousand
+   Sort Key: thousand
+     Group Key: thousand
+   ->  Sort
+         Sort Key: unique1
+         ->  Seq Scan on tenk1
+(13 rows)
+
+explain (costs off)
+  select unique1,
+         count(two), count(four), count(ten),
+         count(hundred), count(thousand), count(twothousand),
+         count(*)
+    from tenk1 group by grouping sets (unique1,hundred,ten,four,two);
+          QUERY PLAN           
+-------------------------------
+ MixedAggregate
+   Hash Key: two
+   Hash Key: four
+   Hash Key: ten
+   Hash Key: hundred
+   Group Key: unique1
+   ->  Sort
+         Sort Key: unique1
+         ->  Seq Scan on tenk1
+(9 rows)
+
+set work_mem = '384kB';
+explain (costs off)
+  select unique1,
+         count(two), count(four), count(ten),
+         count(hundred), count(thousand), count(twothousand),
+         count(*)
+    from tenk1 group by grouping sets (unique1,twothousand,thousand,hundred,ten,four,two);
+          QUERY PLAN           
+-------------------------------
+ MixedAggregate
+   Hash Key: two
+   Hash Key: four
+   Hash Key: ten
+   Hash Key: hundred
+   Hash Key: thousand
+   Group Key: unique1
+   Sort Key: twothousand
+     Group Key: twothousand
+   ->  Sort
+         Sort Key: unique1
+         ->  Seq Scan on tenk1
+(12 rows)
+
 -- end
index 0eeaf9e8300d335003369abf52f714311d5d7554..33f370b49498f88d4ec6ffa5007a84f1e60d9942 100644 (file)
@@ -233,6 +233,7 @@ SELECT few.dataa, count(*), min(id), max(id), generate_series(1,3) FROM few GROU
 (6 rows)
 
 -- grouping sets are a bit special, they produce NULLs in columns not actually NULL
+set enable_hashagg = false;
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab);
  dataa |  b  | g | count 
 -------+-----+---+-------
@@ -311,46 +312,46 @@ SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(d
  b     | bar |   |     2
  b     |     |   |     2
        |     |   |     6
- a     |     | 1 |     2
- b     |     | 1 |     1
-       |     | 1 |     3
- a     |     | 2 |     2
- b     |     | 2 |     1
-       |     | 2 |     3
        | bar | 1 |     2
        | bar | 2 |     2
        | bar |   |     4
        | foo | 1 |     1
        | foo | 2 |     1
        | foo |   |     2
+ a     |     | 1 |     2
+ b     |     | 1 |     1
+       |     | 1 |     3
+ a     |     | 2 |     2
+ b     |     | 2 |     1
+       |     | 2 |     3
 (24 rows)
 
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab, g) ORDER BY dataa;
  dataa |  b  | g | count 
 -------+-----+---+-------
+ a     | foo |   |     2
+ a     |     |   |     4
+ a     |     | 2 |     2
  a     | bar | 1 |     1
  a     | bar | 2 |     1
  a     | bar |   |     2
  a     | foo | 1 |     1
  a     | foo | 2 |     1
- a     | foo |   |     2
- a     |     |   |     4
  a     |     | 1 |     2
- a     |     | 2 |     2
- b     | bar | 2 |     1
+ b     | bar | 1 |     1
  b     |     |   |     2
  b     |     | 1 |     1
- b     |     | 2 |     1
- b     | bar | 1 |     1
+ b     | bar | 2 |     1
  b     | bar |   |     2
-       | foo |   |     2
-       | foo | 1 |     1
+ b     |     | 2 |     1
        |     | 2 |     3
+       |     |   |     6
        | bar | 1 |     2
        | bar | 2 |     2
-       |     |   |     6
-       | foo | 2 |     1
        | bar |   |     4
+       | foo | 1 |     1
+       | foo | 2 |     1
+       | foo |   |     2
        |     | 1 |     3
 (24 rows)
 
@@ -360,29 +361,30 @@ SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(d
  a     | bar | 1 |     1
  a     | foo | 1 |     1
  b     | bar | 1 |     1
+       | bar | 1 |     2
+       | foo | 1 |     1
  a     |     | 1 |     2
  b     |     | 1 |     1
        |     | 1 |     3
-       | bar | 1 |     2
-       | foo | 1 |     1
-       | foo | 2 |     1
-       | bar | 2 |     2
  a     |     | 2 |     2
  b     |     | 2 |     1
- a     | bar | 2 |     1
+       | bar | 2 |     2
        |     | 2 |     3
+       | foo | 2 |     1
+ a     | bar | 2 |     1
  a     | foo | 2 |     1
  b     | bar | 2 |     1
- a     | foo |   |     2
+ a     |     |   |     4
  b     | bar |   |     2
  b     |     |   |     2
        |     |   |     6
- a     |     |   |     4
+ a     | foo |   |     2
+ a     | bar |   |     2
        | bar |   |     4
        | foo |   |     2
- a     | bar |   |     2
 (24 rows)
 
+reset enable_hashagg;
 -- data modification
 CREATE TABLE fewmore AS SELECT generate_series(1,3) AS data;
 INSERT INTO fewmore VALUES(generate_series(4,5));
index cc557cc552d7cf13c7e35d81e52fb7b1247b86cd..0e023d9ddc5ef803b245b5c1927dec3230fb9a8a 100644 (file)
@@ -31,6 +31,14 @@ copy gstest3 from stdin;
 \.
 alter table gstest3 add primary key (a);
 
+create temp table gstest4(id integer, v integer,
+                          unhashable_col bit(4), unsortable_col xid);
+insert into gstest4
+values (1,1,b'0000','1'), (2,2,b'0001','1'),
+       (3,4,b'0010','2'), (4,8,b'0011','2'),
+       (5,16,b'0000','2'), (6,32,b'0001','2'),
+       (7,64,b'0010','1'), (8,128,b'0011','1');
+
 create temp table gstest_empty (a integer, b integer, v integer);
 
 create function gstest_data(v integer, out a integer, out b integer)
@@ -43,8 +51,11 @@ create function gstest_data(v integer, out a integer, out b integer)
 
 -- basic functionality
 
+set enable_hashagg = false;  -- test hashing explicitly later
+
 -- simple rollup with multiple plain aggregates, with and without ordering
 -- (and with ordering differing from grouping)
+
 select a, b, grouping(a,b), sum(v), count(*), max(v)
   from gstest1 group by rollup (a,b);
 select a, b, grouping(a,b), sum(v), count(*), max(v)
@@ -161,7 +172,7 @@ select a, b from (values (1,2),(2,3)) v(a,b) group by a,b, grouping sets(a);
 
 -- Tests for chained aggregates
 select a, b, grouping(a,b), sum(v), count(*), max(v)
-  from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2));
+  from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6;
 select(select (select grouping(a,b) from (values (1)) v2(c)) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY ROLLUP((e+1),(f+1));
 select(select (select grouping(a,b) from (values (1)) v2(c)) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY CUBE((e+1),(f+1)) ORDER BY (e+1),(f+1);
 select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum
@@ -224,4 +235,146 @@ select array(select row(v.a,s1.*) from (select two,four, count(*) from onek grou
 select sum(ten) from onek group by two, rollup(four::text) order by 1;
 select sum(ten) from onek group by rollup(four::text), two order by 1;
 
+-- hashing support
+
+set enable_hashagg = true;
+
+-- failure cases
+
+select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col);
+select array_agg(v order by v) from gstest4 group by grouping sets ((id,unsortable_col),(id));
+
+-- simple cases
+
+select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by grouping sets ((a),(b)) order by 3,1,2;
+explain (costs off) select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by grouping sets ((a),(b)) order by 3,1,2;
+
+select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by cube(a,b) order by 3,1,2;
+explain (costs off) select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by cube(a,b) order by 3,1,2;
+
+-- shouldn't try and hash
+explain (costs off)
+  select a, b, grouping(a,b), array_agg(v order by v)
+    from gstest1 group by cube(a,b);
+
+-- mixed hashable/sortable cases
+select unhashable_col, unsortable_col,
+       grouping(unhashable_col, unsortable_col),
+       count(*), sum(v)
+  from gstest4 group by grouping sets ((unhashable_col),(unsortable_col))
+ order by 3, 5;
+explain (costs off)
+  select unhashable_col, unsortable_col,
+         grouping(unhashable_col, unsortable_col),
+         count(*), sum(v)
+    from gstest4 group by grouping sets ((unhashable_col),(unsortable_col))
+   order by 3,5;
+
+select unhashable_col, unsortable_col,
+       grouping(unhashable_col, unsortable_col),
+       count(*), sum(v)
+  from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col))
+ order by 3,5;
+explain (costs off)
+  select unhashable_col, unsortable_col,
+         grouping(unhashable_col, unsortable_col),
+         count(*), sum(v)
+    from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col))
+   order by 3,5;
+
+-- empty input: first is 0 rows, second 1, third 3 etc.
+select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a);
+explain (costs off)
+  select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a);
+select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),());
+select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),());
+explain (costs off)
+  select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),());
+select sum(v), count(*) from gstest_empty group by grouping sets ((),(),());
+explain (costs off)
+  select sum(v), count(*) from gstest_empty group by grouping sets ((),(),());
+
+-- check that functionally dependent cols are not nulled
+select a, d, grouping(a,b,c)
+  from gstest3
+ group by grouping sets ((a,b), (a,c));
+explain (costs off)
+  select a, d, grouping(a,b,c)
+    from gstest3
+   group by grouping sets ((a,b), (a,c));
+
+-- simple rescan tests
+
+select a, b, sum(v.x)
+  from (values (1),(2)) v(x), gstest_data(v.x)
+ group by grouping sets (a,b);
+explain (costs off)
+  select a, b, sum(v.x)
+    from (values (1),(2)) v(x), gstest_data(v.x)
+   group by grouping sets (a,b);
+
+select *
+  from (values (1),(2)) v(x),
+       lateral (select a, b, sum(v.x) from gstest_data(v.x) group by grouping sets (a,b)) s;
+explain (costs off)
+  select *
+    from (values (1),(2)) v(x),
+         lateral (select a, b, sum(v.x) from gstest_data(v.x) group by grouping sets (a,b)) s;
+
+-- Tests for chained aggregates
+select a, b, grouping(a,b), sum(v), count(*), max(v)
+  from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6;
+explain (costs off)
+  select a, b, grouping(a,b), sum(v), count(*), max(v)
+    from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6;
+select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum
+  from gstest2 group by cube (a,b) order by rsum, a, b;
+explain (costs off)
+  select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum
+    from gstest2 group by cube (a,b) order by rsum, a, b;
+select a, b, sum(v.x)
+  from (values (1),(2)) v(x), gstest_data(v.x)
+ group by cube (a,b) order by a,b;
+explain (costs off)
+  select a, b, sum(v.x)
+    from (values (1),(2)) v(x), gstest_data(v.x)
+   group by cube (a,b) order by a,b;
+
+-- More rescan tests
+select * from (values (1),(2)) v(a) left join lateral (select v.a, four, ten, count(*) from onek group by cube(four,ten)) s on true order by v.a,four,ten;
+select array(select row(v.a,s1.*) from (select two,four, count(*) from onek group by cube(two,four) order by two,four) s1) from (values (1),(2)) v(a);
+
+-- Rescan logic changes when there are no empty grouping sets, so test
+-- that too:
+select * from (values (1),(2)) v(a) left join lateral (select v.a, four, ten, count(*) from onek group by grouping sets(four,ten)) s on true order by v.a,four,ten;
+select array(select row(v.a,s1.*) from (select two,four, count(*) from onek group by grouping sets(two,four) order by two,four) s1) from (values (1),(2)) v(a);
+
+-- test the knapsack
+
+set work_mem = '64kB';
+explain (costs off)
+  select unique1,
+         count(two), count(four), count(ten),
+         count(hundred), count(thousand), count(twothousand),
+         count(*)
+    from tenk1 group by grouping sets (unique1,twothousand,thousand,hundred,ten,four,two);
+explain (costs off)
+  select unique1,
+         count(two), count(four), count(ten),
+         count(hundred), count(thousand), count(twothousand),
+         count(*)
+    from tenk1 group by grouping sets (unique1,hundred,ten,four,two);
+
+set work_mem = '384kB';
+explain (costs off)
+  select unique1,
+         count(two), count(four), count(ten),
+         count(hundred), count(thousand), count(twothousand),
+         count(*)
+    from tenk1 group by grouping sets (unique1,twothousand,thousand,hundred,ten,four,two);
+
 -- end
index e627bb99ed963352bf8158de512b1d7705ba2b83..417e78c53dd192e68f3f73975b9244723d5d55f3 100644 (file)
@@ -66,12 +66,14 @@ SELECT SUM(count(*)) OVER(PARTITION BY generate_series(1,3) ORDER BY generate_se
 SELECT few.dataa, count(*), min(id), max(id), generate_series(1,3) FROM few GROUP BY few.dataa ORDER BY 5, 1;
 
 -- grouping sets are a bit special, they produce NULLs in columns not actually NULL
+set enable_hashagg = false;
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab);
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab) ORDER BY dataa;
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab) ORDER BY g;
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab, g);
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab, g) ORDER BY dataa;
 SELECT dataa, datab b, generate_series(1,2) g, count(*) FROM few GROUP BY CUBE(dataa, datab, g) ORDER BY g;
+reset enable_hashagg;
 
 -- data modification
 CREATE TABLE fewmore AS SELECT generate_series(1,3) AS data;