diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml index 5d1c90282f9..89ced3cd978 100644 --- a/doc/src/sgml/config.sgml +++ b/doc/src/sgml/config.sgml @@ -1751,6 +1751,23 @@ include_dir 'conf.d' + + hashagg_mem_overflow (boolean) + + hashagg_mem_overflow configuration parameter + + + + + If hash aggregation exceeds work_mem at query + execution time, and hashagg_mem_overflow is set + to on, continue consuming more memory rather than + performing disk-based hash aggregation. The default + is off. + + + + max_stack_depth (integer) @@ -4451,6 +4468,24 @@ ANY num_sync ( + enable_hashagg_spill (boolean) + + enable_hashagg_spill configuration parameter + + + + + Enables or disables the query planner's use of hashed aggregation plan + types when the memory usage is expected to + exceed work_mem. This only affects the planner + choice; actual behavior at execution time is dictated by + . The default + is on. + + + + enable_hashjoin (boolean) diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c index 949fefa23ae..c2fb7a088a2 100644 --- a/src/backend/commands/explain.c +++ b/src/backend/commands/explain.c @@ -102,6 +102,7 @@ static void show_tablesample(TableSampleClause *tsc, PlanState *planstate, List *ancestors, ExplainState *es); static void show_sort_info(SortState *sortstate, ExplainState *es); static void show_hash_info(HashState *hashstate, ExplainState *es); +static void show_hashagg_info(AggState *hashstate, ExplainState *es); static void show_tidbitmap_info(BitmapHeapScanState *planstate, ExplainState *es); static void show_instrumentation_count(const char *qlabel, int which, @@ -1844,6 +1845,8 @@ ExplainNode(PlanState *planstate, List *ancestors, case T_Agg: show_agg_keys(castNode(AggState, planstate), ancestors, es); show_upper_qual(plan->qual, "Filter", planstate, ancestors, es); + if (es->analyze) + show_hashagg_info((AggState *) planstate, es); if (plan->qual) show_instrumentation_count("Rows Removed by Filter", 1, planstate, es); @@ -2742,6 +2745,56 @@ show_hash_info(HashState *hashstate, ExplainState *es) } } +/* + * If EXPLAIN ANALYZE, show information on hash aggregate memory usage and + * batches. + */ +static void +show_hashagg_info(AggState *aggstate, ExplainState *es) +{ + Agg *agg = (Agg *)aggstate->ss.ps.plan; + long memPeakKb = (aggstate->hash_mem_peak + 1023) / 1024; + long diskKb = (aggstate->hash_disk_used + 1023) / 1024; + + + Assert(IsA(aggstate, AggState)); + + if (agg->aggstrategy != AGG_HASHED && + agg->aggstrategy != AGG_MIXED) + return; + + if (es->format == EXPLAIN_FORMAT_TEXT) + { + appendStringInfoSpaces(es->str, es->indent * 2); + appendStringInfo( + es->str, + "Memory Usage: %ldkB", + memPeakKb); + + if (aggstate->hash_batches_used > 0) + { + appendStringInfo( + es->str, + " Batches: %d Disk: %ldkB", + aggstate->hash_batches_used, diskKb); + } + + appendStringInfo( + es->str, + "\n"); + } + else + { + ExplainPropertyInteger("Peak Memory Usage", "kB", memPeakKb, es); + if (aggstate->hash_batches_used > 0) + { + ExplainPropertyInteger("HashAgg Batches", NULL, + aggstate->hash_batches_used, es); + ExplainPropertyInteger("Disk Usage", "kB", diskKb, es); + } + } +} + /* * If it's EXPLAIN ANALYZE, show exact/lossy pages for a BitmapHeapScan node */ diff --git a/src/backend/executor/execExpr.c b/src/backend/executor/execExpr.c index 8da2e2dcbba..fb3e81764ad 100644 --- a/src/backend/executor/execExpr.c +++ b/src/backend/executor/execExpr.c @@ -79,7 +79,8 @@ static void ExecInitCoerceToDomain(ExprEvalStep *scratch, CoerceToDomain *ctest, static void ExecBuildAggTransCall(ExprState *state, AggState *aggstate, ExprEvalStep *scratch, FunctionCallInfo fcinfo, AggStatePerTrans pertrans, - int transno, int setno, int setoff, bool ishash); + int transno, int setno, int setoff, bool ishash, + bool spilled); /* @@ -2927,7 +2928,7 @@ ExecInitCoerceToDomain(ExprEvalStep *scratch, CoerceToDomain *ctest, */ ExprState * ExecBuildAggTrans(AggState *aggstate, AggStatePerPhase phase, - bool doSort, bool doHash) + bool doSort, bool doHash, bool spilled) { ExprState *state = makeNode(ExprState); PlanState *parent = &aggstate->ss.ps; @@ -3160,7 +3161,8 @@ ExecBuildAggTrans(AggState *aggstate, AggStatePerPhase phase, for (setno = 0; setno < processGroupingSets; setno++) { ExecBuildAggTransCall(state, aggstate, &scratch, trans_fcinfo, - pertrans, transno, setno, setoff, false); + pertrans, transno, setno, setoff, false, + spilled); setoff++; } } @@ -3178,7 +3180,8 @@ ExecBuildAggTrans(AggState *aggstate, AggStatePerPhase phase, for (setno = 0; setno < numHashes; setno++) { ExecBuildAggTransCall(state, aggstate, &scratch, trans_fcinfo, - pertrans, transno, setno, setoff, true); + pertrans, transno, setno, setoff, true, + spilled); setoff++; } } @@ -3226,7 +3229,8 @@ static void ExecBuildAggTransCall(ExprState *state, AggState *aggstate, ExprEvalStep *scratch, FunctionCallInfo fcinfo, AggStatePerTrans pertrans, - int transno, int setno, int setoff, bool ishash) + int transno, int setno, int setoff, bool ishash, + bool spilled) { int adjust_init_jumpnull = -1; int adjust_strict_jumpnull = -1; @@ -3248,7 +3252,8 @@ ExecBuildAggTransCall(ExprState *state, AggState *aggstate, fcinfo->flinfo->fn_strict && pertrans->initValueIsNull) { - scratch->opcode = EEOP_AGG_INIT_TRANS; + scratch->opcode = spilled ? + EEOP_AGG_INIT_TRANS_SPILLED : EEOP_AGG_INIT_TRANS; scratch->d.agg_init_trans.aggstate = aggstate; scratch->d.agg_init_trans.pertrans = pertrans; scratch->d.agg_init_trans.setno = setno; @@ -3265,7 +3270,8 @@ ExecBuildAggTransCall(ExprState *state, AggState *aggstate, if (pertrans->numSortCols == 0 && fcinfo->flinfo->fn_strict) { - scratch->opcode = EEOP_AGG_STRICT_TRANS_CHECK; + scratch->opcode = spilled ? + EEOP_AGG_STRICT_TRANS_CHECK_SPILLED : EEOP_AGG_STRICT_TRANS_CHECK; scratch->d.agg_strict_trans_check.aggstate = aggstate; scratch->d.agg_strict_trans_check.setno = setno; scratch->d.agg_strict_trans_check.setoff = setoff; @@ -3283,9 +3289,11 @@ ExecBuildAggTransCall(ExprState *state, AggState *aggstate, /* invoke appropriate transition implementation */ if (pertrans->numSortCols == 0 && pertrans->transtypeByVal) - scratch->opcode = EEOP_AGG_PLAIN_TRANS_BYVAL; + scratch->opcode = spilled ? + EEOP_AGG_PLAIN_TRANS_BYVAL_SPILLED : EEOP_AGG_PLAIN_TRANS_BYVAL; else if (pertrans->numSortCols == 0) - scratch->opcode = EEOP_AGG_PLAIN_TRANS; + scratch->opcode = spilled ? + EEOP_AGG_PLAIN_TRANS_SPILLED : EEOP_AGG_PLAIN_TRANS; else if (pertrans->numInputs == 1) scratch->opcode = EEOP_AGG_ORDERED_TRANS_DATUM; else diff --git a/src/backend/executor/execExprInterp.c b/src/backend/executor/execExprInterp.c index dbed5978162..49fbf8e4a42 100644 --- a/src/backend/executor/execExprInterp.c +++ b/src/backend/executor/execExprInterp.c @@ -430,9 +430,13 @@ ExecInterpExpr(ExprState *state, ExprContext *econtext, bool *isnull) &&CASE_EEOP_AGG_STRICT_INPUT_CHECK_ARGS, &&CASE_EEOP_AGG_STRICT_INPUT_CHECK_NULLS, &&CASE_EEOP_AGG_INIT_TRANS, + &&CASE_EEOP_AGG_INIT_TRANS_SPILLED, &&CASE_EEOP_AGG_STRICT_TRANS_CHECK, + &&CASE_EEOP_AGG_STRICT_TRANS_CHECK_SPILLED, &&CASE_EEOP_AGG_PLAIN_TRANS_BYVAL, + &&CASE_EEOP_AGG_PLAIN_TRANS_BYVAL_SPILLED, &&CASE_EEOP_AGG_PLAIN_TRANS, + &&CASE_EEOP_AGG_PLAIN_TRANS_SPILLED, &&CASE_EEOP_AGG_ORDERED_TRANS_DATUM, &&CASE_EEOP_AGG_ORDERED_TRANS_TUPLE, &&CASE_EEOP_LAST @@ -1625,6 +1629,36 @@ ExecInterpExpr(ExprState *state, ExprContext *econtext, bool *isnull) EEO_NEXT(); } + EEO_CASE(EEOP_AGG_INIT_TRANS_SPILLED) + { + AggState *aggstate; + AggStatePerGroup pergroup; + AggStatePerGroup pergroup_allaggs; + + aggstate = op->d.agg_init_trans.aggstate; + pergroup_allaggs = aggstate->all_pergroups[op->d.agg_init_trans.setoff]; + + if (pergroup_allaggs == NULL) + EEO_NEXT(); + + pergroup = &pergroup_allaggs[op->d.agg_init_trans.transno]; + + /* If transValue has not yet been initialized, do so now. */ + if (pergroup->noTransValue) + { + AggStatePerTrans pertrans = op->d.agg_init_trans.pertrans; + + aggstate->curaggcontext = op->d.agg_init_trans.aggcontext; + aggstate->current_set = op->d.agg_init_trans.setno; + + ExecAggInitGroup(aggstate, pertrans, pergroup); + + /* copied trans value from input, done this round */ + EEO_JUMP(op->d.agg_init_trans.jumpnull); + } + + EEO_NEXT(); + } /* check that a strict aggregate's input isn't NULL */ EEO_CASE(EEOP_AGG_STRICT_TRANS_CHECK) @@ -1642,6 +1676,25 @@ ExecInterpExpr(ExprState *state, ExprContext *econtext, bool *isnull) EEO_NEXT(); } + EEO_CASE(EEOP_AGG_STRICT_TRANS_CHECK_SPILLED) + { + AggState *aggstate; + AggStatePerGroup pergroup; + AggStatePerGroup pergroup_allaggs; + + aggstate = op->d.agg_strict_trans_check.aggstate; + pergroup_allaggs = aggstate->all_pergroups[op->d.agg_strict_trans_check.setoff]; + + if (pergroup_allaggs == NULL) + EEO_NEXT(); + + pergroup = &pergroup_allaggs[op->d.agg_strict_trans_check.transno]; + + if (unlikely(pergroup->transValueIsNull)) + EEO_JUMP(op->d.agg_strict_trans_check.jumpnull); + + EEO_NEXT(); + } /* * Evaluate aggregate transition / combine function that has a @@ -1691,6 +1744,52 @@ ExecInterpExpr(ExprState *state, ExprContext *econtext, bool *isnull) EEO_NEXT(); } + EEO_CASE(EEOP_AGG_PLAIN_TRANS_BYVAL_SPILLED) + { + AggState *aggstate; + AggStatePerTrans pertrans; + AggStatePerGroup pergroup; + AggStatePerGroup pergroup_allaggs; + FunctionCallInfo fcinfo; + MemoryContext oldContext; + Datum newVal; + + aggstate = op->d.agg_trans.aggstate; + pertrans = op->d.agg_trans.pertrans; + + pergroup_allaggs = aggstate->all_pergroups[op->d.agg_trans.setoff]; + pergroup = &pergroup_allaggs[op->d.agg_trans.transno]; + + if (pergroup_allaggs == NULL) + EEO_NEXT(); + + Assert(pertrans->transtypeByVal); + + fcinfo = pertrans->transfn_fcinfo; + + /* cf. select_current_set() */ + aggstate->curaggcontext = op->d.agg_trans.aggcontext; + aggstate->current_set = op->d.agg_trans.setno; + + /* set up aggstate->curpertrans for AggGetAggref() */ + aggstate->curpertrans = pertrans; + + /* invoke transition function in per-tuple context */ + oldContext = MemoryContextSwitchTo(aggstate->tmpcontext->ecxt_per_tuple_memory); + + fcinfo->args[0].value = pergroup->transValue; + fcinfo->args[0].isnull = pergroup->transValueIsNull; + fcinfo->isnull = false; /* just in case transfn doesn't set it */ + + newVal = FunctionCallInvoke(fcinfo); + + pergroup->transValue = newVal; + pergroup->transValueIsNull = fcinfo->isnull; + + MemoryContextSwitchTo(oldContext); + + EEO_NEXT(); + } /* * Evaluate aggregate transition / combine function that has a @@ -1756,6 +1855,67 @@ ExecInterpExpr(ExprState *state, ExprContext *econtext, bool *isnull) EEO_NEXT(); } + EEO_CASE(EEOP_AGG_PLAIN_TRANS_SPILLED) + { + AggState *aggstate; + AggStatePerTrans pertrans; + AggStatePerGroup pergroup; + AggStatePerGroup pergroup_allaggs; + FunctionCallInfo fcinfo; + MemoryContext oldContext; + Datum newVal; + + aggstate = op->d.agg_trans.aggstate; + pertrans = op->d.agg_trans.pertrans; + + pergroup_allaggs = aggstate->all_pergroups[op->d.agg_trans.setoff]; + + if (pergroup_allaggs == NULL) + EEO_NEXT(); + + pergroup = &pergroup_allaggs[op->d.agg_trans.transno]; + + Assert(!pertrans->transtypeByVal); + + fcinfo = pertrans->transfn_fcinfo; + + /* cf. select_current_set() */ + aggstate->curaggcontext = op->d.agg_trans.aggcontext; + aggstate->current_set = op->d.agg_trans.setno; + + /* set up aggstate->curpertrans for AggGetAggref() */ + aggstate->curpertrans = pertrans; + + /* invoke transition function in per-tuple context */ + oldContext = MemoryContextSwitchTo(aggstate->tmpcontext->ecxt_per_tuple_memory); + + fcinfo->args[0].value = pergroup->transValue; + fcinfo->args[0].isnull = pergroup->transValueIsNull; + fcinfo->isnull = false; /* just in case transfn doesn't set it */ + + newVal = FunctionCallInvoke(fcinfo); + + /* + * For pass-by-ref datatype, must copy the new value into + * aggcontext and free the prior transValue. But if transfn + * returned a pointer to its first input, we don't need to do + * anything. Also, if transfn returned a pointer to a R/W + * expanded object that is already a child of the aggcontext, + * assume we can adopt that value without copying it. + */ + if (DatumGetPointer(newVal) != DatumGetPointer(pergroup->transValue)) + newVal = ExecAggTransReparent(aggstate, pertrans, + newVal, fcinfo->isnull, + pergroup->transValue, + pergroup->transValueIsNull); + + pergroup->transValue = newVal; + pergroup->transValueIsNull = fcinfo->isnull; + + MemoryContextSwitchTo(oldContext); + + EEO_NEXT(); + } /* process single-column ordered aggregate datum */ EEO_CASE(EEOP_AGG_ORDERED_TRANS_DATUM) diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c index e361143094c..02dba3eac18 100644 --- a/src/backend/executor/execGrouping.c +++ b/src/backend/executor/execGrouping.c @@ -25,8 +25,9 @@ #include "utils/lsyscache.h" #include "utils/memutils.h" -static uint32 TupleHashTableHash(struct tuplehash_hash *tb, const MinimalTuple tuple); static int TupleHashTableMatch(struct tuplehash_hash *tb, const MinimalTuple tuple1, const MinimalTuple tuple2); +static TupleHashEntry LookupTupleHashEntry_internal( + TupleHashTable hashtable, TupleTableSlot *slot, bool *isnew, uint32 hash); /* * Define parameters for tuple hash table code generation. The interface is @@ -284,6 +285,17 @@ ResetTupleHashTable(TupleHashTable hashtable) tuplehash_reset(hashtable->hashtab); } +/* + * Destroy the hash table. Note that the tablecxt passed to + * BuildTupleHashTableExt() should also be reset, otherwise there will be + * leaks. + */ +void +DestroyTupleHashTable(TupleHashTable hashtable) +{ + tuplehash_destroy(hashtable->hashtab); +} + /* * Find or create a hashtable entry for the tuple group containing the * given tuple. The tuple must be the same type as the hashtable entries. @@ -300,10 +312,9 @@ TupleHashEntry LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, bool *isnew) { - TupleHashEntryData *entry; - MemoryContext oldContext; - bool found; - MinimalTuple key; + TupleHashEntry entry; + MemoryContext oldContext; + uint32 hash; /* Need to run the hash functions in short-lived context */ oldContext = MemoryContextSwitchTo(hashtable->tempcxt); @@ -313,32 +324,29 @@ LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, hashtable->in_hash_funcs = hashtable->tab_hash_funcs; hashtable->cur_eq_func = hashtable->tab_eq_func; - key = NULL; /* flag to reference inputslot */ + hash = TupleHashTableHash(hashtable->hashtab, NULL); + entry = LookupTupleHashEntry_internal(hashtable, slot, isnew, hash); - if (isnew) - { - entry = tuplehash_insert(hashtable->hashtab, key, &found); + MemoryContextSwitchTo(oldContext); - if (found) - { - /* found pre-existing entry */ - *isnew = false; - } - else - { - /* created new entry */ - *isnew = true; - /* zero caller data */ - entry->additional = NULL; - MemoryContextSwitchTo(hashtable->tablecxt); - /* Copy the first tuple into the table context */ - entry->firstTuple = ExecCopySlotMinimalTuple(slot); - } - } - else - { - entry = tuplehash_lookup(hashtable->hashtab, key); - } + return entry; +} + +/* + * A variant of LookupTupleHashEntry for callers that have already computed + * the hash value. + */ +TupleHashEntry +LookupTupleHashEntryHash(TupleHashTable hashtable, TupleTableSlot *slot, + bool *isnew, uint32 hash) +{ + TupleHashEntry entry; + MemoryContext oldContext; + + /* Need to run the hash functions in short-lived context */ + oldContext = MemoryContextSwitchTo(hashtable->tempcxt); + + entry = LookupTupleHashEntry_internal(hashtable, slot, isnew, hash); MemoryContextSwitchTo(oldContext); @@ -389,7 +397,7 @@ FindTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, * Also, the caller must select an appropriate memory context for running * the hash functions. (dynahash.c doesn't change CurrentMemoryContext.) */ -static uint32 +uint32 TupleHashTableHash(struct tuplehash_hash *tb, const MinimalTuple tuple) { TupleHashTable hashtable = (TupleHashTable) tb->private_data; @@ -450,6 +458,54 @@ TupleHashTableHash(struct tuplehash_hash *tb, const MinimalTuple tuple) return murmurhash32(hashkey); } +/* + * Does the work of LookupTupleHashEntry and LookupTupleHashEntryHash. Useful + * so that we can avoid switching the memory context multiple times for + * LookupTupleHashEntry. + */ +static TupleHashEntry +LookupTupleHashEntry_internal(TupleHashTable hashtable, TupleTableSlot *slot, + bool *isnew, uint32 hash) +{ + TupleHashEntryData *entry; + bool found; + MinimalTuple key; + + /* set up data needed by hash and match functions */ + hashtable->inputslot = slot; + hashtable->in_hash_funcs = hashtable->tab_hash_funcs; + hashtable->cur_eq_func = hashtable->tab_eq_func; + + key = NULL; /* flag to reference inputslot */ + + if (isnew) + { + entry = tuplehash_insert_hash(hashtable->hashtab, key, hash, &found); + + if (found) + { + /* found pre-existing entry */ + *isnew = false; + } + else + { + /* created new entry */ + *isnew = true; + /* zero caller data */ + entry->additional = NULL; + MemoryContextSwitchTo(hashtable->tablecxt); + /* Copy the first tuple into the table context */ + entry->firstTuple = ExecCopySlotMinimalTuple(slot); + } + } + else + { + entry = tuplehash_lookup_hash(hashtable->hashtab, key, hash); + } + + return entry; +} + /* * See whether two tuples (presumably of the same hash value) match */ diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c index 6ee24eab3d2..f1989b10eac 100644 --- a/src/backend/executor/nodeAgg.c +++ b/src/backend/executor/nodeAgg.c @@ -194,6 +194,18 @@ * transition values. hashcontext is the single context created to support * all hash tables. * + * When the hash table memory exceeds work_mem, we advance the transition + * states only for groups already in the hash table. For tuples that would + * need to create a new hash table entries (and initialize new transition + * states), we spill them to disk to be processed later. The tuples are + * spilled in a partitioned manner, so that subsequent batches are smaller + * and less likely to exceed work_mem (if a batch does exceed work_mem, it + * must be spilled recursively). + * + * Note that it's possible for transition states to start small but then + * grow very large; for instance in the case of ARRAY_AGG. In such cases, + * it's still possible to significantly exceed work_mem. + * * Transition / Combine function invocation: * * For performance reasons transition functions, including combine @@ -229,15 +241,70 @@ #include "optimizer/optimizer.h" #include "parser/parse_agg.h" #include "parser/parse_coerce.h" +#include "storage/buffile.h" #include "utils/acl.h" #include "utils/builtins.h" #include "utils/datum.h" +#include "utils/dynahash.h" #include "utils/expandeddatum.h" #include "utils/lsyscache.h" #include "utils/memutils.h" #include "utils/syscache.h" #include "utils/tuplesort.h" +/* + * Control how many partitions are created when spilling HashAgg to + * disk. + * + * HASH_PARTITION_FACTOR is multiplied by the estimated number of partitions + * needed such that each partition will fit in memory. The factor is set + * higher than one because there's not a high cost to having a few too many + * partitions, and it makes it less likely that a partition will need to be + * spilled recursively. Another benefit of having more, smaller partitions is + * that small hash tables may perform better than large ones due to memory + * caching effects. + * + * HASH_PARTITION_MEM is the approximate amount of work_mem we should reserve + * for the partitions themselves (i.e. buffering of the files backing the + * partitions). This is sloppy, because we must reserve the memory before + * filling the hash table; but we choose the number of partitions at the time + * we need to spill. + * + * We also specify a min and max number of partitions per spill. Too few might + * mean a lot of wasted I/O from repeated spilling of the same tuples. Too + * many will result in lots of memory wasted buffering the spill files (and + * possibly pushing hidden costs to the OS for managing more files). + */ +#define HASH_PARTITION_FACTOR 1.50 +#define HASH_MIN_PARTITIONS 4 +#define HASH_MAX_PARTITIONS 256 +#define HASH_PARTITION_MEM (HASH_MIN_PARTITIONS * BLCKSZ) + +/* + * Represents partitioned spill data for a single hashtable. + */ +typedef struct HashAggSpill +{ + int n_partitions; /* number of output partitions */ + int partition_bits; /* number of bits for partition mask + log2(n_partitions) parent partition bits */ + BufFile **partitions; /* output partition files */ + int64 *ntuples; /* number of tuples in each partition */ +} HashAggSpill; + +/* + * Represents work to be done for one pass of hash aggregation. Initially, + * only the input fields are set. If spilled to disk, also set the spill data. + */ +typedef struct HashAggBatch +{ + BufFile *input_file; /* input partition */ + int input_bits; /* number of bits for input partition mask */ + int64 input_tuples; /* number of tuples in this batch */ + int setno; /* grouping set */ + HashAggSpill spill; /* spill output */ +} HashAggBatch; + 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); @@ -271,12 +338,35 @@ static void finalize_aggregates(AggState *aggstate, 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); +static void build_hash_table(AggState *aggstate, int setno, + int64 ngroups_estimate); +static void prepare_hash_slot(AggState *aggstate); +static void hash_recompile_expressions(AggState *aggstate); +static uint32 calculate_hash(AggState *aggstate); +static long hash_choose_num_buckets(AggState *aggstate, + long estimated_nbuckets, + Size memory); +static int hash_choose_num_spill_partitions(uint64 input_groups, + double hashentrysize); +static AggStatePerGroup lookup_hash_entry(AggState *aggstate, uint32 hash); static void lookup_hash_entries(AggState *aggstate); static TupleTableSlot *agg_retrieve_direct(AggState *aggstate); static void agg_fill_hash_table(AggState *aggstate); +static bool agg_refill_hash_table(AggState *aggstate); static TupleTableSlot *agg_retrieve_hash_table(AggState *aggstate); +static TupleTableSlot *agg_retrieve_hash_table_in_memory(AggState *aggstate); +static void hash_spill_init(HashAggSpill *spill, int input_bits, + uint64 input_tuples, double hashentrysize); +static Size hash_spill_tuple(HashAggSpill *spill, int input_bits, + TupleTableSlot *slot, uint32 hash); +static MinimalTuple hash_read_spilled(BufFile *file, uint32 *hashp); +static HashAggBatch *hash_batch_new(BufFile *input_file, int setno, + int64 input_tuples, int input_bits); +static void hash_finish_initial_spills(AggState *aggstate); +static void hash_spill_finish(AggState *aggstate, HashAggSpill *spill, + int setno, int input_bits); +static void hash_reset_spill(HashAggSpill *spill); +static void hash_reset_spills(AggState *aggstate); static Datum GetAggInitVal(Datum textInitVal, Oid transtype); static void build_pertrans_for_aggref(AggStatePerTrans pertrans, AggState *aggstate, EState *estate, @@ -1201,6 +1291,68 @@ project_aggregates(AggState *aggstate) return NULL; } +static bool +find_aggregated_cols_walker(Node *node, Bitmapset **colnos) +{ + if (node == NULL) + return false; + + if (IsA(node, Var)) + { + Var *var = (Var *) node; + + *colnos = bms_add_member(*colnos, var->varattno); + + return false; + } + return expression_tree_walker(node, find_aggregated_cols_walker, + (void *) colnos); +} + +/* + * find_aggregated_cols + * Construct a bitmapset of the column numbers of aggregated Vars + * appearing in our targetlist and qual (HAVING clause) + */ +static Bitmapset * +find_aggregated_cols(AggState *aggstate) +{ + Agg *node = (Agg *) aggstate->ss.ps.plan; + Bitmapset *colnos = NULL; + ListCell *temp; + + /* + * We only want the columns used by aggregations in the targetlist or qual + */ + if (node->plan.targetlist != NULL) + { + foreach(temp, (List *) node->plan.targetlist) + { + if (IsA(lfirst(temp), TargetEntry)) + { + Node *node = (Node *)((TargetEntry *)lfirst(temp))->expr; + if (IsA(node, Aggref) || IsA(node, GroupingFunc)) + find_aggregated_cols_walker(node, &colnos); + } + } + } + + if (node->plan.qual != NULL) + { + foreach(temp, (List *) node->plan.qual) + { + if (IsA(lfirst(temp), TargetEntry)) + { + Node *node = (Node *)((TargetEntry *)lfirst(temp))->expr; + if (IsA(node, Aggref) || IsA(node, GroupingFunc)) + find_aggregated_cols_walker(node, &colnos); + } + } + } + + return colnos; +} + /* * find_unaggregated_cols * Construct a bitmapset of the column numbers of un-aggregated Vars @@ -1254,46 +1406,80 @@ find_unaggregated_cols_walker(Node *node, Bitmapset **colnos) * for each entry. * * We have a separate hashtable and associated perhash data structure for each - * grouping set for which we're doing hashing. + * grouping set for which we're doing hashing. If setno is -1, build hash + * tables for all grouping sets. Otherwise, build only for the specified + * grouping set. * * The contents of 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) +build_hash_table(AggState *aggstate, int setno, long ngroups_estimate) { - MemoryContext tmpmem = aggstate->tmpcontext->ecxt_per_tuple_memory; - Size additionalsize; - int i; + Agg *agg = (Agg *)aggstate->ss.ps.plan; + MemoryContext tmpmem = aggstate->tmpcontext->ecxt_per_tuple_memory; + Size additionalsize; + int i; Assert(aggstate->aggstrategy == AGG_HASHED || aggstate->aggstrategy == AGG_MIXED); - additionalsize = aggstate->numtrans * sizeof(AggStatePerGroupData); + additionalsize = aggstate->numtrans * sizeof(AggStatePerGroupData) + + agg->transSpace; for (i = 0; i < aggstate->num_hashes; ++i) { AggStatePerHash perhash = &aggstate->perhash[i]; + int64 ngroups; + long nbuckets; + Size memory; Assert(perhash->aggnode->numGroups > 0); if (perhash->hashtable) - ResetTupleHashTable(perhash->hashtable); - else - perhash->hashtable = BuildTupleHashTableExt(&aggstate->ss.ps, - perhash->hashslot->tts_tupleDescriptor, - perhash->numCols, - perhash->hashGrpColIdxHash, - perhash->eqfuncoids, - perhash->hashfunctions, - perhash->aggnode->grpCollations, - perhash->aggnode->numGroups, - additionalsize, - aggstate->ss.ps.state->es_query_cxt, - aggstate->hashcontext->ecxt_per_tuple_memory, - tmpmem, - DO_AGGSPLIT_SKIPFINAL(aggstate->aggsplit)); + DestroyTupleHashTable(perhash->hashtable); + perhash->hashtable = NULL; + + /* + * If we are building a hash table for only a single grouping set, + * skip the others. + */ + if (setno >= 0 && setno != i) + continue; + + /* + * Use an estimate from execution time if we have it; otherwise fall + * back to the planner estimate. + */ + ngroups = ngroups_estimate > 0 ? + ngroups_estimate : perhash->aggnode->numGroups; + + /* divide memory by the number of hash tables we are initializing */ + memory = (long)work_mem * 1024L / + (setno >= 0 ? 1 : aggstate->num_hashes); + + /* choose reasonable number of buckets per hashtable */ + nbuckets = hash_choose_num_buckets(aggstate, ngroups, memory); + + perhash->hashtable = BuildTupleHashTableExt(&aggstate->ss.ps, + perhash->hashslot->tts_tupleDescriptor, + perhash->numCols, + perhash->hashGrpColIdxHash, + perhash->eqfuncoids, + perhash->hashfunctions, + perhash->aggnode->grpCollations, + nbuckets, + additionalsize, + aggstate->ss.ps.state->es_query_cxt, + aggstate->hashcontext->ecxt_per_tuple_memory, + tmpmem, + DO_AGGSPLIT_SKIPFINAL(aggstate->aggsplit)); } + + aggstate->hash_mem_current = MemoryContextMemAllocated( + aggstate->hashcontext->ecxt_per_tuple_memory, true); + aggstate->hash_ngroups_current = 0; + aggstate->hash_no_new_groups = false; } /* @@ -1325,6 +1511,7 @@ static void find_hash_columns(AggState *aggstate) { Bitmapset *base_colnos; + Bitmapset *aggregated_colnos; List *outerTlist = outerPlanState(aggstate)->plan->targetlist; int numHashes = aggstate->num_hashes; EState *estate = aggstate->ss.ps.state; @@ -1332,11 +1519,13 @@ find_hash_columns(AggState *aggstate) /* Find Vars that will be needed in tlist and qual */ base_colnos = find_unaggregated_cols(aggstate); + aggregated_colnos = find_aggregated_cols(aggstate); for (j = 0; j < numHashes; ++j) { AggStatePerHash perhash = &aggstate->perhash[j]; Bitmapset *colnos = bms_copy(base_colnos); + Bitmapset *allNeededColsInput; AttrNumber *grpColIdx = perhash->aggnode->grpColIdx; List *hashTlist = NIL; TupleDesc hashDesc; @@ -1383,6 +1572,19 @@ find_hash_columns(AggState *aggstate) for (i = 0; i < perhash->numCols; i++) colnos = bms_add_member(colnos, grpColIdx[i]); + /* + * Track the necessary columns from the input. This is important for + * spilling tuples so that we don't waste disk space with unneeded + * columns. + */ + allNeededColsInput = bms_union(colnos, aggregated_colnos); + perhash->numNeededColsInput = 0; + perhash->allNeededColsInput = palloc( + bms_num_members(allNeededColsInput) * sizeof(AttrNumber)); + + while ((i = bms_first_member(allNeededColsInput)) >= 0) + perhash->allNeededColsInput[perhash->numNeededColsInput++] = i; + /* * First build mapping for columns directly hashed. These are the * first, because they'll be accessed when computing hash values and @@ -1455,22 +1657,16 @@ hash_agg_entry_size(int numAggs) } /* - * 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. + * Extract the attributes that make up the grouping key into the + * hashslot. This is necessary to compute the hash of the grouping key. */ -static TupleHashEntryData * -lookup_hash_entry(AggState *aggstate) +static void +prepare_hash_slot(AggState *aggstate) { - TupleTableSlot *inputslot = aggstate->tmpcontext->ecxt_outertuple; - AggStatePerHash perhash = &aggstate->perhash[aggstate->current_set]; - TupleTableSlot *hashslot = perhash->hashslot; - TupleHashEntryData *entry; - bool isnew; - int i; + TupleTableSlot *inputslot = aggstate->tmpcontext->ecxt_outertuple; + AggStatePerHash perhash = &aggstate->perhash[aggstate->current_set]; + TupleTableSlot *hashslot = perhash->hashslot; + int i; /* transfer just the needed columns into hashslot */ slot_getsomeattrs(inputslot, perhash->largestGrpColIdx); @@ -1484,14 +1680,185 @@ lookup_hash_entry(AggState *aggstate) hashslot->tts_isnull[i] = inputslot->tts_isnull[varNumber]; } ExecStoreVirtualTuple(hashslot); +} + +/* + * Recompile the expressions for advancing aggregates while hashing. This is + * necessary for certain kinds of state changes that affect the resulting + * expression. For instance, changing aggstate->hash_spilled or + * aggstate->ss.ps.outerops require recompilation. + */ +static void +hash_recompile_expressions(AggState *aggstate) +{ + AggStatePerPhase phase; + + Assert(aggstate->aggstrategy == AGG_HASHED || + aggstate->aggstrategy == AGG_MIXED); + + if (aggstate->aggstrategy == AGG_HASHED) + phase = &aggstate->phases[0]; + else /* AGG_MIXED */ + phase = &aggstate->phases[1]; + + phase->evaltrans = ExecBuildAggTrans( + aggstate, phase, + aggstate->aggstrategy == AGG_MIXED ? true : false, /* dosort */ + true, /* dohash */ + aggstate->hash_spilled /* spilled */); +} + +/* + * Calculate the hash value for a tuple. It's useful to do this outside of the + * hash table so that we can reuse saved hash values rather than recomputing. + */ +static uint32 +calculate_hash(AggState *aggstate) +{ + AggStatePerHash perhash = &aggstate->perhash[aggstate->current_set]; + TupleHashTable hashtable = perhash->hashtable; + MemoryContext oldContext; + uint32 hash; + + /* set up data needed by hash and match functions */ + hashtable->inputslot = perhash->hashslot; + hashtable->in_hash_funcs = hashtable->tab_hash_funcs; + hashtable->cur_eq_func = hashtable->tab_eq_func; + + /* Need to run the hash functions in short-lived context */ + oldContext = MemoryContextSwitchTo(hashtable->tempcxt); + + hash = TupleHashTableHash(hashtable->hashtab, NULL); + + MemoryContextSwitchTo(oldContext); + + return hash; +} + +/* + * Choose a reasonable number of buckets for the initial hash table size. + */ +static long +hash_choose_num_buckets(AggState *aggstate, long ngroups, Size memory) +{ + long max_nbuckets; + int log2_ngroups; + long nbuckets; + + max_nbuckets = memory / aggstate->hashentrysize; + + /* + * Lowest power of two greater than ngroups, without exceeding + * max_nbuckets. + */ + for (log2_ngroups = 1, nbuckets = 2; + nbuckets < ngroups && nbuckets < max_nbuckets; + log2_ngroups++, nbuckets <<= 1); + + if (nbuckets > max_nbuckets && nbuckets > 2) + nbuckets >>= 1; + + return nbuckets; +} + +/* + * Determine the number of partitions to create when spilling. + */ +static int +hash_choose_num_spill_partitions(uint64 input_groups, double hashentrysize) +{ + Size mem_needed; + int partition_limit; + int npartitions; + + /* + * Avoid creating so many partitions that the memory requirements of the + * open partition files (estimated at BLCKSZ for buffering) are greater + * than 1/4 of work_mem. + */ + partition_limit = (work_mem * 1024L * 0.25) / BLCKSZ; + + /* pessimistically estimate that each input tuple creates a new group */ + mem_needed = HASH_PARTITION_FACTOR * input_groups * hashentrysize; + + /* make enough partitions so that each one is likely to fit in memory */ + npartitions = 1 + (mem_needed / (work_mem * 1024L)); + + if (npartitions > partition_limit) + npartitions = partition_limit; + + if (npartitions < HASH_MIN_PARTITIONS) + npartitions = HASH_MIN_PARTITIONS; + if (npartitions > HASH_MAX_PARTITIONS) + npartitions = HASH_MAX_PARTITIONS; + + return npartitions; +} + +/* + * 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. + * + * If the hash table is at the memory limit, then only find existing hashtable + * entries; don't create new ones. If a tuple's group is not already present + * in the hash table for the current grouping set, return NULL and the caller + * will spill it to disk. + */ +static AggStatePerGroup +lookup_hash_entry(AggState *aggstate, uint32 hash) +{ + AggStatePerHash perhash = &aggstate->perhash[aggstate->current_set]; + TupleTableSlot *hashslot = perhash->hashslot; + TupleHashEntryData *entry; + bool isnew = false; + bool *p_isnew; + + /* if hash table already spilled, don't create new entries */ + p_isnew = aggstate->hash_no_new_groups ? NULL : &isnew; /* find or create the hashtable entry using the filtered tuple */ - entry = LookupTupleHashEntry(perhash->hashtable, hashslot, &isnew); + entry = LookupTupleHashEntryHash(perhash->hashtable, hashslot, p_isnew, + hash); + + if (entry == NULL) + return NULL; if (isnew) { - AggStatePerGroup pergroup; - int transno; + AggStatePerGroup pergroup; + int transno; + + aggstate->hash_ngroups_current++; + + aggstate->hash_mem_current = MemoryContextMemAllocated( + aggstate->hashcontext->ecxt_per_tuple_memory, true); + + if (aggstate->hash_mem_current > aggstate->hash_mem_peak) + aggstate->hash_mem_peak = aggstate->hash_mem_current; + + /* + * Check whether we need to spill. For small values of work_mem, the + * empty hash tables might exceed it; so don't spill unless there's at + * least one group in the hash table. + */ + if (aggstate->hash_ngroups_current > 0 && + (aggstate->hash_mem_current > aggstate->hash_mem_limit || + aggstate->hash_ngroups_current > aggstate->hash_ngroups_limit)) + { + aggstate->hash_no_new_groups = true; + if (!aggstate->hash_spilled) + { + aggstate->hash_spilled = true; + aggstate->hash_spills = palloc0( + sizeof(HashAggSpill) * aggstate->num_hashes); + + hash_recompile_expressions(aggstate); + } + } pergroup = (AggStatePerGroup) MemoryContextAlloc(perhash->hashtable->tablecxt, @@ -1511,7 +1878,7 @@ lookup_hash_entry(AggState *aggstate) } } - return entry; + return entry->additional; } /* @@ -1519,18 +1886,74 @@ lookup_hash_entry(AggState *aggstate) * returning an array of pergroup pointers suitable for advance_aggregates. * * Be aware that lookup_hash_entry can reset the tmpcontext. + * + * Some entries may be left NULL if we have reached the limit and have begun + * to spill. The same tuple will belong to different groups for each set, so + * may match a group already in memory for one set and match a group not in + * memory for another set. If we have begun to spill and a tuple doesn't match + * a group in memory for a particular set, it will be spilled. + * + * NB: It's possible to spill the same tuple for several different grouping + * sets. This may seem wasteful, but it's actually a trade-off: if we spill + * the tuple multiple times for multiple grouping sets, it can be partitioned + * for each grouping set, making the refilling of the hash table very + * efficient. */ static void lookup_hash_entries(AggState *aggstate) { - int numHashes = aggstate->num_hashes; AggStatePerGroup *pergroup = aggstate->hash_pergroup; int setno; - for (setno = 0; setno < numHashes; setno++) + for (setno = 0; setno < aggstate->num_hashes; setno++) { + uint32 hash; + select_current_set(aggstate, setno, true); - pergroup[setno] = lookup_hash_entry(aggstate)->additional; + prepare_hash_slot(aggstate); + hash = calculate_hash(aggstate); + pergroup[setno] = lookup_hash_entry(aggstate, hash); + + /* check to see if we need to spill the tuple for this grouping set */ + if (pergroup[setno] == NULL) + { + AggStatePerHash perhash = &aggstate->perhash[setno]; + TupleTableSlot *inputslot = aggstate->tmpcontext->ecxt_outertuple; + TupleTableSlot *spillslot = aggstate->hash_spill_slot; + HashAggSpill *spill = &aggstate->hash_spills[setno]; + int idx; + + if (spill->partitions == NULL) + hash_spill_init(spill, 0, perhash->aggnode->numGroups, + aggstate->hashentrysize); + + /* + * Copy only necessary attributes to spill slot before writing to + * disk. + */ + ExecClearTuple(spillslot); + memset(spillslot->tts_isnull, true, + spillslot->tts_tupleDescriptor->natts); + + /* deserialize needed attributes */ + if (perhash->numNeededColsInput > 0) + { + int maxNeededAttrIdx = perhash->numNeededColsInput - 1; + AttrNumber maxNeededAttr = + perhash->allNeededColsInput[maxNeededAttrIdx]; + slot_getsomeattrs(inputslot, maxNeededAttr); + } + + for (idx = 0; idx < perhash->numNeededColsInput; idx++) + { + AttrNumber att = perhash->allNeededColsInput[idx]; + spillslot->tts_values[att-1] = inputslot->tts_values[att-1]; + spillslot->tts_isnull[att-1] = inputslot->tts_isnull[att-1]; + } + + ExecStoreVirtualTuple(spillslot); + aggstate->hash_disk_used += hash_spill_tuple(spill, 0, spillslot, hash); + } } } @@ -1853,6 +2276,12 @@ agg_retrieve_direct(AggState *aggstate) if (TupIsNull(outerslot)) { /* no more outer-plan tuples available */ + + /* if we built hash tables, finalize any spills */ + if (aggstate->aggstrategy == AGG_MIXED && + aggstate->current_phase == 1) + hash_finish_initial_spills(aggstate); + if (hasGroupingSets) { aggstate->input_done = true; @@ -1955,6 +2384,9 @@ agg_fill_hash_table(AggState *aggstate) ResetExprContext(aggstate->tmpcontext); } + /* finalize spills, if any */ + hash_finish_initial_spills(aggstate); + aggstate->table_filled = true; /* Initialize to walk the first hash table */ select_current_set(aggstate, 0, true); @@ -1962,11 +2394,175 @@ agg_fill_hash_table(AggState *aggstate) &aggstate->perhash[0].hashiter); } +/* + * If any data was spilled during hash aggregation, reset the hash table and + * reprocess one batch of spilled data. After reprocessing a batch, the hash + * table will again contain data, ready to be consumed by + * agg_retrieve_hash_table_in_memory(). + * + * Should only be called after all in memory hash table entries have been + * consumed. + * + * Return false when input is exhausted and there's no more work to be done; + * otherwise return true. + */ +static bool +agg_refill_hash_table(AggState *aggstate) +{ + HashAggBatch *batch; + + if (aggstate->hash_batches == NIL) + return false; + + /* + * Each spill file contains spilled data for only a single grouping + * set. We want to ignore all others, which is done by setting the other + * pergroups to NULL. + */ + memset(aggstate->all_pergroups, 0, + sizeof(AggStatePerGroup) * + (aggstate->maxsets + aggstate->num_hashes)); + + batch = linitial(aggstate->hash_batches); + aggstate->hash_batches = list_delete_first(aggstate->hash_batches); + + /* + * Free memory and rebuild a single hash table for this batch's grouping + * set. + */ + ReScanExprContext(aggstate->hashcontext); + + /* estimate the number of groups to be the number of input tuples */ + build_hash_table(aggstate, batch->setno, batch->input_tuples); + + Assert(aggstate->current_phase == 0); + + if (aggstate->phase->aggstrategy == AGG_MIXED) + { + aggstate->current_phase = 1; + aggstate->phase = &aggstate->phases[aggstate->current_phase]; + } + + /* + * The first pass (agg_fill_hash_table) reads whatever kind of slot comes + * from the outer plan, and considers the slot fixed. But spilled tuples + * are always MinimalTuples, so if that's different from the outer plan we + * need to change it and recompile the aggregate expressions. + */ + if (aggstate->ss.ps.outerops != &TTSOpsMinimalTuple) + { + aggstate->ss.ps.outerops = &TTSOpsMinimalTuple; + hash_recompile_expressions(aggstate); + } + + for (;;) { + TupleTableSlot *slot = aggstate->hash_spill_slot; + MinimalTuple tuple; + uint32 hash; + + CHECK_FOR_INTERRUPTS(); + + tuple = hash_read_spilled(batch->input_file, &hash); + if (tuple == NULL) + break; + + ExecStoreMinimalTuple(tuple, slot, true); + aggstate->tmpcontext->ecxt_outertuple = slot; + + select_current_set(aggstate, batch->setno, true); + prepare_hash_slot(aggstate); + aggstate->hash_pergroup[batch->setno] = lookup_hash_entry(aggstate, hash); + + /* if there's no memory for a new group, spill */ + if (aggstate->hash_pergroup[batch->setno] == NULL) + { + if (batch->spill.partitions == NULL) + { + /* + * Estimate the number of groups for this batch as the total + * number of tuples in its input file. Although that's a worst + * case, it's not bad here for two reasons: (1) overestimating + * is better than underestimating; and (2) we've already + * scanned the relation once, so it's likely that we've + * already finalized many of the common values. + */ + hash_spill_init(&batch->spill, batch->input_bits, + batch->input_tuples, aggstate->hashentrysize); + } + + aggstate->hash_disk_used += hash_spill_tuple( + &batch->spill, batch->input_bits, slot, hash); + } + + /* Advance the aggregates (or combine functions) */ + advance_aggregates(aggstate); + + /* + * Reset per-input-tuple context after each tuple, but note that the + * hash lookups do this too + */ + ResetExprContext(aggstate->tmpcontext); + } + + BufFileClose(batch->input_file); + + aggstate->current_phase = 0; + aggstate->phase = &aggstate->phases[aggstate->current_phase]; + + /* update hashentrysize estimate based on contents */ + if (aggstate->hash_ngroups_current > 0) + { + aggstate->hashentrysize = (double)aggstate->hash_mem_current / + (double)aggstate->hash_ngroups_current; + } + + hash_spill_finish(aggstate, &batch->spill, batch->setno, + batch->input_bits); + + pfree(batch); + + /* Initialize to walk the first hash table */ + select_current_set(aggstate, 0, true); + ResetTupleHashIterator(aggstate->perhash[0].hashtable, + &aggstate->perhash[0].hashiter); + + return true; +} + /* * ExecAgg for hashed case: retrieving groups from hash table + * + * After exhausting in-memory tuples, also try refilling the hash table using + * previously-spilled tuples. Only returns NULL after all in-memory and + * spilled tuples are exhausted. */ static TupleTableSlot * agg_retrieve_hash_table(AggState *aggstate) +{ + TupleTableSlot *result = NULL; + + while (result == NULL) + { + result = agg_retrieve_hash_table_in_memory(aggstate); + if (result == NULL) + { + if (!agg_refill_hash_table(aggstate)) + { + aggstate->agg_done = true; + break; + } + } + } + + return result; +} + +/* + * Retrieve the groups from the in-memory hash tables without considering any + * spilled tuples. + */ +static TupleTableSlot * +agg_retrieve_hash_table_in_memory(AggState *aggstate) { ExprContext *econtext; AggStatePerAgg peragg; @@ -1995,7 +2591,7 @@ agg_retrieve_hash_table(AggState *aggstate) * We loop retrieving groups until we find one satisfying * aggstate->ss.ps.qual */ - while (!aggstate->agg_done) + for (;;) { TupleTableSlot *hashslot = perhash->hashslot; int i; @@ -2026,8 +2622,6 @@ agg_retrieve_hash_table(AggState *aggstate) } else { - /* No more hashtables, so done */ - aggstate->agg_done = true; return NULL; } } @@ -2084,6 +2678,281 @@ agg_retrieve_hash_table(AggState *aggstate) return NULL; } +/* + * hash_spill_init + * + * Called after we determined that spilling is necessary. Chooses the number + * of partitions to create, and initializes them. + */ +static void +hash_spill_init(HashAggSpill *spill, int input_bits, uint64 input_groups, + double hashentrysize) +{ + int npartitions; + int partition_bits; + + npartitions = hash_choose_num_spill_partitions(input_groups, + hashentrysize); + partition_bits = my_log2(npartitions); + + /* make sure that we don't exhaust the hash bits */ + if (partition_bits + input_bits >= 32) + partition_bits = 32 - input_bits; + + /* number of partitions will be a power of two */ + npartitions = 1L << partition_bits; + + spill->partition_bits = partition_bits; + spill->n_partitions = npartitions; + spill->partitions = palloc0(sizeof(BufFile *) * npartitions); + spill->ntuples = palloc0(sizeof(int64) * npartitions); +} + +/* + * hash_spill_tuple + * + * No room for new groups in the hash table. Save for later in the appropriate + * partition spill file. + */ +static Size +hash_spill_tuple(HashAggSpill *spill, int input_bits, TupleTableSlot *slot, + uint32 hash) +{ + int partition; + MinimalTuple tuple; + BufFile *file; + int written; + int total_written = 0; + bool shouldFree; + + Assert(spill->partitions != NULL); + + /* + * When spilling tuples from the input, the slot will be virtual + * (containing only the needed attributes and the rest as NULL), and we + * need to materialize the minimal tuple. When spilling tuples + * recursively, the slot will hold a minimal tuple already. + */ + tuple = ExecFetchSlotMinimalTuple(slot, &shouldFree); + + if (spill->partition_bits == 0) + partition = 0; + else + partition = (hash << input_bits) >> + (32 - spill->partition_bits); + + spill->ntuples[partition]++; + + if (spill->partitions[partition] == NULL) + spill->partitions[partition] = BufFileCreateTemp(false); + file = spill->partitions[partition]; + + written = BufFileWrite(file, (void *) &hash, sizeof(uint32)); + if (written != sizeof(uint32)) + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not write to HashAgg temporary file: %m"))); + total_written += written; + + written = BufFileWrite(file, (void *) tuple, tuple->t_len); + if (written != tuple->t_len) + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not write to HashAgg temporary file: %m"))); + total_written += written; + + if (shouldFree) + pfree(tuple); + + return total_written; +} + +/* + * read_spilled_tuple + * read the next tuple from a batch file. Return NULL if no more. + */ +static MinimalTuple +hash_read_spilled(BufFile *file, uint32 *hashp) +{ + MinimalTuple tuple; + uint32 t_len; + size_t nread; + uint32 hash; + + nread = BufFileRead(file, &hash, sizeof(uint32)); + if (nread == 0) + return NULL; + if (nread != sizeof(uint32)) + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from HashAgg temporary file: %m"))); + if (hashp != NULL) + *hashp = hash; + + nread = BufFileRead(file, &t_len, sizeof(t_len)); + if (nread != sizeof(uint32)) + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from HashAgg temporary file: %m"))); + + tuple = (MinimalTuple) palloc(t_len); + tuple->t_len = t_len; + + nread = BufFileRead(file, (void *)((char *)tuple + sizeof(uint32)), + t_len - sizeof(uint32)); + if (nread != t_len - sizeof(uint32)) + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not read from HashAgg temporary file: %m"))); + + return tuple; +} + +/* + * new_hashagg_batch + * + * Construct a HashAggBatch item, which represents one iteration of HashAgg to + * be done. Should be called in the aggregate's memory context. + */ +static HashAggBatch * +hash_batch_new(BufFile *input_file, int setno, int64 input_tuples, + int input_bits) +{ + HashAggBatch *batch = palloc0(sizeof(HashAggBatch)); + + batch->input_file = input_file; + batch->input_bits = input_bits; + batch->input_tuples = input_tuples; + batch->setno = setno; + + /* batch->spill will be set only after spilling this batch */ + + return batch; +} + +/* + * hash_finish_initial_spills + * + * After a HashAggBatch has been processed, it may have spilled tuples to + * disk. If so, turn the spilled partitions into new batches that must later + * be executed. + */ +static void +hash_finish_initial_spills(AggState *aggstate) +{ + int setno; + + if (aggstate->hash_spills == NULL) + return; + + /* update hashentrysize estimate based on contents */ + Assert(aggstate->hash_ngroups_current > 0); + aggstate->hashentrysize = (double)aggstate->hash_mem_current / + (double)aggstate->hash_ngroups_current; + + for (setno = 0; setno < aggstate->num_hashes; setno++) + hash_spill_finish(aggstate, &aggstate->hash_spills[setno], setno, 0); + + pfree(aggstate->hash_spills); + aggstate->hash_spills = NULL; +} + +/* + * hash_spill_finish + * + * Transform spill files into new batches. + */ +static void +hash_spill_finish(AggState *aggstate, HashAggSpill *spill, int setno, int input_bits) +{ + int i; + + if (spill->n_partitions == 0) + return; /* didn't spill */ + + for (i = 0; i < spill->n_partitions; i++) + { + BufFile *file = spill->partitions[i]; + MemoryContext oldContext; + HashAggBatch *new_batch; + + /* partition is empty */ + if (file == NULL) + continue; + + /* rewind file for reading */ + if (BufFileSeek(file, 0, 0L, SEEK_SET)) + ereport(ERROR, + (errcode_for_file_access(), + errmsg("could not rewind HashAgg temporary file: %m"))); + + oldContext = MemoryContextSwitchTo(aggstate->ss.ps.state->es_query_cxt); + new_batch = hash_batch_new(file, setno, spill->ntuples[i], + spill->partition_bits + input_bits); + aggstate->hash_batches = lappend(aggstate->hash_batches, new_batch); + aggstate->hash_batches_used++; + MemoryContextSwitchTo(oldContext); + } + + pfree(spill->ntuples); + pfree(spill->partitions); +} + +/* + * Clear a HashAggSpill, free its memory, and close its files. + */ +static void +hash_reset_spill(HashAggSpill *spill) +{ + int i; + for (i = 0; i < spill->n_partitions; i++) + { + BufFile *file = spill->partitions[i]; + + if (file != NULL) + BufFileClose(file); + } + if (spill->ntuples != NULL) + pfree(spill->ntuples); + if (spill->partitions != NULL) + pfree(spill->partitions); +} + +/* + * Find and reset all active HashAggSpills. + */ +static void +hash_reset_spills(AggState *aggstate) +{ + ListCell *lc; + + if (aggstate->hash_spills != NULL) + { + int setno; + + for (setno = 0; setno < aggstate->num_hashes; setno++) + hash_reset_spill(&aggstate->hash_spills[setno]); + + pfree(aggstate->hash_spills); + aggstate->hash_spills = NULL; + } + + foreach(lc, aggstate->hash_batches) + { + HashAggBatch *batch = (HashAggBatch*) lfirst(lc); + if (batch->input_file != NULL) + { + BufFileClose(batch->input_file); + batch->input_file = NULL; + } + hash_reset_spill(&batch->spill); + pfree(batch); + } + list_free(aggstate->hash_batches); + aggstate->hash_batches = NIL; +} + + /* ----------------- * ExecInitAgg * @@ -2268,6 +3137,10 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) aggstate->ss.ps.outeropsfixed = false; } + if (use_hashing) + aggstate->hash_spill_slot = ExecInitExtraTupleSlot(estate, scanDesc, + &TTSOpsMinimalTuple); + /* * Initialize result type, slot and projection. */ @@ -2496,8 +3369,36 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) /* this is an array of pointers, not structures */ aggstate->hash_pergroup = pergroups; + aggstate->hashentrysize = + hash_agg_entry_size(aggstate->numtrans) + + node->transSpace; + + /* + * Initialize the thresholds at which we stop creating new hash entries + * and start spilling. + */ + if (hashagg_mem_overflow) + aggstate->hash_mem_limit = SIZE_MAX; + else if (work_mem * 1024L > HASH_PARTITION_MEM * 2) + aggstate->hash_mem_limit = work_mem * 1024L - HASH_PARTITION_MEM; + else + aggstate->hash_mem_limit = work_mem * 1024L; + + /* + * Set a separate limit on the maximum number of groups to + * create. This is important for aggregates where the initial state + * size is small, but aggtransspace is large. + */ + if (hashagg_mem_overflow) + aggstate->hash_ngroups_limit = LONG_MAX; + else if (aggstate->hash_mem_limit > aggstate->hashentrysize) + aggstate->hash_ngroups_limit = + aggstate->hash_mem_limit / aggstate->hashentrysize; + else + aggstate->hash_ngroups_limit = 1; + find_hash_columns(aggstate); - build_hash_table(aggstate); + build_hash_table(aggstate, -1, 0); aggstate->table_filled = false; } @@ -2903,7 +3804,7 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) else Assert(false); - phase->evaltrans = ExecBuildAggTrans(aggstate, phase, dosort, dohash); + phase->evaltrans = ExecBuildAggTrans(aggstate, phase, dosort, dohash, false); } @@ -3398,6 +4299,8 @@ ExecEndAgg(AggState *node) if (node->sort_out) tuplesort_end(node->sort_out); + hash_reset_spills(node); + for (transno = 0; transno < node->numtrans; transno++) { AggStatePerTrans pertrans = &node->pertrans[transno]; @@ -3453,12 +4356,13 @@ ExecReScanAgg(AggState *node) return; /* - * If we do have the hash table, and the subplan does not have any - * parameter changes, and none of our own parameter changes affect - * input expressions of the aggregated functions, then we can just - * rescan the existing hash table; no need to build it again. + * If we do have the hash table, and it never spilled, and the subplan + * does not have any parameter changes, and none of our own parameter + * changes affect input expressions of the aggregated functions, then + * we can just rescan the existing hash table; no need to build it + * again. */ - if (outerPlan->chgParam == NULL && + if (outerPlan->chgParam == NULL && !node->hash_spilled && !bms_overlap(node->ss.ps.chgParam, aggnode->aggParams)) { ResetTupleHashIterator(node->perhash[0].hashtable, @@ -3515,9 +4419,21 @@ ExecReScanAgg(AggState *node) */ if (node->aggstrategy == AGG_HASHED || node->aggstrategy == AGG_MIXED) { + hash_reset_spills(node); + + node->hash_spilled = false; + node->hash_no_new_groups = false; + node->hash_mem_current = 0; + node->hash_ngroups_current = 0; + + /* reset stats */ + node->hash_mem_peak = 0; + node->hash_disk_used = 0; + node->hash_batches_used = 0; + ReScanExprContext(node->hashcontext); /* Rebuild an empty hash table */ - build_hash_table(node); + build_hash_table(node, -1, 0); node->table_filled = false; /* iterator will be reset when the table is filled */ } diff --git a/src/backend/jit/llvm/llvmjit_expr.c b/src/backend/jit/llvm/llvmjit_expr.c index ffd887c71aa..93517d03819 100644 --- a/src/backend/jit/llvm/llvmjit_expr.c +++ b/src/backend/jit/llvm/llvmjit_expr.c @@ -2082,6 +2082,7 @@ llvm_compile_expr(ExprState *state) } case EEOP_AGG_INIT_TRANS: + case EEOP_AGG_INIT_TRANS_SPILLED: { AggState *aggstate; AggStatePerTrans pertrans; @@ -2092,6 +2093,7 @@ llvm_compile_expr(ExprState *state) LLVMValueRef v_allpergroupsp; LLVMValueRef v_pergroupp; + LLVMValueRef v_pergroup_allaggs; LLVMValueRef v_setoff, v_transno; @@ -2119,11 +2121,32 @@ llvm_compile_expr(ExprState *state) "aggstate.all_pergroups"); v_setoff = l_int32_const(op->d.agg_init_trans.setoff); v_transno = l_int32_const(op->d.agg_init_trans.transno); - v_pergroupp = - LLVMBuildGEP(b, - l_load_gep1(b, v_allpergroupsp, v_setoff, ""), - &v_transno, 1, ""); + v_pergroup_allaggs = l_load_gep1(b, v_allpergroupsp, v_setoff, ""); + /* + * When no tuples at all have spilled, we avoid adding this + * extra branch. But after some tuples have spilled, this + * branch is necessary, so we recompile the expression + * using a new opcode. + */ + if (opcode == EEOP_AGG_INIT_TRANS_SPILLED) + { + LLVMBasicBlockRef b_check_notransvalue = l_bb_before_v( + opblocks[i + 1], "op.%d.check_notransvalue", i); + + LLVMBuildCondBr( + b, + LLVMBuildICmp(b, LLVMIntEQ, + LLVMBuildPtrToInt( + b, v_pergroup_allaggs, TypeSizeT, ""), + l_sizet_const(0), ""), + opblocks[i + 1], + b_check_notransvalue); + + LLVMPositionBuilderAtEnd(b, b_check_notransvalue); + } + + v_pergroupp = LLVMBuildGEP(b, v_pergroup_allaggs, &v_transno, 1, ""); v_notransvalue = l_load_struct_gep(b, v_pergroupp, FIELDNO_AGGSTATEPERGROUPDATA_NOTRANSVALUE, @@ -2180,6 +2203,7 @@ llvm_compile_expr(ExprState *state) } case EEOP_AGG_STRICT_TRANS_CHECK: + case EEOP_AGG_STRICT_TRANS_CHECK_SPILLED: { AggState *aggstate; LLVMValueRef v_setoff, @@ -2190,6 +2214,7 @@ llvm_compile_expr(ExprState *state) LLVMValueRef v_transnull; LLVMValueRef v_pergroupp; + LLVMValueRef v_pergroup_allaggs; int jumpnull = op->d.agg_strict_trans_check.jumpnull; @@ -2209,11 +2234,32 @@ llvm_compile_expr(ExprState *state) l_int32_const(op->d.agg_strict_trans_check.setoff); v_transno = l_int32_const(op->d.agg_strict_trans_check.transno); - v_pergroupp = - LLVMBuildGEP(b, - l_load_gep1(b, v_allpergroupsp, v_setoff, ""), - &v_transno, 1, ""); + v_pergroup_allaggs = l_load_gep1(b, v_allpergroupsp, v_setoff, ""); + + /* + * When no tuples at all have spilled, we avoid adding this + * extra branch. But after some tuples have spilled, this + * branch is necessary, so we recompile the expression + * using a new opcode. + */ + if (opcode == EEOP_AGG_STRICT_TRANS_CHECK_SPILLED) + { + LLVMBasicBlockRef b_check_transnull = l_bb_before_v( + opblocks[i + 1], "op.%d.check_transnull", i); + + LLVMBuildCondBr( + b, + LLVMBuildICmp(b, LLVMIntEQ, + LLVMBuildPtrToInt(b, v_pergroup_allaggs, + TypeSizeT, ""), + l_sizet_const(0), ""), + opblocks[jumpnull], + b_check_transnull); + + LLVMPositionBuilderAtEnd(b, b_check_transnull); + } + v_pergroupp = LLVMBuildGEP(b, v_pergroup_allaggs, &v_transno, 1, ""); v_transnull = l_load_struct_gep(b, v_pergroupp, FIELDNO_AGGSTATEPERGROUPDATA_TRANSVALUEISNULL, @@ -2229,7 +2275,9 @@ llvm_compile_expr(ExprState *state) } case EEOP_AGG_PLAIN_TRANS_BYVAL: + case EEOP_AGG_PLAIN_TRANS_BYVAL_SPILLED: case EEOP_AGG_PLAIN_TRANS: + case EEOP_AGG_PLAIN_TRANS_SPILLED: { AggState *aggstate; AggStatePerTrans pertrans; @@ -2255,6 +2303,7 @@ llvm_compile_expr(ExprState *state) LLVMValueRef v_pertransp; LLVMValueRef v_pergroupp; + LLVMValueRef v_pergroup_allaggs; LLVMValueRef v_retval; @@ -2282,10 +2331,33 @@ llvm_compile_expr(ExprState *state) "aggstate.all_pergroups"); v_setoff = l_int32_const(op->d.agg_trans.setoff); v_transno = l_int32_const(op->d.agg_trans.transno); - v_pergroupp = - LLVMBuildGEP(b, - l_load_gep1(b, v_allpergroupsp, v_setoff, ""), - &v_transno, 1, ""); + v_pergroup_allaggs = l_load_gep1(b, v_allpergroupsp, v_setoff, ""); + + /* + * When no tuples at all have spilled, we avoid adding this + * extra branch. But after some tuples have spilled, this + * branch is necessary, so we recompile the expression + * using a new opcode. + */ + if (opcode == EEOP_AGG_PLAIN_TRANS_BYVAL_SPILLED || + opcode == EEOP_AGG_PLAIN_TRANS_SPILLED) + { + LLVMBasicBlockRef b_advance_transval = l_bb_before_v( + opblocks[i + 1], "op.%d.advance_transval", i); + + LLVMBuildCondBr( + b, + LLVMBuildICmp(b, LLVMIntEQ, + LLVMBuildPtrToInt(b, v_pergroup_allaggs, + TypeSizeT, ""), + l_sizet_const(0), ""), + opblocks[i + 1], + b_advance_transval); + + LLVMPositionBuilderAtEnd(b, b_advance_transval); + } + + v_pergroupp = LLVMBuildGEP(b, v_pergroup_allaggs, &v_transno, 1, ""); v_fcinfo = l_ptr_const(fcinfo, l_ptr(StructFunctionCallInfoData)); diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c index c5f65934859..3f0d2899635 100644 --- a/src/backend/optimizer/path/costsize.c +++ b/src/backend/optimizer/path/costsize.c @@ -128,6 +128,7 @@ bool enable_bitmapscan = true; bool enable_tidscan = true; bool enable_sort = true; bool enable_hashagg = true; +bool enable_hashagg_spill = true; bool enable_nestloop = true; bool enable_material = true; bool enable_mergejoin = true; diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c index 8c8b4f8ed69..f93150d4199 100644 --- a/src/backend/optimizer/plan/createplan.c +++ b/src/backend/optimizer/plan/createplan.c @@ -1644,6 +1644,7 @@ create_unique_plan(PlannerInfo *root, UniquePath *best_path, int flags) NIL, NIL, best_path->path.rows, + 0, subplan); } else @@ -2096,6 +2097,7 @@ create_agg_plan(PlannerInfo *root, AggPath *best_path) NIL, NIL, best_path->numGroups, + best_path->transSpace, subplan); copy_generic_path_info(&plan->plan, (Path *) best_path); @@ -2257,6 +2259,7 @@ create_groupingsets_plan(PlannerInfo *root, GroupingSetsPath *best_path) rollup->gsets, NIL, rollup->numGroups, + best_path->transSpace, sort_plan); /* @@ -2295,6 +2298,7 @@ create_groupingsets_plan(PlannerInfo *root, GroupingSetsPath *best_path) rollup->gsets, chain, rollup->numGroups, + best_path->transSpace, subplan); /* Copy cost data from Path to Plan */ @@ -6195,7 +6199,7 @@ make_agg(List *tlist, List *qual, AggStrategy aggstrategy, AggSplit aggsplit, int numGroupCols, AttrNumber *grpColIdx, Oid *grpOperators, Oid *grpCollations, List *groupingSets, List *chain, - double dNumGroups, Plan *lefttree) + double dNumGroups, int32 transSpace, Plan *lefttree) { Agg *node = makeNode(Agg); Plan *plan = &node->plan; @@ -6211,6 +6215,7 @@ make_agg(List *tlist, List *qual, node->grpOperators = grpOperators; node->grpCollations = grpCollations; node->numGroups = numGroups; + node->transSpace = transSpace; node->aggParams = NULL; /* SS_finalize_plan() will fill this */ node->groupingSets = groupingSets; node->chain = chain; diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c index cb54b15507b..b6172fb426a 100644 --- a/src/backend/optimizer/plan/planner.c +++ b/src/backend/optimizer/plan/planner.c @@ -4261,6 +4261,9 @@ consider_groupingsets_paths(PlannerInfo *root, * 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. + * + * TODO: think more about how to plan grouping sets when spilling hash + * tables is an option */ if (hashsize > work_mem * 1024L && gd->rollups) return; /* nope, won't fit */ @@ -6533,7 +6536,8 @@ add_paths_to_grouping_rel(PlannerInfo *root, RelOptInfo *input_rel, * were unable to sort above, then we'd better generate a Path, so * that we at least have one. */ - if (hashaggtablesize < work_mem * 1024L || + if (enable_hashagg_spill || + hashaggtablesize < work_mem * 1024L || grouped_rel->pathlist == NIL) { /* @@ -6566,7 +6570,8 @@ add_paths_to_grouping_rel(PlannerInfo *root, RelOptInfo *input_rel, agg_final_costs, dNumGroups); - if (hashaggtablesize < work_mem * 1024L) + if (enable_hashagg_spill || + hashaggtablesize < work_mem * 1024L) add_path(grouped_rel, (Path *) create_agg_path(root, grouped_rel, @@ -6835,7 +6840,7 @@ create_partial_grouping_paths(PlannerInfo *root, * Tentatively produce a partial HashAgg Path, depending on if it * looks as if the hash table will fit in work_mem. */ - if (hashaggtablesize < work_mem * 1024L && + if ((enable_hashagg_spill || hashaggtablesize < work_mem * 1024L) && cheapest_total_path != NULL) { add_path(partially_grouped_rel, (Path *) @@ -6862,7 +6867,7 @@ create_partial_grouping_paths(PlannerInfo *root, dNumPartialPartialGroups); /* Do the same for partial paths. */ - if (hashaggtablesize < work_mem * 1024L && + if ((enable_hashagg_spill || hashaggtablesize < work_mem * 1024L) && cheapest_partial_path != NULL) { add_partial_path(partially_grouped_rel, (Path *) diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c index 60c93ee7c59..7f5fc6ebb50 100644 --- a/src/backend/optimizer/util/pathnode.c +++ b/src/backend/optimizer/util/pathnode.c @@ -2949,6 +2949,7 @@ create_agg_path(PlannerInfo *root, pathnode->aggstrategy = aggstrategy; pathnode->aggsplit = aggsplit; pathnode->numGroups = numGroups; + pathnode->transSpace = aggcosts ? aggcosts->transitionSpace : 0; pathnode->groupClause = groupClause; pathnode->qual = qual; @@ -3036,6 +3037,7 @@ create_groupingsets_path(PlannerInfo *root, pathnode->aggstrategy = aggstrategy; pathnode->rollups = rollups; pathnode->qual = having_qual; + pathnode->transSpace = agg_costs ? agg_costs->transitionSpace : 0; Assert(rollups != NIL); Assert(aggstrategy != AGG_PLAIN || list_length(rollups) == 1); diff --git a/src/backend/utils/init/globals.c b/src/backend/utils/init/globals.c index 3a091022e24..752f09e3a35 100644 --- a/src/backend/utils/init/globals.c +++ b/src/backend/utils/init/globals.c @@ -120,6 +120,7 @@ bool enableFsync = true; bool allowSystemTableMods = false; int work_mem = 1024; int maintenance_work_mem = 16384; +bool hashagg_mem_overflow = false; int max_parallel_maintenance_workers = 2; /* diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c index 8d951ce404c..467f42944d7 100644 --- a/src/backend/utils/misc/guc.c +++ b/src/backend/utils/misc/guc.c @@ -958,6 +958,26 @@ static struct config_bool ConfigureNamesBool[] = true, NULL, NULL, NULL }, + { + {"enable_hashagg_spill", PGC_USERSET, QUERY_TUNING_METHOD, + gettext_noop("Enables the planner's use of hashed aggregation plans that are expected to exceed work_mem."), + NULL, + GUC_EXPLAIN + }, + &enable_hashagg_spill, + true, + NULL, NULL, NULL + }, + { + {"hashagg_mem_overflow", PGC_USERSET, QUERY_TUNING_METHOD, + gettext_noop("Enables hashed aggregation to overflow work_mem at execution time."), + NULL, + GUC_EXPLAIN + }, + &hashagg_mem_overflow, + false, + NULL, NULL, NULL + }, { {"enable_material", PGC_USERSET, QUERY_TUNING_METHOD, gettext_noop("Enables the planner's use of materialization."), diff --git a/src/include/executor/execExpr.h b/src/include/executor/execExpr.h index d21dbead0a2..e50a7ad6712 100644 --- a/src/include/executor/execExpr.h +++ b/src/include/executor/execExpr.h @@ -226,9 +226,13 @@ typedef enum ExprEvalOp EEOP_AGG_STRICT_INPUT_CHECK_ARGS, EEOP_AGG_STRICT_INPUT_CHECK_NULLS, EEOP_AGG_INIT_TRANS, + EEOP_AGG_INIT_TRANS_SPILLED, EEOP_AGG_STRICT_TRANS_CHECK, + EEOP_AGG_STRICT_TRANS_CHECK_SPILLED, EEOP_AGG_PLAIN_TRANS_BYVAL, + EEOP_AGG_PLAIN_TRANS_BYVAL_SPILLED, EEOP_AGG_PLAIN_TRANS, + EEOP_AGG_PLAIN_TRANS_SPILLED, EEOP_AGG_ORDERED_TRANS_DATUM, EEOP_AGG_ORDERED_TRANS_TUPLE, diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h index 6298c7c8cad..e8d88f2ce26 100644 --- a/src/include/executor/executor.h +++ b/src/include/executor/executor.h @@ -140,11 +140,17 @@ extern TupleHashTable BuildTupleHashTableExt(PlanState *parent, extern TupleHashEntry LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, bool *isnew); +extern TupleHashEntry LookupTupleHashEntryHash(TupleHashTable hashtable, + TupleTableSlot *slot, + bool *isnew, uint32 hash); extern TupleHashEntry FindTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot, ExprState *eqcomp, FmgrInfo *hashfunctions); +extern uint32 TupleHashTableHash(struct tuplehash_hash *tb, + const MinimalTuple tuple); extern void ResetTupleHashTable(TupleHashTable hashtable); +extern void DestroyTupleHashTable(TupleHashTable hashtable); /* * prototypes from functions in execJunk.c @@ -250,7 +256,7 @@ extern ExprState *ExecInitQual(List *qual, PlanState *parent); extern ExprState *ExecInitCheck(List *qual, PlanState *parent); extern List *ExecInitExprList(List *nodes, PlanState *parent); extern ExprState *ExecBuildAggTrans(AggState *aggstate, struct AggStatePerPhaseData *phase, - bool doSort, bool doHash); + bool doSort, bool doHash, bool spilled); extern ExprState *ExecBuildGroupingEqual(TupleDesc ldesc, TupleDesc rdesc, const TupleTableSlotOps *lops, const TupleTableSlotOps *rops, int numCols, diff --git a/src/include/executor/nodeAgg.h b/src/include/executor/nodeAgg.h index 68c9e5f5400..e58180e937a 100644 --- a/src/include/executor/nodeAgg.h +++ b/src/include/executor/nodeAgg.h @@ -302,6 +302,8 @@ typedef struct AggStatePerHashData AttrNumber *hashGrpColIdxInput; /* hash col indices in input slot */ AttrNumber *hashGrpColIdxHash; /* indices in hash table tuples */ Agg *aggnode; /* original Agg node, for numGroups etc. */ + int numNeededColsInput; /* number of columns needed from input */ + AttrNumber *allNeededColsInput; /* all columns needed from input */ } AggStatePerHashData; diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h index ed80f1d6681..77a87cded44 100644 --- a/src/include/miscadmin.h +++ b/src/include/miscadmin.h @@ -244,6 +244,7 @@ extern bool enableFsync; extern PGDLLIMPORT bool allowSystemTableMods; extern PGDLLIMPORT int work_mem; extern PGDLLIMPORT int maintenance_work_mem; +extern PGDLLIMPORT bool hashagg_mem_overflow; extern PGDLLIMPORT int max_parallel_maintenance_workers; extern int VacuumCostPageHit; diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h index 0c2a77aaf8d..8d4a36a3538 100644 --- a/src/include/nodes/execnodes.h +++ b/src/include/nodes/execnodes.h @@ -2066,13 +2066,30 @@ typedef struct AggState HeapTuple grp_firstTuple; /* copy of first tuple of current group */ /* these fields are used in AGG_HASHED and AGG_MIXED modes: */ bool table_filled; /* hash table filled yet? */ - int num_hashes; + int num_hashes; /* number of hash tables active at once */ + bool hash_spilled; /* any hash table ever spilled? */ + double hashentrysize; /* estimate revised during execution */ + struct HashAggSpill *hash_spills; /* HashAggSpill for each hash table, + exists only during first pass if spilled */ + TupleTableSlot *hash_spill_slot; /* slot for reading from spill files */ + bool hash_no_new_groups; /* we hit a limit during the current batch + and we must not create new groups */ + Size hash_mem_current; /* current hash table memory usage */ + Size hash_mem_limit; /* limit before spilling hash table */ + Size hash_mem_peak; /* peak hash table memory usage */ + long hash_ngroups_current; /* number of groups currently in + memory in all hash tables */ + long hash_ngroups_limit; /* limit before spilling hash table */ + uint64 hash_disk_used; /* bytes of disk space used */ + int hash_batches_used; /* batches used during entire execution */ + List *hash_batches; /* hash batches remaining to be processed */ + AggStatePerHash perhash; /* array of per-hashtable data */ AggStatePerGroup *hash_pergroup; /* grouping set indexed array of * per-group pointers */ /* support for evaluation of agg input expressions: */ -#define FIELDNO_AGGSTATE_ALL_PERGROUPS 34 +#define FIELDNO_AGGSTATE_ALL_PERGROUPS 47 AggStatePerGroup *all_pergroups; /* array of first ->pergroups, than * ->hash_pergroup */ ProjectionInfo *combinedproj; /* projection machinery */ diff --git a/src/include/nodes/pathnodes.h b/src/include/nodes/pathnodes.h index 31b631cfe0f..f8557404703 100644 --- a/src/include/nodes/pathnodes.h +++ b/src/include/nodes/pathnodes.h @@ -1663,6 +1663,7 @@ typedef struct AggPath AggStrategy aggstrategy; /* basic strategy, see nodes.h */ AggSplit aggsplit; /* agg-splitting mode, see nodes.h */ double numGroups; /* estimated number of groups in input */ + int32 transSpace; /* estimated transition state size */ List *groupClause; /* a list of SortGroupClause's */ List *qual; /* quals (HAVING quals), if any */ } AggPath; @@ -1700,6 +1701,7 @@ typedef struct GroupingSetsPath AggStrategy aggstrategy; /* basic strategy */ List *rollups; /* list of RollupData */ List *qual; /* quals (HAVING quals), if any */ + int32 transSpace; /* estimated transition state size */ } GroupingSetsPath; /* diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h index 477b4da192c..360a4801f59 100644 --- a/src/include/nodes/plannodes.h +++ b/src/include/nodes/plannodes.h @@ -813,6 +813,7 @@ typedef struct Agg Oid *grpOperators; /* equality operators to compare with */ Oid *grpCollations; long numGroups; /* estimated number of groups in input */ + int32 transSpace; /* estimated transition state size */ Bitmapset *aggParams; /* IDs of Params used in Aggref inputs */ /* Note: planner provides numGroups & aggParams only in HASHED/MIXED case */ List *groupingSets; /* grouping sets to use */ diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h index b3d0b4f6fbc..b72e2d08290 100644 --- a/src/include/optimizer/cost.h +++ b/src/include/optimizer/cost.h @@ -54,6 +54,7 @@ extern PGDLLIMPORT bool enable_bitmapscan; extern PGDLLIMPORT bool enable_tidscan; extern PGDLLIMPORT bool enable_sort; extern PGDLLIMPORT bool enable_hashagg; +extern PGDLLIMPORT bool enable_hashagg_spill; extern PGDLLIMPORT bool enable_nestloop; extern PGDLLIMPORT bool enable_material; extern PGDLLIMPORT bool enable_mergejoin; diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h index e7aaddd50d6..41e4b4a336d 100644 --- a/src/include/optimizer/planmain.h +++ b/src/include/optimizer/planmain.h @@ -55,7 +55,7 @@ extern Agg *make_agg(List *tlist, List *qual, AggStrategy aggstrategy, AggSplit aggsplit, int numGroupCols, AttrNumber *grpColIdx, Oid *grpOperators, Oid *grpCollations, List *groupingSets, List *chain, - double dNumGroups, Plan *lefttree); + double dNumGroups, int32 transSpace, Plan *lefttree); extern Limit *make_limit(Plan *lefttree, Node *limitOffset, Node *limitCount); /* diff --git a/src/test/regress/expected/aggregates.out b/src/test/regress/expected/aggregates.out index d091ae4c6e4..92e5dbad77e 100644 --- a/src/test/regress/expected/aggregates.out +++ b/src/test/regress/expected/aggregates.out @@ -2331,3 +2331,95 @@ explain (costs off) -> Seq Scan on onek (8 rows) +-- +-- Compare results between plans using sorting and plans using hash +-- aggregation. Force spilling in both cases by setting work_mem low. +-- +set work_mem='64kB'; +-- Produce results with sorting. +set enable_hashagg = false; +set jit_above_cost = 0; +explain (costs off) +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; + QUERY PLAN +------------------------------------------------ + GroupAggregate + Group Key: ((g % 100000)) + -> Sort + Sort Key: ((g % 100000)) + -> Function Scan on generate_series g +(5 rows) + +create table agg_group_1 as +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; +set jit_above_cost to default; +create table agg_group_2 as +select (g/2)::numeric as c1, sum(7::int4) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; +create table agg_group_3 as +select (g/2)::numeric as c1, array_agg(g::numeric) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; +-- Produce results with hash aggregation +set enable_hashagg = true; +set enable_sort = false; +set jit_above_cost = 0; +explain (costs off) +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; + QUERY PLAN +------------------------------------------ + HashAggregate + Group Key: (g % 100000) + -> Function Scan on generate_series g +(3 rows) + +create table agg_hash_1 as +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; +set jit_above_cost to default; +create table agg_hash_2 as +select (g/2)::numeric as c1, sum(7::int4) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; +create table agg_hash_3 as +select (g/2)::numeric as c1, array_agg(g::numeric) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; +set enable_sort = true; +set work_mem to default; +-- Compare group aggregation results to hash aggregation results +(select * from agg_hash_1 except select * from agg_group_1) + union all +(select * from agg_group_1 except select * from agg_hash_1); + c1 | c2 | c3 +----+----+---- +(0 rows) + +(select * from agg_hash_2 except select * from agg_group_2) + union all +(select * from agg_group_2 except select * from agg_hash_2); + c1 | c2 | c3 +----+----+---- +(0 rows) + +(select * from agg_hash_3 except select * from agg_group_3) + union all +(select * from agg_group_3 except select * from agg_hash_3); + c1 | c2 | c3 +----+----+---- +(0 rows) + +drop table agg_group_1; +drop table agg_group_2; +drop table agg_group_3; +drop table agg_hash_1; +drop table agg_hash_2; +drop table agg_hash_3; diff --git a/src/test/regress/expected/groupingsets.out b/src/test/regress/expected/groupingsets.out index c1f802c88a7..767f60a96c7 100644 --- a/src/test/regress/expected/groupingsets.out +++ b/src/test/regress/expected/groupingsets.out @@ -1633,4 +1633,127 @@ select v||'a', case when grouping(v||'a') = 1 then 1 else 0 end, count(*) | 1 | 2 (4 rows) +-- +-- Compare results between plans using sorting and plans using hash +-- aggregation. Force spilling in both cases by setting work_mem low. +-- +SET work_mem='64kB'; +-- Produce results with sorting. +set enable_hashagg = false; +set jit_above_cost = 0; +explain (costs off) +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); + QUERY PLAN +--------------------------------------------------------------- + GroupAggregate + Group Key: ((g.g % 1000)), ((g.g % 100)), ((g.g % 10)) + Group Key: ((g.g % 1000)), ((g.g % 100)) + Group Key: ((g.g % 1000)) + Group Key: () + Sort Key: ((g.g % 100)), ((g.g % 10)) + Group Key: ((g.g % 100)), ((g.g % 10)) + Group Key: ((g.g % 100)) + Sort Key: ((g.g % 10)), ((g.g % 1000)) + Group Key: ((g.g % 10)), ((g.g % 1000)) + Group Key: ((g.g % 10)) + -> Sort + Sort Key: ((g.g % 1000)), ((g.g % 100)), ((g.g % 10)) + -> Function Scan on generate_series g +(14 rows) + +create table gs_group_1 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); +set jit_above_cost to default; +create table gs_group_2 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g/20 as g1000, g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by cube (g1000,g100,g10); +create table gs_group_3 as +select g100, g10, array_agg(g) as a, count(*) as c, max(g::text) as m from + (select g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by grouping sets (g100,g10); +-- Produce results with hash aggregation. +set enable_hashagg = true; +set enable_sort = false; +set work_mem='64kB'; +set jit_above_cost = 0; +explain (costs off) +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); + QUERY PLAN +--------------------------------------------------------------- + GroupAggregate + Group Key: ((g.g % 1000)), ((g.g % 100)), ((g.g % 10)) + Group Key: ((g.g % 1000)), ((g.g % 100)) + Group Key: ((g.g % 1000)) + Group Key: () + Sort Key: ((g.g % 100)), ((g.g % 10)) + Group Key: ((g.g % 100)), ((g.g % 10)) + Group Key: ((g.g % 100)) + Sort Key: ((g.g % 10)), ((g.g % 1000)) + Group Key: ((g.g % 10)), ((g.g % 1000)) + Group Key: ((g.g % 10)) + -> Sort + Sort Key: ((g.g % 1000)), ((g.g % 100)), ((g.g % 10)) + -> Function Scan on generate_series g +(14 rows) + +create table gs_hash_1 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); +set jit_above_cost to default; +create table gs_hash_2 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g/20 as g1000, g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by cube (g1000,g100,g10); +create table gs_hash_3 as +select g100, g10, array_agg(g) as a, count(*) as c, max(g::text) as m from + (select g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by grouping sets (g100,g10); +set enable_sort = true; +set work_mem to default; +-- Compare results +(select * from gs_hash_1 except select * from gs_group_1) + union all +(select * from gs_group_1 except select * from gs_hash_1); + g1000 | g100 | g10 | sum | count | max +-------+------+-----+-----+-------+----- +(0 rows) + +(select * from gs_hash_2 except select * from gs_group_2) + union all +(select * from gs_group_2 except select * from gs_hash_2); + g1000 | g100 | g10 | sum | count | max +-------+------+-----+-----+-------+----- +(0 rows) + +(select g100,g10,unnest(a),c,m from gs_hash_3 except + select g100,g10,unnest(a),c,m from gs_group_3) + union all +(select g100,g10,unnest(a),c,m from gs_group_3 except + select g100,g10,unnest(a),c,m from gs_hash_3); + g100 | g10 | unnest | c | m +------+-----+--------+---+--- +(0 rows) + +drop table gs_group_1; +drop table gs_group_2; +drop table gs_group_3; +drop table gs_hash_1; +drop table gs_hash_2; +drop table gs_hash_3; -- end diff --git a/src/test/regress/expected/select_distinct.out b/src/test/regress/expected/select_distinct.out index f3696c6d1de..11c6f50fbfa 100644 --- a/src/test/regress/expected/select_distinct.out +++ b/src/test/regress/expected/select_distinct.out @@ -148,6 +148,68 @@ SELECT count(*) FROM 4 (1 row) +-- +-- Compare results between plans using sorting and plans using hash +-- aggregation. Force spilling in both cases by setting work_mem low. +-- +SET work_mem='64kB'; +-- Produce results with sorting. +SET enable_hashagg=FALSE; +SET jit_above_cost=0; +EXPLAIN (costs off) +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; + QUERY PLAN +------------------------------------------------ + Unique + -> Sort + Sort Key: ((g % 1000)) + -> Function Scan on generate_series g +(4 rows) + +CREATE TABLE distinct_group_1 AS +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; +SET jit_above_cost TO DEFAULT; +CREATE TABLE distinct_group_2 AS +SELECT DISTINCT (g%1000)::text FROM generate_series(0,9999) g; +SET enable_hashagg=TRUE; +-- Produce results with hash aggregation. +SET enable_sort=FALSE; +SET jit_above_cost=0; +EXPLAIN (costs off) +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; + QUERY PLAN +------------------------------------------ + HashAggregate + Group Key: (g % 1000) + -> Function Scan on generate_series g +(3 rows) + +CREATE TABLE distinct_hash_1 AS +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; +SET jit_above_cost TO DEFAULT; +CREATE TABLE distinct_hash_2 AS +SELECT DISTINCT (g%1000)::text FROM generate_series(0,9999) g; +SET enable_sort=TRUE; +SET work_mem TO DEFAULT; +-- Compare results +(SELECT * FROM distinct_hash_1 EXCEPT SELECT * FROM distinct_group_1) + UNION ALL +(SELECT * FROM distinct_group_1 EXCEPT SELECT * FROM distinct_hash_1); + ?column? +---------- +(0 rows) + +(SELECT * FROM distinct_hash_1 EXCEPT SELECT * FROM distinct_group_1) + UNION ALL +(SELECT * FROM distinct_group_1 EXCEPT SELECT * FROM distinct_hash_1); + ?column? +---------- +(0 rows) + +DROP TABLE distinct_hash_1; +DROP TABLE distinct_hash_2; +DROP TABLE distinct_group_1; +DROP TABLE distinct_group_2; -- -- Also, some tests of IS DISTINCT FROM, which doesn't quite deserve its -- very own regression file. diff --git a/src/test/regress/expected/sysviews.out b/src/test/regress/expected/sysviews.out index a1c90eb9057..c40bf6c16eb 100644 --- a/src/test/regress/expected/sysviews.out +++ b/src/test/regress/expected/sysviews.out @@ -75,6 +75,7 @@ select name, setting from pg_settings where name like 'enable%'; enable_bitmapscan | on enable_gathermerge | on enable_hashagg | on + enable_hashagg_spill | on enable_hashjoin | on enable_indexonlyscan | on enable_indexscan | on @@ -89,7 +90,7 @@ select name, setting from pg_settings where name like 'enable%'; enable_seqscan | on enable_sort | on enable_tidscan | on -(17 rows) +(18 rows) -- Test that the pg_timezone_names and pg_timezone_abbrevs views are -- more-or-less working. We can't test their contents in any great detail diff --git a/src/test/regress/sql/aggregates.sql b/src/test/regress/sql/aggregates.sql index 17fb256aec5..bcd336c5812 100644 --- a/src/test/regress/sql/aggregates.sql +++ b/src/test/regress/sql/aggregates.sql @@ -1017,3 +1017,91 @@ select v||'a', case when v||'a' = 'aa' then 1 else 0 end, count(*) explain (costs off) select 1 from tenk1 where (hundred, thousand) in (select twothousand, twothousand from onek); + +-- +-- Compare results between plans using sorting and plans using hash +-- aggregation. Force spilling in both cases by setting work_mem low. +-- + +set work_mem='64kB'; + +-- Produce results with sorting. + +set enable_hashagg = false; + +set jit_above_cost = 0; + +explain (costs off) +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; + +create table agg_group_1 as +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; + +set jit_above_cost to default; + +create table agg_group_2 as +select (g/2)::numeric as c1, sum(7::int4) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; + +create table agg_group_3 as +select (g/2)::numeric as c1, array_agg(g::numeric) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; + +-- Produce results with hash aggregation + +set enable_hashagg = true; +set enable_sort = false; + +set jit_above_cost = 0; + +explain (costs off) +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; + +create table agg_hash_1 as +select g%100000 as c1, sum(g::numeric) as c2, count(*) as c3 + from generate_series(0, 199999) g + group by g%100000; + +set jit_above_cost to default; + +create table agg_hash_2 as +select (g/2)::numeric as c1, sum(7::int4) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; + +create table agg_hash_3 as +select (g/2)::numeric as c1, array_agg(g::numeric) as c2, count(*) as c3 + from generate_series(0, 1999) g + group by g/2; + +set enable_sort = true; +set work_mem to default; + +-- Compare group aggregation results to hash aggregation results + +(select * from agg_hash_1 except select * from agg_group_1) + union all +(select * from agg_group_1 except select * from agg_hash_1); + +(select * from agg_hash_2 except select * from agg_group_2) + union all +(select * from agg_group_2 except select * from agg_hash_2); + +(select * from agg_hash_3 except select * from agg_group_3) + union all +(select * from agg_group_3 except select * from agg_hash_3); + +drop table agg_group_1; +drop table agg_group_2; +drop table agg_group_3; +drop table agg_hash_1; +drop table agg_hash_2; +drop table agg_hash_3; diff --git a/src/test/regress/sql/groupingsets.sql b/src/test/regress/sql/groupingsets.sql index 95ac3fb52f6..bf8bce6ed31 100644 --- a/src/test/regress/sql/groupingsets.sql +++ b/src/test/regress/sql/groupingsets.sql @@ -441,4 +441,103 @@ select v||'a', case when grouping(v||'a') = 1 then 1 else 0 end, count(*) from unnest(array[1,1], array['a','b']) u(i,v) group by rollup(i, v||'a') order by 1,3; +-- +-- Compare results between plans using sorting and plans using hash +-- aggregation. Force spilling in both cases by setting work_mem low. +-- + +SET work_mem='64kB'; + +-- Produce results with sorting. + +set enable_hashagg = false; + +set jit_above_cost = 0; + +explain (costs off) +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); + +create table gs_group_1 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); + +set jit_above_cost to default; + +create table gs_group_2 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g/20 as g1000, g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by cube (g1000,g100,g10); + +create table gs_group_3 as +select g100, g10, array_agg(g) as a, count(*) as c, max(g::text) as m from + (select g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by grouping sets (g100,g10); + +-- Produce results with hash aggregation. + +set enable_hashagg = true; +set enable_sort = false; +set work_mem='64kB'; + +set jit_above_cost = 0; + +explain (costs off) +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); + +create table gs_hash_1 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g%1000 as g1000, g%100 as g100, g%10 as g10, g + from generate_series(0,199999) g) s +group by cube (g1000,g100,g10); + +set jit_above_cost to default; + +create table gs_hash_2 as +select g1000, g100, g10, sum(g::numeric), count(*), max(g::text) from + (select g/20 as g1000, g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by cube (g1000,g100,g10); + +create table gs_hash_3 as +select g100, g10, array_agg(g) as a, count(*) as c, max(g::text) as m from + (select g/200 as g100, g/2000 as g10, g + from generate_series(0,19999) g) s +group by grouping sets (g100,g10); + +set enable_sort = true; +set work_mem to default; + +-- Compare results + +(select * from gs_hash_1 except select * from gs_group_1) + union all +(select * from gs_group_1 except select * from gs_hash_1); + +(select * from gs_hash_2 except select * from gs_group_2) + union all +(select * from gs_group_2 except select * from gs_hash_2); + +(select g100,g10,unnest(a),c,m from gs_hash_3 except + select g100,g10,unnest(a),c,m from gs_group_3) + union all +(select g100,g10,unnest(a),c,m from gs_group_3 except + select g100,g10,unnest(a),c,m from gs_hash_3); + +drop table gs_group_1; +drop table gs_group_2; +drop table gs_group_3; +drop table gs_hash_1; +drop table gs_hash_2; +drop table gs_hash_3; + -- end diff --git a/src/test/regress/sql/select_distinct.sql b/src/test/regress/sql/select_distinct.sql index a605e86449e..33102744ebf 100644 --- a/src/test/regress/sql/select_distinct.sql +++ b/src/test/regress/sql/select_distinct.sql @@ -45,6 +45,68 @@ SELECT count(*) FROM SELECT count(*) FROM (SELECT DISTINCT two, four, two FROM tenk1) ss; +-- +-- Compare results between plans using sorting and plans using hash +-- aggregation. Force spilling in both cases by setting work_mem low. +-- + +SET work_mem='64kB'; + +-- Produce results with sorting. + +SET enable_hashagg=FALSE; + +SET jit_above_cost=0; + +EXPLAIN (costs off) +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; + +CREATE TABLE distinct_group_1 AS +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; + +SET jit_above_cost TO DEFAULT; + +CREATE TABLE distinct_group_2 AS +SELECT DISTINCT (g%1000)::text FROM generate_series(0,9999) g; + +SET enable_hashagg=TRUE; + +-- Produce results with hash aggregation. + +SET enable_sort=FALSE; + +SET jit_above_cost=0; + +EXPLAIN (costs off) +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; + +CREATE TABLE distinct_hash_1 AS +SELECT DISTINCT g%1000 FROM generate_series(0,9999) g; + +SET jit_above_cost TO DEFAULT; + +CREATE TABLE distinct_hash_2 AS +SELECT DISTINCT (g%1000)::text FROM generate_series(0,9999) g; + +SET enable_sort=TRUE; + +SET work_mem TO DEFAULT; + +-- Compare results + +(SELECT * FROM distinct_hash_1 EXCEPT SELECT * FROM distinct_group_1) + UNION ALL +(SELECT * FROM distinct_group_1 EXCEPT SELECT * FROM distinct_hash_1); + +(SELECT * FROM distinct_hash_1 EXCEPT SELECT * FROM distinct_group_1) + UNION ALL +(SELECT * FROM distinct_group_1 EXCEPT SELECT * FROM distinct_hash_1); + +DROP TABLE distinct_hash_1; +DROP TABLE distinct_hash_2; +DROP TABLE distinct_group_1; +DROP TABLE distinct_group_2; + -- -- Also, some tests of IS DISTINCT FROM, which doesn't quite deserve its -- very own regression file.