Commit 849074f9 authored by Tom Lane's avatar Tom Lane

Revise hash join code so that we can increase the number of batches

on-the-fly, and thereby avoid blowing out memory when the planner has
underestimated the hash table size.  Hash join will now obey the
work_mem limit with some faithfulness.  Per my recent proposal
(hash aggregate part isn't done yet though).
parent 31b6d840
This diff is collapsed.
This diff is collapsed.
...@@ -49,7 +49,7 @@ ...@@ -49,7 +49,7 @@
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/optimizer/path/costsize.c,v 1.137 2004/12/31 22:00:04 pgsql Exp $ * $PostgreSQL: pgsql/src/backend/optimizer/path/costsize.c,v 1.138 2005/03/06 22:15:04 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -1074,9 +1074,9 @@ cost_hashjoin(HashPath *path, Query *root) ...@@ -1074,9 +1074,9 @@ cost_hashjoin(HashPath *path, Query *root)
double innerbytes = relation_byte_size(inner_path_rows, double innerbytes = relation_byte_size(inner_path_rows,
inner_path->parent->width); inner_path->parent->width);
int num_hashclauses = list_length(hashclauses); int num_hashclauses = list_length(hashclauses);
int virtualbuckets; int numbuckets;
int physicalbuckets;
int numbatches; int numbatches;
double virtualbuckets;
Selectivity innerbucketsize; Selectivity innerbucketsize;
Selectivity joininfactor; Selectivity joininfactor;
ListCell *hcl; ListCell *hcl;
...@@ -1123,9 +1123,9 @@ cost_hashjoin(HashPath *path, Query *root) ...@@ -1123,9 +1123,9 @@ cost_hashjoin(HashPath *path, Query *root)
/* Get hash table size that executor would use for inner relation */ /* Get hash table size that executor would use for inner relation */
ExecChooseHashTableSize(inner_path_rows, ExecChooseHashTableSize(inner_path_rows,
inner_path->parent->width, inner_path->parent->width,
&virtualbuckets, &numbuckets,
&physicalbuckets,
&numbatches); &numbatches);
virtualbuckets = (double) numbuckets * (double) numbatches;
/* /*
* Determine bucketsize fraction for inner relation. We use the * Determine bucketsize fraction for inner relation. We use the
...@@ -1196,13 +1196,13 @@ cost_hashjoin(HashPath *path, Query *root) ...@@ -1196,13 +1196,13 @@ cost_hashjoin(HashPath *path, Query *root)
} }
/* /*
* if inner relation is too big then we will need to "batch" the join, * If inner relation is too big then we will need to "batch" the join,
* which implies writing and reading most of the tuples to disk an * which implies writing and reading most of the tuples to disk an
* extra time. Charge one cost unit per page of I/O (correct since it * extra time. Charge one cost unit per page of I/O (correct since it
* should be nice and sequential...). Writing the inner rel counts as * should be nice and sequential...). Writing the inner rel counts as
* startup cost, all the rest as run cost. * startup cost, all the rest as run cost.
*/ */
if (numbatches) if (numbatches > 1)
{ {
double outerpages = page_size(outer_path_rows, double outerpages = page_size(outer_path_rows,
outer_path->parent->width); outer_path->parent->width);
...@@ -1228,7 +1228,9 @@ cost_hashjoin(HashPath *path, Query *root) ...@@ -1228,7 +1228,9 @@ cost_hashjoin(HashPath *path, Query *root)
* The number of tuple comparisons needed is the number of outer * The number of tuple comparisons needed is the number of outer
* tuples times the typical number of tuples in a hash bucket, which * tuples times the typical number of tuples in a hash bucket, which
* is the inner relation size times its bucketsize fraction. At each * is the inner relation size times its bucketsize fraction. At each
* one, we need to evaluate the hashjoin quals. * one, we need to evaluate the hashjoin quals. (Note: charging the
* full qual eval cost at each tuple is pessimistic, since we don't
* evaluate the quals unless the hash values match exactly.)
*/ */
startup_cost += hash_qual_cost.startup; startup_cost += hash_qual_cost.startup;
run_cost += hash_qual_cost.per_tuple * run_cost += hash_qual_cost.per_tuple *
......
...@@ -15,7 +15,7 @@ ...@@ -15,7 +15,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/utils/adt/selfuncs.c,v 1.171 2005/02/01 23:07:58 tgl Exp $ * $PostgreSQL: pgsql/src/backend/utils/adt/selfuncs.c,v 1.172 2005/03/06 22:15:04 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -2154,7 +2154,7 @@ estimate_num_groups(Query *root, List *groupExprs, double input_rows) ...@@ -2154,7 +2154,7 @@ estimate_num_groups(Query *root, List *groupExprs, double input_rows)
* inner rel is well-dispersed (or the alternatives seem much worse). * inner rel is well-dispersed (or the alternatives seem much worse).
*/ */
Selectivity Selectivity
estimate_hash_bucketsize(Query *root, Node *hashkey, int nbuckets) estimate_hash_bucketsize(Query *root, Node *hashkey, double nbuckets)
{ {
VariableStatData vardata; VariableStatData vardata;
double estfract, double estfract,
...@@ -2212,8 +2212,8 @@ estimate_hash_bucketsize(Query *root, Node *hashkey, int nbuckets) ...@@ -2212,8 +2212,8 @@ estimate_hash_bucketsize(Query *root, Node *hashkey, int nbuckets)
* the number of buckets is less than the expected number of distinct * the number of buckets is less than the expected number of distinct
* values; otherwise it is 1/ndistinct. * values; otherwise it is 1/ndistinct.
*/ */
if (ndistinct > (double) nbuckets) if (ndistinct > nbuckets)
estfract = 1.0 / (double) nbuckets; estfract = 1.0 / nbuckets;
else else
estfract = 1.0 / ndistinct; estfract = 1.0 / ndistinct;
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/executor/hashjoin.h,v 1.34 2004/12/31 22:03:29 pgsql Exp $ * $PostgreSQL: pgsql/src/include/executor/hashjoin.h,v 1.35 2005/03/06 22:15:05 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -20,11 +20,12 @@ ...@@ -20,11 +20,12 @@
/* ---------------------------------------------------------------- /* ----------------------------------------------------------------
* hash-join hash table structures * hash-join hash table structures
* *
* Each active hashjoin has a HashJoinTable control block which is * Each active hashjoin has a HashJoinTable control block, which is
* palloc'd in the executor's per-query context. All other storage needed * palloc'd in the executor's per-query context. All other storage needed
* for the hashjoin is kept in private memory contexts, two for each hashjoin. * for the hashjoin is kept in private memory contexts, two for each hashjoin.
* This makes it easy and fast to release the storage when we don't need it * This makes it easy and fast to release the storage when we don't need it
* anymore. * anymore. (Exception: data associated with the temp files lives in the
* per-query context too, since we always call buffile.c in that context.)
* *
* The hashtable contexts are made children of the per-query context, ensuring * The hashtable contexts are made children of the per-query context, ensuring
* that they will be discarded at end of statement even if the join is * that they will be discarded at end of statement even if the join is
...@@ -35,40 +36,64 @@ ...@@ -35,40 +36,64 @@
* "hashCxt", while storage that is only wanted for the current batch is * "hashCxt", while storage that is only wanted for the current batch is
* allocated in the "batchCxt". By resetting the batchCxt at the end of * allocated in the "batchCxt". By resetting the batchCxt at the end of
* each batch, we free all the per-batch storage reliably and without tedium. * each batch, we free all the per-batch storage reliably and without tedium.
*
* During first scan of inner relation, we get its tuples from executor.
* If nbatch > 1 then tuples that don't belong in first batch get saved
* into inner-batch temp files. The same statements apply for the
* first scan of the outer relation, except we write tuples to outer-batch
* temp files. After finishing the first scan, we do the following for
* each remaining batch:
* 1. Read tuples from inner batch file, load into hash buckets.
* 2. Read tuples from outer batch file, match to hash buckets and output.
*
* It is possible to increase nbatch on the fly if the in-memory hash table
* gets too big. The hash-value-to-batch computation is arranged so that this
* can only cause a tuple to go into a later batch than previously thought,
* never into an earlier batch. When we increase nbatch, we rescan the hash
* table and dump out any tuples that are now of a later batch to the correct
* inner batch file. Subsequently, while reading either inner or outer batch
* files, we might find tuples that no longer belong to the current batch;
* if so, we just dump them out to the correct batch file.
* ---------------------------------------------------------------- * ----------------------------------------------------------------
*/ */
/* these are in nodes/execnodes.h: */
/* typedef struct HashJoinTupleData *HashJoinTuple; */
/* typedef struct HashJoinTableData *HashJoinTable; */
typedef struct HashJoinTupleData typedef struct HashJoinTupleData
{ {
struct HashJoinTupleData *next; /* link to next tuple in same struct HashJoinTupleData *next; /* link to next tuple in same bucket */
* bucket */ uint32 hashvalue; /* tuple's hash code */
HeapTupleData htup; /* tuple header */ HeapTupleData htup; /* tuple header */
} HashJoinTupleData; } HashJoinTupleData;
typedef HashJoinTupleData *HashJoinTuple;
typedef struct HashJoinTableData typedef struct HashJoinTableData
{ {
int nbuckets; /* buckets in use during this batch */ int nbuckets; /* # buckets in the in-memory hash table */
int totalbuckets; /* total number of (virtual) buckets */ /* buckets[i] is head of list of tuples in i'th in-memory bucket */
HashJoinTuple *buckets; /* buckets[i] is head of list of tuples */ struct HashJoinTupleData **buckets;
/* buckets array is per-batch storage, as are all the tuples */ /* buckets array is per-batch storage, as are all the tuples */
int nbatch; /* number of batches; 0 means 1-pass join */ int nbatch; /* number of batches */
int curbatch; /* current batch #, or 0 during 1st pass */ int curbatch; /* current batch #; 0 during 1st pass */
int nbatch_original; /* nbatch when we started inner scan */
int nbatch_outstart; /* nbatch when we started outer scan */
bool growEnabled; /* flag to shut off nbatch increases */
bool hashNonEmpty; /* did inner plan produce any rows? */ bool hashNonEmpty; /* did inner plan produce any rows? */
/* /*
* all these arrays are allocated for the life of the hash join, but * These arrays are allocated for the life of the hash join, but
* only if nbatch > 0: * only if nbatch > 1. A file is opened only when we first write
* a tuple into it (otherwise its pointer remains NULL). Note that
* the zero'th array elements never get used, since we will process
* rather than dump out any tuples of batch zero.
*/ */
BufFile **innerBatchFile; /* buffered virtual temp file per batch */ BufFile **innerBatchFile; /* buffered virtual temp file per batch */
BufFile **outerBatchFile; /* buffered virtual temp file per batch */ BufFile **outerBatchFile; /* buffered virtual temp file per batch */
long *outerBatchSize; /* count of tuples in each outer batch
* file */
long *innerBatchSize; /* count of tuples in each inner batch
* file */
/* /*
* Info about the datatype-specific hash functions for the datatypes * Info about the datatype-specific hash functions for the datatypes
...@@ -79,21 +104,11 @@ typedef struct HashJoinTableData ...@@ -79,21 +104,11 @@ typedef struct HashJoinTableData
*/ */
FmgrInfo *hashfunctions; /* lookup data for hash functions */ FmgrInfo *hashfunctions; /* lookup data for hash functions */
/* Size spaceUsed; /* memory space currently used by tuples */
* During 1st scan of inner relation, we get tuples from executor. If Size spaceAllowed; /* upper limit for space used */
* nbatch > 0 then tuples that don't belong in first nbuckets logical
* buckets get dumped into inner-batch temp files. The same statements
* apply for the 1st scan of the outer relation, except we write
* tuples to outer-batch temp files. If nbatch > 0 then we do the
* following for each batch: 1. Read tuples from inner batch file,
* load into hash buckets. 2. Read tuples from outer batch file, match
* to hash buckets and output.
*/
MemoryContext hashCxt; /* context for whole-hash-join storage */ MemoryContext hashCxt; /* context for whole-hash-join storage */
MemoryContext batchCxt; /* context for this-batch-only storage */ MemoryContext batchCxt; /* context for this-batch-only storage */
} HashJoinTableData; } HashJoinTableData;
typedef HashJoinTableData *HashJoinTable;
#endif /* HASHJOIN_H */ #endif /* HASHJOIN_H */
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/executor/nodeHash.h,v 1.35 2004/12/31 22:03:29 pgsql Exp $ * $PostgreSQL: pgsql/src/include/executor/nodeHash.h,v 1.36 2005/03/06 22:15:05 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -25,18 +25,20 @@ extern void ExecReScanHash(HashState *node, ExprContext *exprCtxt); ...@@ -25,18 +25,20 @@ extern void ExecReScanHash(HashState *node, ExprContext *exprCtxt);
extern HashJoinTable ExecHashTableCreate(Hash *node, List *hashOperators); extern HashJoinTable ExecHashTableCreate(Hash *node, List *hashOperators);
extern void ExecHashTableDestroy(HashJoinTable hashtable); extern void ExecHashTableDestroy(HashJoinTable hashtable);
extern void ExecHashTableInsert(HashJoinTable hashtable, extern void ExecHashTableInsert(HashJoinTable hashtable,
ExprContext *econtext, HeapTuple tuple,
List *hashkeys); uint32 hashvalue);
extern int ExecHashGetBucket(HashJoinTable hashtable, extern uint32 ExecHashGetHashValue(HashJoinTable hashtable,
ExprContext *econtext, ExprContext *econtext,
List *hashkeys); List *hashkeys);
extern int ExecHashGetBatch(int bucketno, HashJoinTable hashtable); extern void ExecHashGetBucketAndBatch(HashJoinTable hashtable,
extern HeapTuple ExecScanHashBucket(HashJoinState *hjstate, List *hjclauses, uint32 hashvalue,
ExprContext *econtext); int *bucketno,
extern void ExecHashTableReset(HashJoinTable hashtable, long ntuples); int *batchno);
extern HeapTuple ExecScanHashBucket(HashJoinState *hjstate,
ExprContext *econtext);
extern void ExecHashTableReset(HashJoinTable hashtable);
extern void ExecChooseHashTableSize(double ntuples, int tupwidth, extern void ExecChooseHashTableSize(double ntuples, int tupwidth,
int *virtualbuckets, int *numbuckets,
int *physicalbuckets,
int *numbatches); int *numbatches);
#endif /* NODEHASH_H */ #endif /* NODEHASH_H */
/*------------------------------------------------------------------------- /*-------------------------------------------------------------------------
* *
* nodeHashjoin.h * nodeHashjoin.h
* * prototypes for nodeHashjoin.c
* *
* *
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/executor/nodeHashjoin.h,v 1.28 2004/12/31 22:03:29 pgsql Exp $ * $PostgreSQL: pgsql/src/include/executor/nodeHashjoin.h,v 1.29 2005/03/06 22:15:05 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -15,6 +15,7 @@ ...@@ -15,6 +15,7 @@
#define NODEHASHJOIN_H #define NODEHASHJOIN_H
#include "nodes/execnodes.h" #include "nodes/execnodes.h"
#include "storage/buffile.h"
extern int ExecCountSlotsHashJoin(HashJoin *node); extern int ExecCountSlotsHashJoin(HashJoin *node);
extern HashJoinState *ExecInitHashJoin(HashJoin *node, EState *estate); extern HashJoinState *ExecInitHashJoin(HashJoin *node, EState *estate);
...@@ -22,6 +23,7 @@ extern TupleTableSlot *ExecHashJoin(HashJoinState *node); ...@@ -22,6 +23,7 @@ extern TupleTableSlot *ExecHashJoin(HashJoinState *node);
extern void ExecEndHashJoin(HashJoinState *node); extern void ExecEndHashJoin(HashJoinState *node);
extern void ExecReScanHashJoin(HashJoinState *node, ExprContext *exprCtxt); extern void ExecReScanHashJoin(HashJoinState *node, ExprContext *exprCtxt);
extern void ExecHashJoinSaveTuple(HeapTuple heapTuple, BufFile *file); extern void ExecHashJoinSaveTuple(HeapTuple heapTuple, uint32 hashvalue,
BufFile **fileptr);
#endif /* NODEHASHJOIN_H */ #endif /* NODEHASHJOIN_H */
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/nodes/execnodes.h,v 1.122 2004/12/31 22:03:34 pgsql Exp $ * $PostgreSQL: pgsql/src/include/nodes/execnodes.h,v 1.123 2005/03/06 22:15:05 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -15,7 +15,6 @@ ...@@ -15,7 +15,6 @@
#define EXECNODES_H #define EXECNODES_H
#include "access/relscan.h" #include "access/relscan.h"
#include "executor/hashjoin.h"
#include "executor/tuptable.h" #include "executor/tuptable.h"
#include "fmgr.h" #include "fmgr.h"
#include "nodes/bitmapset.h" #include "nodes/bitmapset.h"
...@@ -985,11 +984,13 @@ typedef struct MergeJoinState ...@@ -985,11 +984,13 @@ typedef struct MergeJoinState
* HashJoinState information * HashJoinState information
* *
* hj_HashTable hash table for the hashjoin * hj_HashTable hash table for the hashjoin
* (NULL if table not built yet)
* hj_CurHashValue hash value for current outer tuple
* hj_CurBucketNo bucket# for current outer tuple * hj_CurBucketNo bucket# for current outer tuple
* hj_CurTuple last inner tuple matched to current outer * hj_CurTuple last inner tuple matched to current outer
* tuple, or NULL if starting search * tuple, or NULL if starting search
* (CurBucketNo and CurTuple are meaningless * (CurHashValue, CurBucketNo and CurTuple are
* unless OuterTupleSlot is nonempty!) * undefined if OuterTupleSlot is empty!)
* hj_OuterHashKeys the outer hash keys in the hashjoin condition * hj_OuterHashKeys the outer hash keys in the hashjoin condition
* hj_InnerHashKeys the inner hash keys in the hashjoin condition * hj_InnerHashKeys the inner hash keys in the hashjoin condition
* hj_HashOperators the join operators in the hashjoin condition * hj_HashOperators the join operators in the hashjoin condition
...@@ -998,14 +999,19 @@ typedef struct MergeJoinState ...@@ -998,14 +999,19 @@ typedef struct MergeJoinState
* hj_NullInnerTupleSlot prepared null tuple for left outer joins * hj_NullInnerTupleSlot prepared null tuple for left outer joins
* hj_NeedNewOuter true if need new outer tuple on next call * hj_NeedNewOuter true if need new outer tuple on next call
* hj_MatchedOuter true if found a join match for current outer * hj_MatchedOuter true if found a join match for current outer
* hj_hashdone true if hash-table-build phase is done
* ---------------- * ----------------
*/ */
/* these structs are defined in executor/hashjoin.h: */
typedef struct HashJoinTupleData *HashJoinTuple;
typedef struct HashJoinTableData *HashJoinTable;
typedef struct HashJoinState typedef struct HashJoinState
{ {
JoinState js; /* its first field is NodeTag */ JoinState js; /* its first field is NodeTag */
List *hashclauses; /* list of ExprState nodes */ List *hashclauses; /* list of ExprState nodes */
HashJoinTable hj_HashTable; HashJoinTable hj_HashTable;
uint32 hj_CurHashValue;
int hj_CurBucketNo; int hj_CurBucketNo;
HashJoinTuple hj_CurTuple; HashJoinTuple hj_CurTuple;
List *hj_OuterHashKeys; /* list of ExprState nodes */ List *hj_OuterHashKeys; /* list of ExprState nodes */
...@@ -1016,7 +1022,6 @@ typedef struct HashJoinState ...@@ -1016,7 +1022,6 @@ typedef struct HashJoinState
TupleTableSlot *hj_NullInnerTupleSlot; TupleTableSlot *hj_NullInnerTupleSlot;
bool hj_NeedNewOuter; bool hj_NeedNewOuter;
bool hj_MatchedOuter; bool hj_MatchedOuter;
bool hj_hashdone;
} HashJoinState; } HashJoinState;
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2005, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/utils/selfuncs.h,v 1.21 2004/12/31 22:03:46 pgsql Exp $ * $PostgreSQL: pgsql/src/include/utils/selfuncs.h,v 1.22 2005/03/06 22:15:05 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -121,7 +121,7 @@ extern double estimate_num_groups(Query *root, List *groupExprs, ...@@ -121,7 +121,7 @@ extern double estimate_num_groups(Query *root, List *groupExprs,
double input_rows); double input_rows);
extern Selectivity estimate_hash_bucketsize(Query *root, Node *hashkey, extern Selectivity estimate_hash_bucketsize(Query *root, Node *hashkey,
int nbuckets); double nbuckets);
extern Datum btcostestimate(PG_FUNCTION_ARGS); extern Datum btcostestimate(PG_FUNCTION_ARGS);
extern Datum rtcostestimate(PG_FUNCTION_ARGS); extern Datum rtcostestimate(PG_FUNCTION_ARGS);
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment