#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 TupleHashTable CurTupleHashTable = NULL;
-
-static uint32 TupleHashTableHash(const void *key, Size keysize);
-static int TupleHashTableMatch(const void *key1, const void *key2,
- Size keysize);
+/*
+ * Define parameters for tuple hash table code generation. The interface is
+ * *also* declared in execnodes.h (to generate the types, which are externally
+ * visible).
+ */
+#define SH_PREFIX tuplehash
+#define SH_ELEMENT_TYPE TupleHashEntryData
+#define SH_KEY_TYPE MinimalTuple
+#define SH_KEY firstTuple
+#define SH_HASH_KEY(tb, key) TupleHashTableHash(tb, key)
+#define SH_EQUAL(tb, a, b) TupleHashTableMatch(tb, a, b) == 0
+#define SH_SCOPE extern
+#define SH_STORE_HASH
+#define SH_GET_HASH(tb, a) a->hash
+#define SH_DEFINE
+#include "lib/simplehash.h"
/*****************************************************************************
* eqfunctions: equality comparison functions to use
* hashfunctions: datatype-specific hashing functions to use
* nbuckets: initial estimate of hashtable size
- * entrysize: size of each entry (at least sizeof(TupleHashEntryData))
+ * additionalsize: size of data stored in ->additional
* tablecxt: memory context in which to store table and table entries
* tempcxt: short-lived context for evaluation hash and comparison functions
*
BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
FmgrInfo *eqfunctions,
FmgrInfo *hashfunctions,
- long nbuckets, Size entrysize,
+ long nbuckets, Size additionalsize,
MemoryContext tablecxt, MemoryContext tempcxt)
{
TupleHashTable hashtable;
- HASHCTL hash_ctl;
+ Size entrysize = sizeof(TupleHashEntryData) + additionalsize;
Assert(nbuckets > 0);
- Assert(entrysize >= sizeof(TupleHashEntryData));
/* Limit initial table size request to not more than work_mem */
nbuckets = Min(nbuckets, (long) ((work_mem * 1024L) / entrysize));
- hashtable = (TupleHashTable) MemoryContextAlloc(tablecxt,
- sizeof(TupleHashTableData));
+ hashtable = (TupleHashTable)
+ MemoryContextAlloc(tablecxt, sizeof(TupleHashTableData));
hashtable->numCols = numCols;
hashtable->keyColIdx = keyColIdx;
hashtable->in_hash_funcs = NULL;
hashtable->cur_eq_funcs = NULL;
- MemSet(&hash_ctl, 0, sizeof(hash_ctl));
- hash_ctl.keysize = sizeof(TupleHashEntryData);
- hash_ctl.entrysize = entrysize;
- hash_ctl.hash = TupleHashTableHash;
- hash_ctl.match = TupleHashTableMatch;
- hash_ctl.hcxt = tablecxt;
- hashtable->hashtab = hash_create("TupleHashTable", nbuckets,
- &hash_ctl,
- HASH_ELEM | HASH_FUNCTION | HASH_COMPARE | HASH_CONTEXT);
+ hashtable->hashtab = tuplehash_create(tablecxt, nbuckets);
+ hashtable->hashtab->private = hashtable;
return hashtable;
}
*
* If isnew isn't NULL, then a new entry is created if no existing entry
* matches. On return, *isnew is true if the entry is newly created,
- * false if it existed already. Any extra space in a new entry has been
- * zeroed.
+ * false if it existed already. ->additional_data in the new entry has
+ * been zeroed.
*/
TupleHashEntry
LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot,
bool *isnew)
{
- TupleHashEntry entry;
+ TupleHashEntryData *entry;
MemoryContext oldContext;
- TupleHashTable saveCurHT;
- TupleHashEntryData dummy;
bool found;
+ MinimalTuple key;
/* If first time through, clone the input slot to make table slot */
if (hashtable->tableslot == NULL)
/* Need to run the hash functions in short-lived context */
oldContext = MemoryContextSwitchTo(hashtable->tempcxt);
- /*
- * Set up data needed by hash and match functions
- *
- * We save and restore CurTupleHashTable just in case someone manages to
- * invoke this code re-entrantly.
- */
+ /* set up data needed by hash and match functions */
hashtable->inputslot = slot;
hashtable->in_hash_funcs = hashtable->tab_hash_funcs;
hashtable->cur_eq_funcs = hashtable->tab_eq_funcs;
- saveCurHT = CurTupleHashTable;
- CurTupleHashTable = hashtable;
-
- /* Search the hash table */
- dummy.firstTuple = NULL; /* flag to reference inputslot */
- entry = (TupleHashEntry) hash_search(hashtable->hashtab,
- &dummy,
- isnew ? HASH_ENTER : HASH_FIND,
- &found);
+ key = NULL; /* flag to reference inputslot */
if (isnew)
{
+ entry = tuplehash_insert(hashtable->hashtab, key, &found);
+
if (found)
{
/* found pre-existing entry */
}
else
{
- /*
- * created new entry
- *
- * Zero any caller-requested space in the entry. (This zaps the
- * "key data" dynahash.c copied into the new entry, but we don't
- * care since we're about to overwrite it anyway.)
- */
- MemSet(entry, 0, hashtable->entrysize);
-
- /* Copy the first tuple into the table context */
+ /* 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);
-
- *isnew = true;
}
}
-
- CurTupleHashTable = saveCurHT;
+ else
+ {
+ entry = tuplehash_lookup(hashtable->hashtab, key);
+ }
MemoryContextSwitchTo(oldContext);
{
TupleHashEntry entry;
MemoryContext oldContext;
- TupleHashTable saveCurHT;
- TupleHashEntryData dummy;
+ MinimalTuple key;
/* Need to run the hash functions in short-lived context */
oldContext = MemoryContextSwitchTo(hashtable->tempcxt);
- /*
- * Set up data needed by hash and match functions
- *
- * We save and restore CurTupleHashTable just in case someone manages to
- * invoke this code re-entrantly.
- */
+ /* Set up data needed by hash and match functions */
hashtable->inputslot = slot;
hashtable->in_hash_funcs = hashfunctions;
hashtable->cur_eq_funcs = eqfunctions;
- saveCurHT = CurTupleHashTable;
- CurTupleHashTable = hashtable;
-
/* Search the hash table */
- dummy.firstTuple = NULL; /* flag to reference inputslot */
- entry = (TupleHashEntry) hash_search(hashtable->hashtab,
- &dummy,
- HASH_FIND,
- NULL);
-
- CurTupleHashTable = saveCurHT;
-
+ key = NULL; /* flag to reference inputslot */
+ entry = tuplehash_lookup(hashtable->hashtab, key);
MemoryContextSwitchTo(oldContext);
return entry;
* This convention avoids the need to materialize virtual input tuples unless
* they actually need to get copied into the table.
*
- * CurTupleHashTable must be set before calling this, since dynahash.c
- * doesn't provide any API that would let us get at the hashtable otherwise.
- *
* Also, the caller must select an appropriate memory context for running
* the hash functions. (dynahash.c doesn't change CurrentMemoryContext.)
*/
static uint32
-TupleHashTableHash(const void *key, Size keysize)
+TupleHashTableHash(struct tuplehash_hash *tb, const MinimalTuple tuple)
{
- MinimalTuple tuple = ((const TupleHashEntryData *) key)->firstTuple;
- TupleTableSlot *slot;
- TupleHashTable hashtable = CurTupleHashTable;
+ TupleHashTable hashtable = (TupleHashTable) tb->private;
int numCols = hashtable->numCols;
AttrNumber *keyColIdx = hashtable->keyColIdx;
- FmgrInfo *hashfunctions;
uint32 hashkey = 0;
+ TupleTableSlot *slot;
+ FmgrInfo *hashfunctions;
int i;
if (tuple == NULL)
}
else
{
- /* Process a tuple already stored in the table */
- /* (this case never actually occurs in current dynahash.c code) */
+ /*
+ * Process a tuple already stored in the table.
+ *
+ * (this case never actually occurs due to the way simplehash.h is
+ * used, as the hash-value is stored in the entries)
+ */
slot = hashtable->tableslot;
ExecStoreMinimalTuple(tuple, slot, false);
hashfunctions = hashtable->tab_hash_funcs;
*
* As above, the passed pointers are pointers to TupleHashEntryData.
*
- * CurTupleHashTable must be set before calling this, since dynahash.c
- * doesn't provide any API that would let us get at the hashtable otherwise.
- *
* Also, the caller must select an appropriate memory context for running
* the compare functions. (dynahash.c doesn't change CurrentMemoryContext.)
*/
static int
-TupleHashTableMatch(const void *key1, const void *key2, Size keysize)
+TupleHashTableMatch(struct tuplehash_hash *tb, const MinimalTuple tuple1, const MinimalTuple tuple2)
{
- MinimalTuple tuple1 = ((const TupleHashEntryData *) key1)->firstTuple;
-
-#ifdef USE_ASSERT_CHECKING
- MinimalTuple tuple2 = ((const TupleHashEntryData *) key2)->firstTuple;
-#endif
TupleTableSlot *slot1;
TupleTableSlot *slot2;
- TupleHashTable hashtable = CurTupleHashTable;
+ TupleHashTable hashtable = (TupleHashTable) tb->private;
/*
- * We assume that dynahash.c will only ever call us with the first
+ * We assume that simplehash.h will only ever call us with the first
* argument being an actual table entry, and the second argument being
* LookupTupleHashEntry's dummy TupleHashEntryData. The other direction
- * could be supported too, but is not currently used by dynahash.c.
+ * could be supported too, but is not currently required.
*/
Assert(tuple1 != NULL);
slot1 = hashtable->tableslot;
Sort *sortnode; /* Sort node for input ordering for phase */
} AggStatePerPhaseData;
-/*
- * To implement hashed aggregation, we need a hashtable that stores a
- * representative tuple and an array of AggStatePerGroup structs for each
- * distinct set of GROUP BY column values. We compute the hash key from
- * the GROUP BY columns.
- */
-typedef struct AggHashEntryData *AggHashEntry;
-
-typedef struct AggHashEntryData
-{
- TupleHashEntryData shared; /* common header for hash table entries */
- /* per-aggregate transition status array */
- AggStatePerGroupData pergroup[FLEXIBLE_ARRAY_MEMBER];
-} AggHashEntryData;
static void initialize_phase(AggState *aggstate, int newphase);
static TupleTableSlot *fetch_input_tuple(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 AggHashEntry lookup_hash_entry(AggState *aggstate,
+static TupleHashEntryData *lookup_hash_entry(AggState *aggstate,
TupleTableSlot *inputslot);
static TupleTableSlot *agg_retrieve_direct(AggState *aggstate);
static void agg_fill_hash_table(AggState *aggstate);
/*
* Initialize the hash table to empty.
*
+ * To implement hashed aggregation, we need a hashtable that stores a
+ * representative tuple and an array of AggStatePerGroup structs for each
+ * distinct set of GROUP BY column values. We compute the hash key from the
+ * GROUP BY columns. The per-group data is allocated in lookup_hash_entry(),
+ * for each entry.
+ *
* The hash table always lives in the aggcontext memory context.
*/
static void
{
Agg *node = (Agg *) aggstate->ss.ps.plan;
MemoryContext tmpmem = aggstate->tmpcontext->ecxt_per_tuple_memory;
- Size entrysize;
+ Size additionalsize;
Assert(node->aggstrategy == AGG_HASHED);
Assert(node->numGroups > 0);
- entrysize = offsetof(AggHashEntryData, pergroup) +
- aggstate->numaggs * sizeof(AggStatePerGroupData);
+ additionalsize = aggstate->numaggs * sizeof(AggStatePerGroupData);
aggstate->hashtable = BuildTupleHashTable(node->numCols,
node->grpColIdx,
aggstate->phase->eqfunctions,
aggstate->hashfunctions,
node->numGroups,
- entrysize,
+ additionalsize,
aggstate->aggcontexts[0]->ecxt_per_tuple_memory,
tmpmem);
}
*
* Note that the estimate does not include space for pass-by-reference
* transition data values, nor for the representative tuple of each group.
+ * Nor does this account of the target fill-factor and growth policy of the
+ * hash table.
*/
Size
hash_agg_entry_size(int numAggs)
Size entrysize;
/* This must match build_hash_table */
- entrysize = offsetof(AggHashEntryData, pergroup) +
+ entrysize = sizeof(TupleHashEntryData) +
numAggs * sizeof(AggStatePerGroupData);
entrysize = MAXALIGN(entrysize);
- /* Account for hashtable overhead (assuming fill factor = 1) */
- entrysize += 3 * sizeof(void *);
+
return entrysize;
}
*
* When called, CurrentMemoryContext should be the per-query context.
*/
-static AggHashEntry
+static TupleHashEntryData *
lookup_hash_entry(AggState *aggstate, TupleTableSlot *inputslot)
{
TupleTableSlot *hashslot = aggstate->hashslot;
ListCell *l;
- AggHashEntry entry;
+ TupleHashEntryData *entry;
bool isnew;
/* if first time through, initialize hashslot by cloning input slot */
}
/* find or create the hashtable entry using the filtered tuple */
- entry = (AggHashEntry) LookupTupleHashEntry(aggstate->hashtable,
- hashslot,
- &isnew);
+ entry = LookupTupleHashEntry(aggstate->hashtable, hashslot, &isnew);
if (isnew)
{
+ entry->additional = (AggStatePerGroup)
+ MemoryContextAlloc(aggstate->hashtable->tablecxt,
+ sizeof(AggStatePerGroupData) * aggstate->numtrans);
/* initialize aggregates for new tuple group */
- initialize_aggregates(aggstate, entry->pergroup, 0);
+ initialize_aggregates(aggstate, (AggStatePerGroup) entry->additional,
+ 0);
}
return entry;
agg_fill_hash_table(AggState *aggstate)
{
ExprContext *tmpcontext;
- AggHashEntry entry;
+ TupleHashEntryData *entry;
TupleTableSlot *outerslot;
/*
/* Advance the aggregates */
if (DO_AGGSPLIT_COMBINE(aggstate->aggsplit))
- combine_aggregates(aggstate, entry->pergroup);
+ combine_aggregates(aggstate, (AggStatePerGroup) entry->additional);
else
- advance_aggregates(aggstate, entry->pergroup);
+ advance_aggregates(aggstate, (AggStatePerGroup) entry->additional);
/* Reset per-input-tuple context after each tuple */
ResetExprContext(tmpcontext);
ExprContext *econtext;
AggStatePerAgg peragg;
AggStatePerGroup pergroup;
- AggHashEntry entry;
+ TupleHashEntryData *entry;
TupleTableSlot *firstSlot;
TupleTableSlot *result;
/*
* Find the next entry in the hash table
*/
- entry = (AggHashEntry) ScanTupleHashTable(&aggstate->hashiter);
+ entry = ScanTupleHashTable(aggstate->hashtable, &aggstate->hashiter);
if (entry == NULL)
{
/* No more entries in hashtable, so done */
* Store the copied first input tuple in the tuple table slot reserved
* for it, so that it can be used in ExecProject.
*/
- ExecStoreMinimalTuple(entry->shared.firstTuple,
+ ExecStoreMinimalTuple(entry->firstTuple,
firstSlot,
false);
- pergroup = entry->pergroup;
+ pergroup = (AggStatePerGroup) entry->additional;
finalize_aggregates(aggstate, peragg, pergroup, 0);
* nodeRecursiveunion.c
* routines to handle RecursiveUnion nodes.
*
+ * To implement UNION (without ALL), we need a hashtable that stores tuples
+ * already seen. The hash key is computed from the grouping columns.
+ *
+ *
* Portions Copyright (c) 1996-2016, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
#include "utils/memutils.h"
-/*
- * To implement UNION (without ALL), we need a hashtable that stores tuples
- * already seen. The hash key is computed from the grouping columns.
- */
-typedef struct RUHashEntryData *RUHashEntry;
-
-typedef struct RUHashEntryData
-{
- TupleHashEntryData shared; /* common header for hash table entries */
-} RUHashEntryData;
-
/*
* Initialize the hash table to empty.
rustate->eqfunctions,
rustate->hashfunctions,
node->numGroups,
- sizeof(RUHashEntryData),
+ 0,
rustate->tableContext,
rustate->tempContext);
}
long numRight; /* number of right-input dups in group */
} SetOpStatePerGroupData;
-/*
- * To implement hashed mode, we need a hashtable that stores a
- * representative tuple and the duplicate counts for each distinct set
- * of grouping columns. We compute the hash key from the grouping columns.
- */
-typedef struct SetOpHashEntryData *SetOpHashEntry;
-
-typedef struct SetOpHashEntryData
-{
- TupleHashEntryData shared; /* common header for hash table entries */
- SetOpStatePerGroupData pergroup;
-} SetOpHashEntryData;
-
static TupleTableSlot *setop_retrieve_direct(SetOpState *setopstate);
static void setop_fill_hash_table(SetOpState *setopstate);
setopstate->eqfunctions,
setopstate->hashfunctions,
node->numGroups,
- sizeof(SetOpHashEntryData),
+ 0,
setopstate->tableContext,
setopstate->tempContext);
}
* get state info from node
*/
outerPlan = outerPlanState(setopstate);
- pergroup = setopstate->pergroup;
+ pergroup = (SetOpStatePerGroup) setopstate->pergroup;
resultTupleSlot = setopstate->ps.ps_ResultTupleSlot;
/*
{
TupleTableSlot *outerslot;
int flag;
- SetOpHashEntry entry;
+ TupleHashEntryData *entry;
bool isnew;
outerslot = ExecProcNode(outerPlan);
Assert(in_first_rel);
/* Find or build hashtable entry for this tuple's group */
- entry = (SetOpHashEntry)
- LookupTupleHashEntry(setopstate->hashtable, outerslot, &isnew);
+ entry = LookupTupleHashEntry(setopstate->hashtable, outerslot,
+ &isnew);
/* If new tuple group, initialize counts */
if (isnew)
- initialize_counts(&entry->pergroup);
+ {
+ entry->additional = (SetOpStatePerGroup)
+ MemoryContextAlloc(setopstate->hashtable->tablecxt,
+ sizeof(SetOpStatePerGroupData));
+ initialize_counts((SetOpStatePerGroup) entry->additional);
+ }
/* Advance the counts */
- advance_counts(&entry->pergroup, flag);
+ advance_counts((SetOpStatePerGroup) entry->additional, flag);
}
else
{
in_first_rel = false;
/* For tuples not seen previously, do not make hashtable entry */
- entry = (SetOpHashEntry)
- LookupTupleHashEntry(setopstate->hashtable, outerslot, NULL);
+ entry = LookupTupleHashEntry(setopstate->hashtable, outerslot,
+ NULL);
/* Advance the counts if entry is already present */
if (entry)
- advance_counts(&entry->pergroup, flag);
+ advance_counts((SetOpStatePerGroup) entry->additional, flag);
}
/* Must reset temp context after each hashtable lookup */
static TupleTableSlot *
setop_retrieve_hash_table(SetOpState *setopstate)
{
- SetOpHashEntry entry;
+ TupleHashEntryData *entry;
TupleTableSlot *resultTupleSlot;
/*
/*
* Find the next entry in the hash table
*/
- entry = (SetOpHashEntry) ScanTupleHashTable(&setopstate->hashiter);
+ entry = ScanTupleHashTable(setopstate->hashtable, &setopstate->hashiter);
if (entry == NULL)
{
/* No more entries in hashtable, so done */
* See if we should emit any copies of this tuple, and if so return
* the first copy.
*/
- set_output_count(setopstate, &entry->pergroup);
+ set_output_count(setopstate, (SetOpStatePerGroup) entry->additional);
if (setopstate->numOutput > 0)
{
setopstate->numOutput--;
- return ExecStoreMinimalTuple(entry->shared.firstTuple,
+ return ExecStoreMinimalTuple(entry->firstTuple,
resultTupleSlot,
false);
}
node->tab_eq_funcs,
node->tab_hash_funcs,
nbuckets,
- sizeof(TupleHashEntryData),
+ 0,
node->hashtablecxt,
node->hashtempcxt);
node->tab_eq_funcs,
node->tab_hash_funcs,
nbuckets,
- sizeof(TupleHashEntryData),
+ 0,
node->hashtablecxt,
node->hashtempcxt);
}
TupleHashEntry entry;
InitTupleHashIterator(hashtable, &hashiter);
- while ((entry = ScanTupleHashTable(&hashiter)) != NULL)
+ while ((entry = ScanTupleHashTable(hashtable, &hashiter)) != NULL)
{
ExecStoreMinimalTuple(entry->firstTuple, hashtable->tableslot, false);
if (!execTuplesUnequal(slot, hashtable->tableslot,
/* plus the per-hash-entry overhead */
hashentrysize += hash_agg_entry_size(agg_costs->numAggs);
+ /*
+ * Note that this disregards the effect of fill-factor and growth policy
+ * of the hash-table. That's probably ok, given default the default
+ * fill-factor is relatively high. It'd be hard to meaningfully factor in
+ * "double-in-size" growth policies here.
+ */
return hashentrysize * dNumGroups;
}
extern TupleHashTable BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
FmgrInfo *eqfunctions,
FmgrInfo *hashfunctions,
- long nbuckets, Size entrysize,
+ long nbuckets, Size additionalsize,
MemoryContext tablecxt,
MemoryContext tempcxt);
extern TupleHashEntry LookupTupleHashEntry(TupleHashTable hashtable,
typedef struct TupleHashEntryData
{
- /* firstTuple must be the first field in this struct! */
MinimalTuple firstTuple; /* copy of first tuple in this group */
- /* there may be additional data beyond the end of this struct */
-} TupleHashEntryData; /* VARIABLE LENGTH STRUCT */
+ void *additional; /* user data */
+ uint32 status; /* hash status */
+ uint32 hash; /* hash value (cached) */
+} TupleHashEntryData;
+
+/* define paramters necessary to generate the tuple hash table interface */
+#define SH_PREFIX tuplehash
+#define SH_ELEMENT_TYPE TupleHashEntryData
+#define SH_KEY_TYPE MinimalTuple
+#define SH_SCOPE extern
+#define SH_DECLARE
+#include "lib/simplehash.h"
typedef struct TupleHashTableData
{
- HTAB *hashtab; /* underlying dynahash table */
+ tuplehash_hash *hashtab; /* underlying hash table */
int numCols; /* number of columns in lookup key */
AttrNumber *keyColIdx; /* attr numbers of key columns */
FmgrInfo *tab_hash_funcs; /* hash functions for table datatype(s) */
FmgrInfo *cur_eq_funcs; /* equality functions for input vs. table */
} TupleHashTableData;
-typedef HASH_SEQ_STATUS TupleHashIterator;
+typedef tuplehash_iterator TupleHashIterator;
/*
* Use InitTupleHashIterator/TermTupleHashIterator for a read/write scan.
* explicit scan termination is needed).
*/
#define InitTupleHashIterator(htable, iter) \
- hash_seq_init(iter, (htable)->hashtab)
+ tuplehash_start_iterate(htable->hashtab, iter)
#define TermTupleHashIterator(iter) \
- hash_seq_term(iter)
+ ((void) 0)
#define ResetTupleHashIterator(htable, iter) \
- do { \
- hash_freeze((htable)->hashtab); \
- hash_seq_init(iter, (htable)->hashtab); \
- } while (0)
-#define ScanTupleHashTable(iter) \
- ((TupleHashEntry) hash_seq_search(iter))
+ InitTupleHashIterator(htable, iter)
+#define ScanTupleHashTable(htable, iter) \
+ tuplehash_iterate(htable->hashtab, iter)
/* ----------------------------------------------------------------
ts_db_fctx
ts_tokentype
tsearch_readline_state
+tuplehash_hash
+tuplehash_iterator
txid
tzEntry
u1byte