hash join: probe both inputs first - Mailing list pgsql-patches

From Neil Conway
Subject hash join: probe both inputs first
Date
Msg-id 42AE5F93.3060403@samurai.com
Whole thread Raw
Responses Re: hash join: probe both inputs first  (Neil Conway <neilc@samurai.com>)
List pgsql-patches
Per earlier discussion on pgsql-hackers[1], this patch changes the
implementation of hash join to attempt to avoid redundant 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 existing hash join code will avoid scanning the outer relation if
the inner relation is empty, but it doesn't try to avoid scanning the
outer relation.

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 correct: since ExecHash() is invoked to return a tuple,
and then MultiExecHash() is invoked, we would return one too many tuples
to EXPLAIN ANALYZE. So the current patch hacks around that by just
checking for this situation and subtracting 1 from the row count.
Suggestions for a cleaner approach would be welcome.

-Neil

[1] http://archives.postgresql.org/pgsql-hackers/2005-03/msg01040.php
Index: src/backend/executor/nodeHash.c
===================================================================
RCS file: /var/lib/cvs/pgsql/src/backend/executor/nodeHash.c,v
retrieving revision 1.93
diff -c -r1.93 nodeHash.c
*** src/backend/executor/nodeHash.c    16 Apr 2005 20:07:35 -0000    1.93
--- src/backend/executor/nodeHash.c    14 Jun 2005 04:33:02 -0000
***************
*** 37,50 ****
  /* ----------------------------------------------------------------
   *        ExecHash
   *
!  *        stub for pro forma compliance
   * ----------------------------------------------------------------
   */
  TupleTableSlot *
  ExecHash(HashState *node)
  {
!     elog(ERROR, "Hash node does not support ExecProcNode call convention");
!     return NULL;
  }

  /* ----------------------------------------------------------------
--- 37,58 ----
  /* ----------------------------------------------------------------
   *        ExecHash
   *
!  *        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 *
  ExecHash(HashState *node)
  {
!     if (TupIsNull(node->firstTuple))
!         node->firstTuple = ExecProcNode(outerPlanState(node));
!
!     return node->firstTuple;
  }

  /* ----------------------------------------------------------------
***************
*** 63,68 ****
--- 71,77 ----
      TupleTableSlot *slot;
      ExprContext *econtext;
      uint32        hashvalue;
+     bool        cleared_first_tuple = false;

      /* must provide our own instrumentation support */
      if (node->ps.instrument)
***************
*** 85,93 ****
       */
      for (;;)
      {
!         slot = ExecProcNode(outerNode);
!         if (TupIsNull(slot))
!             break;
          hashtable->totalTuples += 1;
          /* We have to compute the hash value */
          econtext->ecxt_innertuple = slot;
--- 94,112 ----
       */
      for (;;)
      {
!         /* use and clear the tuple produced by ExecHash(), if any */
!         if (!TupIsNull(node->firstTuple))
!         {
!             slot = node->firstTuple;
!             node->firstTuple = NULL;
!             cleared_first_tuple = true;
!         }
!         else
!         {
!             slot = ExecProcNode(outerNode);
!             if (TupIsNull(slot))
!                 break;
!         }
          hashtable->totalTuples += 1;
          /* We have to compute the hash value */
          econtext->ecxt_innertuple = slot;
***************
*** 97,103 ****

      /* must provide our own instrumentation support */
      if (node->ps.instrument)
!         InstrStopNodeMulti(node->ps.instrument, hashtable->totalTuples);

      /*
       * We do not return the hash table directly because it's not a subtype
--- 116,134 ----

      /* must provide our own instrumentation support */
      if (node->ps.instrument)
!     {
!         /*
!          * 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
***************
*** 130,135 ****
--- 161,167 ----
      hashstate->ps.state = estate;
      hashstate->hashtable = NULL;
      hashstate->hashkeys = NIL;    /* will be set by parent HashJoin */
+     hashstate->firstTuple = NULL;

      /*
       * Miscellaneous initialization
***************
*** 189,194 ****
--- 221,228 ----
  {
      PlanState  *outerPlan;

+     node->firstTuple = NULL;
+
      /*
       * free exprcontext
       */
***************
*** 830,835 ****
--- 864,871 ----
  void
  ExecReScanHash(HashState *node, ExprContext *exprCtxt)
  {
+     node->firstTuple = NULL;
+
      /*
       * if chgParam of subnode is not null then plan will be re-scanned by
       * first ExecProcNode.
Index: src/backend/executor/nodeHashjoin.c
===================================================================
RCS file: /var/lib/cvs/pgsql/src/backend/executor/nodeHashjoin.c,v
retrieving revision 1.71
diff -c -r1.71 nodeHashjoin.c
*** src/backend/executor/nodeHashjoin.c    16 Apr 2005 20:07:35 -0000    1.71
--- src/backend/executor/nodeHashjoin.c    14 Jun 2005 04:31:16 -0000
***************
*** 31,37 ****
                            uint32 *hashvalue,
                            TupleTableSlot *tupleSlot);
  static int    ExecHashJoinNewBatch(HashJoinState *hjstate);
!

  /* ----------------------------------------------------------------
   *        ExecHashJoin
--- 31,37 ----
                            uint32 *hashvalue,
                            TupleTableSlot *tupleSlot);
  static int    ExecHashJoinNewBatch(HashJoinState *hjstate);
! static TupleTableSlot *ExecHashJoinReadOuterPlan(HashJoinState *hjstate);

  /* ----------------------------------------------------------------
   *        ExecHashJoin
***************
*** 57,64 ****
      HashJoinTable hashtable;
      HeapTuple    curtuple;
      TupleTableSlot *outerTupleSlot;
-     uint32        hashvalue;
-     int            batchno;

      /*
       * get information from HashJoin node
--- 57,62 ----
***************
*** 107,137 ****
       */
      ResetExprContext(econtext);

-     /*
-      * if this is the first call, build the hash table for inner relation
-      */
      if (hashtable == NULL)
      {
          /*
!          * create the hash table
           */
          hashtable = ExecHashTableCreate((Hash *) hashNode->ps.plan,
                                          node->hj_HashOperators);
          node->hj_HashTable = hashtable;

          /*
!          * execute the Hash node, to build the hash table
           */
-         hashNode->hashtable = hashtable;
          (void) MultiExecProcNode((PlanState *) hashNode);

          /*
!          * If the inner relation is completely empty, and we're not doing
!          * an outer join, we can quit without scanning the outer relation.
           */
          if (hashtable->totalTuples == 0 && node->js.jointype != JOIN_LEFT)
          {
!             ExecHashTableDestroy(hashtable);
              node->hj_HashTable = NULL;
              return NULL;
          }
--- 105,172 ----
       */
      ResetExprContext(econtext);

      if (hashtable == NULL)
      {
          /*
!          * 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,
                                          node->hj_HashOperators);
          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;
+         }

          /*
!          * Okay, we can't avoid it, so execute the Hash node to build
!          * the hash table
           */
          (void) MultiExecProcNode((PlanState *) hashNode);

          /*
!          * If the inner relation is empty but its startup cost was
!          * 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)
          {
!             ExecHashTableDestroy(node->hj_HashTable);
              node->hj_HashTable = NULL;
              return NULL;
          }
***************
*** 153,198 ****
           */
          if (node->hj_NeedNewOuter)
          {
!             outerTupleSlot = ExecHashJoinOuterGetTuple(outerNode,
!                                                        node,
!                                                        &hashvalue);
              if (TupIsNull(outerTupleSlot))
!             {
!                 /* 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 */
!             }
          }

          /*
--- 188,196 ----
           */
          if (node->hj_NeedNewOuter)
          {
!             outerTupleSlot = ExecHashJoinReadOuterPlan(node);
              if (TupIsNull(outerTupleSlot))
!                 return NULL; /* end of join */
          }

          /*
***************
*** 488,493 ****
--- 486,564 ----
  }

  /*
+  * 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
   *
   *        get the next outer tuple for hashjoin: either by
***************
*** 769,775 ****
      return ExecStoreTuple(heapTuple, tupleSlot, InvalidBuffer, true);
  }

-
  void
  ExecReScanHashJoin(HashJoinState *node, ExprContext *exprCtxt)
  {
--- 840,845 ----
Index: src/include/nodes/execnodes.h
===================================================================
RCS file: /var/lib/cvs/pgsql/src/include/nodes/execnodes.h,v
retrieving revision 1.133
diff -c -r1.133 execnodes.h
*** src/include/nodes/execnodes.h    14 May 2005 21:29:23 -0000    1.133
--- src/include/nodes/execnodes.h    14 Jun 2005 03:44:57 -0000
***************
*** 1218,1223 ****
--- 1218,1224 ----
      HashJoinTable hashtable;    /* hash table for the hashjoin */
      List       *hashkeys;        /* list of ExprState nodes */
      /* hashkeys is same as parent's hj_InnerHashKeys */
+     TupleTableSlot *firstTuple;    /* tuple produced by ExecHash() */
  } HashState;

  /* ----------------

pgsql-patches by date:

Previous
From: Bruce Momjian
Date:
Subject: Re: logfile for psql patch update
Next
From: Neil Conway
Date:
Subject: Re: plpgsql raise - parameters can be expressions