Improve hash join's handling of tuples with null join keys - Mailing list pgsql-hackers
From | Tom Lane |
---|---|
Subject | Improve hash join's handling of tuples with null join keys |
Date | |
Msg-id | 3061845.1746486714@sss.pgh.pa.us Whole thread Raw |
Responses |
Re: Improve hash join's handling of tuples with null join keys
|
List | pgsql-hackers |
The attached patch is a response to the discussion at [1], where it emerged that lots of rows with null join keys can send a hash join into too-many-batches hell, if they are on the outer side of the join so that they must be null-extended not just discarded. This isn't really surprising given that such rows will certainly end up in the same hash bucket, and no amount of splitting can reduce the size of that bucket. (I'm a bit surprised that the growEnabled heuristic didn't kick in, but it seems it didn't, at least not up to several million batches.) Thinking about that, it occurred to me to wonder why we are putting null-keyed tuples into the hash table at all. They cannot match anything, so all we really have to do with them is emit one null-extended copy. Awhile later I had the attached, which shoves such rows into a tuplestore that's separate from the hash table proper, ensuring that they can't bollix our algorithms for when to grow the hash table. (For tuples coming from the right input, we need to use a tuplestore in case we're asked to rescan the existing hashtable. For tuples coming from the left input, we could theoretically emit 'em and forget 'em immediately, but that'd require some major code restructuring so I decided to just use a tuplestore there too.) This passes check-world, and I've extended a couple of existing test cases to ensure that the new code paths are exercised. I've not done any real performance testing, though. regards, tom lane [1] https://www.postgresql.org/message-id/flat/18909-e5e1b702c9441b8a%40postgresql.org From cd1cf545583a328e756a07818669c1667fb51bb8 Mon Sep 17 00:00:00 2001 From: Tom Lane <tgl@sss.pgh.pa.us> Date: Mon, 5 May 2025 16:02:46 -0400 Subject: [PATCH v1] Improve hash join's handling of tuples with null join keys. In a plain join, we can just summarily discard an input tuple with null join key(s), since it cannot match anything from the other side of the join (assuming a strict join operator). However, if the tuple comes from the outer side of an outer join then we have to emit it with null-extension of the other side. Up to now, hash joins did that by inserting the tuple into the hash table as though it were a normal tuple. This is unnecessarily inefficient though, since the required processing is far simpler than for a potentially-matchable tuple. Worse, if there are a lot of such tuples they will bloat the hash bucket they go into, possibly causing useless repeated attempts to split that bucket or increase the number of batches. We have a report of a large join vainly creating many thousands of batches when faced with such input. This patch improves the situation by keeping such tuples out of the hash table altogether, instead pushing them into a separate tuplestore from which we return them later. (One might consider trying to return them immediately; but that would require substantial refactoring, and it doesn't work anyway for the case where we rescan an unmodified hash table.) This works even in parallel hash joins, because whichever worker reads a null-keyed tuple can just return it; there's no need for consultation with other workers. Thus the tuplestores are local storage even in a parallel join. --- src/backend/executor/execExpr.c | 20 +- src/backend/executor/nodeHash.c | 66 +++++-- src/backend/executor/nodeHashjoin.c | 247 +++++++++++++++++++++--- src/backend/utils/sort/tuplestore.c | 32 +++ src/include/executor/executor.h | 2 +- src/include/executor/hashjoin.h | 9 + src/include/executor/nodeHash.h | 1 + src/include/nodes/execnodes.h | 8 + src/include/utils/tuplestore.h | 3 + src/test/regress/expected/join.out | 7 +- src/test/regress/expected/join_hash.out | 15 +- src/test/regress/sql/join.sql | 4 +- src/test/regress/sql/join_hash.sql | 1 + 13 files changed, 356 insertions(+), 59 deletions(-) diff --git a/src/backend/executor/execExpr.c b/src/backend/executor/execExpr.c index f1569879b52..2cd5eb07985 100644 --- a/src/backend/executor/execExpr.c +++ b/src/backend/executor/execExpr.c @@ -4282,25 +4282,25 @@ ExecBuildHash32FromAttrs(TupleDesc desc, const TupleTableSlotOps *ops, * 'hash_exprs'. When multiple expressions are present, the hash values * returned by each hash function are combined to produce a single hash value. * + * If any hash_expr yields NULL and the corresponding hash function is strict, + * the created ExprState will return NULL. + * * desc: tuple descriptor for the to-be-hashed expressions * ops: TupleTableSlotOps for the TupleDesc * hashfunc_oids: Oid for each hash function to call, one for each 'hash_expr' - * collations: collation to use when calling the hash function. - * hash_expr: list of expressions to hash the value of - * opstrict: array corresponding to the 'hashfunc_oids' to store op_strict() + * collations: collation to use when calling the hash function + * hash_exprs: list of expressions to hash the value of + * opstrict: strictness flag for each hash function * parent: PlanState node that the 'hash_exprs' will be evaluated at * init_value: Normally 0, but can be set to other values to seed the hash * with some other value. Using non-zero is slightly less efficient but can * be useful. - * keep_nulls: if true, evaluation of the returned ExprState will abort early - * returning NULL if the given hash function is strict and the Datum to hash - * is null. When set to false, any NULL input Datums are skipped. */ ExprState * ExecBuildHash32Expr(TupleDesc desc, const TupleTableSlotOps *ops, const Oid *hashfunc_oids, const List *collations, const List *hash_exprs, const bool *opstrict, - PlanState *parent, uint32 init_value, bool keep_nulls) + PlanState *parent, uint32 init_value) { ExprState *state = makeNode(ExprState); ExprEvalStep scratch = {0}; @@ -4377,8 +4377,8 @@ ExecBuildHash32Expr(TupleDesc desc, const TupleTableSlotOps *ops, fmgr_info(funcid, finfo); /* - * Build the steps to evaluate the hash function's argument have it so - * the value of that is stored in the 0th argument of the hash func. + * Build the steps to evaluate the hash function's argument, placing + * the value in the 0th argument of the hash func. */ ExecInitExprRec(expr, state, @@ -4413,7 +4413,7 @@ ExecBuildHash32Expr(TupleDesc desc, const TupleTableSlotOps *ops, scratch.d.hashdatum.fcinfo_data = fcinfo; scratch.d.hashdatum.fn_addr = finfo->fn_addr; - scratch.opcode = opstrict[i] && !keep_nulls ? strict_opcode : opcode; + scratch.opcode = opstrict[i] ? strict_opcode : opcode; scratch.d.hashdatum.jumpdone = -1; ExprEvalPushStep(state, &scratch); diff --git a/src/backend/executor/nodeHash.c b/src/backend/executor/nodeHash.c index 8d2201ab67f..003814a4d31 100644 --- a/src/backend/executor/nodeHash.c +++ b/src/backend/executor/nodeHash.c @@ -154,8 +154,11 @@ MultiExecPrivateHash(HashState *node) econtext = node->ps.ps_ExprContext; /* - * Get all tuples from the node below the Hash node and insert into the - * hash table (or temp files). + * Get all tuples from the node below the Hash node and insert the + * potentially-matchable ones into the hash table (or temp files). Tuples + * that can't possibly match because they have null join keys are dumped + * into a separate tuplestore, or just summarily discarded if we don't + * need to emit them with null-extension. */ for (;;) { @@ -175,6 +178,7 @@ MultiExecPrivateHash(HashState *node) if (!isnull) { + /* normal case with a non-null join key */ uint32 hashvalue = DatumGetUInt32(hashdatum); int bucketNumber; @@ -193,6 +197,14 @@ MultiExecPrivateHash(HashState *node) } hashtable->totalTuples += 1; } + else if (node->keep_null_tuples) + { + /* null join key, but we must save tuple to be emitted later */ + if (node->null_tuple_store == NULL) + node->null_tuple_store = ExecHashBuildNullTupleStore(hashtable); + tuplestore_puttupleslot(node->null_tuple_store, slot); + } + /* else we can discard the tuple immediately */ } /* resize the hash table if needed (NTUP_PER_BUCKET exceeded) */ @@ -223,7 +235,6 @@ MultiExecParallelHash(HashState *node) HashJoinTable hashtable; TupleTableSlot *slot; ExprContext *econtext; - uint32 hashvalue; Barrier *build_barrier; int i; @@ -283,6 +294,7 @@ MultiExecParallelHash(HashState *node) for (;;) { bool isnull; + uint32 hashvalue; slot = ExecProcNode(outerNode); if (TupIsNull(slot)) @@ -296,8 +308,19 @@ MultiExecParallelHash(HashState *node) &isnull)); if (!isnull) + { + /* normal case with a non-null join key */ ExecParallelHashTableInsert(hashtable, slot, hashvalue); - hashtable->partialTuples++; + hashtable->partialTuples++; + } + else if (node->keep_null_tuples) + { + /* null join key, but save tuple to be emitted later */ + if (node->null_tuple_store == NULL) + node->null_tuple_store = ExecHashBuildNullTupleStore(hashtable); + tuplestore_puttupleslot(node->null_tuple_store, slot); + } + /* else we can discard the tuple immediately */ } /* @@ -405,14 +428,10 @@ ExecInitHash(Hash *node, EState *estate, int eflags) Assert(node->plan.qual == NIL); - /* - * Delay initialization of hash_expr until ExecInitHashJoin(). We cannot - * build the ExprState here as we don't yet know the join type we're going - * to be hashing values for and we need to know that before calling - * ExecBuildHash32Expr as the keep_nulls parameter depends on the join - * type. - */ + /* these fields will be filled by ExecInitHashJoin() */ hashstate->hash_expr = NULL; + hashstate->null_tuple_store = NULL; + hashstate->keep_null_tuples = false; return hashstate; } @@ -2748,6 +2767,31 @@ ExecHashRemoveNextSkewBucket(HashJoinTable hashtable) } } +/* + * Build a tuplestore suitable for holding null-keyed input tuples. + * (This function doesn't care whether it's for outer or inner tuples.) + * + * Note that in a parallel hash join, each worker has its own tuplestore(s) + * for these. There's no need to interact with other workers to decide + * what to do with them. So they're always in private storage. + */ +Tuplestorestate * +ExecHashBuildNullTupleStore(HashJoinTable hashtable) +{ + Tuplestorestate *tstore; + MemoryContext oldcxt; + + /* + * We keep the tuplestore in the hashCxt to ensure it won't go away too + * soon. Size it at work_mem/16 so that it doesn't bloat the node's space + * consumption too much. + */ + oldcxt = MemoryContextSwitchTo(hashtable->hashCxt); + tstore = tuplestore_begin_heap(false, false, work_mem / 16); + MemoryContextSwitchTo(oldcxt); + return tstore; +} + /* * Reserve space in the DSM segment for instrumentation data. */ diff --git a/src/backend/executor/nodeHashjoin.c b/src/backend/executor/nodeHashjoin.c index 5661ad76830..6a42041c927 100644 --- a/src/backend/executor/nodeHashjoin.c +++ b/src/backend/executor/nodeHashjoin.c @@ -182,7 +182,9 @@ #define HJ_SCAN_BUCKET 3 #define HJ_FILL_OUTER_TUPLE 4 #define HJ_FILL_INNER_TUPLES 5 -#define HJ_NEED_NEW_BATCH 6 +#define HJ_FILL_OUTER_NULL_TUPLES 6 +#define HJ_FILL_INNER_NULL_TUPLES 7 +#define HJ_NEED_NEW_BATCH 8 /* Returns true if doing null-fill on outer relation */ #define HJ_FILL_OUTER(hjstate) ((hjstate)->hj_NullInnerTupleSlot != NULL) @@ -346,9 +348,16 @@ ExecHashJoinImpl(PlanState *pstate, bool parallel) /* * If the inner relation is completely empty, and we're not * doing a left outer join, we can quit without scanning the - * outer relation. + * outer relation. (If the inner relation contains only + * null-keyed tuples that we need to emit, we'll fall through + * and do the outer-relation scan. In principle we could go + * emit those tuples then quit, but it would complicate the + * state machine logic. The case seems rare enough to not be + * worth optimizing.) */ - if (hashtable->totalTuples == 0 && !HJ_FILL_OUTER(node)) + if (hashtable->totalTuples == 0 && + hashNode->null_tuple_store == NULL && + !HJ_FILL_OUTER(node)) { if (parallel) { @@ -440,12 +449,17 @@ ExecHashJoinImpl(PlanState *pstate, bool parallel) if (parallel) { /* - * Only one process is currently allow to handle + * Only one process is currently allowed to handle * each batch's unmatched tuples, in a parallel - * join. + * join. However, each process must deal with any + * null-keyed tuples it found. */ if (ExecParallelPrepHashTableForUnmatched(node)) node->hj_JoinState = HJ_FILL_INNER_TUPLES; + else if (node->hj_NullOuterTupleStore) + node->hj_JoinState = HJ_FILL_OUTER_NULL_TUPLES; + else if (hashNode->null_tuple_store) + node->hj_JoinState = HJ_FILL_INNER_NULL_TUPLES; else node->hj_JoinState = HJ_NEED_NEW_BATCH; } @@ -456,7 +470,14 @@ ExecHashJoinImpl(PlanState *pstate, bool parallel) } } else - node->hj_JoinState = HJ_NEED_NEW_BATCH; + { + /* might have outer null-keyed tuples to fill */ + Assert(hashNode->null_tuple_store == NULL); + if (node->hj_NullOuterTupleStore) + node->hj_JoinState = HJ_FILL_OUTER_NULL_TUPLES; + else + node->hj_JoinState = HJ_NEED_NEW_BATCH; + } continue; } @@ -632,8 +653,13 @@ ExecHashJoinImpl(PlanState *pstate, bool parallel) if (!(parallel ? ExecParallelScanHashTableForUnmatched(node, econtext) : ExecScanHashTableForUnmatched(node, econtext))) { - /* no more unmatched tuples */ - node->hj_JoinState = HJ_NEED_NEW_BATCH; + /* no more unmatched tuples, but maybe there are nulls */ + if (node->hj_NullOuterTupleStore) + node->hj_JoinState = HJ_FILL_OUTER_NULL_TUPLES; + else if (hashNode->null_tuple_store) + node->hj_JoinState = HJ_FILL_INNER_NULL_TUPLES; + else + node->hj_JoinState = HJ_NEED_NEW_BATCH; continue; } @@ -649,6 +675,93 @@ ExecHashJoinImpl(PlanState *pstate, bool parallel) InstrCountFiltered2(node, 1); break; + case HJ_FILL_OUTER_NULL_TUPLES: + + /* + * We have finished a batch, but we are doing left/full join, + * so any null-keyed outer tuples have to be emitted before we + * continue to the next batch. + * + * (We could delay this till the end of the join, but there + * seems little percentage in that.) + * + * We have to use tuplestore_gettupleslot_force because + * hj_OuterTupleSlot may not be able to store a MinimalTuple. + */ + while (tuplestore_gettupleslot_force(node->hj_NullOuterTupleStore, + true, false, + node->hj_OuterTupleSlot)) + { + /* + * Generate a fake join tuple with nulls for the inner + * tuple, and return it if it passes the non-join quals. + */ + econtext->ecxt_outertuple = node->hj_OuterTupleSlot; + econtext->ecxt_innertuple = node->hj_NullInnerTupleSlot; + + if (otherqual == NULL || ExecQual(otherqual, econtext)) + return ExecProject(node->js.ps.ps_ProjInfo); + else + InstrCountFiltered2(node, 1); + + ResetExprContext(econtext); + + /* allow this loop to be cancellable */ + CHECK_FOR_INTERRUPTS(); + } + + /* We don't need the tuplestore any more, so discard it. */ + tuplestore_end(node->hj_NullOuterTupleStore); + node->hj_NullOuterTupleStore = NULL; + + /* Fill inner tuples too if it's a full join, else advance. */ + if (hashNode->null_tuple_store) + node->hj_JoinState = HJ_FILL_INNER_NULL_TUPLES; + else + node->hj_JoinState = HJ_NEED_NEW_BATCH; + break; + + case HJ_FILL_INNER_NULL_TUPLES: + + /* + * We have finished a batch, but we are doing + * right/right-anti/full join, so any null-keyed inner tuples + * have to be emitted before we continue to the next batch. + * + * (We could delay this till the end of the join, but there + * seems little percentage in that.) + */ + while (tuplestore_gettupleslot(hashNode->null_tuple_store, + true, false, + node->hj_HashTupleSlot)) + { + /* + * Generate a fake join tuple with nulls for the outer + * tuple, and return it if it passes the non-join quals. + */ + econtext->ecxt_outertuple = node->hj_NullOuterTupleSlot; + econtext->ecxt_innertuple = node->hj_HashTupleSlot; + + if (otherqual == NULL || ExecQual(otherqual, econtext)) + return ExecProject(node->js.ps.ps_ProjInfo); + else + InstrCountFiltered2(node, 1); + + ResetExprContext(econtext); + + /* allow this loop to be cancellable */ + CHECK_FOR_INTERRUPTS(); + } + + /* + * Ideally we'd discard the tuplestore now, but we can't + * because we might need it for rescans. + */ + + /* Now we can advance to the next batch. */ + node->hj_JoinState = HJ_NEED_NEW_BATCH; + break; + case HJ_NEED_NEW_BATCH: /* @@ -831,10 +944,7 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) /* * Build ExprStates to obtain hash values for either side of the join. - * This must be done here as ExecBuildHash32Expr needs to know how to - * handle NULL inputs and the required handling of that depends on the - * jointype. We don't know the join type in ExecInitHash() and we - * must build the ExprStates before ExecHashTableCreate() so we + * Note: must build the ExprStates before ExecHashTableCreate() so we * properly attribute any SubPlans that exist in the hash expressions * to the correct PlanState. */ @@ -846,7 +956,7 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) /* * Determine the hash function for each side of the join for the given - * hash operator. + * join operator, and detect whether the join operator is strict. */ foreach(lc, node->hashoperators) { @@ -864,11 +974,7 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) /* * Build an ExprState to generate the hash value for the expressions - * on the outer of the join. This ExprState must finish generating - * the hash value when HJ_FILL_OUTER() is true. Otherwise, - * ExecBuildHash32Expr will set up the ExprState to abort early if it - * finds a NULL. In these cases, we don't need to store these tuples - * in the hash table as the jointype does not require it. + * on the outer side of the join. */ hjstate->hj_OuterHash = ExecBuildHash32Expr(hjstate->js.ps.ps_ResultTupleDesc, @@ -878,8 +984,7 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) node->hashkeys, hash_strict, &hjstate->js.ps, - 0, - HJ_FILL_OUTER(hjstate)); + 0); /* As above, but for the inner side of the join */ hashstate->hash_expr = @@ -890,8 +995,11 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) hash->hashkeys, hash_strict, &hashstate->ps, - 0, - HJ_FILL_INNER(hjstate)); + 0); + + /* Remember whether we need to save tuples with null join keys */ + hjstate->hj_KeepNullTuples = HJ_FILL_OUTER(hjstate); + hashstate->keep_null_tuples = HJ_FILL_INNER(hjstate); /* * Set up the skew table hash function while we have a record of the @@ -924,6 +1032,7 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) * initialize hash-specific info */ hjstate->hj_HashTable = NULL; + hjstate->hj_NullOuterTupleStore = NULL; hjstate->hj_FirstOuterTupleSlot = NULL; hjstate->hj_CurHashValue = 0; @@ -947,6 +1056,23 @@ ExecInitHashJoin(HashJoin *node, EState *estate, int eflags) void ExecEndHashJoin(HashJoinState *node) { + HashState *hashNode = castNode(HashState, innerPlanState(node)); + + /* + * Free tuple stores if we made them (must do this before + * ExecHashTableDestroy deletes hashCxt) + */ + if (node->hj_NullOuterTupleStore) + { + tuplestore_end(node->hj_NullOuterTupleStore); + node->hj_NullOuterTupleStore = NULL; + } + if (hashNode->null_tuple_store) + { + tuplestore_end(hashNode->null_tuple_store); + hashNode->null_tuple_store = NULL; + } + /* * Free hash table */ @@ -1015,11 +1141,19 @@ ExecHashJoinOuterGetTuple(PlanState *outerNode, if (!isnull) { + /* normal case with a non-null join key */ /* remember outer relation is not empty for possible rescan */ hjstate->hj_OuterNotEmpty = true; return slot; } + else if (hjstate->hj_KeepNullTuples) + { + /* null join key, but we must save tuple to be emitted later */ + if (hjstate->hj_NullOuterTupleStore == NULL) + hjstate->hj_NullOuterTupleStore = ExecHashBuildNullTupleStore(hashtable); + tuplestore_puttupleslot(hjstate->hj_NullOuterTupleStore, slot); + } /* * That tuple couldn't match because of a NULL, so discard it and @@ -1087,7 +1221,17 @@ ExecParallelHashJoinOuterGetTuple(PlanState *outerNode, &isnull)); if (!isnull) + { + /* normal case with a non-null join key */ return slot; + } + else if (hjstate->hj_KeepNullTuples) + { + /* null join key, but we must save tuple to be emitted later */ + if (hjstate->hj_NullOuterTupleStore == NULL) + hjstate->hj_NullOuterTupleStore = ExecHashBuildNullTupleStore(hashtable); + tuplestore_puttupleslot(hjstate->hj_NullOuterTupleStore, slot); + } /* * That tuple couldn't match because of a NULL, so discard it and @@ -1496,6 +1640,17 @@ ExecReScanHashJoin(HashJoinState *node) PlanState *outerPlan = outerPlanState(node); PlanState *innerPlan = innerPlanState(node); + /* + * We're always going to rescan the outer rel, so drop the associated + * null-keys tuplestore; we'll rebuild it during the rescan. (Must do + * this before ExecHashTableDestroy deletes hashCxt.) + */ + if (node->hj_NullOuterTupleStore) + { + tuplestore_end(node->hj_NullOuterTupleStore); + node->hj_NullOuterTupleStore = NULL; + } + /* * In a multi-batch join, we currently have to do rescans the hard way, * primarily because batch temp files may have already been released. But @@ -1505,6 +1660,10 @@ ExecReScanHashJoin(HashJoinState *node) */ if (node->hj_HashTable != NULL) { + HashState *hashNode = castNode(HashState, innerPlan); + + Assert(hashNode->hashtable == node->hj_HashTable); + if (node->hj_HashTable->nbatch == 1 && innerPlan->chgParam == NULL) { @@ -1529,15 +1688,20 @@ ExecReScanHashJoin(HashJoinState *node) */ node->hj_OuterNotEmpty = false; + /* + * Also, rewind inner null-key tuplestore so that we can return + * those tuples again. + */ + if (hashNode->null_tuple_store) + tuplestore_rescan(hashNode->null_tuple_store); + /* ExecHashJoin can skip the BUILD_HASHTABLE step */ node->hj_JoinState = HJ_NEED_NEW_OUTER; } else { /* must destroy and rebuild hash table */ - HashState *hashNode = castNode(HashState, innerPlan); - Assert(hashNode->hashtable == node->hj_HashTable); /* accumulate stats from old hash table, if wanted */ /* (this should match ExecShutdownHash) */ if (hashNode->ps.instrument && !hashNode->hinstrument) @@ -1546,6 +1710,14 @@ ExecReScanHashJoin(HashJoinState *node) if (hashNode->hinstrument) ExecHashAccumInstrumentation(hashNode->hinstrument, hashNode->hashtable); + + /* free inner null-key tuplestore before ExecHashTableDestroy */ + if (hashNode->null_tuple_store) + { + tuplestore_end(hashNode->null_tuple_store); + hashNode->null_tuple_store = NULL; + } + /* for safety, be sure to clear child plan node's pointer too */ hashNode->hashtable = NULL; @@ -1601,7 +1773,6 @@ ExecParallelHashJoinPartitionOuter(HashJoinState *hjstate) ExprContext *econtext = hjstate->js.ps.ps_ExprContext; HashJoinTable hashtable = hjstate->hj_HashTable; TupleTableSlot *slot; - uint32 hashvalue; int i; Assert(hjstate->hj_FirstOuterTupleSlot == NULL); @@ -1610,6 +1781,7 @@ ExecParallelHashJoinPartitionOuter(HashJoinState *hjstate) for (;;) { bool isnull; + uint32 hashvalue; slot = ExecProcNode(outerState); if (TupIsNull(slot)) @@ -1624,6 +1796,7 @@ ExecParallelHashJoinPartitionOuter(HashJoinState *hjstate) if (!isnull) { + /* normal case with a non-null join key */ int batchno; int bucketno; bool shouldFree; @@ -1637,6 +1810,15 @@ ExecParallelHashJoinPartitionOuter(HashJoinState *hjstate) if (shouldFree) heap_free_minimal_tuple(mintup); } + else if (hjstate->hj_KeepNullTuples) + { + /* null join key, but we must save tuple to be emitted later */ + if (hjstate->hj_NullOuterTupleStore == NULL) + hjstate->hj_NullOuterTupleStore = ExecHashBuildNullTupleStore(hashtable); + tuplestore_puttupleslot(hjstate->hj_NullOuterTupleStore, slot); + } + /* else we can just discard the tuple immediately */ + CHECK_FOR_INTERRUPTS(); } @@ -1715,6 +1897,7 @@ ExecHashJoinReInitializeDSM(HashJoinState *state, ParallelContext *pcxt) { int plan_node_id = state->js.ps.plan->plan_node_id; ParallelHashJoinState *pstate; + HashState *hashNode; /* Nothing to do if we failed to create a DSM segment. */ if (pcxt->seg == NULL) @@ -1744,6 +1927,20 @@ ExecHashJoinReInitializeDSM(HashJoinState *state, ParallelContext *pcxt) /* Clear any shared batch files. */ SharedFileSetDeleteAll(&pstate->fileset); + /* We'd better clear our local null-key tuplestores, too. */ + if (state->hj_NullOuterTupleStore) + { + tuplestore_end(state->hj_NullOuterTupleStore); + state->hj_NullOuterTupleStore = NULL; + } + hashNode = (HashState *) innerPlanState(state); + if (hashNode->null_tuple_store) + { + tuplestore_end(hashNode->null_tuple_store); + hashNode->null_tuple_store = NULL; + } + + /* Reset build_barrier to PHJ_BUILD_ELECT so we can go around again. */ BarrierInit(&pstate->build_barrier, 0); } diff --git a/src/backend/utils/sort/tuplestore.c b/src/backend/utils/sort/tuplestore.c index c9aecab8d66..0e19ecc2f8d 100644 --- a/src/backend/utils/sort/tuplestore.c +++ b/src/backend/utils/sort/tuplestore.c @@ -1152,6 +1152,38 @@ tuplestore_gettupleslot(Tuplestorestate *state, bool forward, } } +/* + * tuplestore_gettupleslot_force - exported function to fetch a tuple + * + * This is identical to tuplestore_gettupleslot except the given slot can be + * any kind of slot; it need not be one that will accept a MinimalTuple. + */ +bool +tuplestore_gettupleslot_force(Tuplestorestate *state, bool forward, + bool copy, TupleTableSlot *slot) +{ + MinimalTuple tuple; + bool should_free; + + tuple = (MinimalTuple) tuplestore_gettuple(state, forward, &should_free); + + if (tuple) + { + if (copy && !should_free) + { + tuple = heap_copy_minimal_tuple(tuple, 0); + should_free = true; + } + ExecForceStoreMinimalTuple(tuple, slot, should_free); + return true; + } + else + { + ExecClearTuple(slot); + return false; + } +} + /* * tuplestore_advance - exported function to adjust position without fetching * diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h index ae99407db89..24010a7fdd6 100644 --- a/src/include/executor/executor.h +++ b/src/include/executor/executor.h @@ -366,7 +366,7 @@ extern ExprState *ExecBuildHash32Expr(TupleDesc desc, const List *collations, const List *hash_exprs, const bool *opstrict, PlanState *parent, - uint32 init_value, bool keep_nulls); + uint32 init_value); extern ExprState *ExecBuildGroupingEqual(TupleDesc ldesc, TupleDesc rdesc, const TupleTableSlotOps *lops, const TupleTableSlotOps *rops, int numCols, diff --git a/src/include/executor/hashjoin.h b/src/include/executor/hashjoin.h index ecff4842fd3..5f59b61f671 100644 --- a/src/include/executor/hashjoin.h +++ b/src/include/executor/hashjoin.h @@ -68,6 +68,15 @@ * inner batch file. Subsequently, while reading either inner or outer batch * files, we might find tuples that no longer belong to the current batch; * if so, we just dump them out to the correct batch file. + * + * If an input tuple has a null join key, then it cannot match anything from + * the other side of the join. Normally we can just discard such a tuple + * immediately, but if it comes from the outer side of an outer join then we + * must emit it with null-extension of the other side. For various reasons + * it's not convenient to do that immediately on seeing the tuple, so we dump + * the tuple into a tuplestore and emit it later. (In the unlikely but + * supported case of a non-strict join operator, we treat null keys as normal + * data.) * ---------------------------------------------------------------- */ diff --git a/src/include/executor/nodeHash.h b/src/include/executor/nodeHash.h index 3c1a09415aa..55b89febd1a 100644 --- a/src/include/executor/nodeHash.h +++ b/src/include/executor/nodeHash.h @@ -64,6 +64,7 @@ extern void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, int *numbatches, int *num_skew_mcvs); extern int ExecHashGetSkewBucket(HashJoinTable hashtable, uint32 hashvalue); +extern Tuplestorestate *ExecHashBuildNullTupleStore(HashJoinTable hashtable); extern void ExecHashEstimate(HashState *node, ParallelContext *pcxt); extern void ExecHashInitializeDSM(HashState *node, ParallelContext *pcxt); extern void ExecHashInitializeWorker(HashState *node, ParallelWorkerContext *pwcxt); diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h index 5b6cadb5a6c..d31c6c2e59e 100644 --- a/src/include/nodes/execnodes.h +++ b/src/include/nodes/execnodes.h @@ -2239,8 +2239,11 @@ typedef struct MergeJoinState * hj_NullOuterTupleSlot prepared null tuple for right/right-anti/full * outer joins * hj_NullInnerTupleSlot prepared null tuple for left/full outer joins + * hj_NullOuterTupleStore tuplestore holding outer tuples that have + * null join keys (but must be emitted anyway) * hj_FirstOuterTupleSlot first tuple retrieved from outer plan * hj_JoinState current state of ExecHashJoin state machine + * hj_KeepNullTuples true to keep outer tuples with null join keys * hj_MatchedOuter true if found a join match for current outer * hj_OuterNotEmpty true if outer relation known not empty * ---------------- @@ -2264,8 +2267,10 @@ typedef struct HashJoinState TupleTableSlot *hj_HashTupleSlot; TupleTableSlot *hj_NullOuterTupleSlot; TupleTableSlot *hj_NullInnerTupleSlot; + Tuplestorestate *hj_NullOuterTupleStore; TupleTableSlot *hj_FirstOuterTupleSlot; int hj_JoinState; + bool hj_KeepNullTuples; bool hj_MatchedOuter; bool hj_OuterNotEmpty; } HashJoinState; @@ -2815,6 +2820,9 @@ typedef struct HashState FmgrInfo *skew_hashfunction; /* lookup data for skew hash function */ Oid skew_collation; /* collation to call skew_hashfunction with */ + Tuplestorestate *null_tuple_store; /* where to put null-keyed tuples */ + bool keep_null_tuples; /* do we need to save such tuples? */ + /* * In a parallelized hash join, the leader retains a pointer to the * shared-memory stats area in its shared_info field, and then copies the diff --git a/src/include/utils/tuplestore.h b/src/include/utils/tuplestore.h index 865ba7b8265..b9e152c1701 100644 --- a/src/include/utils/tuplestore.h +++ b/src/include/utils/tuplestore.h @@ -73,6 +73,9 @@ extern bool tuplestore_in_memory(Tuplestorestate *state); extern bool tuplestore_gettupleslot(Tuplestorestate *state, bool forward, bool copy, TupleTableSlot *slot); +extern bool tuplestore_gettupleslot_force(Tuplestorestate *state, bool forward, + bool copy, TupleTableSlot *slot); + extern bool tuplestore_advance(Tuplestorestate *state, bool forward); extern bool tuplestore_skiptuples(Tuplestorestate *state, diff --git a/src/test/regress/expected/join.out b/src/test/regress/expected/join.out index f35a0b18c37..334d38b1052 100644 --- a/src/test/regress/expected/join.out +++ b/src/test/regress/expected/join.out @@ -4590,7 +4590,7 @@ order by fault; explain (costs off) select * from (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys) -left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +left join (values (1, 10), (2, 20), (2, null)) as v2(v2x,v2y) on v2x = v1x left join unnest(v1ys) as u1(u1y) on u1y = v2y; QUERY PLAN ------------------------------------------------------------- @@ -4606,13 +4606,14 @@ left join unnest(v1ys) as u1(u1y) on u1y = v2y; select * from (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys) -left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +left join (values (1, 10), (2, 20), (2, null)) as v2(v2x,v2y) on v2x = v1x left join unnest(v1ys) as u1(u1y) on u1y = v2y; v1x | v1ys | v2x | v2y | u1y -----+---------+-----+-----+----- 1 | {10,20} | 1 | 10 | 10 2 | {20,30} | 2 | 20 | 20 -(2 rows) + 2 | {20,30} | 2 | | +(3 rows) -- -- test handling of potential equivalence clauses above outer joins diff --git a/src/test/regress/expected/join_hash.out b/src/test/regress/expected/join_hash.out index 4fc34a0e72a..3df9f653d35 100644 --- a/src/test/regress/expected/join_hash.out +++ b/src/test/regress/expected/join_hash.out @@ -53,6 +53,7 @@ $$; -- estimated size. create table simple as select generate_series(1, 20000) AS id, 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'; +insert into simple values (null, null); alter table simple set (parallel_workers = 2); analyze simple; -- Make a relation whose size we will under-estimate. We want stats @@ -308,7 +309,7 @@ $$); select count(*) from simple r full outer join simple s using (id); count ------- - 20000 + 20002 (1 row) rollback to settings; @@ -786,7 +787,7 @@ explain (costs off) select count(*) from simple r full outer join simple s using (id); count ------- - 20000 + 20002 (1 row) rollback to settings; @@ -809,7 +810,7 @@ explain (costs off) select count(*) from simple r full outer join simple s using (id); count ------- - 20000 + 20002 (1 row) rollback to settings; @@ -834,7 +835,7 @@ explain (costs off) select count(*) from simple r full outer join simple s using (id); count ------- - 20000 + 20002 (1 row) rollback to settings; @@ -857,7 +858,7 @@ explain (costs off) select count(*) from simple r full outer join simple s on (r.id = 0 - s.id); count ------- - 40000 + 40002 (1 row) rollback to settings; @@ -880,7 +881,7 @@ explain (costs off) select count(*) from simple r full outer join simple s on (r.id = 0 - s.id); count ------- - 40000 + 40002 (1 row) rollback to settings; @@ -905,7 +906,7 @@ explain (costs off) select count(*) from simple r full outer join simple s on (r.id = 0 - s.id); count ------- - 40000 + 40002 (1 row) rollback to settings; diff --git a/src/test/regress/sql/join.sql b/src/test/regress/sql/join.sql index cc5128add4d..c4946d39e77 100644 --- a/src/test/regress/sql/join.sql +++ b/src/test/regress/sql/join.sql @@ -1554,12 +1554,12 @@ order by fault; explain (costs off) select * from (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys) -left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +left join (values (1, 10), (2, 20), (2, null)) as v2(v2x,v2y) on v2x = v1x left join unnest(v1ys) as u1(u1y) on u1y = v2y; select * from (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys) -left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +left join (values (1, 10), (2, 20), (2, null)) as v2(v2x,v2y) on v2x = v1x left join unnest(v1ys) as u1(u1y) on u1y = v2y; -- diff --git a/src/test/regress/sql/join_hash.sql b/src/test/regress/sql/join_hash.sql index 6b0688ab0a6..11e3a164c76 100644 --- a/src/test/regress/sql/join_hash.sql +++ b/src/test/regress/sql/join_hash.sql @@ -57,6 +57,7 @@ $$; -- estimated size. create table simple as select generate_series(1, 20000) AS id, 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'; +insert into simple values (null, null); alter table simple set (parallel_workers = 2); analyze simple; -- 2.43.5
pgsql-hackers by date: