summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/backend/executor/execGrouping.c145
-rw-r--r--src/backend/executor/nodeAgg.c59
-rw-r--r--src/backend/executor/nodeRecursiveunion.c13
-rw-r--r--src/backend/executor/nodeSetOp.c43
-rw-r--r--src/backend/executor/nodeSubplan.c6
-rw-r--r--src/include/executor/executor.h2
-rw-r--r--src/include/nodes/execnodes.h38
-rw-r--r--src/test/regress/expected/matview.out4
-rw-r--r--src/test/regress/expected/psql.out2
-rw-r--r--src/test/regress/expected/tsrf.out6
-rw-r--r--src/test/regress/expected/union.out26
-rw-r--r--src/test/regress/expected/with.out2
-rw-r--r--src/test/regress/sql/psql.sql2
13 files changed, 144 insertions, 204 deletions
diff --git a/src/backend/executor/execGrouping.c b/src/backend/executor/execGrouping.c
index 808275a094..f80da1943a 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_KEYTYPE MinimalTuple
+#define SH_KEY firstTuple
+#define SH_CONTAINS TupleHashEntryData
+#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,18 @@ 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 +313,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 +328,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)
@@ -358,26 +361,17 @@ LookupTupleHashEntry(TupleHashTable hashtable, TupleTableSlot *slot,
/*
* 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.
*/
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;
if (isnew)
{
+ entry = tuplehash_insert(hashtable->hashtab, key, &found);
+
if (found)
{
/* found pre-existing entry */
@@ -385,24 +379,20 @@ 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;
}
}
+ else
+ {
+ entry = tuplehash_lookup(hashtable->hashtab, key);
+ }
- CurTupleHashTable = saveCurHT;
MemoryContextSwitchTo(oldContext);
@@ -425,34 +415,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 +443,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)
@@ -495,7 +466,6 @@ 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) */
slot = hashtable->tableslot;
ExecStoreMinimalTuple(tuple, slot, false);
hashfunctions = hashtable->tab_hash_funcs;
@@ -530,30 +500,23 @@ 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;
ExecStoreMinimalTuple(tuple1, slot1, false);
diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c
index ce2fc281a4..ec1c72e9bb 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);
@@ -1653,20 +1639,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);
}
@@ -1730,11 +1715,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 *);
+ /* FIXME: Do we want to handle fill factor overhead here? */
return entrysize;
}
@@ -1744,18 +1728,21 @@ 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 */
if (hashslot->tts_tupleDescriptor == NULL)
{
- ExecSetSlotDescriptor(hashslot, inputslot->tts_tupleDescriptor);
+ MemoryContext oldContext = MemoryContextSwitchTo(hashslot->tts_mcxt);
+ /* get rid of constraints */
+ ExecSetSlotDescriptor(hashslot, CreateTupleDescCopy(inputslot->tts_tupleDescriptor));
+ MemoryContextSwitchTo(oldContext);
/* Make sure all unused columns are NULLs */
ExecStoreAllNullTuple(hashslot);
}
@@ -1771,14 +1758,14 @@ 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 = MemoryContextAlloc(aggstate->hashtable->tablecxt,
+ sizeof(AggStatePerGroupData) * aggstate->numtrans);
/* initialize aggregates for new tuple group */
- initialize_aggregates(aggstate, entry->pergroup, 0);
+ initialize_aggregates(aggstate, entry->additional, 0);
}
return entry;
@@ -2176,7 +2163,7 @@ static void
agg_fill_hash_table(AggState *aggstate)
{
ExprContext *tmpcontext;
- AggHashEntry entry;
+ TupleHashEntryData *entry;
TupleTableSlot *outerslot;
/*
@@ -2203,9 +2190,9 @@ agg_fill_hash_table(AggState *aggstate)
/* Advance the aggregates */
if (DO_AGGSPLIT_COMBINE(aggstate->aggsplit))
- combine_aggregates(aggstate, entry->pergroup);
+ combine_aggregates(aggstate, entry->additional);
else
- advance_aggregates(aggstate, entry->pergroup);
+ advance_aggregates(aggstate, entry->additional);
/* Reset per-input-tuple context after each tuple */
ResetExprContext(tmpcontext);
@@ -2225,7 +2212,7 @@ agg_retrieve_hash_table(AggState *aggstate)
ExprContext *econtext;
AggStatePerAgg peragg;
AggStatePerGroup pergroup;
- AggHashEntry entry;
+ TupleHashEntryData *entry;
TupleTableSlot *firstSlot;
TupleTableSlot *result;
@@ -2246,7 +2233,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 +2254,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 = entry->additional;
finalize_aggregates(aggstate, peragg, pergroup, 0);
diff --git a/src/backend/executor/nodeRecursiveunion.c b/src/backend/executor/nodeRecursiveunion.c
index 39be191c45..6e7d9d2731 100644
--- a/src/backend/executor/nodeRecursiveunion.c
+++ b/src/backend/executor/nodeRecursiveunion.c
@@ -20,17 +20,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 +37,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 633580b436..37d0477d1d 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);
}
@@ -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,19 @@ 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 = MemoryContextAlloc(setopstate->hashtable->tablecxt,
+ sizeof(SetOpStatePerGroupData));
+ initialize_counts(entry->additional);
+ }
/* Advance the counts */
- advance_counts(&entry->pergroup, flag);
+ advance_counts(entry->additional, flag);
}
else
{
@@ -399,12 +390,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(entry->additional, flag);
}
/* Must reset temp context after each hashtable lookup */
@@ -422,7 +413,7 @@ setop_fill_hash_table(SetOpState *setopstate)
static TupleTableSlot *
setop_retrieve_hash_table(SetOpState *setopstate)
{
- SetOpHashEntry entry;
+ TupleHashEntryData *entry;
TupleTableSlot *resultTupleSlot;
/*
@@ -438,7 +429,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 +441,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, 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 2cf169f956..8ca8fc460c 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/include/executor/executor.h b/src/include/executor/executor.h
index 39521ed08e..136276be53 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 4fa366178f..3148f3a86b 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -499,14 +499,29 @@ 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
+ * functions.
+ */
+#define SH_PREFIX tuplehash
+#define SH_KEYTYPE MinimalTuple
+#define SH_KEY key
+#define SH_CONTAINS TupleHashEntryData
+#define SH_SCOPE extern
+#define SH_DECLARE
+#include "lib/simplehash.h"
+
+typedef tuplehash_iterator TupleHashIterator;
typedef struct TupleHashTableData
{
- HTAB *hashtab; /* underlying dynahash table */
+ struct 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,24 +536,19 @@ typedef struct TupleHashTableData
FmgrInfo *cur_eq_funcs; /* equality functions for input vs. table */
} TupleHashTableData;
-typedef HASH_SEQ_STATUS TupleHashIterator;
-
/*
* Use InitTupleHashIterator/TermTupleHashIterator for a read/write scan.
* Use ResetTupleHashIterator if the table can be frozen (in this case no
* 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/test/regress/expected/matview.out b/src/test/regress/expected/matview.out
index e7d0ad1d86..8668b77e9c 100644
--- a/src/test/regress/expected/matview.out
+++ b/src/test/regress/expected/matview.out
@@ -47,9 +47,9 @@ CREATE UNIQUE INDEX mvtest_tm_type ON mvtest_tm (type);
SELECT * FROM mvtest_tm;
type | totamt
------+--------
- y | 12
- z | 11
x | 5
+ z | 11
+ y | 12
(3 rows)
-- create various views
diff --git a/src/test/regress/expected/psql.out b/src/test/regress/expected/psql.out
index 017b79ea9c..464436ab3b 100644
--- a/src/test/regress/expected/psql.out
+++ b/src/test/regress/expected/psql.out
@@ -123,7 +123,7 @@ unicode_header_linestyle single
prepare q as select array_to_string(array_agg(repeat('x',2*n)),E'\n') as "ab
c", array_to_string(array_agg(repeat('y',20-2*n)),E'\n') as "a
-bc" from generate_series(1,10) as n(n) group by n>1 ;
+bc" from generate_series(1,10) as n(n) group by n>1 order by n>1;
\pset linestyle ascii
\pset expanded off
\pset columns 40
diff --git a/src/test/regress/expected/tsrf.out b/src/test/regress/expected/tsrf.out
index d9a5f137dc..07db3a1fc7 100644
--- a/src/test/regress/expected/tsrf.out
+++ b/src/test/regress/expected/tsrf.out
@@ -190,12 +190,12 @@ SELECT SUM(count(*)) OVER(PARTITION BY generate_series(1,3) ORDER BY generate_se
SELECT few.dataa, count(*), min(id), max(id), generate_series(1,3) FROM few GROUP BY few.dataa ORDER BY 5;
dataa | count | min | max | generate_series
-------+-------+-----+-----+-----------------
- b | 1 | 3 | 3 | 1
a | 2 | 1 | 2 | 1
- b | 1 | 3 | 3 | 2
+ b | 1 | 3 | 3 | 1
a | 2 | 1 | 2 | 2
- b | 1 | 3 | 3 | 3
+ b | 1 | 3 | 3 | 2
a | 2 | 1 | 2 | 3
+ b | 1 | 3 | 3 | 3
(6 rows)
-- grouping sets are a bit special, they produce NULLs in columns not actually NULL
diff --git a/src/test/regress/expected/union.out b/src/test/regress/expected/union.out
index 016571bd4a..cd8262c53b 100644
--- a/src/test/regress/expected/union.out
+++ b/src/test/regress/expected/union.out
@@ -32,16 +32,16 @@ SELECT 1 AS two UNION ALL SELECT 1;
SELECT 1 AS three UNION SELECT 2 UNION SELECT 3;
three
-------
- 1
2
+ 1
3
(3 rows)
SELECT 1 AS two UNION SELECT 2 UNION SELECT 2;
two
-----
- 1
2
+ 1
(2 rows)
SELECT 1 AS three UNION SELECT 2 UNION ALL SELECT 2;
@@ -97,9 +97,9 @@ SELECT 1.0::float8 AS two UNION ALL SELECT 1;
SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3;
three
-------
- 1.1
- 2
3
+ 2
+ 1.1
(3 rows)
SELECT 1.1::float8 AS two UNION SELECT 2 UNION SELECT 2.0::float8 ORDER BY 1;
@@ -120,8 +120,8 @@ SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2;
SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2);
two
-----
- 1.1
2
+ 1.1
(2 rows)
--
@@ -263,16 +263,16 @@ ORDER BY 1;
SELECT q2 FROM int8_tbl INTERSECT SELECT q1 FROM int8_tbl;
q2
------------------
- 4567890123456789
123
+ 4567890123456789
(2 rows)
SELECT q2 FROM int8_tbl INTERSECT ALL SELECT q1 FROM int8_tbl;
q2
------------------
+ 123
4567890123456789
4567890123456789
- 123
(3 rows)
SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1;
@@ -305,16 +305,16 @@ SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl;
SELECT q1 FROM int8_tbl EXCEPT ALL SELECT q2 FROM int8_tbl;
q1
------------------
- 4567890123456789
123
+ 4567890123456789
(2 rows)
SELECT q1 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q2 FROM int8_tbl;
q1
------------------
+ 123
4567890123456789
4567890123456789
- 123
(3 rows)
SELECT q1 FROM int8_tbl EXCEPT ALL SELECT q1 FROM int8_tbl FOR NO KEY UPDATE;
@@ -343,8 +343,8 @@ SELECT f1 FROM float8_tbl EXCEPT SELECT f1 FROM int4_tbl ORDER BY 1;
SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl;
q1
-------------------
- 4567890123456789
123
+ 4567890123456789
456
4567890123456789
123
@@ -355,15 +355,15 @@ SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FR
SELECT q1 FROM int8_tbl INTERSECT (((SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl)));
q1
------------------
- 4567890123456789
123
+ 4567890123456789
(2 rows)
(((SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl))) UNION ALL SELECT q2 FROM int8_tbl;
q1
-------------------
- 4567890123456789
123
+ 4567890123456789
456
4567890123456789
123
@@ -419,8 +419,8 @@ HINT: There is a column named "q2" in table "*SELECT* 2", but it cannot be refe
SELECT q1 FROM int8_tbl EXCEPT (((SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1)));
q1
------------------
- 4567890123456789
123
+ 4567890123456789
(2 rows)
--
diff --git a/src/test/regress/expected/with.out b/src/test/regress/expected/with.out
index 137420d9b7..31723be6f1 100644
--- a/src/test/regress/expected/with.out
+++ b/src/test/regress/expected/with.out
@@ -1247,8 +1247,8 @@ WITH outermost(x) AS (
SELECT * FROM outermost;
x
---
- 1
2
+ 1
3
(3 rows)
diff --git a/src/test/regress/sql/psql.sql b/src/test/regress/sql/psql.sql
index 4dc0745f1d..900aa7ee1e 100644
--- a/src/test/regress/sql/psql.sql
+++ b/src/test/regress/sql/psql.sql
@@ -67,7 +67,7 @@ select 'drop table gexec_test', 'select ''2000-01-01''::date as party_over'
prepare q as select array_to_string(array_agg(repeat('x',2*n)),E'\n') as "ab
c", array_to_string(array_agg(repeat('y',20-2*n)),E'\n') as "a
-bc" from generate_series(1,10) as n(n) group by n>1 ;
+bc" from generate_series(1,10) as n(n) group by n>1 order by n>1;
\pset linestyle ascii