1 /*-------------------------------------------------------------------------
4 * Routines to handle hash join nodes
6 * Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
7 * Portions Copyright (c) 1994, Regents of the University of California
11 * src/backend/executor/nodeHashjoin.c
15 * This is based on the "hybrid hash join" algorithm described shortly in the
18 * https://en.wikipedia.org/wiki/Hash_join#Hybrid_hash_join
20 * and in detail in the referenced paper:
22 * "An Adaptive Hash Join Algorithm for Multiuser Environments"
23 * Hansjörg Zeller; Jim Gray (1990). Proceedings of the 16th VLDB conference.
26 * If the inner side tuples of a hash join do not fit in memory, the hash join
27 * can be executed in multiple batches.
29 * If the statistics on the inner side relation are accurate, planner chooses a
30 * multi-batch strategy and estimates the number of batches.
32 * The query executor measures the real size of the hashtable and increases the
33 * number of batches if the hashtable grows too large.
35 * The number of batches is always a power of two, so an increase in the number
36 * of batches doubles it.
38 * Serial hash join measures batch size lazily -- waiting until it is loading a
39 * batch to determine if it will fit in memory. While inserting tuples into the
40 * hashtable, serial hash join will, if that tuple were to exceed work_mem,
41 * dump out the hashtable and reassign them either to other batch files or the
42 * current batch resident in the hashtable.
44 * Parallel hash join, on the other hand, completes all changes to the number
45 * of batches during the build phase. If it increases the number of batches, it
46 * dumps out all the tuples from all batches and reassigns them to entirely new
47 * batch files. Then it checks every batch to ensure it will fit in the space
48 * budget for the query.
50 * In both parallel and serial hash join, the executor currently makes a best
51 * effort. If a particular batch will not fit in memory, it tries doubling the
52 * number of batches. If after a batch increase, there is a batch which
53 * retained all or none of its tuples, the executor disables growth in the
54 * number of batches globally. After growth is disabled, all batches that would
55 * have previously triggered an increase in the number of batches instead
56 * exceed the space allowed.
60 * Hash joins can participate in parallel query execution in several ways. A
61 * parallel-oblivious hash join is one where the node is unaware that it is
62 * part of a parallel plan. In this case, a copy of the inner plan is used to
63 * build a copy of the hash table in every backend, and the outer plan could
64 * either be built from a partial or complete path, so that the results of the
65 * hash join are correspondingly either partial or complete. A parallel-aware
66 * hash join is one that behaves differently, coordinating work between
67 * backends, and appears as Parallel Hash Join in EXPLAIN output. A Parallel
68 * Hash Join always appears with a Parallel Hash node.
70 * Parallel-aware hash joins use the same per-backend state machine to track
71 * progress through the hash join algorithm as parallel-oblivious hash joins.
72 * In a parallel-aware hash join, there is also a shared state machine that
73 * co-operating backends use to synchronize their local state machines and
74 * program counters. The shared state machine is managed with a Barrier IPC
75 * primitive. When all attached participants arrive at a barrier, the phase
76 * advances and all waiting participants are released.
78 * When a participant begins working on a parallel hash join, it must first
79 * figure out how much progress has already been made, because participants
80 * don't wait for each other to begin. For this reason there are switch
81 * statements at key points in the code where we have to synchronize our local
82 * state machine with the phase, and then jump to the correct part of the
83 * algorithm so that we can get started.
85 * One barrier called build_barrier is used to coordinate the hashing phases.
86 * The phase is represented by an integer which begins at zero and increments
87 * one by one, but in the code it is referred to by symbolic names as follows.
88 * An asterisk indicates a phase that is performed by a single arbitrarily
91 * PHJ_BUILD_ELECT -- initial state
92 * PHJ_BUILD_ALLOCATE* -- one sets up the batches and table 0
93 * PHJ_BUILD_HASH_INNER -- all hash the inner rel
94 * PHJ_BUILD_HASH_OUTER -- (multi-batch only) all hash the outer
95 * PHJ_BUILD_RUN -- building done, probing can begin
96 * PHJ_BUILD_FREE* -- all work complete, one frees batches
98 * While in the phase PHJ_BUILD_HASH_INNER a separate pair of barriers may
99 * be used repeatedly as required to coordinate expansions in the number of
100 * batches or buckets. Their phases are as follows:
102 * PHJ_GROW_BATCHES_ELECT -- initial state
103 * PHJ_GROW_BATCHES_REALLOCATE* -- one allocates new batches
104 * PHJ_GROW_BATCHES_REPARTITION -- all repartition
105 * PHJ_GROW_BATCHES_DECIDE* -- one detects skew and cleans up
106 * PHJ_GROW_BATCHES_FINISH -- finished one growth cycle
108 * PHJ_GROW_BUCKETS_ELECT -- initial state
109 * PHJ_GROW_BUCKETS_REALLOCATE* -- one allocates new buckets
110 * PHJ_GROW_BUCKETS_REINSERT -- all insert tuples
112 * If the planner got the number of batches and buckets right, those won't be
113 * necessary, but on the other hand we might finish up needing to expand the
114 * buckets or batches multiple times while hashing the inner relation to stay
115 * within our memory budget and load factor target. For that reason it's a
116 * separate pair of barriers using circular phases.
118 * The PHJ_BUILD_HASH_OUTER phase is required only for multi-batch joins,
119 * because we need to divide the outer relation into batches up front in order
120 * to be able to process batches entirely independently. In contrast, the
121 * parallel-oblivious algorithm simply throws tuples 'forward' to 'later'
122 * batches whenever it encounters them while scanning and probing, which it
123 * can do because it processes batches in serial order.
125 * Once PHJ_BUILD_RUN is reached, backends then split up and process
126 * different batches, or gang up and work together on probing batches if there
127 * aren't enough to go around. For each batch there is a separate barrier
128 * with the following phases:
130 * PHJ_BATCH_ELECT -- initial state
131 * PHJ_BATCH_ALLOCATE* -- one allocates buckets
132 * PHJ_BATCH_LOAD -- all load the hash table from disk
133 * PHJ_BATCH_PROBE -- all probe
134 * PHJ_BATCH_SCAN* -- one does right/right-anti/full unmatched scan
135 * PHJ_BATCH_FREE* -- one frees memory
137 * Batch 0 is a special case, because it starts out in phase
138 * PHJ_BATCH_PROBE; populating batch 0's hash table is done during
139 * PHJ_BUILD_HASH_INNER so we can skip loading.
141 * Initially we try to plan for a single-batch hash join using the combined
142 * hash_mem of all participants to create a large shared hash table. If that
143 * turns out either at planning or execution time to be impossible then we
144 * fall back to regular hash_mem sized hash tables.
146 * To avoid deadlocks, we never wait for any barrier unless it is known that
147 * all other backends attached to it are actively executing the node or have
148 * finished. Practically, that means that we never emit a tuple while attached
149 * to a barrier, unless the barrier has reached a phase that means that no
150 * process will wait on it again. We emit tuples while attached to the build
151 * barrier in phase PHJ_BUILD_RUN, and to a per-batch barrier in phase
152 * PHJ_BATCH_PROBE. These are advanced to PHJ_BUILD_FREE and PHJ_BATCH_SCAN
153 * respectively without waiting, using BarrierArriveAndDetach() and
154 * BarrierArriveAndDetachExceptLast() respectively. The last to detach
155 * receives a different return value so that it knows that it's safe to
156 * clean up. Any straggler process that attaches after that phase is reached
157 * will see that it's too late to participate or access the relevant shared
160 *-------------------------------------------------------------------------
163 #include "postgres.h"
165 #include "access/htup_details.h"
166 #include "access/parallel.h"
167 #include "executor/executor.h"
168 #include "executor/hashjoin.h"
169 #include "executor/nodeHash.h"
170 #include "executor/nodeHashjoin.h"
171 #include "miscadmin.h"
172 #include "utils/lsyscache.h"
173 #include "utils/sharedtuplestore.h"
174 #include "utils/wait_event.h"
178 * States of the ExecHashJoin state machine
180 #define HJ_BUILD_HASHTABLE 1
181 #define HJ_NEED_NEW_OUTER 2
182 #define HJ_SCAN_BUCKET 3
183 #define HJ_FILL_OUTER_TUPLE 4
184 #define HJ_FILL_INNER_TUPLES 5
185 #define HJ_NEED_NEW_BATCH 6
187 /* Returns true if doing null-fill on outer relation */
188 #define HJ_FILL_OUTER(hjstate) ((hjstate)->hj_NullInnerTupleSlot != NULL)
189 /* Returns true if doing null-fill on inner relation */
190 #define HJ_FILL_INNER(hjstate) ((hjstate)->hj_NullOuterTupleSlot != NULL)
192 static TupleTableSlot
*ExecHashJoinOuterGetTuple(PlanState
*outerNode
,
193 HashJoinState
*hjstate
,
195 static TupleTableSlot
*ExecParallelHashJoinOuterGetTuple(PlanState
*outerNode
,
196 HashJoinState
*hjstate
,
198 static TupleTableSlot
*ExecHashJoinGetSavedTuple(HashJoinState
*hjstate
,
201 TupleTableSlot
*tupleSlot
);
202 static bool ExecHashJoinNewBatch(HashJoinState
*hjstate
);
203 static bool ExecParallelHashJoinNewBatch(HashJoinState
*hjstate
);
204 static void ExecParallelHashJoinPartitionOuter(HashJoinState
*hjstate
);
207 /* ----------------------------------------------------------------
210 * This function implements the Hybrid Hashjoin algorithm. It is marked
211 * with an always-inline attribute so that ExecHashJoin() and
212 * ExecParallelHashJoin() can inline it. Compilers that respect the
213 * attribute should create versions specialized for parallel == true and
214 * parallel == false with unnecessary branches removed.
216 * Note: the relation we build hash table on is the "inner"
217 * the other one is "outer".
218 * ----------------------------------------------------------------
220 static pg_attribute_always_inline TupleTableSlot
*
221 ExecHashJoinImpl(PlanState
*pstate
, bool parallel
)
223 HashJoinState
*node
= castNode(HashJoinState
, pstate
);
224 PlanState
*outerNode
;
227 ExprState
*otherqual
;
228 ExprContext
*econtext
;
229 HashJoinTable hashtable
;
230 TupleTableSlot
*outerTupleSlot
;
233 ParallelHashJoinState
*parallel_state
;
236 * get information from HashJoin node
238 joinqual
= node
->js
.joinqual
;
239 otherqual
= node
->js
.ps
.qual
;
240 hashNode
= (HashState
*) innerPlanState(node
);
241 outerNode
= outerPlanState(node
);
242 hashtable
= node
->hj_HashTable
;
243 econtext
= node
->js
.ps
.ps_ExprContext
;
244 parallel_state
= hashNode
->parallel_state
;
247 * Reset per-tuple memory context to free any expression evaluation
248 * storage allocated in the previous tuple cycle.
250 ResetExprContext(econtext
);
253 * run the hash join state machine
258 * It's possible to iterate this loop many times before returning a
259 * tuple, in some pathological cases such as needing to move much of
260 * the current batch to a later batch. So let's check for interrupts
263 CHECK_FOR_INTERRUPTS();
265 switch (node
->hj_JoinState
)
267 case HJ_BUILD_HASHTABLE
:
270 * First time through: build hash table for inner relation.
272 Assert(hashtable
== NULL
);
275 * If the outer relation is completely empty, and it's not
276 * right/right-anti/full join, we can quit without building
277 * the hash table. However, for an inner join it is only a
278 * win to check this when the outer relation's startup cost is
279 * less than the projected cost of building the hash table.
280 * Otherwise it's best to build the hash table first and see
281 * if the inner relation is empty. (When it's a left join, we
282 * should always make this check, since we aren't going to be
283 * able to skip the join on the strength of an empty inner
286 * If we are rescanning the join, we make use of information
287 * gained on the previous scan: don't bother to try the
288 * prefetch if the previous scan found the outer relation
289 * nonempty. This is not 100% reliable since with new
290 * parameters the outer relation might yield different
291 * results, but it's a good heuristic.
293 * The only way to make the check is to try to fetch a tuple
294 * from the outer plan node. If we succeed, we have to stash
295 * it away for later consumption by ExecHashJoinOuterGetTuple.
297 if (HJ_FILL_INNER(node
))
299 /* no chance to not build the hash table */
300 node
->hj_FirstOuterTupleSlot
= NULL
;
305 * The empty-outer optimization is not implemented for
306 * shared hash tables, because no one participant can
307 * determine that there are no outer tuples, and it's not
308 * yet clear that it's worth the synchronization overhead
309 * of reaching consensus to figure that out. So we have
310 * to build the hash table.
312 node
->hj_FirstOuterTupleSlot
= NULL
;
314 else if (HJ_FILL_OUTER(node
) ||
315 (outerNode
->plan
->startup_cost
< hashNode
->ps
.plan
->total_cost
&&
316 !node
->hj_OuterNotEmpty
))
318 node
->hj_FirstOuterTupleSlot
= ExecProcNode(outerNode
);
319 if (TupIsNull(node
->hj_FirstOuterTupleSlot
))
321 node
->hj_OuterNotEmpty
= false;
325 node
->hj_OuterNotEmpty
= true;
328 node
->hj_FirstOuterTupleSlot
= NULL
;
331 * Create the hash table. If using Parallel Hash, then
332 * whoever gets here first will create the hash table and any
333 * later arrivals will merely attach to it.
335 hashtable
= ExecHashTableCreate(hashNode
);
336 node
->hj_HashTable
= hashtable
;
339 * Execute the Hash node, to build the hash table. If using
340 * Parallel Hash, then we'll try to help hashing unless we
343 hashNode
->hashtable
= hashtable
;
344 (void) MultiExecProcNode((PlanState
*) hashNode
);
347 * If the inner relation is completely empty, and we're not
348 * doing a left outer join, we can quit without scanning the
351 if (hashtable
->totalTuples
== 0 && !HJ_FILL_OUTER(node
))
356 * Advance the build barrier to PHJ_BUILD_RUN before
357 * proceeding so we can negotiate resource cleanup.
359 Barrier
*build_barrier
= ¶llel_state
->build_barrier
;
361 while (BarrierPhase(build_barrier
) < PHJ_BUILD_RUN
)
362 BarrierArriveAndWait(build_barrier
, 0);
368 * need to remember whether nbatch has increased since we
369 * began scanning the outer relation
371 hashtable
->nbatch_outstart
= hashtable
->nbatch
;
374 * Reset OuterNotEmpty for scan. (It's OK if we fetched a
375 * tuple above, because ExecHashJoinOuterGetTuple will
376 * immediately set it again.)
378 node
->hj_OuterNotEmpty
= false;
382 Barrier
*build_barrier
;
384 build_barrier
= ¶llel_state
->build_barrier
;
385 Assert(BarrierPhase(build_barrier
) == PHJ_BUILD_HASH_OUTER
||
386 BarrierPhase(build_barrier
) == PHJ_BUILD_RUN
||
387 BarrierPhase(build_barrier
) == PHJ_BUILD_FREE
);
388 if (BarrierPhase(build_barrier
) == PHJ_BUILD_HASH_OUTER
)
391 * If multi-batch, we need to hash the outer relation
394 if (hashtable
->nbatch
> 1)
395 ExecParallelHashJoinPartitionOuter(node
);
396 BarrierArriveAndWait(build_barrier
,
397 WAIT_EVENT_HASH_BUILD_HASH_OUTER
);
399 else if (BarrierPhase(build_barrier
) == PHJ_BUILD_FREE
)
402 * If we attached so late that the job is finished and
403 * the batch state has been freed, we can return
409 /* Each backend should now select a batch to work on. */
410 Assert(BarrierPhase(build_barrier
) == PHJ_BUILD_RUN
);
411 hashtable
->curbatch
= -1;
412 node
->hj_JoinState
= HJ_NEED_NEW_BATCH
;
417 node
->hj_JoinState
= HJ_NEED_NEW_OUTER
;
421 case HJ_NEED_NEW_OUTER
:
424 * We don't have an outer tuple, try to get the next one
428 ExecParallelHashJoinOuterGetTuple(outerNode
, node
,
432 ExecHashJoinOuterGetTuple(outerNode
, node
, &hashvalue
);
434 if (TupIsNull(outerTupleSlot
))
436 /* end of batch, or maybe whole join */
437 if (HJ_FILL_INNER(node
))
439 /* set up to scan for unmatched inner tuples */
443 * Only one process is currently allow to handle
444 * each batch's unmatched tuples, in a parallel
447 if (ExecParallelPrepHashTableForUnmatched(node
))
448 node
->hj_JoinState
= HJ_FILL_INNER_TUPLES
;
450 node
->hj_JoinState
= HJ_NEED_NEW_BATCH
;
454 ExecPrepHashTableForUnmatched(node
);
455 node
->hj_JoinState
= HJ_FILL_INNER_TUPLES
;
459 node
->hj_JoinState
= HJ_NEED_NEW_BATCH
;
463 econtext
->ecxt_outertuple
= outerTupleSlot
;
464 node
->hj_MatchedOuter
= false;
467 * Find the corresponding bucket for this tuple in the main
468 * hash table or skew hash table.
470 node
->hj_CurHashValue
= hashvalue
;
471 ExecHashGetBucketAndBatch(hashtable
, hashvalue
,
472 &node
->hj_CurBucketNo
, &batchno
);
473 node
->hj_CurSkewBucketNo
= ExecHashGetSkewBucket(hashtable
,
475 node
->hj_CurTuple
= NULL
;
478 * The tuple might not belong to the current batch (where
479 * "current batch" includes the skew buckets if any).
481 if (batchno
!= hashtable
->curbatch
&&
482 node
->hj_CurSkewBucketNo
== INVALID_SKEW_BUCKET_NO
)
485 MinimalTuple mintuple
= ExecFetchSlotMinimalTuple(outerTupleSlot
,
489 * Need to postpone this outer tuple to a later batch.
490 * Save it in the corresponding outer-batch file.
492 Assert(parallel_state
== NULL
);
493 Assert(batchno
> hashtable
->curbatch
);
494 ExecHashJoinSaveTuple(mintuple
, hashvalue
,
495 &hashtable
->outerBatchFile
[batchno
],
499 heap_free_minimal_tuple(mintuple
);
501 /* Loop around, staying in HJ_NEED_NEW_OUTER state */
505 /* OK, let's scan the bucket for matches */
506 node
->hj_JoinState
= HJ_SCAN_BUCKET
;
513 * Scan the selected hash bucket for matches to current outer
517 if (!ExecParallelScanHashBucket(node
, econtext
))
519 /* out of matches; check for possible outer-join fill */
520 node
->hj_JoinState
= HJ_FILL_OUTER_TUPLE
;
526 if (!ExecScanHashBucket(node
, econtext
))
528 /* out of matches; check for possible outer-join fill */
529 node
->hj_JoinState
= HJ_FILL_OUTER_TUPLE
;
535 * In a right-semijoin, we only need the first match for each
538 if (node
->js
.jointype
== JOIN_RIGHT_SEMI
&&
539 HeapTupleHeaderHasMatch(HJTUPLE_MINTUPLE(node
->hj_CurTuple
)))
543 * We've got a match, but still need to test non-hashed quals.
544 * ExecScanHashBucket already set up all the state needed to
547 * If we pass the qual, then save state for next call and have
548 * ExecProject form the projection, store it in the tuple
549 * table, and return the slot.
551 * Only the joinquals determine tuple match status, but all
552 * quals must pass to actually return the tuple.
554 if (joinqual
== NULL
|| ExecQual(joinqual
, econtext
))
556 node
->hj_MatchedOuter
= true;
559 * This is really only needed if HJ_FILL_INNER(node) or if
560 * we are in a right-semijoin, but we'll avoid the branch
561 * and just set it always.
563 if (!HeapTupleHeaderHasMatch(HJTUPLE_MINTUPLE(node
->hj_CurTuple
)))
564 HeapTupleHeaderSetMatch(HJTUPLE_MINTUPLE(node
->hj_CurTuple
));
566 /* In an antijoin, we never return a matched tuple */
567 if (node
->js
.jointype
== JOIN_ANTI
)
569 node
->hj_JoinState
= HJ_NEED_NEW_OUTER
;
574 * If we only need to consider the first matching inner
575 * tuple, then advance to next outer tuple after we've
576 * processed this one.
578 if (node
->js
.single_match
)
579 node
->hj_JoinState
= HJ_NEED_NEW_OUTER
;
582 * In a right-antijoin, we never return a matched tuple.
583 * If it's not an inner_unique join, we need to stay on
584 * the current outer tuple to continue scanning the inner
587 if (node
->js
.jointype
== JOIN_RIGHT_ANTI
)
590 if (otherqual
== NULL
|| ExecQual(otherqual
, econtext
))
591 return ExecProject(node
->js
.ps
.ps_ProjInfo
);
593 InstrCountFiltered2(node
, 1);
596 InstrCountFiltered1(node
, 1);
599 case HJ_FILL_OUTER_TUPLE
:
602 * The current outer tuple has run out of matches, so check
603 * whether to emit a dummy outer-join tuple. Whether we emit
604 * one or not, the next state is NEED_NEW_OUTER.
606 node
->hj_JoinState
= HJ_NEED_NEW_OUTER
;
608 if (!node
->hj_MatchedOuter
&&
612 * Generate a fake join tuple with nulls for the inner
613 * tuple, and return it if it passes the non-join quals.
615 econtext
->ecxt_innertuple
= node
->hj_NullInnerTupleSlot
;
617 if (otherqual
== NULL
|| ExecQual(otherqual
, econtext
))
618 return ExecProject(node
->js
.ps
.ps_ProjInfo
);
620 InstrCountFiltered2(node
, 1);
624 case HJ_FILL_INNER_TUPLES
:
627 * We have finished a batch, but we are doing
628 * right/right-anti/full join, so any unmatched inner tuples
629 * in the hashtable have to be emitted before we continue to
632 if (!(parallel
? ExecParallelScanHashTableForUnmatched(node
, econtext
)
633 : ExecScanHashTableForUnmatched(node
, econtext
)))
635 /* no more unmatched tuples */
636 node
->hj_JoinState
= HJ_NEED_NEW_BATCH
;
641 * Generate a fake join tuple with nulls for the outer tuple,
642 * and return it if it passes the non-join quals.
644 econtext
->ecxt_outertuple
= node
->hj_NullOuterTupleSlot
;
646 if (otherqual
== NULL
|| ExecQual(otherqual
, econtext
))
647 return ExecProject(node
->js
.ps
.ps_ProjInfo
);
649 InstrCountFiltered2(node
, 1);
652 case HJ_NEED_NEW_BATCH
:
655 * Try to advance to next batch. Done if there are no more.
659 if (!ExecParallelHashJoinNewBatch(node
))
660 return NULL
; /* end of parallel-aware join */
664 if (!ExecHashJoinNewBatch(node
))
665 return NULL
; /* end of parallel-oblivious join */
667 node
->hj_JoinState
= HJ_NEED_NEW_OUTER
;
671 elog(ERROR
, "unrecognized hashjoin state: %d",
672 (int) node
->hj_JoinState
);
677 /* ----------------------------------------------------------------
680 * Parallel-oblivious version.
681 * ----------------------------------------------------------------
683 static TupleTableSlot
* /* return: a tuple or NULL */
684 ExecHashJoin(PlanState
*pstate
)
687 * On sufficiently smart compilers this should be inlined with the
688 * parallel-aware branches removed.
690 return ExecHashJoinImpl(pstate
, false);
693 /* ----------------------------------------------------------------
694 * ExecParallelHashJoin
696 * Parallel-aware version.
697 * ----------------------------------------------------------------
699 static TupleTableSlot
* /* return: a tuple or NULL */
700 ExecParallelHashJoin(PlanState
*pstate
)
703 * On sufficiently smart compilers this should be inlined with the
704 * parallel-oblivious branches removed.
706 return ExecHashJoinImpl(pstate
, true);
709 /* ----------------------------------------------------------------
712 * Init routine for HashJoin node.
713 * ----------------------------------------------------------------
716 ExecInitHashJoin(HashJoin
*node
, EState
*estate
, int eflags
)
718 HashJoinState
*hjstate
;
723 const TupleTableSlotOps
*ops
;
725 /* check for unsupported flags */
726 Assert(!(eflags
& (EXEC_FLAG_BACKWARD
| EXEC_FLAG_MARK
)));
729 * create state structure
731 hjstate
= makeNode(HashJoinState
);
732 hjstate
->js
.ps
.plan
= (Plan
*) node
;
733 hjstate
->js
.ps
.state
= estate
;
736 * See ExecHashJoinInitializeDSM() and ExecHashJoinInitializeWorker()
737 * where this function may be replaced with a parallel version, if we
738 * managed to launch a parallel query.
740 hjstate
->js
.ps
.ExecProcNode
= ExecHashJoin
;
741 hjstate
->js
.jointype
= node
->join
.jointype
;
744 * Miscellaneous initialization
746 * create expression context for node
748 ExecAssignExprContext(estate
, &hjstate
->js
.ps
);
751 * initialize child nodes
753 * Note: we could suppress the REWIND flag for the inner input, which
754 * would amount to betting that the hash will be a single batch. Not
755 * clear if this would be a win or not.
757 outerNode
= outerPlan(node
);
758 hashNode
= (Hash
*) innerPlan(node
);
760 outerPlanState(hjstate
) = ExecInitNode(outerNode
, estate
, eflags
);
761 outerDesc
= ExecGetResultType(outerPlanState(hjstate
));
762 innerPlanState(hjstate
) = ExecInitNode((Plan
*) hashNode
, estate
, eflags
);
763 innerDesc
= ExecGetResultType(innerPlanState(hjstate
));
766 * Initialize result slot, type and projection.
768 ExecInitResultTupleSlotTL(&hjstate
->js
.ps
, &TTSOpsVirtual
);
769 ExecAssignProjectionInfo(&hjstate
->js
.ps
, NULL
);
772 * tuple table initialization
774 ops
= ExecGetResultSlotOps(outerPlanState(hjstate
), NULL
);
775 hjstate
->hj_OuterTupleSlot
= ExecInitExtraTupleSlot(estate
, outerDesc
,
779 * detect whether we need only consider the first matching inner tuple
781 hjstate
->js
.single_match
= (node
->join
.inner_unique
||
782 node
->join
.jointype
== JOIN_SEMI
);
784 /* set up null tuples for outer joins, if needed */
785 switch (node
->join
.jointype
)
789 case JOIN_RIGHT_SEMI
:
793 hjstate
->hj_NullInnerTupleSlot
=
794 ExecInitNullTupleSlot(estate
, innerDesc
, &TTSOpsVirtual
);
797 case JOIN_RIGHT_ANTI
:
798 hjstate
->hj_NullOuterTupleSlot
=
799 ExecInitNullTupleSlot(estate
, outerDesc
, &TTSOpsVirtual
);
802 hjstate
->hj_NullOuterTupleSlot
=
803 ExecInitNullTupleSlot(estate
, outerDesc
, &TTSOpsVirtual
);
804 hjstate
->hj_NullInnerTupleSlot
=
805 ExecInitNullTupleSlot(estate
, innerDesc
, &TTSOpsVirtual
);
808 elog(ERROR
, "unrecognized join type: %d",
809 (int) node
->join
.jointype
);
813 * now for some voodoo. our temporary tuple slot is actually the result
814 * tuple slot of the Hash node (which is our inner plan). we can do this
815 * because Hash nodes don't return tuples via ExecProcNode() -- instead
816 * the hash join node uses ExecScanHashBucket() to get at the contents of
817 * the hash table. -cim 6/9/91
820 HashState
*hashstate
= (HashState
*) innerPlanState(hjstate
);
821 Hash
*hash
= (Hash
*) hashstate
->ps
.plan
;
822 TupleTableSlot
*slot
= hashstate
->ps
.ps_ResultTupleSlot
;
823 Oid
*outer_hashfuncid
;
824 Oid
*inner_hashfuncid
;
830 hjstate
->hj_HashTupleSlot
= slot
;
833 * Build ExprStates to obtain hash values for either side of the join.
834 * This must be done here as ExecBuildHash32Expr needs to know how to
835 * handle NULL inputs and the required handling of that depends on the
836 * jointype. We don't know the join type in ExecInitHash() and we
837 * must build the ExprStates before ExecHashTableCreate() so we
838 * properly attribute any SubPlans that exist in the hash expressions
839 * to the correct PlanState.
841 nkeys
= list_length(node
->hashoperators
);
843 outer_hashfuncid
= palloc_array(Oid
, nkeys
);
844 inner_hashfuncid
= palloc_array(Oid
, nkeys
);
845 hash_strict
= palloc_array(bool, nkeys
);
848 * Determine the hash function for each side of the join for the given
851 foreach(lc
, node
->hashoperators
)
853 Oid hashop
= lfirst_oid(lc
);
854 int i
= foreach_current_index(lc
);
856 if (!get_op_hash_functions(hashop
,
857 &outer_hashfuncid
[i
],
858 &inner_hashfuncid
[i
]))
860 "could not find hash function for hash operator %u",
862 hash_strict
[i
] = op_strict(hashop
);
866 * Build an ExprState to generate the hash value for the expressions
867 * on the outer of the join. This ExprState must finish generating
868 * the hash value when HJ_FILL_OUTER() is true. Otherwise,
869 * ExecBuildHash32Expr will set up the ExprState to abort early if it
870 * finds a NULL. In these cases, we don't need to store these tuples
871 * in the hash table as the jointype does not require it.
873 hjstate
->hj_OuterHash
=
874 ExecBuildHash32Expr(hjstate
->js
.ps
.ps_ResultTupleDesc
,
875 hjstate
->js
.ps
.resultops
,
877 node
->hashcollations
,
882 HJ_FILL_OUTER(hjstate
));
884 /* As above, but for the inner side of the join */
885 hashstate
->hash_expr
=
886 ExecBuildHash32Expr(hashstate
->ps
.ps_ResultTupleDesc
,
887 hashstate
->ps
.resultops
,
889 node
->hashcollations
,
894 HJ_FILL_INNER(hjstate
));
897 * Set up the skew table hash function while we have a record of the
898 * first key's hash function Oid.
900 if (OidIsValid(hash
->skewTable
))
902 hashstate
->skew_hashfunction
= palloc0(sizeof(FmgrInfo
));
903 hashstate
->skew_collation
= linitial_oid(node
->hashcollations
);
904 fmgr_info(outer_hashfuncid
[0], hashstate
->skew_hashfunction
);
907 /* no need to keep these */
908 pfree(outer_hashfuncid
);
909 pfree(inner_hashfuncid
);
914 * initialize child expressions
916 hjstate
->js
.ps
.qual
=
917 ExecInitQual(node
->join
.plan
.qual
, (PlanState
*) hjstate
);
918 hjstate
->js
.joinqual
=
919 ExecInitQual(node
->join
.joinqual
, (PlanState
*) hjstate
);
920 hjstate
->hashclauses
=
921 ExecInitQual(node
->hashclauses
, (PlanState
*) hjstate
);
924 * initialize hash-specific info
926 hjstate
->hj_HashTable
= NULL
;
927 hjstate
->hj_FirstOuterTupleSlot
= NULL
;
929 hjstate
->hj_CurHashValue
= 0;
930 hjstate
->hj_CurBucketNo
= 0;
931 hjstate
->hj_CurSkewBucketNo
= INVALID_SKEW_BUCKET_NO
;
932 hjstate
->hj_CurTuple
= NULL
;
934 hjstate
->hj_JoinState
= HJ_BUILD_HASHTABLE
;
935 hjstate
->hj_MatchedOuter
= false;
936 hjstate
->hj_OuterNotEmpty
= false;
941 /* ----------------------------------------------------------------
944 * clean up routine for HashJoin node
945 * ----------------------------------------------------------------
948 ExecEndHashJoin(HashJoinState
*node
)
953 if (node
->hj_HashTable
)
955 ExecHashTableDestroy(node
->hj_HashTable
);
956 node
->hj_HashTable
= NULL
;
962 ExecEndNode(outerPlanState(node
));
963 ExecEndNode(innerPlanState(node
));
967 * ExecHashJoinOuterGetTuple
969 * get the next outer tuple for a parallel oblivious hashjoin: either by
970 * executing the outer plan node in the first pass, or from the temp
971 * files for the hashjoin batches.
973 * Returns a null slot if no more outer tuples (within the current batch).
975 * On success, the tuple's hash value is stored at *hashvalue --- this is
976 * either originally computed, or re-read from the temp file.
978 static TupleTableSlot
*
979 ExecHashJoinOuterGetTuple(PlanState
*outerNode
,
980 HashJoinState
*hjstate
,
983 HashJoinTable hashtable
= hjstate
->hj_HashTable
;
984 int curbatch
= hashtable
->curbatch
;
985 TupleTableSlot
*slot
;
987 if (curbatch
== 0) /* if it is the first pass */
990 * Check to see if first outer tuple was already fetched by
991 * ExecHashJoin() and not used yet.
993 slot
= hjstate
->hj_FirstOuterTupleSlot
;
994 if (!TupIsNull(slot
))
995 hjstate
->hj_FirstOuterTupleSlot
= NULL
;
997 slot
= ExecProcNode(outerNode
);
999 while (!TupIsNull(slot
))
1004 * We have to compute the tuple's hash value.
1006 ExprContext
*econtext
= hjstate
->js
.ps
.ps_ExprContext
;
1008 econtext
->ecxt_outertuple
= slot
;
1010 ResetExprContext(econtext
);
1012 *hashvalue
= DatumGetUInt32(ExecEvalExprSwitchContext(hjstate
->hj_OuterHash
,
1018 /* remember outer relation is not empty for possible rescan */
1019 hjstate
->hj_OuterNotEmpty
= true;
1025 * That tuple couldn't match because of a NULL, so discard it and
1026 * continue with the next one.
1028 slot
= ExecProcNode(outerNode
);
1031 else if (curbatch
< hashtable
->nbatch
)
1033 BufFile
*file
= hashtable
->outerBatchFile
[curbatch
];
1036 * In outer-join cases, we could get here even though the batch file
1042 slot
= ExecHashJoinGetSavedTuple(hjstate
,
1045 hjstate
->hj_OuterTupleSlot
);
1046 if (!TupIsNull(slot
))
1050 /* End of this batch */
1055 * ExecHashJoinOuterGetTuple variant for the parallel case.
1057 static TupleTableSlot
*
1058 ExecParallelHashJoinOuterGetTuple(PlanState
*outerNode
,
1059 HashJoinState
*hjstate
,
1062 HashJoinTable hashtable
= hjstate
->hj_HashTable
;
1063 int curbatch
= hashtable
->curbatch
;
1064 TupleTableSlot
*slot
;
1067 * In the Parallel Hash case we only run the outer plan directly for
1068 * single-batch hash joins. Otherwise we have to go to batch files, even
1071 if (curbatch
== 0 && hashtable
->nbatch
== 1)
1073 slot
= ExecProcNode(outerNode
);
1075 while (!TupIsNull(slot
))
1079 ExprContext
*econtext
= hjstate
->js
.ps
.ps_ExprContext
;
1081 econtext
->ecxt_outertuple
= slot
;
1083 ResetExprContext(econtext
);
1085 *hashvalue
= DatumGetUInt32(ExecEvalExprSwitchContext(hjstate
->hj_OuterHash
,
1093 * That tuple couldn't match because of a NULL, so discard it and
1094 * continue with the next one.
1096 slot
= ExecProcNode(outerNode
);
1099 else if (curbatch
< hashtable
->nbatch
)
1103 tuple
= sts_parallel_scan_next(hashtable
->batches
[curbatch
].outer_tuples
,
1107 ExecForceStoreMinimalTuple(tuple
,
1108 hjstate
->hj_OuterTupleSlot
,
1110 slot
= hjstate
->hj_OuterTupleSlot
;
1114 ExecClearTuple(hjstate
->hj_OuterTupleSlot
);
1117 /* End of this batch */
1118 hashtable
->batches
[curbatch
].outer_eof
= true;
1124 * ExecHashJoinNewBatch
1125 * switch to a new hashjoin batch
1127 * Returns true if successful, false if there are no more batches.
1130 ExecHashJoinNewBatch(HashJoinState
*hjstate
)
1132 HashJoinTable hashtable
= hjstate
->hj_HashTable
;
1136 TupleTableSlot
*slot
;
1139 nbatch
= hashtable
->nbatch
;
1140 curbatch
= hashtable
->curbatch
;
1145 * We no longer need the previous outer batch file; close it right
1146 * away to free disk space.
1148 if (hashtable
->outerBatchFile
[curbatch
])
1149 BufFileClose(hashtable
->outerBatchFile
[curbatch
]);
1150 hashtable
->outerBatchFile
[curbatch
] = NULL
;
1152 else /* we just finished the first batch */
1155 * Reset some of the skew optimization state variables, since we no
1156 * longer need to consider skew tuples after the first batch. The
1157 * memory context reset we are about to do will release the skew
1160 hashtable
->skewEnabled
= false;
1161 hashtable
->skewBucket
= NULL
;
1162 hashtable
->skewBucketNums
= NULL
;
1163 hashtable
->nSkewBuckets
= 0;
1164 hashtable
->spaceUsedSkew
= 0;
1168 * We can always skip over any batches that are completely empty on both
1169 * sides. We can sometimes skip over batches that are empty on only one
1170 * side, but there are exceptions:
1172 * 1. In a left/full outer join, we have to process outer batches even if
1173 * the inner batch is empty. Similarly, in a right/right-anti/full outer
1174 * join, we have to process inner batches even if the outer batch is
1177 * 2. If we have increased nbatch since the initial estimate, we have to
1178 * scan inner batches since they might contain tuples that need to be
1179 * reassigned to later inner batches.
1181 * 3. Similarly, if we have increased nbatch since starting the outer
1182 * scan, we have to rescan outer batches in case they contain tuples that
1183 * need to be reassigned.
1186 while (curbatch
< nbatch
&&
1187 (hashtable
->outerBatchFile
[curbatch
] == NULL
||
1188 hashtable
->innerBatchFile
[curbatch
] == NULL
))
1190 if (hashtable
->outerBatchFile
[curbatch
] &&
1191 HJ_FILL_OUTER(hjstate
))
1192 break; /* must process due to rule 1 */
1193 if (hashtable
->innerBatchFile
[curbatch
] &&
1194 HJ_FILL_INNER(hjstate
))
1195 break; /* must process due to rule 1 */
1196 if (hashtable
->innerBatchFile
[curbatch
] &&
1197 nbatch
!= hashtable
->nbatch_original
)
1198 break; /* must process due to rule 2 */
1199 if (hashtable
->outerBatchFile
[curbatch
] &&
1200 nbatch
!= hashtable
->nbatch_outstart
)
1201 break; /* must process due to rule 3 */
1202 /* We can ignore this batch. */
1203 /* Release associated temp files right away. */
1204 if (hashtable
->innerBatchFile
[curbatch
])
1205 BufFileClose(hashtable
->innerBatchFile
[curbatch
]);
1206 hashtable
->innerBatchFile
[curbatch
] = NULL
;
1207 if (hashtable
->outerBatchFile
[curbatch
])
1208 BufFileClose(hashtable
->outerBatchFile
[curbatch
]);
1209 hashtable
->outerBatchFile
[curbatch
] = NULL
;
1213 if (curbatch
>= nbatch
)
1214 return false; /* no more batches */
1216 hashtable
->curbatch
= curbatch
;
1219 * Reload the hash table with the new inner batch (which could be empty)
1221 ExecHashTableReset(hashtable
);
1223 innerFile
= hashtable
->innerBatchFile
[curbatch
];
1225 if (innerFile
!= NULL
)
1227 if (BufFileSeek(innerFile
, 0, 0, SEEK_SET
))
1229 (errcode_for_file_access(),
1230 errmsg("could not rewind hash-join temporary file")));
1232 while ((slot
= ExecHashJoinGetSavedTuple(hjstate
,
1235 hjstate
->hj_HashTupleSlot
)))
1238 * NOTE: some tuples may be sent to future batches. Also, it is
1239 * possible for hashtable->nbatch to be increased here!
1241 ExecHashTableInsert(hashtable
, slot
, hashvalue
);
1245 * after we build the hash table, the inner batch file is no longer
1248 BufFileClose(innerFile
);
1249 hashtable
->innerBatchFile
[curbatch
] = NULL
;
1253 * Rewind outer batch file (if present), so that we can start reading it.
1255 if (hashtable
->outerBatchFile
[curbatch
] != NULL
)
1257 if (BufFileSeek(hashtable
->outerBatchFile
[curbatch
], 0, 0, SEEK_SET
))
1259 (errcode_for_file_access(),
1260 errmsg("could not rewind hash-join temporary file")));
1267 * Choose a batch to work on, and attach to it. Returns true if successful,
1268 * false if there are no more batches.
1271 ExecParallelHashJoinNewBatch(HashJoinState
*hjstate
)
1273 HashJoinTable hashtable
= hjstate
->hj_HashTable
;
1278 * If we were already attached to a batch, remember not to bother checking
1279 * it again, and detach from it (possibly freeing the hash table if we are
1282 if (hashtable
->curbatch
>= 0)
1284 hashtable
->batches
[hashtable
->curbatch
].done
= true;
1285 ExecHashTableDetachBatch(hashtable
);
1289 * Search for a batch that isn't done. We use an atomic counter to start
1290 * our search at a different batch in every participant when there are
1291 * more batches than participants.
1293 batchno
= start_batchno
=
1294 pg_atomic_fetch_add_u32(&hashtable
->parallel_state
->distributor
, 1) %
1300 TupleTableSlot
*slot
;
1302 if (!hashtable
->batches
[batchno
].done
)
1304 SharedTuplestoreAccessor
*inner_tuples
;
1305 Barrier
*batch_barrier
=
1306 &hashtable
->batches
[batchno
].shared
->batch_barrier
;
1308 switch (BarrierAttach(batch_barrier
))
1310 case PHJ_BATCH_ELECT
:
1312 /* One backend allocates the hash table. */
1313 if (BarrierArriveAndWait(batch_barrier
,
1314 WAIT_EVENT_HASH_BATCH_ELECT
))
1315 ExecParallelHashTableAlloc(hashtable
, batchno
);
1318 case PHJ_BATCH_ALLOCATE
:
1319 /* Wait for allocation to complete. */
1320 BarrierArriveAndWait(batch_barrier
,
1321 WAIT_EVENT_HASH_BATCH_ALLOCATE
);
1324 case PHJ_BATCH_LOAD
:
1325 /* Start (or join in) loading tuples. */
1326 ExecParallelHashTableSetCurrentBatch(hashtable
, batchno
);
1327 inner_tuples
= hashtable
->batches
[batchno
].inner_tuples
;
1328 sts_begin_parallel_scan(inner_tuples
);
1329 while ((tuple
= sts_parallel_scan_next(inner_tuples
,
1332 ExecForceStoreMinimalTuple(tuple
,
1333 hjstate
->hj_HashTupleSlot
,
1335 slot
= hjstate
->hj_HashTupleSlot
;
1336 ExecParallelHashTableInsertCurrentBatch(hashtable
, slot
,
1339 sts_end_parallel_scan(inner_tuples
);
1340 BarrierArriveAndWait(batch_barrier
,
1341 WAIT_EVENT_HASH_BATCH_LOAD
);
1344 case PHJ_BATCH_PROBE
:
1347 * This batch is ready to probe. Return control to
1348 * caller. We stay attached to batch_barrier so that the
1349 * hash table stays alive until everyone's finished
1350 * probing it, but no participant is allowed to wait at
1351 * this barrier again (or else a deadlock could occur).
1352 * All attached participants must eventually detach from
1353 * the barrier and one worker must advance the phase so
1354 * that the final phase is reached.
1356 ExecParallelHashTableSetCurrentBatch(hashtable
, batchno
);
1357 sts_begin_parallel_scan(hashtable
->batches
[batchno
].outer_tuples
);
1360 case PHJ_BATCH_SCAN
:
1363 * In principle, we could help scan for unmatched tuples,
1364 * since that phase is already underway (the thing we
1365 * can't do under current deadlock-avoidance rules is wait
1366 * for others to arrive at PHJ_BATCH_SCAN, because
1367 * PHJ_BATCH_PROBE emits tuples, but in this case we just
1368 * got here without waiting). That is not yet done. For
1369 * now, we just detach and go around again. We have to
1370 * use ExecHashTableDetachBatch() because there's a small
1371 * chance we'll be the last to detach, and then we're
1372 * responsible for freeing memory.
1374 ExecParallelHashTableSetCurrentBatch(hashtable
, batchno
);
1375 hashtable
->batches
[batchno
].done
= true;
1376 ExecHashTableDetachBatch(hashtable
);
1379 case PHJ_BATCH_FREE
:
1382 * Already done. Detach and go around again (if any
1385 BarrierDetach(batch_barrier
);
1386 hashtable
->batches
[batchno
].done
= true;
1387 hashtable
->curbatch
= -1;
1391 elog(ERROR
, "unexpected batch phase %d",
1392 BarrierPhase(batch_barrier
));
1395 batchno
= (batchno
+ 1) % hashtable
->nbatch
;
1396 } while (batchno
!= start_batchno
);
1402 * ExecHashJoinSaveTuple
1403 * save a tuple to a batch file.
1405 * The data recorded in the file for each tuple is its hash value,
1406 * then the tuple in MinimalTuple format.
1408 * fileptr points to a batch file in one of the hashtable arrays.
1410 * The batch files (and their buffers) are allocated in the spill context
1411 * created for the hashtable.
1414 ExecHashJoinSaveTuple(MinimalTuple tuple
, uint32 hashvalue
,
1415 BufFile
**fileptr
, HashJoinTable hashtable
)
1417 BufFile
*file
= *fileptr
;
1420 * The batch file is lazily created. If this is the first tuple written to
1421 * this batch, the batch file is created and its buffer is allocated in
1422 * the spillCxt context, NOT in the batchCxt.
1424 * During the build phase, buffered files are created for inner batches.
1425 * Each batch's buffered file is closed (and its buffer freed) after the
1426 * batch is loaded into memory during the outer side scan. Therefore, it
1427 * is necessary to allocate the batch file buffer in a memory context
1428 * which outlives the batch itself.
1430 * Also, we use spillCxt instead of hashCxt for a better accounting of the
1431 * spilling memory consumption.
1435 MemoryContext oldctx
= MemoryContextSwitchTo(hashtable
->spillCxt
);
1437 file
= BufFileCreateTemp(false);
1440 MemoryContextSwitchTo(oldctx
);
1443 BufFileWrite(file
, &hashvalue
, sizeof(uint32
));
1444 BufFileWrite(file
, tuple
, tuple
->t_len
);
1448 * ExecHashJoinGetSavedTuple
1449 * read the next tuple from a batch file. Return NULL if no more.
1451 * On success, *hashvalue is set to the tuple's hash value, and the tuple
1452 * itself is stored in the given slot.
1454 static TupleTableSlot
*
1455 ExecHashJoinGetSavedTuple(HashJoinState
*hjstate
,
1458 TupleTableSlot
*tupleSlot
)
1465 * We check for interrupts here because this is typically taken as an
1466 * alternative code path to an ExecProcNode() call, which would include
1469 CHECK_FOR_INTERRUPTS();
1472 * Since both the hash value and the MinimalTuple length word are uint32,
1473 * we can read them both in one BufFileRead() call without any type
1476 nread
= BufFileReadMaybeEOF(file
, header
, sizeof(header
), true);
1477 if (nread
== 0) /* end of file */
1479 ExecClearTuple(tupleSlot
);
1482 *hashvalue
= header
[0];
1483 tuple
= (MinimalTuple
) palloc(header
[1]);
1484 tuple
->t_len
= header
[1];
1485 BufFileReadExact(file
,
1486 (char *) tuple
+ sizeof(uint32
),
1487 header
[1] - sizeof(uint32
));
1488 ExecForceStoreMinimalTuple(tuple
, tupleSlot
, true);
1494 ExecReScanHashJoin(HashJoinState
*node
)
1496 PlanState
*outerPlan
= outerPlanState(node
);
1497 PlanState
*innerPlan
= innerPlanState(node
);
1500 * In a multi-batch join, we currently have to do rescans the hard way,
1501 * primarily because batch temp files may have already been released. But
1502 * if it's a single-batch join, and there is no parameter change for the
1503 * inner subnode, then we can just re-use the existing hash table without
1506 if (node
->hj_HashTable
!= NULL
)
1508 if (node
->hj_HashTable
->nbatch
== 1 &&
1509 innerPlan
->chgParam
== NULL
)
1512 * Okay to reuse the hash table; needn't rescan inner, either.
1514 * However, if it's a right/right-anti/full join, we'd better
1515 * reset the inner-tuple match flags contained in the table.
1517 if (HJ_FILL_INNER(node
))
1518 ExecHashTableResetMatchFlags(node
->hj_HashTable
);
1521 * Also, we need to reset our state about the emptiness of the
1522 * outer relation, so that the new scan of the outer will update
1523 * it correctly if it turns out to be empty this time. (There's no
1524 * harm in clearing it now because ExecHashJoin won't need the
1525 * info. In the other cases, where the hash table doesn't exist
1526 * or we are destroying it, we leave this state alone because
1527 * ExecHashJoin will need it the first time through.)
1529 node
->hj_OuterNotEmpty
= false;
1531 /* ExecHashJoin can skip the BUILD_HASHTABLE step */
1532 node
->hj_JoinState
= HJ_NEED_NEW_OUTER
;
1536 /* must destroy and rebuild hash table */
1537 HashState
*hashNode
= castNode(HashState
, innerPlan
);
1539 Assert(hashNode
->hashtable
== node
->hj_HashTable
);
1540 /* accumulate stats from old hash table, if wanted */
1541 /* (this should match ExecShutdownHash) */
1542 if (hashNode
->ps
.instrument
&& !hashNode
->hinstrument
)
1543 hashNode
->hinstrument
= (HashInstrumentation
*)
1544 palloc0(sizeof(HashInstrumentation
));
1545 if (hashNode
->hinstrument
)
1546 ExecHashAccumInstrumentation(hashNode
->hinstrument
,
1547 hashNode
->hashtable
);
1548 /* for safety, be sure to clear child plan node's pointer too */
1549 hashNode
->hashtable
= NULL
;
1551 ExecHashTableDestroy(node
->hj_HashTable
);
1552 node
->hj_HashTable
= NULL
;
1553 node
->hj_JoinState
= HJ_BUILD_HASHTABLE
;
1556 * if chgParam of subnode is not null then plan will be re-scanned
1557 * by first ExecProcNode.
1559 if (innerPlan
->chgParam
== NULL
)
1560 ExecReScan(innerPlan
);
1564 /* Always reset intra-tuple state */
1565 node
->hj_CurHashValue
= 0;
1566 node
->hj_CurBucketNo
= 0;
1567 node
->hj_CurSkewBucketNo
= INVALID_SKEW_BUCKET_NO
;
1568 node
->hj_CurTuple
= NULL
;
1570 node
->hj_MatchedOuter
= false;
1571 node
->hj_FirstOuterTupleSlot
= NULL
;
1574 * if chgParam of subnode is not null then plan will be re-scanned by
1575 * first ExecProcNode.
1577 if (outerPlan
->chgParam
== NULL
)
1578 ExecReScan(outerPlan
);
1582 ExecShutdownHashJoin(HashJoinState
*node
)
1584 if (node
->hj_HashTable
)
1587 * Detach from shared state before DSM memory goes away. This makes
1588 * sure that we don't have any pointers into DSM memory by the time
1589 * ExecEndHashJoin runs.
1591 ExecHashTableDetachBatch(node
->hj_HashTable
);
1592 ExecHashTableDetach(node
->hj_HashTable
);
1597 ExecParallelHashJoinPartitionOuter(HashJoinState
*hjstate
)
1599 PlanState
*outerState
= outerPlanState(hjstate
);
1600 ExprContext
*econtext
= hjstate
->js
.ps
.ps_ExprContext
;
1601 HashJoinTable hashtable
= hjstate
->hj_HashTable
;
1602 TupleTableSlot
*slot
;
1606 Assert(hjstate
->hj_FirstOuterTupleSlot
== NULL
);
1608 /* Execute outer plan, writing all tuples to shared tuplestores. */
1613 slot
= ExecProcNode(outerState
);
1614 if (TupIsNull(slot
))
1616 econtext
->ecxt_outertuple
= slot
;
1618 ResetExprContext(econtext
);
1620 hashvalue
= DatumGetUInt32(ExecEvalExprSwitchContext(hjstate
->hj_OuterHash
,
1629 MinimalTuple mintup
= ExecFetchSlotMinimalTuple(slot
, &shouldFree
);
1631 ExecHashGetBucketAndBatch(hashtable
, hashvalue
, &bucketno
,
1633 sts_puttuple(hashtable
->batches
[batchno
].outer_tuples
,
1634 &hashvalue
, mintup
);
1637 heap_free_minimal_tuple(mintup
);
1639 CHECK_FOR_INTERRUPTS();
1642 /* Make sure all outer partitions are readable by any backend. */
1643 for (i
= 0; i
< hashtable
->nbatch
; ++i
)
1644 sts_end_write(hashtable
->batches
[i
].outer_tuples
);
1648 ExecHashJoinEstimate(HashJoinState
*state
, ParallelContext
*pcxt
)
1650 shm_toc_estimate_chunk(&pcxt
->estimator
, sizeof(ParallelHashJoinState
));
1651 shm_toc_estimate_keys(&pcxt
->estimator
, 1);
1655 ExecHashJoinInitializeDSM(HashJoinState
*state
, ParallelContext
*pcxt
)
1657 int plan_node_id
= state
->js
.ps
.plan
->plan_node_id
;
1658 HashState
*hashNode
;
1659 ParallelHashJoinState
*pstate
;
1662 * Disable shared hash table mode if we failed to create a real DSM
1663 * segment, because that means that we don't have a DSA area to work with.
1665 if (pcxt
->seg
== NULL
)
1668 ExecSetExecProcNode(&state
->js
.ps
, ExecParallelHashJoin
);
1671 * Set up the state needed to coordinate access to the shared hash
1672 * table(s), using the plan node ID as the toc key.
1674 pstate
= shm_toc_allocate(pcxt
->toc
, sizeof(ParallelHashJoinState
));
1675 shm_toc_insert(pcxt
->toc
, plan_node_id
, pstate
);
1678 * Set up the shared hash join state with no batches initially.
1679 * ExecHashTableCreate() will prepare at least one later and set nbatch
1680 * and space_allowed.
1683 pstate
->space_allowed
= 0;
1684 pstate
->batches
= InvalidDsaPointer
;
1685 pstate
->old_batches
= InvalidDsaPointer
;
1686 pstate
->nbuckets
= 0;
1687 pstate
->growth
= PHJ_GROWTH_OK
;
1688 pstate
->chunk_work_queue
= InvalidDsaPointer
;
1689 pg_atomic_init_u32(&pstate
->distributor
, 0);
1690 pstate
->nparticipants
= pcxt
->nworkers
+ 1;
1691 pstate
->total_tuples
= 0;
1692 LWLockInitialize(&pstate
->lock
,
1693 LWTRANCHE_PARALLEL_HASH_JOIN
);
1694 BarrierInit(&pstate
->build_barrier
, 0);
1695 BarrierInit(&pstate
->grow_batches_barrier
, 0);
1696 BarrierInit(&pstate
->grow_buckets_barrier
, 0);
1698 /* Set up the space we'll use for shared temporary files. */
1699 SharedFileSetInit(&pstate
->fileset
, pcxt
->seg
);
1701 /* Initialize the shared state in the hash node. */
1702 hashNode
= (HashState
*) innerPlanState(state
);
1703 hashNode
->parallel_state
= pstate
;
1706 /* ----------------------------------------------------------------
1707 * ExecHashJoinReInitializeDSM
1709 * Reset shared state before beginning a fresh scan.
1710 * ----------------------------------------------------------------
1713 ExecHashJoinReInitializeDSM(HashJoinState
*state
, ParallelContext
*pcxt
)
1715 int plan_node_id
= state
->js
.ps
.plan
->plan_node_id
;
1716 ParallelHashJoinState
*pstate
;
1718 /* Nothing to do if we failed to create a DSM segment. */
1719 if (pcxt
->seg
== NULL
)
1722 pstate
= shm_toc_lookup(pcxt
->toc
, plan_node_id
, false);
1725 * It would be possible to reuse the shared hash table in single-batch
1726 * cases by resetting and then fast-forwarding build_barrier to
1727 * PHJ_BUILD_FREE and batch 0's batch_barrier to PHJ_BATCH_PROBE, but
1728 * currently shared hash tables are already freed by now (by the last
1729 * participant to detach from the batch). We could consider keeping it
1730 * around for single-batch joins. We'd also need to adjust
1731 * finalize_plan() so that it doesn't record a dummy dependency for
1732 * Parallel Hash nodes, preventing the rescan optimization. For now we
1736 /* Detach, freeing any remaining shared memory. */
1737 if (state
->hj_HashTable
!= NULL
)
1739 ExecHashTableDetachBatch(state
->hj_HashTable
);
1740 ExecHashTableDetach(state
->hj_HashTable
);
1743 /* Clear any shared batch files. */
1744 SharedFileSetDeleteAll(&pstate
->fileset
);
1746 /* Reset build_barrier to PHJ_BUILD_ELECT so we can go around again. */
1747 BarrierInit(&pstate
->build_barrier
, 0);
1751 ExecHashJoinInitializeWorker(HashJoinState
*state
,
1752 ParallelWorkerContext
*pwcxt
)
1754 HashState
*hashNode
;
1755 int plan_node_id
= state
->js
.ps
.plan
->plan_node_id
;
1756 ParallelHashJoinState
*pstate
=
1757 shm_toc_lookup(pwcxt
->toc
, plan_node_id
, false);
1759 /* Attach to the space for shared temporary files. */
1760 SharedFileSetAttach(&pstate
->fileset
, pwcxt
->seg
);
1762 /* Attach to the shared state in the hash node. */
1763 hashNode
= (HashState
*) innerPlanState(state
);
1764 hashNode
->parallel_state
= pstate
;
1766 ExecSetExecProcNode(&state
->js
.ps
, ExecParallelHashJoin
);