diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 808275a094b64fb8b18d616118edf90494348f66..5a4e7364766b838f9fc074c2c7f6b1762c81a2c8 100644
--- a/src/backend/executor/execGrouping.c
+++ b/src/backend/executor/execGrouping.c
@@ -23,12 +23,25 @@
 #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"
 
 
 /*****************************************************************************
@@ -260,7 +273,7 @@ execTuplesHashPrepare(int numCols,
  *	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
  *
@@ -275,20 +288,19 @@ TupleHashTable
 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;
@@ -302,15 +314,8 @@ BuildTupleHashTable(int numCols, AttrNumber *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;
 }
@@ -324,18 +329,17 @@ BuildTupleHashTable(int numCols, AttrNumber *keyColIdx,
  *
  * 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)
@@ -356,28 +360,17 @@ LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot,
 	/* 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 */
@@ -385,24 +378,19 @@ LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot,
 		}
 		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);
 
@@ -425,34 +413,19 @@ FindTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot,
 {
 	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;
@@ -468,22 +441,18 @@ FindTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot,
  * 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)
@@ -494,8 +463,12 @@ TupleHashTableHash(const void *key, Size keysize)
 	}
 	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;
@@ -530,29 +503,21 @@ TupleHashTableHash(const void *key, Size keysize)
  *
  * 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;
diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c
index ce2fc281a43bb445d6d592921bf885e136303b62..b06e1c1562a0d128f533d3efe5c000f3d3f154af 100644
--- a/src/backend/executor/nodeAgg.c
+++ b/src/backend/executor/nodeAgg.c
@@ -434,20 +434,6 @@ typedef struct AggStatePerPhaseData
 	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);
@@ -487,7 +473,7 @@ 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 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);
@@ -1646,6 +1632,12 @@ find_unaggregated_cols_walker(Node *node, Bitmapset **colnos)
 /*
  * 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
@@ -1653,20 +1645,19 @@ build_hash_table(AggState *aggstate)
 {
 	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);
 }
@@ -1723,6 +1714,8 @@ find_hash_columns(AggState *aggstate)
  *
  * 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)
@@ -1730,11 +1723,10 @@ 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;
 }
 
@@ -1744,12 +1736,12 @@ hash_agg_entry_size(int numAggs)
  *
  * 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 */
@@ -1771,14 +1763,16 @@ lookup_hash_entry(AggState *aggstate, TupleTableSlot *inputslot)
 	}
 
 	/* 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;
@@ -2176,7 +2170,7 @@ static void
 agg_fill_hash_table(AggState *aggstate)
 {
 	ExprContext *tmpcontext;
-	AggHashEntry entry;
+	TupleHashEntryData *entry;
 	TupleTableSlot *outerslot;
 
 	/*
@@ -2203,9 +2197,9 @@ agg_fill_hash_table(AggState *aggstate)
 
 		/* 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);
@@ -2225,7 +2219,7 @@ agg_retrieve_hash_table(AggState *aggstate)
 	ExprContext *econtext;
 	AggStatePerAgg peragg;
 	AggStatePerGroup pergroup;
-	AggHashEntry entry;
+	TupleHashEntryData *entry;
 	TupleTableSlot *firstSlot;
 	TupleTableSlot *result;
 
@@ -2246,7 +2240,7 @@ agg_retrieve_hash_table(AggState *aggstate)
 		/*
 		 * 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 */
@@ -2267,11 +2261,11 @@ agg_retrieve_hash_table(AggState *aggstate)
 		 * 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);
 
diff --git a/src/backend/executor/nodeRecursiveunion.c b/src/backend/executor/nodeRecursiveunion.c
index 39be191c45b0459b0f7ddb3e2360fb0f32804a67..acded079e245ddce63f41062144f3a2599bcf8dc 100644
--- a/src/backend/executor/nodeRecursiveunion.c
+++ b/src/backend/executor/nodeRecursiveunion.c
@@ -3,6 +3,10 @@
  * 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
  *
@@ -20,17 +24,6 @@
 #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.
@@ -48,7 +41,7 @@ build_hash_table(RecursiveUnionState *rustate)
 											 rustate->eqfunctions,
 											 rustate->hashfunctions,
 											 node->numGroups,
-											 sizeof(RUHashEntryData),
+											 0,
 											 rustate->tableContext,
 											 rustate->tempContext);
 }
diff --git a/src/backend/executor/nodeSetOp.c b/src/backend/executor/nodeSetOp.c
index 633580b4362af9b9dd9efd01510b2cc1f94dbca2..e94555ead894ea632aa1c36b0799b4029a093aa0 100644
--- a/src/backend/executor/nodeSetOp.c
+++ b/src/backend/executor/nodeSetOp.c
@@ -66,19 +66,6 @@ typedef struct SetOpStatePerGroupData
 	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);
@@ -141,7 +128,7 @@ build_hash_table(SetOpState *setopstate)
 												setopstate->eqfunctions,
 												setopstate->hashfunctions,
 												node->numGroups,
-												sizeof(SetOpHashEntryData),
+												0,
 												setopstate->tableContext,
 												setopstate->tempContext);
 }
@@ -238,7 +225,7 @@ setop_retrieve_direct(SetOpState *setopstate)
 	 * get state info from node
 	 */
 	outerPlan = outerPlanState(setopstate);
-	pergroup = setopstate->pergroup;
+	pergroup = (SetOpStatePerGroup) setopstate->pergroup;
 	resultTupleSlot = setopstate->ps.ps_ResultTupleSlot;
 
 	/*
@@ -367,7 +354,7 @@ setop_fill_hash_table(SetOpState *setopstate)
 	{
 		TupleTableSlot *outerslot;
 		int			flag;
-		SetOpHashEntry entry;
+		TupleHashEntryData *entry;
 		bool		isnew;
 
 		outerslot = ExecProcNode(outerPlan);
@@ -383,15 +370,20 @@ setop_fill_hash_table(SetOpState *setopstate)
 			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
 		{
@@ -399,12 +391,12 @@ setop_fill_hash_table(SetOpState *setopstate)
 			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 */
@@ -422,7 +414,7 @@ setop_fill_hash_table(SetOpState *setopstate)
 static TupleTableSlot *
 setop_retrieve_hash_table(SetOpState *setopstate)
 {
-	SetOpHashEntry entry;
+	TupleHashEntryData *entry;
 	TupleTableSlot *resultTupleSlot;
 
 	/*
@@ -438,7 +430,7 @@ setop_retrieve_hash_table(SetOpState *setopstate)
 		/*
 		 * 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 */
@@ -450,12 +442,12 @@ setop_retrieve_hash_table(SetOpState *setopstate)
 		 * 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);
 		}
diff --git a/src/backend/executor/nodeSubplan.c b/src/backend/executor/nodeSubplan.c
index 2cf169f956620c5720c8d0a29c6f1f7b205b309e..8ca8fc460ca90ae729fc1d48b6314b26a80f1915 100644
--- a/src/backend/executor/nodeSubplan.c
+++ b/src/backend/executor/nodeSubplan.c
@@ -508,7 +508,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
 										  node->tab_eq_funcs,
 										  node->tab_hash_funcs,
 										  nbuckets,
-										  sizeof(TupleHashEntryData),
+										  0,
 										  node->hashtablecxt,
 										  node->hashtempcxt);
 
@@ -527,7 +527,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
 											  node->tab_eq_funcs,
 											  node->tab_hash_funcs,
 											  nbuckets,
-											  sizeof(TupleHashEntryData),
+											  0,
 											  node->hashtablecxt,
 											  node->hashtempcxt);
 	}
@@ -626,7 +626,7 @@ findPartialMatch(TupleHashTable hashtable, TupleTableSlot *slot,
 	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,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index f657ffc44632633d863c9a129c78c2bd7aaedb78..644b8b6763d6aab7d94a59c6fb65f84be5a6faed 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -3292,6 +3292,12 @@ estimate_hashagg_tablesize(Path *path, const AggClauseCosts *agg_costs,
 	/* 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;
 }
 
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 39521ed08e36c1b893934db6de20c53d9ab0664c..136276be53c3df281be31c44f59321f49139af21 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -140,7 +140,7 @@ extern void execTuplesHashPrepare(int numCols,
 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,
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 4fa366178f5ee14f4fa3aef56454c54eae731218..f6f73f3c590fed593ce3d6b66ba18bd0502f8dfb 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -499,14 +499,23 @@ typedef struct TupleHashTableData *TupleHashTable;
 
 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) */
@@ -521,7 +530,7 @@ typedef struct TupleHashTableData
 	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.
@@ -529,16 +538,13 @@ typedef HASH_SEQ_STATUS TupleHashIterator;
  * 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)
 
 
 /* ----------------------------------------------------------------
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index e61ebbfe2ae17cbb78912aa0dc00e7de24fc06fa..6c6d519aac1a1c46d4772d190a540c51602133cc 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2813,6 +2813,8 @@ tsKEY
 ts_db_fctx
 ts_tokentype
 tsearch_readline_state
+tuplehash_hash
+tuplehash_iterator
 txid
 tzEntry
 u1byte