1
0
mirror of https://github.com/postgres/postgres.git synced 2025-10-25 13:17:41 +03:00

Change the implementation of hash join to attempt to avoid unnecessary

work if either of the join relations are empty. The logic is:

(1) if the inner relation's startup cost is less than the outer
    relation's startup cost and this is not an outer join, read
    a single tuple from the inner relation via ExecHash()
      - if NULL, we're done

(2) read a single tuple from the outer relation
      - if NULL, we're done

(3) build the hash table on the inner relation
      - if hash table is empty and this is not an outer join,
        we're done

(4) otherwise, do hash join as usual

The implementation uses the new MultiExecProcNode API, per a
suggestion from Tom: invoking ExecHash() now produces the first
tuple from the Hash node's child node, whereas MultiExecHash()
builds the hash table.

I had to put in a bit of a kludge to get the row count returned
for EXPLAIN ANALYZE to be correct: since ExecHash() is invoked to
return a tuple, and then MultiExecHash() is invoked, we would
return one too many tuples to EXPLAIN ANALYZE. I hacked around
this by just manually detecting this situation and subtracting 1
from the EXPLAIN ANALYZE row count.
This commit is contained in:
Neil Conway
2005-06-15 07:27:44 +00:00
parent 4aaff55359
commit c119c5bd49
3 changed files with 169 additions and 62 deletions

View File

@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/executor/nodeHash.c,v 1.93 2005/04/16 20:07:35 tgl Exp $ * $PostgreSQL: pgsql/src/backend/executor/nodeHash.c,v 1.94 2005/06/15 07:27:44 neilc Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@@ -37,14 +37,22 @@ static void ExecHashIncreaseNumBatches(HashJoinTable hashtable);
/* ---------------------------------------------------------------- /* ----------------------------------------------------------------
* ExecHash * ExecHash
* *
* stub for pro forma compliance * produce the first tuple from our child node (and _only_ the
* first tuple). This is of limited general use -- it does not
* hash its output, and produces only a single tuple. It is
* provided so that hash join can probe the inner hash input to
* determine whether it is empty without needing to build the
* entire hash table first, which is what MultiExecHash() would
* do.
* ---------------------------------------------------------------- * ----------------------------------------------------------------
*/ */
TupleTableSlot * TupleTableSlot *
ExecHash(HashState *node) ExecHash(HashState *node)
{ {
elog(ERROR, "Hash node does not support ExecProcNode call convention"); if (TupIsNull(node->firstTuple))
return NULL; node->firstTuple = ExecProcNode(outerPlanState(node));
return node->firstTuple;
} }
/* ---------------------------------------------------------------- /* ----------------------------------------------------------------
@@ -63,6 +71,7 @@ MultiExecHash(HashState *node)
TupleTableSlot *slot; TupleTableSlot *slot;
ExprContext *econtext; ExprContext *econtext;
uint32 hashvalue; uint32 hashvalue;
bool cleared_first_tuple = false;
/* must provide our own instrumentation support */ /* must provide our own instrumentation support */
if (node->ps.instrument) if (node->ps.instrument)
@@ -85,9 +94,19 @@ MultiExecHash(HashState *node)
*/ */
for (;;) for (;;)
{ {
slot = ExecProcNode(outerNode); /* use and clear the tuple produced by ExecHash(), if any */
if (TupIsNull(slot)) if (!TupIsNull(node->firstTuple))
break; {
slot = node->firstTuple;
node->firstTuple = NULL;
cleared_first_tuple = true;
}
else
{
slot = ExecProcNode(outerNode);
if (TupIsNull(slot))
break;
}
hashtable->totalTuples += 1; hashtable->totalTuples += 1;
/* We have to compute the hash value */ /* We have to compute the hash value */
econtext->ecxt_innertuple = slot; econtext->ecxt_innertuple = slot;
@@ -97,7 +116,19 @@ MultiExecHash(HashState *node)
/* must provide our own instrumentation support */ /* must provide our own instrumentation support */
if (node->ps.instrument) if (node->ps.instrument)
InstrStopNodeMulti(node->ps.instrument, hashtable->totalTuples); {
/*
* XXX: kludge -- if ExecHash() was invoked, we've already
* included the tuple that it produced in the row output count
* for this node, so subtract 1 from the # of hashed tuples.
*/
if (cleared_first_tuple)
InstrStopNodeMulti(node->ps.instrument,
hashtable->totalTuples - 1);
else
InstrStopNodeMulti(node->ps.instrument,
hashtable->totalTuples);
}
/* /*
* We do not return the hash table directly because it's not a subtype * We do not return the hash table directly because it's not a subtype
@@ -130,6 +161,7 @@ ExecInitHash(Hash *node, EState *estate)
hashstate->ps.state = estate; hashstate->ps.state = estate;
hashstate->hashtable = NULL; hashstate->hashtable = NULL;
hashstate->hashkeys = NIL; /* will be set by parent HashJoin */ hashstate->hashkeys = NIL; /* will be set by parent HashJoin */
hashstate->firstTuple = NULL;
/* /*
* Miscellaneous initialization * Miscellaneous initialization
@@ -189,6 +221,8 @@ ExecEndHash(HashState *node)
{ {
PlanState *outerPlan; PlanState *outerPlan;
node->firstTuple = NULL;
/* /*
* free exprcontext * free exprcontext
*/ */
@@ -830,6 +864,8 @@ ExecHashTableReset(HashJoinTable hashtable)
void void
ExecReScanHash(HashState *node, ExprContext *exprCtxt) ExecReScanHash(HashState *node, ExprContext *exprCtxt)
{ {
node->firstTuple = NULL;
/* /*
* if chgParam of subnode is not null then plan will be re-scanned by * if chgParam of subnode is not null then plan will be re-scanned by
* first ExecProcNode. * first ExecProcNode.

View File

@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/executor/nodeHashjoin.c,v 1.71 2005/04/16 20:07:35 tgl Exp $ * $PostgreSQL: pgsql/src/backend/executor/nodeHashjoin.c,v 1.72 2005/06/15 07:27:44 neilc Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@@ -31,7 +31,7 @@ static TupleTableSlot *ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
uint32 *hashvalue, uint32 *hashvalue,
TupleTableSlot *tupleSlot); TupleTableSlot *tupleSlot);
static int ExecHashJoinNewBatch(HashJoinState *hjstate); static int ExecHashJoinNewBatch(HashJoinState *hjstate);
static TupleTableSlot *ExecHashJoinReadOuterPlan(HashJoinState *hjstate);
/* ---------------------------------------------------------------- /* ----------------------------------------------------------------
* ExecHashJoin * ExecHashJoin
@@ -57,8 +57,6 @@ ExecHashJoin(HashJoinState *node)
HashJoinTable hashtable; HashJoinTable hashtable;
HeapTuple curtuple; HeapTuple curtuple;
TupleTableSlot *outerTupleSlot; TupleTableSlot *outerTupleSlot;
uint32 hashvalue;
int batchno;
/* /*
* get information from HashJoin node * get information from HashJoin node
@@ -107,31 +105,68 @@ ExecHashJoin(HashJoinState *node)
*/ */
ResetExprContext(econtext); ResetExprContext(econtext);
/*
* if this is the first call, build the hash table for inner relation
*/
if (hashtable == NULL) if (hashtable == NULL)
{ {
/* /*
* create the hash table * This is the first call to the node. When _either_ of the
* hash join inputs are empty, we want to avoid doing
* unnecessary work (e.g. building the hash table for the
* inner join relation). We therefore read a single tuple from
* both inputs before proceeding further. We choose which
* input to probe first based on the startup cost of the plan
* node.
*
* Note that if we're executing an outer join and the inner
* relation is empty, we still have work to do.
*/
/* Consider probing the inner relation first */
if (hashNode->ps.plan->startup_cost <= outerNode->plan->startup_cost &&
node->js.jointype != JOIN_LEFT)
{
/*
* ExecHash() lets us get a single tuple from the inner
* relation without building the entire hash table
*/
TupleTableSlot *tup = ExecProcNode(&hashNode->ps);
if (TupIsNull(tup))
return NULL;
}
/*
* Before we can check the outer relation, we need to build
* the hash table. This is somewhat a waste of time if the
* outer relation is empty, but it would be awkward to avoid.
*/ */
hashtable = ExecHashTableCreate((Hash *) hashNode->ps.plan, hashtable = ExecHashTableCreate((Hash *) hashNode->ps.plan,
node->hj_HashOperators); node->hj_HashOperators);
node->hj_HashTable = hashtable; node->hj_HashTable = hashtable;
hashNode->hashtable = hashtable;
/* Now check the outer relation */
outerTupleSlot = ExecHashJoinReadOuterPlan(node);
if (TupIsNull(outerTupleSlot))
{
ExecHashTableDestroy(node->hj_HashTable);
node->hj_HashTable = NULL;
return NULL;
}
/* /*
* execute the Hash node, to build the hash table * Okay, we can't avoid it, so execute the Hash node to build
* the hash table
*/ */
hashNode->hashtable = hashtable;
(void) MultiExecProcNode((PlanState *) hashNode); (void) MultiExecProcNode((PlanState *) hashNode);
/* /*
* If the inner relation is completely empty, and we're not doing * If the inner relation is empty but its startup cost was
* an outer join, we can quit without scanning the outer relation. * less than the outer relation's startup cost, we can arrive
* here -- we're done unless this is an outer join
*/ */
if (hashtable->totalTuples == 0 && node->js.jointype != JOIN_LEFT) if (hashtable->totalTuples == 0 && node->js.jointype != JOIN_LEFT)
{ {
ExecHashTableDestroy(hashtable); ExecHashTableDestroy(node->hj_HashTable);
node->hj_HashTable = NULL; node->hj_HashTable = NULL;
return NULL; return NULL;
} }
@@ -153,46 +188,9 @@ ExecHashJoin(HashJoinState *node)
*/ */
if (node->hj_NeedNewOuter) if (node->hj_NeedNewOuter)
{ {
outerTupleSlot = ExecHashJoinOuterGetTuple(outerNode, outerTupleSlot = ExecHashJoinReadOuterPlan(node);
node,
&hashvalue);
if (TupIsNull(outerTupleSlot)) if (TupIsNull(outerTupleSlot))
{ return NULL; /* end of join */
/* end of join */
return NULL;
}
node->js.ps.ps_OuterTupleSlot = outerTupleSlot;
econtext->ecxt_outertuple = outerTupleSlot;
node->hj_NeedNewOuter = false;
node->hj_MatchedOuter = false;
/*
* now we have an outer tuple, find the corresponding bucket
* for this tuple from the hash table
*/
node->hj_CurHashValue = hashvalue;
ExecHashGetBucketAndBatch(hashtable, hashvalue,
&node->hj_CurBucketNo, &batchno);
node->hj_CurTuple = NULL;
/*
* Now we've got an outer tuple and the corresponding hash
* bucket, but this tuple may not belong to the current batch.
*/
if (batchno != hashtable->curbatch)
{
/*
* Need to postpone this outer tuple to a later batch.
* Save it in the corresponding outer-batch file.
*/
Assert(batchno > hashtable->curbatch);
ExecHashJoinSaveTuple(ExecFetchSlotTuple(outerTupleSlot),
hashvalue,
&hashtable->outerBatchFile[batchno]);
node->hj_NeedNewOuter = true;
continue; /* loop around for a new outer tuple */
}
} }
/* /*
@@ -487,6 +485,79 @@ ExecEndHashJoin(HashJoinState *node)
ExecEndNode(innerPlanState(node)); ExecEndNode(innerPlanState(node));
} }
/*
* ExecHashJoinReadOuterPlan
*
* do all the work necessary to produce the next tuple from the
* outer hash join relation that is in the current batch. Returns
* NULL if there are no more tuples in the outer relation.
*/
static TupleTableSlot *
ExecHashJoinReadOuterPlan(HashJoinState *hjstate)
{
PlanState *outerNode;
ExprContext *econtext;
HashJoinTable hashtable;
outerNode = outerPlanState(hjstate);
econtext = hjstate->js.ps.ps_ExprContext;
hashtable = hjstate->hj_HashTable;
for (;;)
{
TupleTableSlot *result;
uint32 hashvalue;
int batchno;
result = ExecHashJoinOuterGetTuple(outerNode,
hjstate,
&hashvalue);
if (TupIsNull(result))
{
/* end of join */
return NULL;
}
hjstate->js.ps.ps_OuterTupleSlot = result;
econtext->ecxt_outertuple = result;
hjstate->hj_NeedNewOuter = false;
hjstate->hj_MatchedOuter = false;
/*
* now we have an outer tuple, find the corresponding bucket
* for this tuple from the hash table
*/
hjstate->hj_CurHashValue = hashvalue;
ExecHashGetBucketAndBatch(hashtable, hashvalue,
&hjstate->hj_CurBucketNo, &batchno);
hjstate->hj_CurTuple = NULL;
/*
* Now we've got an outer tuple and the corresponding hash
* bucket, but this tuple may not belong to the current batch.
*/
if (batchno != hashtable->curbatch)
{
/*
* Need to postpone this outer tuple to a later batch.
* Save it in the corresponding outer-batch file.
*/
Assert(batchno > hashtable->curbatch);
ExecHashJoinSaveTuple(ExecFetchSlotTuple(result),
hashvalue,
&hashtable->outerBatchFile[batchno]);
hjstate->hj_NeedNewOuter = true;
continue; /* Get the next outer tuple */
}
/*
* Otherwise, we have a tuple in the current batch, so we're
* done
*/
return result;
}
}
/* /*
* ExecHashJoinOuterGetTuple * ExecHashJoinOuterGetTuple
* *
@@ -769,7 +840,6 @@ ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
return ExecStoreTuple(heapTuple, tupleSlot, InvalidBuffer, true); return ExecStoreTuple(heapTuple, tupleSlot, InvalidBuffer, true);
} }
void void
ExecReScanHashJoin(HashJoinState *node, ExprContext *exprCtxt) ExecReScanHashJoin(HashJoinState *node, ExprContext *exprCtxt)
{ {

View File

@@ -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.133 2005/05/14 21:29:23 tgl Exp $ * $PostgreSQL: pgsql/src/include/nodes/execnodes.h,v 1.134 2005/06/15 07:27:44 neilc Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
@@ -1218,6 +1218,7 @@ typedef struct HashState
HashJoinTable hashtable; /* hash table for the hashjoin */ HashJoinTable hashtable; /* hash table for the hashjoin */
List *hashkeys; /* list of ExprState nodes */ List *hashkeys; /* list of ExprState nodes */
/* hashkeys is same as parent's hj_InnerHashKeys */ /* hashkeys is same as parent's hj_InnerHashKeys */
TupleTableSlot *firstTuple; /* tuple produced by ExecHash() */
} HashState; } HashState;
/* ---------------- /* ----------------