Add parallel-aware hash joins.

Introduce parallel-aware hash joins that appear in EXPLAIN plans as Parallel
Hash Join with Parallel Hash.  While hash joins could already appear in
parallel queries, they were previously always parallel-oblivious and had a
partial subplan only on the outer side, meaning that the work of the inner
subplan was duplicated in every worker.

After this commit, the planner will consider using a partial subplan on the
inner side too, using the Parallel Hash node to divide the work over the
available CPU cores and combine its results in shared memory.  If the join
needs to be split into multiple batches in order to respect work_mem, then
workers process different batches as much as possible and then work together
on the remaining batches.

The advantages of a parallel-aware hash join over a parallel-oblivious hash
join used in a parallel query are that it:

 * avoids wasting memory on duplicated hash tables
 * avoids wasting disk space on duplicated batch files
 * divides the work of building the hash table over the CPUs

One disadvantage is that there is some communication between the participating
CPUs which might outweigh the benefits of parallelism in the case of small
hash tables.  This is avoided by the planner's existing reluctance to supply
partial plans for small scans, but it may be necessary to estimate
synchronization costs in future if that situation changes.  Another is that
outer batch 0 must be written to disk if multiple batches are required.

A potential future advantage of parallel-aware hash joins is that right and
full outer joins could be supported, since there is a single set of matched
bits for each hashtable, but that is not yet implemented.

A new GUC enable_parallel_hash is defined to control the feature, defaulting
to on.

Author: Thomas Munro
Reviewed-By: Andres Freund, Robert Haas
Tested-By: Rafia Sabih, Prabhat Sahu
Discussion:
    https://postgr.es/m/CAEepm=2W=cOkiZxcg6qiFQP-dHUe09aqTrEMM7yJDrHMhDv_RA@mail.gmail.com
    https://postgr.es/m/CAEepm=37HKyJ4U6XOLi=JgfSHM3o6B-GaeO-6hkOmneTDkH+Uw@mail.gmail.com
This commit is contained in:
Andres Freund 2017-12-20 23:39:21 -08:00
parent f94eec490b
commit 1804284042
30 changed files with 3087 additions and 112 deletions

View File

@ -3647,6 +3647,21 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
<varlistentry id="guc-enable-parallel-hash" xreflabel="enable_parallel_hash">
<term><varname>enable_parallel_hash</varname> (<type>boolean</type>)
<indexterm>
<primary><varname>enable_parallel_hash</varname> configuration parameter</primary>
</indexterm>
</term>
<listitem>
<para>
Enables or disables the query planner's use of hash-join plan
types with parallel hash. Has no effect if hash-join plans are not
also enabled. The default is <literal>on</literal>.
</para>
</listitem>
</varlistentry>
<varlistentry id="guc-enable-partition-wise-join" xreflabel="enable_partition_wise_join">
<term><varname>enable_partition_wise_join</varname> (<type>boolean</type>)
<indexterm>

View File

@ -1263,7 +1263,7 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry>Waiting in an extension.</entry>
</row>
<row>
<entry morerows="17"><literal>IPC</literal></entry>
<entry morerows="32"><literal>IPC</literal></entry>
<entry><literal>BgWorkerShutdown</literal></entry>
<entry>Waiting for background worker to shut down.</entry>
</row>
@ -1279,6 +1279,66 @@ postgres 27093 0.0 0.0 30096 2752 ? Ss 11:34 0:00 postgres: ser
<entry><literal>ExecuteGather</literal></entry>
<entry>Waiting for activity from child process when executing <literal>Gather</literal> node.</entry>
</row>
<row>
<entry><literal>Hash/Batch/Allocating</literal></entry>
<entry>Waiting for an elected Parallel Hash participant to allocate a hash table.</entry>
</row>
<row>
<entry><literal>Hash/Batch/Electing</literal></entry>
<entry>Electing a Parallel Hash participant to allocate a hash table.</entry>
</row>
<row>
<entry><literal>Hash/Batch/Loading</literal></entry>
<entry>Waiting for other Parallel Hash participants to finish loading a hash table.</entry>
</row>
<row>
<entry><literal>Hash/Build/Allocating</literal></entry>
<entry>Waiting for an elected Parallel Hash participant to allocate the initial hash table.</entry>
</row>
<row>
<entry><literal>Hash/Build/Electing</literal></entry>
<entry>Electing a Parallel Hash participant to allocate the initial hash table.</entry>
</row>
<row>
<entry><literal>Hash/Build/HashingInner</literal></entry>
<entry>Waiting for other Parallel Hash participants to finish hashing the inner relation.</entry>
</row>
<row>
<entry><literal>Hash/Build/HashingOuter</literal></entry>
<entry>Waiting for other Parallel Hash participants to finish partitioning the outer relation.</entry>
</row>
<row>
<entry><literal>Hash/GrowBatches/Allocating</literal></entry>
<entry>Waiting for an elected Parallel Hash participant to allocate more batches.</entry>
</row>
<row>
<entry><literal>Hash/GrowBatches/Deciding</literal></entry>
<entry>Electing a Parallel Hash participant to decide on future batch growth.</entry>
</row>
<row>
<entry><literal>Hash/GrowBatches/Electing</literal></entry>
<entry>Electing a Parallel Hash participant to allocate more batches.</entry>
</row>
<row>
<entry><literal>Hash/GrowBatches/Finishing</literal></entry>
<entry>Waiting for an elected Parallel Hash participant to decide on future batch growth.</entry>
</row>
<row>
<entry><literal>Hash/GrowBatches/Repartitioning</literal></entry>
<entry>Waiting for other Parallel Hash participants to finishing repartitioning.</entry>
</row>
<row>
<entry><literal>Hash/GrowBuckets/Allocating</literal></entry>
<entry>Waiting for an elected Parallel Hash participant to finish allocating more buckets.</entry>
</row>
<row>
<entry><literal>Hash/GrowBuckets/Electing</literal></entry>
<entry>Electing a Parallel Hash participant to allocate more buckets.</entry>
</row>
<row>
<entry><literal>Hash/GrowBuckets/Reinserting</literal></entry>
<entry>Waiting for other Parallel Hash participants to finish inserting tuples into new buckets.</entry>
</row>
<row>
<entry><literal>LogicalSyncData</literal></entry>
<entry>Waiting for logical replication remote server to send data for initial table synchronization.</entry>

View File

@ -31,6 +31,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "executor/nodeIndexscan.h"
#include "executor/nodeIndexonlyscan.h"
#include "executor/nodeSeqscan.h"
@ -266,6 +267,11 @@ ExecParallelEstimate(PlanState *planstate, ExecParallelEstimateContext *e)
ExecBitmapHeapEstimate((BitmapHeapScanState *) planstate,
e->pcxt);
break;
case T_HashJoinState:
if (planstate->plan->parallel_aware)
ExecHashJoinEstimate((HashJoinState *) planstate,
e->pcxt);
break;
case T_HashState:
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecHashEstimate((HashState *) planstate, e->pcxt);
@ -474,6 +480,11 @@ ExecParallelInitializeDSM(PlanState *planstate,
ExecBitmapHeapInitializeDSM((BitmapHeapScanState *) planstate,
d->pcxt);
break;
case T_HashJoinState:
if (planstate->plan->parallel_aware)
ExecHashJoinInitializeDSM((HashJoinState *) planstate,
d->pcxt);
break;
case T_HashState:
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecHashInitializeDSM((HashState *) planstate, d->pcxt);
@ -898,6 +909,11 @@ ExecParallelReInitializeDSM(PlanState *planstate,
ExecBitmapHeapReInitializeDSM((BitmapHeapScanState *) planstate,
pcxt);
break;
case T_HashJoinState:
if (planstate->plan->parallel_aware)
ExecHashJoinReInitializeDSM((HashJoinState *) planstate,
pcxt);
break;
case T_HashState:
case T_SortState:
/* these nodes have DSM state, but no reinitialization is required */
@ -1196,6 +1212,11 @@ ExecParallelInitializeWorker(PlanState *planstate, ParallelWorkerContext *pwcxt)
ExecBitmapHeapInitializeWorker((BitmapHeapScanState *) planstate,
pwcxt);
break;
case T_HashJoinState:
if (planstate->plan->parallel_aware)
ExecHashJoinInitializeWorker((HashJoinState *) planstate,
pwcxt);
break;
case T_HashState:
/* even when not parallel-aware, for EXPLAIN ANALYZE */
ExecHashInitializeWorker((HashState *) planstate, pwcxt);

View File

@ -770,6 +770,9 @@ ExecShutdownNode(PlanState *node)
case T_HashState:
ExecShutdownHash((HashState *) node);
break;
case T_HashJoinState:
ExecShutdownHashJoin((HashJoinState *) node);
break;
default:
break;
}

File diff suppressed because it is too large Load Diff

View File

@ -10,18 +10,112 @@
* IDENTIFICATION
* src/backend/executor/nodeHashjoin.c
*
* PARALLELISM
*
* Hash joins can participate in parallel query execution in several ways. A
* parallel-oblivious hash join is one where the node is unaware that it is
* part of a parallel plan. In this case, a copy of the inner plan is used to
* build a copy of the hash table in every backend, and the outer plan could
* either be built from a partial or complete path, so that the results of the
* hash join are correspondingly either partial or complete. A parallel-aware
* hash join is one that behaves differently, coordinating work between
* backends, and appears as Parallel Hash Join in EXPLAIN output. A Parallel
* Hash Join always appears with a Parallel Hash node.
*
* Parallel-aware hash joins use the same per-backend state machine to track
* progress through the hash join algorithm as parallel-oblivious hash joins.
* In a parallel-aware hash join, there is also a shared state machine that
* co-operating backends use to synchronize their local state machines and
* program counters. The shared state machine is managed with a Barrier IPC
* primitive. When all attached participants arrive at a barrier, the phase
* advances and all waiting participants are released.
*
* When a participant begins working on a parallel hash join, it must first
* figure out how much progress has already been made, because participants
* don't wait for each other to begin. For this reason there are switch
* statements at key points in the code where we have to synchronize our local
* state machine with the phase, and then jump to the correct part of the
* algorithm so that we can get started.
*
* One barrier called build_barrier is used to coordinate the hashing phases.
* The phase is represented by an integer which begins at zero and increments
* one by one, but in the code it is referred to by symbolic names as follows:
*
* PHJ_BUILD_ELECTING -- initial state
* PHJ_BUILD_ALLOCATING -- one sets up the batches and table 0
* PHJ_BUILD_HASHING_INNER -- all hash the inner rel
* PHJ_BUILD_HASHING_OUTER -- (multi-batch only) all hash the outer
* PHJ_BUILD_DONE -- building done, probing can begin
*
* While in the phase PHJ_BUILD_HASHING_INNER a separate pair of barriers may
* be used repeatedly as required to coordinate expansions in the number of
* batches or buckets. Their phases are as follows:
*
* PHJ_GROW_BATCHES_ELECTING -- initial state
* PHJ_GROW_BATCHES_ALLOCATING -- one allocates new batches
* PHJ_GROW_BATCHES_REPARTITIONING -- all repartition
* PHJ_GROW_BATCHES_FINISHING -- one cleans up, detects skew
*
* PHJ_GROW_BUCKETS_ELECTING -- initial state
* PHJ_GROW_BUCKETS_ALLOCATING -- one allocates new buckets
* PHJ_GROW_BUCKETS_REINSERTING -- all insert tuples
*
* If the planner got the number of batches and buckets right, those won't be
* necessary, but on the other hand we might finish up needing to expand the
* buckets or batches multiple times while hashing the inner relation to stay
* within our memory budget and load factor target. For that reason it's a
* separate pair of barriers using circular phases.
*
* The PHJ_BUILD_HASHING_OUTER phase is required only for multi-batch joins,
* because we need to divide the outer relation into batches up front in order
* to be able to process batches entirely independently. In contrast, the
* parallel-oblivious algorithm simply throws tuples 'forward' to 'later'
* batches whenever it encounters them while scanning and probing, which it
* can do because it processes batches in serial order.
*
* Once PHJ_BUILD_DONE is reached, backends then split up and process
* different batches, or gang up and work together on probing batches if there
* aren't enough to go around. For each batch there is a separate barrier
* with the following phases:
*
* PHJ_BATCH_ELECTING -- initial state
* PHJ_BATCH_ALLOCATING -- one allocates buckets
* PHJ_BATCH_LOADING -- all load the hash table from disk
* PHJ_BATCH_PROBING -- all probe
* PHJ_BATCH_DONE -- end
*
* Batch 0 is a special case, because it starts out in phase
* PHJ_BATCH_PROBING; populating batch 0's hash table is done during
* PHJ_BUILD_HASHING_INNER so we can skip loading.
*
* Initially we try to plan for a single-batch hash join using the combined
* work_mem of all participants to create a large shared hash table. If that
* turns out either at planning or execution time to be impossible then we
* fall back to regular work_mem sized hash tables.
*
* To avoid deadlocks, we never wait for any barrier unless it is known that
* all other backends attached to it are actively executing the node or have
* already arrived. Practically, that means that we never return a tuple
* while attached to a barrier, unless the barrier has reached its final
* state. In the slightly special case of the per-batch barrier, we return
* tuples while in PHJ_BATCH_PROBING phase, but that's OK because we use
* BarrierArriveAndDetach() to advance it to PHJ_BATCH_DONE without waiting.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/htup_details.h"
#include "access/parallel.h"
#include "executor/executor.h"
#include "executor/hashjoin.h"
#include "executor/nodeHash.h"
#include "executor/nodeHashjoin.h"
#include "miscadmin.h"
#include "pgstat.h"
#include "utils/memutils.h"
#include "utils/sharedtuplestore.h"
/*
@ -42,24 +136,34 @@
static TupleTableSlot *ExecHashJoinOuterGetTuple(PlanState *outerNode,
HashJoinState *hjstate,
uint32 *hashvalue);
static TupleTableSlot *ExecParallelHashJoinOuterGetTuple(PlanState *outerNode,
HashJoinState *hjstate,
uint32 *hashvalue);
static TupleTableSlot *ExecHashJoinGetSavedTuple(HashJoinState *hjstate,
BufFile *file,
uint32 *hashvalue,
TupleTableSlot *tupleSlot);
static bool ExecHashJoinNewBatch(HashJoinState *hjstate);
static bool ExecParallelHashJoinNewBatch(HashJoinState *hjstate);
static void ExecParallelHashJoinPartitionOuter(HashJoinState *node);
/* ----------------------------------------------------------------
* ExecHashJoin
* ExecHashJoinImpl
*
* This function implements the Hybrid Hashjoin algorithm.
* This function implements the Hybrid Hashjoin algorithm. It is marked
* with an always-inline attribute so that ExecHashJoin() and
* ExecParallelHashJoin() can inline it. Compilers that respect the
* attribute should create versions specialized for parallel == true and
* parallel == false with unnecessary branches removed.
*
* Note: the relation we build hash table on is the "inner"
* the other one is "outer".
* ----------------------------------------------------------------
*/
static TupleTableSlot * /* return: a tuple or NULL */
ExecHashJoin(PlanState *pstate)
pg_attribute_always_inline
static inline TupleTableSlot *
ExecHashJoinImpl(PlanState *pstate, bool parallel)
{
HashJoinState *node = castNode(HashJoinState, pstate);
PlanState *outerNode;
@ -71,6 +175,7 @@ ExecHashJoin(PlanState *pstate)
TupleTableSlot *outerTupleSlot;
uint32 hashvalue;
int batchno;
ParallelHashJoinState *parallel_state;
/*
* get information from HashJoin node
@ -81,6 +186,7 @@ ExecHashJoin(PlanState *pstate)
outerNode = outerPlanState(node);
hashtable = node->hj_HashTable;
econtext = node->js.ps.ps_ExprContext;
parallel_state = hashNode->parallel_state;
/*
* Reset per-tuple memory context to free any expression evaluation
@ -138,6 +244,18 @@ ExecHashJoin(PlanState *pstate)
/* no chance to not build the hash table */
node->hj_FirstOuterTupleSlot = NULL;
}
else if (parallel)
{
/*
* The empty-outer optimization is not implemented for
* shared hash tables, because no one participant can
* determine that there are no outer tuples, and it's not
* yet clear that it's worth the synchronization overhead
* of reaching consensus to figure that out. So we have
* to build the hash table.
*/
node->hj_FirstOuterTupleSlot = NULL;
}
else if (HJ_FILL_OUTER(node) ||
(outerNode->plan->startup_cost < hashNode->ps.plan->total_cost &&
!node->hj_OuterNotEmpty))
@ -155,15 +273,19 @@ ExecHashJoin(PlanState *pstate)
node->hj_FirstOuterTupleSlot = NULL;
/*
* create the hash table
* Create the hash table. If using Parallel Hash, then
* whoever gets here first will create the hash table and any
* later arrivals will merely attach to it.
*/
hashtable = ExecHashTableCreate((Hash *) hashNode->ps.plan,
hashtable = ExecHashTableCreate(hashNode,
node->hj_HashOperators,
HJ_FILL_INNER(node));
node->hj_HashTable = hashtable;
/*
* execute the Hash node, to build the hash table
* Execute the Hash node, to build the hash table. If using
* Parallel Hash, then we'll try to help hashing unless we
* arrived too late.
*/
hashNode->hashtable = hashtable;
(void) MultiExecProcNode((PlanState *) hashNode);
@ -189,7 +311,34 @@ ExecHashJoin(PlanState *pstate)
*/
node->hj_OuterNotEmpty = false;
node->hj_JoinState = HJ_NEED_NEW_OUTER;
if (parallel)
{
Barrier *build_barrier;
build_barrier = &parallel_state->build_barrier;
Assert(BarrierPhase(build_barrier) == PHJ_BUILD_HASHING_OUTER ||
BarrierPhase(build_barrier) == PHJ_BUILD_DONE);
if (BarrierPhase(build_barrier) == PHJ_BUILD_HASHING_OUTER)
{
/*
* If multi-batch, we need to hash the outer relation
* up front.
*/
if (hashtable->nbatch > 1)
ExecParallelHashJoinPartitionOuter(node);
BarrierArriveAndWait(build_barrier,
WAIT_EVENT_HASH_BUILD_HASHING_OUTER);
}
Assert(BarrierPhase(build_barrier) == PHJ_BUILD_DONE);
/* Each backend should now select a batch to work on. */
hashtable->curbatch = -1;
node->hj_JoinState = HJ_NEED_NEW_BATCH;
continue;
}
else
node->hj_JoinState = HJ_NEED_NEW_OUTER;
/* FALL THRU */
@ -198,9 +347,14 @@ ExecHashJoin(PlanState *pstate)
/*
* We don't have an outer tuple, try to get the next one
*/
outerTupleSlot = ExecHashJoinOuterGetTuple(outerNode,
node,
&hashvalue);
if (parallel)
outerTupleSlot =
ExecParallelHashJoinOuterGetTuple(outerNode, node,
&hashvalue);
else
outerTupleSlot =
ExecHashJoinOuterGetTuple(outerNode, node, &hashvalue);
if (TupIsNull(outerTupleSlot))
{
/* end of batch, or maybe whole join */
@ -240,10 +394,12 @@ ExecHashJoin(PlanState *pstate)
* Need to postpone this outer tuple to a later batch.
* Save it in the corresponding outer-batch file.
*/
Assert(parallel_state == NULL);
Assert(batchno > hashtable->curbatch);
ExecHashJoinSaveTuple(ExecFetchSlotMinimalTuple(outerTupleSlot),
hashvalue,
&hashtable->outerBatchFile[batchno]);
/* Loop around, staying in HJ_NEED_NEW_OUTER state */
continue;
}
@ -258,11 +414,23 @@ ExecHashJoin(PlanState *pstate)
/*
* Scan the selected hash bucket for matches to current outer
*/
if (!ExecScanHashBucket(node, econtext))
if (parallel)
{
/* out of matches; check for possible outer-join fill */
node->hj_JoinState = HJ_FILL_OUTER_TUPLE;
continue;
if (!ExecParallelScanHashBucket(node, econtext))
{
/* out of matches; check for possible outer-join fill */
node->hj_JoinState = HJ_FILL_OUTER_TUPLE;
continue;
}
}
else
{
if (!ExecScanHashBucket(node, econtext))
{
/* out of matches; check for possible outer-join fill */
node->hj_JoinState = HJ_FILL_OUTER_TUPLE;
continue;
}
}
/*
@ -362,8 +530,16 @@ ExecHashJoin(PlanState *pstate)
/*
* Try to advance to next batch. Done if there are no more.
*/
if (!ExecHashJoinNewBatch(node))
return NULL; /* end of join */
if (parallel)
{
if (!ExecParallelHashJoinNewBatch(node))
return NULL; /* end of parallel-aware join */
}
else
{
if (!ExecHashJoinNewBatch(node))
return NULL; /* end of parallel-oblivious join */
}
node->hj_JoinState = HJ_NEED_NEW_OUTER;
break;
@ -374,6 +550,38 @@ ExecHashJoin(PlanState *pstate)
}
}
/* ----------------------------------------------------------------
* ExecHashJoin
*
* Parallel-oblivious version.
* ----------------------------------------------------------------
*/
static TupleTableSlot * /* return: a tuple or NULL */
ExecHashJoin(PlanState *pstate)
{
/*
* On sufficiently smart compilers this should be inlined with the
* parallel-aware branches removed.
*/
return ExecHashJoinImpl(pstate, false);
}
/* ----------------------------------------------------------------
* ExecParallelHashJoin
*
* Parallel-aware version.
* ----------------------------------------------------------------
*/
static TupleTableSlot * /* return: a tuple or NULL */
ExecParallelHashJoin(PlanState *pstate)
{
/*
* On sufficiently smart compilers this should be inlined with the
* parallel-oblivious branches removed.
*/
return ExecHashJoinImpl(pstate, true);
}
/* ----------------------------------------------------------------
* ExecInitHashJoin
*
@ -400,6 +608,12 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags)
hjstate = makeNode(HashJoinState);
hjstate->js.ps.plan = (Plan *) node;
hjstate->js.ps.state = estate;
/*
* See ExecHashJoinInitializeDSM() and ExecHashJoinInitializeWorker()
* where this function may be replaced with a parallel version, if we
* managed to launch a parallel query.
*/
hjstate->js.ps.ExecProcNode = ExecHashJoin;
/*
@ -581,9 +795,9 @@ ExecEndHashJoin(HashJoinState *node)
/*
* ExecHashJoinOuterGetTuple
*
* get the next outer tuple for hashjoin: either by
* executing the outer plan node in the first pass, or from
* the temp files for the hashjoin batches.
* get the next outer tuple for a parallel oblivious hashjoin: either by
* executing the outer plan node in the first pass, or from the temp
* files for the hashjoin batches.
*
* Returns a null slot if no more outer tuples (within the current batch).
*
@ -661,6 +875,67 @@ ExecHashJoinOuterGetTuple(PlanState *outerNode,
return NULL;
}
/*
* ExecHashJoinOuterGetTuple variant for the parallel case.
*/
static TupleTableSlot *
ExecParallelHashJoinOuterGetTuple(PlanState *outerNode,
HashJoinState *hjstate,
uint32 *hashvalue)
{
HashJoinTable hashtable = hjstate->hj_HashTable;
int curbatch = hashtable->curbatch;
TupleTableSlot *slot;
/*
* In the Parallel Hash case we only run the outer plan directly for
* single-batch hash joins. Otherwise we have to go to batch files, even
* for batch 0.
*/
if (curbatch == 0 && hashtable->nbatch == 1)
{
slot = ExecProcNode(outerNode);
while (!TupIsNull(slot))
{
ExprContext *econtext = hjstate->js.ps.ps_ExprContext;
econtext->ecxt_outertuple = slot;
if (ExecHashGetHashValue(hashtable, econtext,
hjstate->hj_OuterHashKeys,
true, /* outer tuple */
HJ_FILL_OUTER(hjstate),
hashvalue))
return slot;
/*
* That tuple couldn't match because of a NULL, so discard it and
* continue with the next one.
*/
slot = ExecProcNode(outerNode);
}
}
else if (curbatch < hashtable->nbatch)
{
MinimalTuple tuple;
tuple = sts_parallel_scan_next(hashtable->batches[curbatch].outer_tuples,
hashvalue);
if (tuple != NULL)
{
slot = ExecStoreMinimalTuple(tuple,
hjstate->hj_OuterTupleSlot,
false);
return slot;
}
else
ExecClearTuple(hjstate->hj_OuterTupleSlot);
}
/* End of this batch */
return NULL;
}
/*
* ExecHashJoinNewBatch
* switch to a new hashjoin batch
@ -803,6 +1078,135 @@ ExecHashJoinNewBatch(HashJoinState *hjstate)
return true;
}
/*
* Choose a batch to work on, and attach to it. Returns true if successful,
* false if there are no more batches.
*/
static bool
ExecParallelHashJoinNewBatch(HashJoinState *hjstate)
{
HashJoinTable hashtable = hjstate->hj_HashTable;
int start_batchno;
int batchno;
/*
* If we started up so late that the batch tracking array has been freed
* already by ExecHashTableDetach(), then we are finished. See also
* ExecParallelHashEnsureBatchAccessors().
*/
if (hashtable->batches == NULL)
return false;
/*
* If we were already attached to a batch, remember not to bother checking
* it again, and detach from it (possibly freeing the hash table if we are
* last to detach).
*/
if (hashtable->curbatch >= 0)
{
hashtable->batches[hashtable->curbatch].done = true;
ExecHashTableDetachBatch(hashtable);
}
/*
* Search for a batch that isn't done. We use an atomic counter to start
* our search at a different batch in every participant when there are
* more batches than participants.
*/
batchno = start_batchno =
pg_atomic_fetch_add_u32(&hashtable->parallel_state->distributor, 1) %
hashtable->nbatch;
do
{
uint32 hashvalue;
MinimalTuple tuple;
TupleTableSlot *slot;
if (!hashtable->batches[batchno].done)
{
SharedTuplestoreAccessor *inner_tuples;
Barrier *batch_barrier =
&hashtable->batches[batchno].shared->batch_barrier;
switch (BarrierAttach(batch_barrier))
{
case PHJ_BATCH_ELECTING:
/* One backend allocates the hash table. */
if (BarrierArriveAndWait(batch_barrier,
WAIT_EVENT_HASH_BATCH_ELECTING))
ExecParallelHashTableAlloc(hashtable, batchno);
/* Fall through. */
case PHJ_BATCH_ALLOCATING:
/* Wait for allocation to complete. */
BarrierArriveAndWait(batch_barrier,
WAIT_EVENT_HASH_BATCH_ALLOCATING);
/* Fall through. */
case PHJ_BATCH_LOADING:
/* Start (or join in) loading tuples. */
ExecParallelHashTableSetCurrentBatch(hashtable, batchno);
inner_tuples = hashtable->batches[batchno].inner_tuples;
sts_begin_parallel_scan(inner_tuples);
while ((tuple = sts_parallel_scan_next(inner_tuples,
&hashvalue)))
{
slot = ExecStoreMinimalTuple(tuple,
hjstate->hj_HashTupleSlot,
false);
ExecParallelHashTableInsertCurrentBatch(hashtable, slot,
hashvalue);
}
sts_end_parallel_scan(inner_tuples);
BarrierArriveAndWait(batch_barrier,
WAIT_EVENT_HASH_BATCH_LOADING);
/* Fall through. */
case PHJ_BATCH_PROBING:
/*
* This batch is ready to probe. Return control to
* caller. We stay attached to batch_barrier so that the
* hash table stays alive until everyone's finished
* probing it, but no participant is allowed to wait at
* this barrier again (or else a deadlock could occur).
* All attached participants must eventually call
* BarrierArriveAndDetach() so that the final phase
* PHJ_BATCH_DONE can be reached.
*/
ExecParallelHashTableSetCurrentBatch(hashtable, batchno);
sts_begin_parallel_scan(hashtable->batches[batchno].outer_tuples);
return true;
case PHJ_BATCH_DONE:
/*
* Already done. Detach and go around again (if any
* remain).
*/
BarrierDetach(batch_barrier);
/*
* We didn't work on this batch, but we need to observe
* its size for EXPLAIN.
*/
ExecParallelHashUpdateSpacePeak(hashtable, batchno);
hashtable->batches[batchno].done = true;
hashtable->curbatch = -1;
break;
default:
elog(ERROR, "unexpected batch phase %d",
BarrierPhase(batch_barrier));
}
}
batchno = (batchno + 1) % hashtable->nbatch;
} while (batchno != start_batchno);
return false;
}
/*
* ExecHashJoinSaveTuple
* save a tuple to a batch file.
@ -964,3 +1368,176 @@ ExecReScanHashJoin(HashJoinState *node)
if (node->js.ps.lefttree->chgParam == NULL)
ExecReScan(node->js.ps.lefttree);
}
void
ExecShutdownHashJoin(HashJoinState *node)
{
if (node->hj_HashTable)
{
/*
* Detach from shared state before DSM memory goes away. This makes
* sure that we don't have any pointers into DSM memory by the time
* ExecEndHashJoin runs.
*/
ExecHashTableDetachBatch(node->hj_HashTable);
ExecHashTableDetach(node->hj_HashTable);
}
}
static void
ExecParallelHashJoinPartitionOuter(HashJoinState *hjstate)
{
PlanState *outerState = outerPlanState(hjstate);
ExprContext *econtext = hjstate->js.ps.ps_ExprContext;
HashJoinTable hashtable = hjstate->hj_HashTable;
TupleTableSlot *slot;
uint32 hashvalue;
int i;
Assert(hjstate->hj_FirstOuterTupleSlot == NULL);
/* Execute outer plan, writing all tuples to shared tuplestores. */
for (;;)
{
slot = ExecProcNode(outerState);
if (TupIsNull(slot))
break;
econtext->ecxt_outertuple = slot;
if (ExecHashGetHashValue(hashtable, econtext,
hjstate->hj_OuterHashKeys,
true, /* outer tuple */
false, /* outer join, currently unsupported */
&hashvalue))
{
int batchno;
int bucketno;
ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno,
&batchno);
sts_puttuple(hashtable->batches[batchno].outer_tuples,
&hashvalue, ExecFetchSlotMinimalTuple(slot));
}
CHECK_FOR_INTERRUPTS();
}
/* Make sure all outer partitions are readable by any backend. */
for (i = 0; i < hashtable->nbatch; ++i)
sts_end_write(hashtable->batches[i].outer_tuples);
}
void
ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt)
{
shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelHashJoinState));
shm_toc_estimate_keys(&pcxt->estimator, 1);
}
void
ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt)
{
int plan_node_id = state->js.ps.plan->plan_node_id;
HashState *hashNode;
ParallelHashJoinState *pstate;
/*
* Disable shared hash table mode if we failed to create a real DSM
* segment, because that means that we don't have a DSA area to work with.
*/
if (pcxt->seg == NULL)
return;
ExecSetExecProcNode(&state->js.ps, ExecParallelHashJoin);
/*
* Set up the state needed to coordinate access to the shared hash
* table(s), using the plan node ID as the toc key.
*/
pstate = shm_toc_allocate(pcxt->toc, sizeof(ParallelHashJoinState));
shm_toc_insert(pcxt->toc, plan_node_id, pstate);
/*
* Set up the shared hash join state with no batches initially.
* ExecHashTableCreate() will prepare at least one later and set nbatch
* and space_allowed.
*/
pstate->nbatch = 0;
pstate->space_allowed = 0;
pstate->batches = InvalidDsaPointer;
pstate->old_batches = InvalidDsaPointer;
pstate->nbuckets = 0;
pstate->growth = PHJ_GROWTH_OK;
pstate->chunk_work_queue = InvalidDsaPointer;
pg_atomic_init_u32(&pstate->distributor, 0);
pstate->nparticipants = pcxt->nworkers + 1;
pstate->total_tuples = 0;
LWLockInitialize(&pstate->lock,
LWTRANCHE_PARALLEL_HASH_JOIN);
BarrierInit(&pstate->build_barrier, 0);
BarrierInit(&pstate->grow_batches_barrier, 0);
BarrierInit(&pstate->grow_buckets_barrier, 0);
/* Set up the space we'll use for shared temporary files. */
SharedFileSetInit(&pstate->fileset, pcxt->seg);
/* Initialize the shared state in the hash node. */
hashNode = (HashState *) innerPlanState(state);
hashNode->parallel_state = pstate;
}
/* ----------------------------------------------------------------
* ExecHashJoinReInitializeDSM
*
* Reset shared state before beginning a fresh scan.
* ----------------------------------------------------------------
*/
void
ExecHashJoinReInitializeDSM(HashJoinState *state, ParallelContext *cxt)
{
int plan_node_id = state->js.ps.plan->plan_node_id;
ParallelHashJoinState *pstate =
shm_toc_lookup(cxt->toc, plan_node_id, false);
/*
* It would be possible to reuse the shared hash table in single-batch
* cases by resetting and then fast-forwarding build_barrier to
* PHJ_BUILD_DONE and batch 0's batch_barrier to PHJ_BATCH_PROBING, but
* currently shared hash tables are already freed by now (by the last
* participant to detach from the batch). We could consider keeping it
* around for single-batch joins. We'd also need to adjust
* finalize_plan() so that it doesn't record a dummy dependency for
* Parallel Hash nodes, preventing the rescan optimization. For now we
* don't try.
*/
/* Detach, freeing any remaining shared memory. */
if (state->hj_HashTable != NULL)
{
ExecHashTableDetachBatch(state->hj_HashTable);
ExecHashTableDetach(state->hj_HashTable);
}
/* Clear any shared batch files. */
SharedFileSetDeleteAll(&pstate->fileset);
/* Reset build_barrier to PHJ_BUILD_ELECTING so we can go around again. */
BarrierInit(&pstate->build_barrier, 0);
}
void
ExecHashJoinInitializeWorker(HashJoinState *state,
ParallelWorkerContext *pwcxt)
{
HashState *hashNode;
int plan_node_id = state->js.ps.plan->plan_node_id;
ParallelHashJoinState *pstate =
shm_toc_lookup(pwcxt->toc, plan_node_id, false);
/* Attach to the space for shared temporary files. */
SharedFileSetAttach(&pstate->fileset, pwcxt->seg);
/* Attach to the shared state in the hash node. */
hashNode = (HashState *) innerPlanState(state);
hashNode->parallel_state = pstate;
ExecSetExecProcNode(&state->js.ps, ExecParallelHashJoin);
}

View File

@ -1057,6 +1057,7 @@ _copyHash(const Hash *from)
COPY_SCALAR_FIELD(skewTable);
COPY_SCALAR_FIELD(skewColumn);
COPY_SCALAR_FIELD(skewInherit);
COPY_SCALAR_FIELD(rows_total);
return newnode;
}

View File

@ -927,6 +927,7 @@ _outHash(StringInfo str, const Hash *node)
WRITE_OID_FIELD(skewTable);
WRITE_INT_FIELD(skewColumn);
WRITE_BOOL_FIELD(skewInherit);
WRITE_FLOAT_FIELD(rows_total, "%.0f");
}
static void

View File

@ -2213,6 +2213,7 @@ _readHash(void)
READ_OID_FIELD(skewTable);
READ_INT_FIELD(skewColumn);
READ_BOOL_FIELD(skewInherit);
READ_FLOAT_FIELD(rows_total);
READ_DONE();
}

View File

@ -129,6 +129,7 @@ bool enable_hashjoin = true;
bool enable_gathermerge = true;
bool enable_partition_wise_join = false;
bool enable_parallel_append = true;
bool enable_parallel_hash = true;
typedef struct
{
@ -3130,16 +3131,19 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
JoinType jointype,
List *hashclauses,
Path *outer_path, Path *inner_path,
JoinPathExtraData *extra)
JoinPathExtraData *extra,
bool parallel_hash)
{
Cost startup_cost = 0;
Cost run_cost = 0;
double outer_path_rows = outer_path->rows;
double inner_path_rows = inner_path->rows;
double inner_path_rows_total = inner_path_rows;
int num_hashclauses = list_length(hashclauses);
int numbuckets;
int numbatches;
int num_skew_mcvs;
size_t space_allowed; /* unused */
/* cost of source data */
startup_cost += outer_path->startup_cost;
@ -3160,6 +3164,15 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
* inner_path_rows;
run_cost += cpu_operator_cost * num_hashclauses * outer_path_rows;
/*
* If this is a parallel hash build, then the value we have for
* inner_rows_total currently refers only to the rows returned by each
* participant. For shared hash table size estimation, we need the total
* number, so we need to undo the division.
*/
if (parallel_hash)
inner_path_rows_total *= get_parallel_divisor(inner_path);
/*
* Get hash table size that executor would use for inner relation.
*
@ -3170,9 +3183,12 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
* XXX at some point it might be interesting to try to account for skew
* optimization in the cost estimate, but for now, we don't.
*/
ExecChooseHashTableSize(inner_path_rows,
ExecChooseHashTableSize(inner_path_rows_total,
inner_path->pathtarget->width,
true, /* useskew */
parallel_hash, /* try_combined_work_mem */
outer_path->parallel_workers,
&space_allowed,
&numbuckets,
&numbatches,
&num_skew_mcvs);
@ -3204,6 +3220,7 @@ initial_cost_hashjoin(PlannerInfo *root, JoinCostWorkspace *workspace,
workspace->run_cost = run_cost;
workspace->numbuckets = numbuckets;
workspace->numbatches = numbatches;
workspace->inner_rows_total = inner_path_rows_total;
}
/*
@ -3226,6 +3243,7 @@ final_cost_hashjoin(PlannerInfo *root, HashPath *path,
Path *inner_path = path->jpath.innerjoinpath;
double outer_path_rows = outer_path->rows;
double inner_path_rows = inner_path->rows;
double inner_path_rows_total = workspace->inner_rows_total;
List *hashclauses = path->path_hashclauses;
Cost startup_cost = workspace->startup_cost;
Cost run_cost = workspace->run_cost;
@ -3266,6 +3284,9 @@ final_cost_hashjoin(PlannerInfo *root, HashPath *path,
/* mark the path with estimated # of batches */
path->num_batches = numbatches;
/* store the total number of tuples (sum of partial row estimates) */
path->inner_rows_total = inner_path_rows_total;
/* and compute the number of "virtual" buckets in the whole join */
virtualbuckets = (double) numbuckets * (double) numbatches;

View File

@ -747,7 +747,7 @@ try_hashjoin_path(PlannerInfo *root,
* never have any output pathkeys, per comments in create_hashjoin_path.
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path, extra);
outer_path, inner_path, extra, false);
if (add_path_precheck(joinrel,
workspace.startup_cost, workspace.total_cost,
@ -761,6 +761,7 @@ try_hashjoin_path(PlannerInfo *root,
extra,
outer_path,
inner_path,
false, /* parallel_hash */
extra->restrictlist,
required_outer,
hashclauses));
@ -776,6 +777,10 @@ try_hashjoin_path(PlannerInfo *root,
* try_partial_hashjoin_path
* Consider a partial hashjoin join path; if it appears useful, push it into
* the joinrel's partial_pathlist via add_partial_path().
* The outer side is partial. If parallel_hash is true, then the inner path
* must be partial and will be run in parallel to create one or more shared
* hash tables; otherwise the inner path must be complete and a copy of it
* is run in every process to create separate identical private hash tables.
*/
static void
try_partial_hashjoin_path(PlannerInfo *root,
@ -784,7 +789,8 @@ try_partial_hashjoin_path(PlannerInfo *root,
Path *inner_path,
List *hashclauses,
JoinType jointype,
JoinPathExtraData *extra)
JoinPathExtraData *extra,
bool parallel_hash)
{
JoinCostWorkspace workspace;
@ -808,7 +814,7 @@ try_partial_hashjoin_path(PlannerInfo *root,
* cost. Bail out right away if it looks terrible.
*/
initial_cost_hashjoin(root, &workspace, jointype, hashclauses,
outer_path, inner_path, extra);
outer_path, inner_path, extra, true);
if (!add_partial_path_precheck(joinrel, workspace.total_cost, NIL))
return;
@ -821,6 +827,7 @@ try_partial_hashjoin_path(PlannerInfo *root,
extra,
outer_path,
inner_path,
parallel_hash,
extra->restrictlist,
NULL,
hashclauses));
@ -1839,6 +1846,10 @@ hash_inner_and_outer(PlannerInfo *root,
* able to properly guarantee uniqueness. Similarly, we can't handle
* JOIN_FULL and JOIN_RIGHT, because they can produce false null
* extended rows. Also, the resulting path must not be parameterized.
* We would be able to support JOIN_FULL and JOIN_RIGHT for Parallel
* Hash, since in that case we're back to a single hash table with a
* single set of match bits for each batch, but that will require
* figuring out a deadlock-free way to wait for the probe to finish.
*/
if (joinrel->consider_parallel &&
save_jointype != JOIN_UNIQUE_OUTER &&
@ -1848,11 +1859,27 @@ hash_inner_and_outer(PlannerInfo *root,
bms_is_empty(joinrel->lateral_relids))
{
Path *cheapest_partial_outer;
Path *cheapest_partial_inner = NULL;
Path *cheapest_safe_inner = NULL;
cheapest_partial_outer =
(Path *) linitial(outerrel->partial_pathlist);
/*
* Can we use a partial inner plan too, so that we can build a
* shared hash table in parallel?
*/
if (innerrel->partial_pathlist != NIL && enable_parallel_hash)
{
cheapest_partial_inner =
(Path *) linitial(innerrel->partial_pathlist);
try_partial_hashjoin_path(root, joinrel,
cheapest_partial_outer,
cheapest_partial_inner,
hashclauses, jointype, extra,
true /* parallel_hash */ );
}
/*
* Normally, given that the joinrel is parallel-safe, the cheapest
* total inner path will also be parallel-safe, but if not, we'll
@ -1870,7 +1897,8 @@ hash_inner_and_outer(PlannerInfo *root,
try_partial_hashjoin_path(root, joinrel,
cheapest_partial_outer,
cheapest_safe_inner,
hashclauses, jointype, extra);
hashclauses, jointype, extra,
false /* parallel_hash */ );
}
}
}

View File

@ -4192,6 +4192,17 @@ create_hashjoin_plan(PlannerInfo *root,
copy_plan_costsize(&hash_plan->plan, inner_plan);
hash_plan->plan.startup_cost = hash_plan->plan.total_cost;
/*
* If parallel-aware, the executor will also need an estimate of the total
* number of rows expected from all participants so that it can size the
* shared hash table.
*/
if (best_path->jpath.path.parallel_aware)
{
hash_plan->plan.parallel_aware = true;
hash_plan->rows_total = best_path->inner_rows_total;
}
join_plan = make_hashjoin(tlist,
joinclauses,
otherclauses,

View File

@ -2278,6 +2278,7 @@ create_mergejoin_path(PlannerInfo *root,
* 'extra' contains various information about the join
* 'outer_path' is the cheapest outer path
* 'inner_path' is the cheapest inner path
* 'parallel_hash' to select Parallel Hash of inner path (shared hash table)
* 'restrict_clauses' are the RestrictInfo nodes to apply at the join
* 'required_outer' is the set of required outer rels
* 'hashclauses' are the RestrictInfo nodes to use as hash clauses
@ -2291,6 +2292,7 @@ create_hashjoin_path(PlannerInfo *root,
JoinPathExtraData *extra,
Path *outer_path,
Path *inner_path,
bool parallel_hash,
List *restrict_clauses,
Relids required_outer,
List *hashclauses)
@ -2308,7 +2310,8 @@ create_hashjoin_path(PlannerInfo *root,
extra->sjinfo,
required_outer,
&restrict_clauses);
pathnode->jpath.path.parallel_aware = false;
pathnode->jpath.path.parallel_aware =
joinrel->consider_parallel && parallel_hash;
pathnode->jpath.path.parallel_safe = joinrel->consider_parallel &&
outer_path->parallel_safe && inner_path->parallel_safe;
/* This is a foolish way to estimate parallel_workers, but for now... */

View File

@ -3586,6 +3586,51 @@ pgstat_get_wait_ipc(WaitEventIPC w)
case WAIT_EVENT_EXECUTE_GATHER:
event_name = "ExecuteGather";
break;
case WAIT_EVENT_HASH_BATCH_ALLOCATING:
event_name = "Hash/Batch/Allocating";
break;
case WAIT_EVENT_HASH_BATCH_ELECTING:
event_name = "Hash/Batch/Electing";
break;
case WAIT_EVENT_HASH_BATCH_LOADING:
event_name = "Hash/Batch/Loading";
break;
case WAIT_EVENT_HASH_BUILD_ALLOCATING:
event_name = "Hash/Build/Allocating";
break;
case WAIT_EVENT_HASH_BUILD_ELECTING:
event_name = "Hash/Build/Electing";
break;
case WAIT_EVENT_HASH_BUILD_HASHING_INNER:
event_name = "Hash/Build/HashingInner";
break;
case WAIT_EVENT_HASH_BUILD_HASHING_OUTER:
event_name = "Hash/Build/HashingOuter";
break;
case WAIT_EVENT_HASH_GROW_BATCHES_ALLOCATING:
event_name = "Hash/GrowBatches/Allocating";
break;
case WAIT_EVENT_HASH_GROW_BATCHES_DECIDING:
event_name = "Hash/GrowBatches/Deciding";
break;
case WAIT_EVENT_HASH_GROW_BATCHES_ELECTING:
event_name = "Hash/GrowBatches/Electing";
break;
case WAIT_EVENT_HASH_GROW_BATCHES_FINISHING:
event_name = "Hash/GrowBatches/Finishing";
break;
case WAIT_EVENT_HASH_GROW_BATCHES_REPARTITIONING:
event_name = "Hash/GrowBatches/Repartitioning";
break;
case WAIT_EVENT_HASH_GROW_BUCKETS_ALLOCATING:
event_name = "Hash/GrowBuckets/Allocating";
break;
case WAIT_EVENT_HASH_GROW_BUCKETS_ELECTING:
event_name = "Hash/GrowBuckets/Electing";
break;
case WAIT_EVENT_HASH_GROW_BUCKETS_REINSERTING:
event_name = "Hash/GrowBuckets/Reinserting";
break;
case WAIT_EVENT_LOGICAL_SYNC_DATA:
event_name = "LogicalSyncData";
break;

View File

@ -929,7 +929,15 @@ static struct config_bool ConfigureNamesBool[] =
true,
NULL, NULL, NULL
},
{
{"enable_parallel_hash", PGC_USERSET, QUERY_TUNING_METHOD,
gettext_noop("Enables the planner's user of parallel hash plans."),
NULL
},
&enable_parallel_hash,
true,
NULL, NULL, NULL
},
{
{"geqo", PGC_USERSET, QUERY_TUNING_GEQO,
gettext_noop("Enables genetic query optimization."),

View File

@ -301,6 +301,7 @@
#enable_sort = on
#enable_tidscan = on
#enable_partition_wise_join = off
#enable_parallel_hash = on
# - Planner Cost Constants -

View File

@ -15,7 +15,10 @@
#define HASHJOIN_H
#include "nodes/execnodes.h"
#include "port/atomics.h"
#include "storage/barrier.h"
#include "storage/buffile.h"
#include "storage/lwlock.h"
/* ----------------------------------------------------------------
* hash-join hash table structures
@ -63,7 +66,12 @@
typedef struct HashJoinTupleData
{
struct HashJoinTupleData *next; /* link to next tuple in same bucket */
/* link to next tuple in same bucket */
union
{
struct HashJoinTupleData *unshared;
dsa_pointer shared;
} next;
uint32 hashvalue; /* tuple's hash code */
/* Tuple data, in MinimalTuple format, follows on a MAXALIGN boundary */
} HashJoinTupleData;
@ -112,8 +120,12 @@ typedef struct HashMemoryChunkData
size_t maxlen; /* size of the buffer holding the tuples */
size_t used; /* number of buffer bytes already used */
struct HashMemoryChunkData *next; /* pointer to the next chunk (linked
* list) */
/* pointer to the next chunk (linked list) */
union
{
struct HashMemoryChunkData *unshared;
dsa_pointer shared;
} next;
char data[FLEXIBLE_ARRAY_MEMBER]; /* buffer allocated at the end */
} HashMemoryChunkData;
@ -121,8 +133,148 @@ typedef struct HashMemoryChunkData
typedef struct HashMemoryChunkData *HashMemoryChunk;
#define HASH_CHUNK_SIZE (32 * 1024L)
#define HASH_CHUNK_HEADER_SIZE (offsetof(HashMemoryChunkData, data))
#define HASH_CHUNK_THRESHOLD (HASH_CHUNK_SIZE / 4)
/*
* For each batch of a Parallel Hash Join, we have a ParallelHashJoinBatch
* object in shared memory to coordinate access to it. Since they are
* followed by variable-sized objects, they are arranged in contiguous memory
* but not accessed directly as an array.
*/
typedef struct ParallelHashJoinBatch
{
dsa_pointer buckets; /* array of hash table buckets */
Barrier batch_barrier; /* synchronization for joining this batch */
dsa_pointer chunks; /* chunks of tuples loaded */
size_t size; /* size of buckets + chunks in memory */
size_t estimated_size; /* size of buckets + chunks while writing */
size_t ntuples; /* number of tuples loaded */
size_t old_ntuples; /* number of tuples before repartitioning */
bool space_exhausted;
/*
* Variable-sized SharedTuplestore objects follow this struct in memory.
* See the accessor macros below.
*/
} ParallelHashJoinBatch;
/* Accessor for inner batch tuplestore following a ParallelHashJoinBatch. */
#define ParallelHashJoinBatchInner(batch) \
((SharedTuplestore *) \
((char *) (batch) + MAXALIGN(sizeof(ParallelHashJoinBatch))))
/* Accessor for outer batch tuplestore following a ParallelHashJoinBatch. */
#define ParallelHashJoinBatchOuter(batch, nparticipants) \
((SharedTuplestore *) \
((char *) ParallelHashJoinBatchInner(batch) + \
MAXALIGN(sts_estimate(nparticipants))))
/* Total size of a ParallelHashJoinBatch and tuplestores. */
#define EstimateParallelHashJoinBatch(hashtable) \
(MAXALIGN(sizeof(ParallelHashJoinBatch)) + \
MAXALIGN(sts_estimate((hashtable)->parallel_state->nparticipants)) * 2)
/* Accessor for the nth ParallelHashJoinBatch given the base. */
#define NthParallelHashJoinBatch(base, n) \
((ParallelHashJoinBatch *) \
((char *) (base) + \
EstimateParallelHashJoinBatch(hashtable) * (n)))
/*
* Each backend requires a small amount of per-batch state to interact with
* each ParalellHashJoinBatch.
*/
typedef struct ParallelHashJoinBatchAccessor
{
ParallelHashJoinBatch *shared; /* pointer to shared state */
/* Per-backend partial counters to reduce contention. */
size_t preallocated; /* pre-allocated space for this backend */
size_t ntuples; /* number of tuples */
size_t size; /* size of partition in memory */
size_t estimated_size; /* size of partition on disk */
size_t old_ntuples; /* how many tuples before repartioning? */
bool at_least_one_chunk; /* has this backend allocated a chunk? */
bool done; /* flag to remember that a batch is done */
SharedTuplestoreAccessor *inner_tuples;
SharedTuplestoreAccessor *outer_tuples;
} ParallelHashJoinBatchAccessor;
/*
* While hashing the inner relation, any participant might determine that it's
* time to increase the number of buckets to reduce the load factor or batches
* to reduce the memory size. This is indicated by setting the growth flag to
* these values.
*/
typedef enum ParallelHashGrowth
{
/* The current dimensions are sufficient. */
PHJ_GROWTH_OK,
/* The load factor is too high, so we need to add buckets. */
PHJ_GROWTH_NEED_MORE_BUCKETS,
/* The memory budget would be exhausted, so we need to repartition. */
PHJ_GROWTH_NEED_MORE_BATCHES,
/* Repartitioning didn't help last time, so don't try to do that again. */
PHJ_GROWTH_DISABLED
} ParallelHashGrowth;
/*
* The shared state used to coordinate a Parallel Hash Join. This is stored
* in the DSM segment.
*/
typedef struct ParallelHashJoinState
{
dsa_pointer batches; /* array of ParallelHashJoinBatch */
dsa_pointer old_batches; /* previous generation during repartition */
int nbatch; /* number of batches now */
int old_nbatch; /* previous number of batches */
int nbuckets; /* number of buckets */
ParallelHashGrowth growth; /* control batch/bucket growth */
dsa_pointer chunk_work_queue; /* chunk work queue */
int nparticipants;
size_t space_allowed;
size_t total_tuples; /* total number of inner tuples */
LWLock lock; /* lock protecting the above */
Barrier build_barrier; /* synchronization for the build phases */
Barrier grow_batches_barrier;
Barrier grow_buckets_barrier;
pg_atomic_uint32 distributor; /* counter for load balancing */
SharedFileSet fileset; /* space for shared temporary files */
} ParallelHashJoinState;
/* The phases for building batches, used by build_barrier. */
#define PHJ_BUILD_ELECTING 0
#define PHJ_BUILD_ALLOCATING 1
#define PHJ_BUILD_HASHING_INNER 2
#define PHJ_BUILD_HASHING_OUTER 3
#define PHJ_BUILD_DONE 4
/* The phases for probing each batch, used by for batch_barrier. */
#define PHJ_BATCH_ELECTING 0
#define PHJ_BATCH_ALLOCATING 1
#define PHJ_BATCH_LOADING 2
#define PHJ_BATCH_PROBING 3
#define PHJ_BATCH_DONE 4
/* The phases of batch growth while hashing, for grow_batches_barrier. */
#define PHJ_GROW_BATCHES_ELECTING 0
#define PHJ_GROW_BATCHES_ALLOCATING 1
#define PHJ_GROW_BATCHES_REPARTITIONING 2
#define PHJ_GROW_BATCHES_DECIDING 3
#define PHJ_GROW_BATCHES_FINISHING 4
#define PHJ_GROW_BATCHES_PHASE(n) ((n) % 5) /* circular phases */
/* The phases of bucket growth while hashing, for grow_buckets_barrier. */
#define PHJ_GROW_BUCKETS_ELECTING 0
#define PHJ_GROW_BUCKETS_ALLOCATING 1
#define PHJ_GROW_BUCKETS_REINSERTING 2
#define PHJ_GROW_BUCKETS_PHASE(n) ((n) % 3) /* circular phases */
typedef struct HashJoinTableData
{
int nbuckets; /* # buckets in the in-memory hash table */
@ -133,8 +285,13 @@ typedef struct HashJoinTableData
int log2_nbuckets_optimal; /* log2(nbuckets_optimal) */
/* buckets[i] is head of list of tuples in i'th in-memory bucket */
struct HashJoinTupleData **buckets;
/* buckets array is per-batch storage, as are all the tuples */
union
{
/* unshared array is per-batch storage, as are all the tuples */
struct HashJoinTupleData **unshared;
/* shared array is per-query DSA area, as are all the tuples */
dsa_pointer_atomic *shared;
} buckets;
bool keepNulls; /* true to store unmatchable NULL tuples */
@ -153,6 +310,7 @@ typedef struct HashJoinTableData
bool growEnabled; /* flag to shut off nbatch increases */
double totalTuples; /* # tuples obtained from inner plan */
double partialTuples; /* # tuples obtained from inner plan by me */
double skewTuples; /* # tuples inserted into skew tuples */
/*
@ -185,6 +343,13 @@ typedef struct HashJoinTableData
/* used for dense allocation of tuples (into linked chunks) */
HashMemoryChunk chunks; /* one list for the whole batch */
/* Shared and private state for Parallel Hash. */
HashMemoryChunk current_chunk; /* this backend's current chunk */
dsa_area *area; /* DSA area to allocate memory from */
ParallelHashJoinState *parallel_state;
ParallelHashJoinBatchAccessor *batches;
dsa_pointer current_chunk_shared;
} HashJoinTableData;
#endif /* HASHJOIN_H */

View File

@ -17,17 +17,33 @@
#include "access/parallel.h"
#include "nodes/execnodes.h"
struct SharedHashJoinBatch;
extern HashState *ExecInitHash(Hash *node, EState *estate, int eflags);
extern Node *MultiExecHash(HashState *node);
extern void ExecEndHash(HashState *node);
extern void ExecReScanHash(HashState *node);
extern HashJoinTable ExecHashTableCreate(Hash *node, List *hashOperators,
extern HashJoinTable ExecHashTableCreate(HashState *state, List *hashOperators,
bool keepNulls);
extern void ExecParallelHashTableAlloc(HashJoinTable hashtable,
int batchno);
extern void ExecHashTableDestroy(HashJoinTable hashtable);
extern void ExecHashTableDetach(HashJoinTable hashtable);
extern void ExecHashTableDetachBatch(HashJoinTable hashtable);
extern void ExecParallelHashTableSetCurrentBatch(HashJoinTable hashtable,
int batchno);
void ExecParallelHashUpdateSpacePeak(HashJoinTable hashtable, int batchno);
extern void ExecHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
uint32 hashvalue);
extern void ExecParallelHashTableInsert(HashJoinTable hashtable,
TupleTableSlot *slot,
uint32 hashvalue);
extern void ExecParallelHashTableInsertCurrentBatch(HashJoinTable hashtable,
TupleTableSlot *slot,
uint32 hashvalue);
extern bool ExecHashGetHashValue(HashJoinTable hashtable,
ExprContext *econtext,
List *hashkeys,
@ -39,12 +55,16 @@ extern void ExecHashGetBucketAndBatch(HashJoinTable hashtable,
int *bucketno,
int *batchno);
extern bool ExecScanHashBucket(HashJoinState *hjstate, ExprContext *econtext);
extern bool ExecParallelScanHashBucket(HashJoinState *hjstate, ExprContext *econtext);
extern void ExecPrepHashTableForUnmatched(HashJoinState *hjstate);
extern bool ExecScanHashTableForUnmatched(HashJoinState *hjstate,
ExprContext *econtext);
extern void ExecHashTableReset(HashJoinTable hashtable);
extern void ExecHashTableResetMatchFlags(HashJoinTable hashtable);
extern void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew,
bool try_combined_work_mem,
int parallel_workers,
size_t *space_allowed,
int *numbuckets,
int *numbatches,
int *num_skew_mcvs);
@ -55,6 +75,6 @@ extern void ExecHashInitializeWorker(HashState *node, ParallelWorkerContext *pwc
extern void ExecHashRetrieveInstrumentation(HashState *node);
extern void ExecShutdownHash(HashState *node);
extern void ExecHashGetInstrumentation(HashInstrumentation *instrument,
HashJoinTable hashtable);
HashJoinTable hashtable);
#endif /* NODEHASH_H */

View File

@ -20,6 +20,12 @@
extern HashJoinState *ExecInitHashJoin(HashJoin *node, EState *estate, int eflags);
extern void ExecEndHashJoin(HashJoinState *node);
extern void ExecReScanHashJoin(HashJoinState *node);
extern void ExecShutdownHashJoin(HashJoinState *node);
extern void ExecHashJoinEstimate(HashJoinState *state, ParallelContext *pcxt);
extern void ExecHashJoinInitializeDSM(HashJoinState *state, ParallelContext *pcxt);
extern void ExecHashJoinReInitializeDSM(HashJoinState *state, ParallelContext *pcxt);
extern void ExecHashJoinInitializeWorker(HashJoinState *state,
ParallelWorkerContext *pwcxt);
extern void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue,
BufFile **fileptr);

View File

@ -25,6 +25,7 @@
#include "utils/hsearch.h"
#include "utils/queryenvironment.h"
#include "utils/reltrigger.h"
#include "utils/sharedtuplestore.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
#include "utils/tuplesort.h"
@ -43,6 +44,8 @@ struct ExprState; /* forward references in this file */
struct ExprContext;
struct ExprEvalStep; /* avoid including execExpr.h everywhere */
struct ParallelHashJoinState;
typedef Datum (*ExprStateEvalFunc) (struct ExprState *expression,
struct ExprContext *econtext,
bool *isNull);
@ -2026,6 +2029,9 @@ typedef struct HashState
SharedHashInfo *shared_info; /* one entry per worker */
HashInstrumentation *hinstrument; /* this worker's entry */
/* Parallel hash state. */
struct ParallelHashJoinState *parallel_state;
} HashState;
/* ----------------

View File

@ -880,6 +880,7 @@ typedef struct Hash
AttrNumber skewColumn; /* outer join key's column #, or zero */
bool skewInherit; /* is outer join rel an inheritance tree? */
/* all other info is in the parent HashJoin node */
double rows_total; /* estimate total rows if parallel_aware */
} Hash;
/* ----------------

View File

@ -1464,6 +1464,7 @@ typedef struct HashPath
JoinPath jpath;
List *path_hashclauses; /* join clauses used for hashing */
int num_batches; /* number of batches expected */
double inner_rows_total; /* total inner rows expected */
} HashPath;
/*
@ -2315,6 +2316,7 @@ typedef struct JoinCostWorkspace
/* private for cost_hashjoin code */
int numbuckets;
int numbatches;
double inner_rows_total;
} JoinCostWorkspace;
#endif /* RELATION_H */

View File

@ -69,6 +69,7 @@ extern bool enable_hashjoin;
extern bool enable_gathermerge;
extern bool enable_partition_wise_join;
extern bool enable_parallel_append;
extern bool enable_parallel_hash;
extern int constraint_exclusion;
extern double clamp_row_est(double nrows);
@ -153,7 +154,8 @@ extern void initial_cost_hashjoin(PlannerInfo *root,
JoinType jointype,
List *hashclauses,
Path *outer_path, Path *inner_path,
JoinPathExtraData *extra);
JoinPathExtraData *extra,
bool parallel_hash);
extern void final_cost_hashjoin(PlannerInfo *root, HashPath *path,
JoinCostWorkspace *workspace,
JoinPathExtraData *extra);

View File

@ -153,6 +153,7 @@ extern HashPath *create_hashjoin_path(PlannerInfo *root,
JoinPathExtraData *extra,
Path *outer_path,
Path *inner_path,
bool parallel_hash,
List *restrict_clauses,
Relids required_outer,
List *hashclauses);

View File

@ -803,6 +803,21 @@ typedef enum
WAIT_EVENT_BGWORKER_STARTUP,
WAIT_EVENT_BTREE_PAGE,
WAIT_EVENT_EXECUTE_GATHER,
WAIT_EVENT_HASH_BATCH_ALLOCATING,
WAIT_EVENT_HASH_BATCH_ELECTING,
WAIT_EVENT_HASH_BATCH_LOADING,
WAIT_EVENT_HASH_BUILD_ALLOCATING,
WAIT_EVENT_HASH_BUILD_ELECTING,
WAIT_EVENT_HASH_BUILD_HASHING_INNER,
WAIT_EVENT_HASH_BUILD_HASHING_OUTER,
WAIT_EVENT_HASH_GROW_BATCHES_ELECTING,
WAIT_EVENT_HASH_GROW_BATCHES_FINISHING,
WAIT_EVENT_HASH_GROW_BATCHES_REPARTITIONING,
WAIT_EVENT_HASH_GROW_BATCHES_ALLOCATING,
WAIT_EVENT_HASH_GROW_BATCHES_DECIDING,
WAIT_EVENT_HASH_GROW_BUCKETS_ELECTING,
WAIT_EVENT_HASH_GROW_BUCKETS_REINSERTING,
WAIT_EVENT_HASH_GROW_BUCKETS_ALLOCATING,
WAIT_EVENT_LOGICAL_SYNC_DATA,
WAIT_EVENT_LOGICAL_SYNC_STATE_CHANGE,
WAIT_EVENT_MQ_INTERNAL,

View File

@ -211,6 +211,7 @@ typedef enum BuiltinTrancheIds
LWTRANCHE_BUFFER_MAPPING,
LWTRANCHE_LOCK_MANAGER,
LWTRANCHE_PREDICATE_LOCK_MANAGER,
LWTRANCHE_PARALLEL_HASH_JOIN,
LWTRANCHE_PARALLEL_QUERY_DSA,
LWTRANCHE_SESSION_DSA,
LWTRANCHE_SESSION_RECORD_TABLE,

View File

@ -5884,6 +5884,9 @@ insert into extremely_skewed
update pg_class
set reltuples = 2, relpages = pg_relation_size('extremely_skewed') / 8192
where relname = 'extremely_skewed';
-- Make a relation with a couple of enormous tuples.
create table wide as select generate_series(1, 2) as id, rpad('', 320000, 'x') as t;
alter table wide set (parallel_workers = 2);
-- The "optimal" case: the hash table fits in memory; we plan for 1
-- batch, we stick to that number, and peak memory usage stays within
-- our work_mem budget
@ -5924,6 +5927,7 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '4MB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join simple s using (id);
QUERY PLAN
@ -5955,6 +5959,43 @@ $$);
f | f
(1 row)
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '4MB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join simple s using (id);
QUERY PLAN
-------------------------------------------------------------
Finalize Aggregate
-> Gather
Workers Planned: 2
-> Partial Aggregate
-> Parallel Hash Join
Hash Cond: (r.id = s.id)
-> Parallel Seq Scan on simple r
-> Parallel Hash
-> Parallel Seq Scan on simple s
(9 rows)
select count(*) from simple r join simple s using (id);
count
-------
20000
(1 row)
select original > 1 as initially_multibatch, final > original as increased_batches
from hash_join_batches(
$$
select count(*) from simple r join simple s using (id);
$$);
initially_multibatch | increased_batches
----------------------+-------------------
f | f
(1 row)
rollback to settings;
-- The "good" case: batches required, but we plan the right number; we
-- plan for some number of batches, and we stick to that number, and
@ -5996,6 +6037,7 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join simple s using (id);
QUERY PLAN
@ -6027,6 +6069,43 @@ $$);
t | f
(1 row)
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join simple s using (id);
QUERY PLAN
-------------------------------------------------------------
Finalize Aggregate
-> Gather
Workers Planned: 2
-> Partial Aggregate
-> Parallel Hash Join
Hash Cond: (r.id = s.id)
-> Parallel Seq Scan on simple r
-> Parallel Hash
-> Parallel Seq Scan on simple s
(9 rows)
select count(*) from simple r join simple s using (id);
count
-------
20000
(1 row)
select original > 1 as initially_multibatch, final > original as increased_batches
from hash_join_batches(
$$
select count(*) from simple r join simple s using (id);
$$);
initially_multibatch | increased_batches
----------------------+-------------------
t | f
(1 row)
rollback to settings;
-- The "bad" case: during execution we need to increase number of
-- batches; in this case we plan for 1 batch, and increase at least a
@ -6069,6 +6148,7 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join bigger_than_it_looks s using (id);
QUERY PLAN
@ -6100,6 +6180,43 @@ $$);
f | t
(1 row)
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 1;
set local work_mem = '192kB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join bigger_than_it_looks s using (id);
QUERY PLAN
---------------------------------------------------------------------------
Finalize Aggregate
-> Gather
Workers Planned: 1
-> Partial Aggregate
-> Parallel Hash Join
Hash Cond: (r.id = s.id)
-> Parallel Seq Scan on simple r
-> Parallel Hash
-> Parallel Seq Scan on bigger_than_it_looks s
(9 rows)
select count(*) from simple r join bigger_than_it_looks s using (id);
count
-------
20000
(1 row)
select original > 1 as initially_multibatch, final > original as increased_batches
from hash_join_batches(
$$
select count(*) from simple r join bigger_than_it_looks s using (id);
$$);
initially_multibatch | increased_batches
----------------------+-------------------
f | t
(1 row)
rollback to settings;
-- The "ugly" case: increasing the number of batches during execution
-- doesn't help, so stop trying to fit in work_mem and hope for the
@ -6142,6 +6259,7 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join extremely_skewed s using (id);
QUERY PLAN
@ -6171,6 +6289,42 @@ $$);
1 | 2
(1 row)
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 1;
set local work_mem = '128kB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join extremely_skewed s using (id);
QUERY PLAN
-----------------------------------------------------------------------
Finalize Aggregate
-> Gather
Workers Planned: 1
-> Partial Aggregate
-> Parallel Hash Join
Hash Cond: (r.id = s.id)
-> Parallel Seq Scan on simple r
-> Parallel Hash
-> Parallel Seq Scan on extremely_skewed s
(9 rows)
select count(*) from simple r join extremely_skewed s using (id);
count
-------
20000
(1 row)
select * from hash_join_batches(
$$
select count(*) from simple r join extremely_skewed s using (id);
$$);
original | final
----------+-------
1 | 4
(1 row)
rollback to settings;
-- A couple of other hash join tests unrelated to work_mem management.
-- Check that EXPLAIN ANALYZE has data even if the leader doesn't participate
@ -6192,10 +6346,11 @@ rollback to settings;
-- that we can check that instrumentation comes back correctly.
create table foo as select generate_series(1, 3) as id, 'xxxxx'::text as t;
alter table foo set (parallel_workers = 0);
create table bar as select generate_series(1, 5000) as id, 'xxxxx'::text as t;
create table bar as select generate_series(1, 10000) as id, 'xxxxx'::text as t;
alter table bar set (parallel_workers = 2);
-- multi-batch with rescan, parallel-oblivious
savepoint settings;
set enable_parallel_hash = off;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
@ -6246,6 +6401,7 @@ $$);
rollback to settings;
-- single-batch with rescan, parallel-oblivious
savepoint settings;
set enable_parallel_hash = off;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
@ -6293,6 +6449,108 @@ $$);
f
(1 row)
rollback to settings;
-- multi-batch with rescan, parallel-aware
savepoint settings;
set enable_parallel_hash = on;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
set parallel_tuple_cost = 0;
set max_parallel_workers_per_gather = 2;
set enable_material = off;
set enable_mergejoin = off;
set work_mem = '64kB';
explain (costs off)
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
QUERY PLAN
--------------------------------------------------------------------------
Aggregate
-> Nested Loop Left Join
Join Filter: ((foo.id < (b1.id + 1)) AND (foo.id > (b1.id - 1)))
-> Seq Scan on foo
-> Gather
Workers Planned: 2
-> Parallel Hash Join
Hash Cond: (b1.id = b2.id)
-> Parallel Seq Scan on bar b1
-> Parallel Hash
-> Parallel Seq Scan on bar b2
(11 rows)
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
count
-------
3
(1 row)
select final > 1 as multibatch
from hash_join_batches(
$$
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
$$);
multibatch
------------
t
(1 row)
rollback to settings;
-- single-batch with rescan, parallel-aware
savepoint settings;
set enable_parallel_hash = on;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
set parallel_tuple_cost = 0;
set max_parallel_workers_per_gather = 2;
set enable_material = off;
set enable_mergejoin = off;
set work_mem = '4MB';
explain (costs off)
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
QUERY PLAN
--------------------------------------------------------------------------
Aggregate
-> Nested Loop Left Join
Join Filter: ((foo.id < (b1.id + 1)) AND (foo.id > (b1.id - 1)))
-> Seq Scan on foo
-> Gather
Workers Planned: 2
-> Parallel Hash Join
Hash Cond: (b1.id = b2.id)
-> Parallel Seq Scan on bar b1
-> Parallel Hash
-> Parallel Seq Scan on bar b2
(11 rows)
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
count
-------
3
(1 row)
select final > 1 as multibatch
from hash_join_batches(
$$
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
$$);
multibatch
------------
f
(1 row)
rollback to settings;
-- A full outer join where every record is matched.
-- non-parallel
@ -6383,5 +6641,49 @@ select count(*) from simple r full outer join simple s on (r.id = 0 - s.id);
40000
(1 row)
rollback to settings;
-- exercise special code paths for huge tuples (note use of non-strict
-- expression and left join required to get the detoasted tuple into
-- the hash table)
-- parallel with parallel-aware hash join (hits ExecParallelHashLoadTuple and
-- sts_puttuple oversized tuple cases because it's multi-batch)
savepoint settings;
set max_parallel_workers_per_gather = 2;
set enable_parallel_hash = on;
set work_mem = '128kB';
explain (costs off)
select length(max(s.t))
from wide left join (select id, coalesce(t, '') || '' as t from wide) s using (id);
QUERY PLAN
----------------------------------------------------------------
Finalize Aggregate
-> Gather
Workers Planned: 2
-> Partial Aggregate
-> Parallel Hash Left Join
Hash Cond: (wide.id = wide_1.id)
-> Parallel Seq Scan on wide
-> Parallel Hash
-> Parallel Seq Scan on wide wide_1
(9 rows)
select length(max(s.t))
from wide left join (select id, coalesce(t, '') || '' as t from wide) s using (id);
length
--------
320000
(1 row)
select final > 1 as multibatch
from hash_join_batches(
$$
select length(max(s.t))
from wide left join (select id, coalesce(t, '') || '' as t from wide) s using (id);
$$);
multibatch
------------
t
(1 row)
rollback to settings;
rollback;

View File

@ -82,11 +82,12 @@ select name, setting from pg_settings where name like 'enable%';
enable_mergejoin | on
enable_nestloop | on
enable_parallel_append | on
enable_parallel_hash | on
enable_partition_wise_join | off
enable_seqscan | on
enable_sort | on
enable_tidscan | on
(14 rows)
(15 rows)
-- Test that the pg_timezone_names and pg_timezone_abbrevs views are
-- more-or-less working. We can't test their contents in any great detail

View File

@ -2028,6 +2028,10 @@ update pg_class
set reltuples = 2, relpages = pg_relation_size('extremely_skewed') / 8192
where relname = 'extremely_skewed';
-- Make a relation with a couple of enormous tuples.
create table wide as select generate_series(1, 2) as id, rpad('', 320000, 'x') as t;
alter table wide set (parallel_workers = 2);
-- The "optimal" case: the hash table fits in memory; we plan for 1
-- batch, we stick to that number, and peak memory usage stays within
-- our work_mem budget
@ -2050,6 +2054,22 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '4MB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join simple s using (id);
select count(*) from simple r join simple s using (id);
select original > 1 as initially_multibatch, final > original as increased_batches
from hash_join_batches(
$$
select count(*) from simple r join simple s using (id);
$$);
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '4MB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join simple s using (id);
select count(*) from simple r join simple s using (id);
@ -2082,6 +2102,22 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join simple s using (id);
select count(*) from simple r join simple s using (id);
select original > 1 as initially_multibatch, final > original as increased_batches
from hash_join_batches(
$$
select count(*) from simple r join simple s using (id);
$$);
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join simple s using (id);
select count(*) from simple r join simple s using (id);
@ -2115,6 +2151,22 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join bigger_than_it_looks s using (id);
select count(*) from simple r join bigger_than_it_looks s using (id);
select original > 1 as initially_multibatch, final > original as increased_batches
from hash_join_batches(
$$
select count(*) from simple r join bigger_than_it_looks s using (id);
$$);
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 1;
set local work_mem = '192kB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join bigger_than_it_looks s using (id);
select count(*) from simple r join bigger_than_it_looks s using (id);
@ -2148,6 +2200,21 @@ rollback to settings;
savepoint settings;
set local max_parallel_workers_per_gather = 2;
set local work_mem = '128kB';
set local enable_parallel_hash = off;
explain (costs off)
select count(*) from simple r join extremely_skewed s using (id);
select count(*) from simple r join extremely_skewed s using (id);
select * from hash_join_batches(
$$
select count(*) from simple r join extremely_skewed s using (id);
$$);
rollback to settings;
-- parallel with parallel-aware hash join
savepoint settings;
set local max_parallel_workers_per_gather = 1;
set local work_mem = '128kB';
set local enable_parallel_hash = on;
explain (costs off)
select count(*) from simple r join extremely_skewed s using (id);
select count(*) from simple r join extremely_skewed s using (id);
@ -2175,11 +2242,12 @@ rollback to settings;
create table foo as select generate_series(1, 3) as id, 'xxxxx'::text as t;
alter table foo set (parallel_workers = 0);
create table bar as select generate_series(1, 5000) as id, 'xxxxx'::text as t;
create table bar as select generate_series(1, 10000) as id, 'xxxxx'::text as t;
alter table bar set (parallel_workers = 2);
-- multi-batch with rescan, parallel-oblivious
savepoint settings;
set enable_parallel_hash = off;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
@ -2206,6 +2274,61 @@ rollback to settings;
-- single-batch with rescan, parallel-oblivious
savepoint settings;
set enable_parallel_hash = off;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
set parallel_tuple_cost = 0;
set max_parallel_workers_per_gather = 2;
set enable_material = off;
set enable_mergejoin = off;
set work_mem = '4MB';
explain (costs off)
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
select final > 1 as multibatch
from hash_join_batches(
$$
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
$$);
rollback to settings;
-- multi-batch with rescan, parallel-aware
savepoint settings;
set enable_parallel_hash = on;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
set parallel_tuple_cost = 0;
set max_parallel_workers_per_gather = 2;
set enable_material = off;
set enable_mergejoin = off;
set work_mem = '64kB';
explain (costs off)
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
select final > 1 as multibatch
from hash_join_batches(
$$
select count(*) from foo
left join (select b1.id, b1.t from bar b1 join bar b2 using (id)) ss
on foo.id < ss.id + 1 and foo.id > ss.id - 1;
$$);
rollback to settings;
-- single-batch with rescan, parallel-aware
savepoint settings;
set enable_parallel_hash = on;
set parallel_leader_participation = off;
set min_parallel_table_scan_size = 0;
set parallel_setup_cost = 0;
@ -2266,4 +2389,27 @@ explain (costs off)
select count(*) from simple r full outer join simple s on (r.id = 0 - s.id);
rollback to settings;
-- exercise special code paths for huge tuples (note use of non-strict
-- expression and left join required to get the detoasted tuple into
-- the hash table)
-- parallel with parallel-aware hash join (hits ExecParallelHashLoadTuple and
-- sts_puttuple oversized tuple cases because it's multi-batch)
savepoint settings;
set max_parallel_workers_per_gather = 2;
set enable_parallel_hash = on;
set work_mem = '128kB';
explain (costs off)
select length(max(s.t))
from wide left join (select id, coalesce(t, '') || '' as t from wide) s using (id);
select length(max(s.t))
from wide left join (select id, coalesce(t, '') || '' as t from wide) s using (id);
select final > 1 as multibatch
from hash_join_batches(
$$
select length(max(s.t))
from wide left join (select id, coalesce(t, '') || '' as t from wide) s using (id);
$$);
rollback to settings;
rollback;

View File

@ -1542,6 +1542,10 @@ ParallelBitmapHeapState
ParallelCompletionPtr
ParallelContext
ParallelExecutorInfo
ParallelHashGrowth
ParallelHashJoinBatch
ParallelHashJoinBatchAccessor
ParallelHashJoinState
ParallelHeapScanDesc
ParallelIndexScanDesc
ParallelSlot