PostgreSQL Source Code git master
nodeHash.h File Reference
#include "access/parallel.h"
#include "nodes/execnodes.h"
Include dependency graph for nodeHash.h:
This graph shows which files directly or indirectly include this file:

Go to the source code of this file.

Functions

HashStateExecInitHash (Hash *node, EState *estate, int eflags)
 
NodeMultiExecHash (HashState *node)
 
void ExecEndHash (HashState *node)
 
void ExecReScanHash (HashState *node)
 
HashJoinTable ExecHashTableCreate (HashState *state)
 
void ExecParallelHashTableAlloc (HashJoinTable hashtable, int batchno)
 
void ExecHashTableDestroy (HashJoinTable hashtable)
 
void ExecHashTableDetach (HashJoinTable hashtable)
 
void ExecHashTableDetachBatch (HashJoinTable hashtable)
 
void ExecParallelHashTableSetCurrentBatch (HashJoinTable hashtable, int batchno)
 
void ExecHashTableInsert (HashJoinTable hashtable, TupleTableSlot *slot, uint32 hashvalue)
 
void ExecParallelHashTableInsert (HashJoinTable hashtable, TupleTableSlot *slot, uint32 hashvalue)
 
void ExecParallelHashTableInsertCurrentBatch (HashJoinTable hashtable, TupleTableSlot *slot, uint32 hashvalue)
 
void ExecHashGetBucketAndBatch (HashJoinTable hashtable, uint32 hashvalue, int *bucketno, int *batchno)
 
bool ExecScanHashBucket (HashJoinState *hjstate, ExprContext *econtext)
 
bool ExecParallelScanHashBucket (HashJoinState *hjstate, ExprContext *econtext)
 
void ExecPrepHashTableForUnmatched (HashJoinState *hjstate)
 
bool ExecParallelPrepHashTableForUnmatched (HashJoinState *hjstate)
 
bool ExecScanHashTableForUnmatched (HashJoinState *hjstate, ExprContext *econtext)
 
bool ExecParallelScanHashTableForUnmatched (HashJoinState *hjstate, ExprContext *econtext)
 
void ExecHashTableReset (HashJoinTable hashtable)
 
void ExecHashTableResetMatchFlags (HashJoinTable hashtable)
 
void ExecChooseHashTableSize (double ntuples, int tupwidth, bool useskew, bool try_combined_hash_mem, int parallel_workers, size_t *space_allowed, int *numbuckets, int *numbatches, int *num_skew_mcvs)
 
int ExecHashGetSkewBucket (HashJoinTable hashtable, uint32 hashvalue)
 
void ExecHashEstimate (HashState *node, ParallelContext *pcxt)
 
void ExecHashInitializeDSM (HashState *node, ParallelContext *pcxt)
 
void ExecHashInitializeWorker (HashState *node, ParallelWorkerContext *pwcxt)
 
void ExecHashRetrieveInstrumentation (HashState *node)
 
void ExecShutdownHash (HashState *node)
 
void ExecHashAccumInstrumentation (HashInstrumentation *instrument, HashJoinTable hashtable)
 

Function Documentation

◆ ExecChooseHashTableSize()

void ExecChooseHashTableSize ( double  ntuples,
int  tupwidth,
bool  useskew,
bool  try_combined_hash_mem,
int  parallel_workers,
size_t *  space_allowed,
int *  numbuckets,
int *  numbatches,
int *  num_skew_mcvs 
)

Definition at line 657 of file nodeHash.c.

664{
665 int tupsize;
666 double inner_rel_bytes;
667 size_t hash_table_bytes;
668 size_t bucket_bytes;
669 size_t max_pointers;
670 int nbatch = 1;
671 int nbuckets;
672 double dbuckets;
673
674 /* Force a plausible relation size if no info */
675 if (ntuples <= 0.0)
676 ntuples = 1000.0;
677
678 /*
679 * Estimate tupsize based on footprint of tuple in hashtable... note this
680 * does not allow for any palloc overhead. The manipulations of spaceUsed
681 * don't count palloc overhead either.
682 */
683 tupsize = HJTUPLE_OVERHEAD +
685 MAXALIGN(tupwidth);
686 inner_rel_bytes = ntuples * tupsize;
687
688 /*
689 * Compute in-memory hashtable size limit from GUCs.
690 */
691 hash_table_bytes = get_hash_memory_limit();
692
693 /*
694 * Parallel Hash tries to use the combined hash_mem of all workers to
695 * avoid the need to batch. If that won't work, it falls back to hash_mem
696 * per worker and tries to process batches in parallel.
697 */
698 if (try_combined_hash_mem)
699 {
700 /* Careful, this could overflow size_t */
701 double newlimit;
702
703 newlimit = (double) hash_table_bytes * (double) (parallel_workers + 1);
704 newlimit = Min(newlimit, (double) SIZE_MAX);
705 hash_table_bytes = (size_t) newlimit;
706 }
707
708 *space_allowed = hash_table_bytes;
709
710 /*
711 * If skew optimization is possible, estimate the number of skew buckets
712 * that will fit in the memory allowed, and decrement the assumed space
713 * available for the main hash table accordingly.
714 *
715 * We make the optimistic assumption that each skew bucket will contain
716 * one inner-relation tuple. If that turns out to be low, we will recover
717 * at runtime by reducing the number of skew buckets.
718 *
719 * hashtable->skewBucket will have up to 8 times as many HashSkewBucket
720 * pointers as the number of MCVs we allow, since ExecHashBuildSkewHash
721 * will round up to the next power of 2 and then multiply by 4 to reduce
722 * collisions.
723 */
724 if (useskew)
725 {
726 size_t bytes_per_mcv;
727 size_t skew_mcvs;
728
729 /*----------
730 * Compute number of MCVs we could hold in hash_table_bytes
731 *
732 * Divisor is:
733 * size of a hash tuple +
734 * worst-case size of skewBucket[] per MCV +
735 * size of skewBucketNums[] entry +
736 * size of skew bucket struct itself
737 *----------
738 */
739 bytes_per_mcv = tupsize +
740 (8 * sizeof(HashSkewBucket *)) +
741 sizeof(int) +
743 skew_mcvs = hash_table_bytes / bytes_per_mcv;
744
745 /*
746 * Now scale by SKEW_HASH_MEM_PERCENT (we do it in this order so as
747 * not to worry about size_t overflow in the multiplication)
748 */
749 skew_mcvs = (skew_mcvs * SKEW_HASH_MEM_PERCENT) / 100;
750
751 /* Now clamp to integer range */
752 skew_mcvs = Min(skew_mcvs, INT_MAX);
753
754 *num_skew_mcvs = (int) skew_mcvs;
755
756 /* Reduce hash_table_bytes by the amount needed for the skew table */
757 if (skew_mcvs > 0)
758 hash_table_bytes -= skew_mcvs * bytes_per_mcv;
759 }
760 else
761 *num_skew_mcvs = 0;
762
763 /*
764 * Set nbuckets to achieve an average bucket load of NTUP_PER_BUCKET when
765 * memory is filled, assuming a single batch; but limit the value so that
766 * the pointer arrays we'll try to allocate do not exceed hash_table_bytes
767 * nor MaxAllocSize.
768 *
769 * Note that both nbuckets and nbatch must be powers of 2 to make
770 * ExecHashGetBucketAndBatch fast.
771 */
772 max_pointers = hash_table_bytes / sizeof(HashJoinTuple);
773 max_pointers = Min(max_pointers, MaxAllocSize / sizeof(HashJoinTuple));
774 /* If max_pointers isn't a power of 2, must round it down to one */
775 max_pointers = pg_prevpower2_size_t(max_pointers);
776
777 /* Also ensure we avoid integer overflow in nbatch and nbuckets */
778 /* (this step is redundant given the current value of MaxAllocSize) */
779 max_pointers = Min(max_pointers, INT_MAX / 2 + 1);
780
781 dbuckets = ceil(ntuples / NTUP_PER_BUCKET);
782 dbuckets = Min(dbuckets, max_pointers);
783 nbuckets = (int) dbuckets;
784 /* don't let nbuckets be really small, though ... */
785 nbuckets = Max(nbuckets, 1024);
786 /* ... and force it to be a power of 2. */
787 nbuckets = pg_nextpower2_32(nbuckets);
788
789 /*
790 * If there's not enough space to store the projected number of tuples and
791 * the required bucket headers, we will need multiple batches.
792 */
793 bucket_bytes = sizeof(HashJoinTuple) * nbuckets;
794 if (inner_rel_bytes + bucket_bytes > hash_table_bytes)
795 {
796 /* We'll need multiple batches */
797 size_t sbuckets;
798 double dbatch;
799 int minbatch;
800 size_t bucket_size;
801
802 /*
803 * If Parallel Hash with combined hash_mem would still need multiple
804 * batches, we'll have to fall back to regular hash_mem budget.
805 */
806 if (try_combined_hash_mem)
807 {
808 ExecChooseHashTableSize(ntuples, tupwidth, useskew,
809 false, parallel_workers,
810 space_allowed,
811 numbuckets,
812 numbatches,
813 num_skew_mcvs);
814 return;
815 }
816
817 /*
818 * Estimate the number of buckets we'll want to have when hash_mem is
819 * entirely full. Each bucket will contain a bucket pointer plus
820 * NTUP_PER_BUCKET tuples, whose projected size already includes
821 * overhead for the hash code, pointer to the next tuple, etc.
822 */
823 bucket_size = (tupsize * NTUP_PER_BUCKET + sizeof(HashJoinTuple));
824 if (hash_table_bytes <= bucket_size)
825 sbuckets = 1; /* avoid pg_nextpower2_size_t(0) */
826 else
827 sbuckets = pg_nextpower2_size_t(hash_table_bytes / bucket_size);
828 sbuckets = Min(sbuckets, max_pointers);
829 nbuckets = (int) sbuckets;
830 nbuckets = pg_nextpower2_32(nbuckets);
831 bucket_bytes = nbuckets * sizeof(HashJoinTuple);
832
833 /*
834 * Buckets are simple pointers to hashjoin tuples, while tupsize
835 * includes the pointer, hash code, and MinimalTupleData. So buckets
836 * should never really exceed 25% of hash_mem (even for
837 * NTUP_PER_BUCKET=1); except maybe for hash_mem values that are not
838 * 2^N bytes, where we might get more because of doubling. So let's
839 * look for 50% here.
840 */
841 Assert(bucket_bytes <= hash_table_bytes / 2);
842
843 /* Calculate required number of batches. */
844 dbatch = ceil(inner_rel_bytes / (hash_table_bytes - bucket_bytes));
845 dbatch = Min(dbatch, max_pointers);
846 minbatch = (int) dbatch;
847 nbatch = pg_nextpower2_32(Max(2, minbatch));
848 }
849
850 /*
851 * Optimize the total amount of memory consumed by the hash node.
852 *
853 * The nbatch calculation above focuses on the in-memory hash table,
854 * assuming no per-batch overhead. But each batch may have two files, each
855 * with a BLCKSZ buffer. For large nbatch values these buffers may use
856 * significantly more memory than the hash table.
857 *
858 * The total memory usage may be expressed by this formula:
859 *
860 * (inner_rel_bytes / nbatch) + (2 * nbatch * BLCKSZ)
861 *
862 * where (inner_rel_bytes / nbatch) is the size of the in-memory hash
863 * table and (2 * nbatch * BLCKSZ) is the amount of memory used by file
864 * buffers.
865 *
866 * The nbatch calculation however ignores the second part. And for very
867 * large inner_rel_bytes, there may be no nbatch that keeps total memory
868 * usage under the budget (work_mem * hash_mem_multiplier). To deal with
869 * that, we will adjust nbatch to minimize total memory consumption across
870 * both the hashtable and file buffers.
871 *
872 * As we increase the size of the hashtable, the number of batches
873 * decreases, and the total memory usage follows a U-shaped curve. We find
874 * the minimum nbatch by "walking back" -- checking if halving nbatch
875 * would lower the total memory usage. We stop when it no longer helps.
876 *
877 * We only reduce the number of batches. Adding batches reduces memory
878 * usage only when most of the memory is used by the hash table, with
879 * total memory usage within the limit or not far from it. We don't want
880 * to start batching when not needed, even if that would reduce memory
881 * usage.
882 *
883 * While growing the hashtable, we also adjust the number of buckets to
884 * maintain a load factor of NTUP_PER_BUCKET while squeezing tuples back
885 * from batches into the hashtable.
886 *
887 * Note that we can only change nbuckets during initial hashtable sizing.
888 * Once we start building the hash, nbuckets is fixed (we may still grow
889 * the hash table).
890 *
891 * We double several parameters (space_allowed, nbuckets, num_skew_mcvs),
892 * which introduces a risk of overflow. We avoid this by exiting the loop.
893 * We could do something smarter (e.g. capping nbuckets and continue), but
894 * the complexity is not worth it. Such cases are extremely rare, and this
895 * is a best-effort attempt to reduce memory usage.
896 */
897 while (nbatch > 1)
898 {
899 /* Check that buckets won't overflow MaxAllocSize */
900 if (nbuckets > (MaxAllocSize / sizeof(HashJoinTuple) / 2))
901 break;
902
903 /* num_skew_mcvs should be less than nbuckets */
904 Assert((*num_skew_mcvs) < (INT_MAX / 2));
905
906 /*
907 * Check that space_allowed won't overflow SIZE_MAX.
908 *
909 * We don't use hash_table_bytes here, because it does not include the
910 * skew buckets. And we want to limit the overall memory limit.
911 */
912 if ((*space_allowed) > (SIZE_MAX / 2))
913 break;
914
915 /*
916 * Will halving the number of batches and doubling the size of the
917 * hashtable reduce overall memory usage?
918 *
919 * This is the same as (S = space_allowed):
920 *
921 * (S + 2 * nbatch * BLCKSZ) < (S * 2 + nbatch * BLCKSZ)
922 *
923 * but avoiding intermediate overflow.
924 */
925 if (nbatch < (*space_allowed) / BLCKSZ)
926 break;
927
928 /*
929 * MaxAllocSize is sufficiently small that we are not worried about
930 * overflowing nbuckets.
931 */
932 nbuckets *= 2;
933
934 *num_skew_mcvs = (*num_skew_mcvs) * 2;
935 *space_allowed = (*space_allowed) * 2;
936
937 nbatch /= 2;
938 }
939
940 Assert(nbuckets > 0);
941 Assert(nbatch > 0);
942
943 *numbuckets = nbuckets;
944 *numbatches = nbatch;
945}
#define Min(x, y)
Definition: c.h:1008
#define MAXALIGN(LEN)
Definition: c.h:815
#define Max(x, y)
Definition: c.h:1002
struct HashJoinTupleData * HashJoinTuple
Definition: execnodes.h:2256
#define MaxAllocSize
Definition: fe_memutils.h:22
Assert(PointerIsAligned(start, uint64))
#define HJTUPLE_OVERHEAD
Definition: hashjoin.h:90
#define SKEW_BUCKET_OVERHEAD
Definition: hashjoin.h:119
#define SKEW_HASH_MEM_PERCENT
Definition: hashjoin.h:121
#define SizeofMinimalTupleHeader
Definition: htup_details.h:699
void ExecChooseHashTableSize(double ntuples, int tupwidth, bool useskew, bool try_combined_hash_mem, int parallel_workers, size_t *space_allowed, int *numbuckets, int *numbatches, int *num_skew_mcvs)
Definition: nodeHash.c:657
#define NTUP_PER_BUCKET
Definition: nodeHash.c:654
size_t get_hash_memory_limit(void)
Definition: nodeHash.c:3621
static uint32 pg_nextpower2_32(uint32 num)
Definition: pg_bitutils.h:189
#define pg_nextpower2_size_t
Definition: pg_bitutils.h:441
#define pg_prevpower2_size_t
Definition: pg_bitutils.h:442

References Assert(), ExecChooseHashTableSize(), get_hash_memory_limit(), HJTUPLE_OVERHEAD, Max, MAXALIGN, MaxAllocSize, Min, NTUP_PER_BUCKET, pg_nextpower2_32(), pg_nextpower2_size_t, pg_prevpower2_size_t, SizeofMinimalTupleHeader, SKEW_BUCKET_OVERHEAD, and SKEW_HASH_MEM_PERCENT.

Referenced by ExecChooseHashTableSize(), ExecHashTableCreate(), and initial_cost_hashjoin().

◆ ExecEndHash()

void ExecEndHash ( HashState node)

Definition at line 426 of file nodeHash.c.

427{
429
430 /*
431 * shut down the subplan
432 */
435}
void ExecEndNode(PlanState *node)
Definition: execProcnode.c:562
#define outerPlanState(node)
Definition: execnodes.h:1261
#define outerPlan(node)
Definition: plannodes.h:261

References ExecEndNode(), outerPlan, and outerPlanState.

Referenced by ExecEndNode().

◆ ExecHashAccumInstrumentation()

void ExecHashAccumInstrumentation ( HashInstrumentation instrument,
HashJoinTable  hashtable 
)

Definition at line 2876 of file nodeHash.c.

2878{
2879 instrument->nbuckets = Max(instrument->nbuckets,
2880 hashtable->nbuckets);
2881 instrument->nbuckets_original = Max(instrument->nbuckets_original,
2882 hashtable->nbuckets_original);
2883 instrument->nbatch = Max(instrument->nbatch,
2884 hashtable->nbatch);
2885 instrument->nbatch_original = Max(instrument->nbatch_original,
2886 hashtable->nbatch_original);
2887 instrument->space_peak = Max(instrument->space_peak,
2888 hashtable->spacePeak);
2889}

References Max, HashJoinTableData::nbatch, HashInstrumentation::nbatch, HashJoinTableData::nbatch_original, HashInstrumentation::nbatch_original, HashJoinTableData::nbuckets, HashInstrumentation::nbuckets, HashJoinTableData::nbuckets_original, HashInstrumentation::nbuckets_original, HashInstrumentation::space_peak, and HashJoinTableData::spacePeak.

Referenced by ExecReScanHashJoin(), and ExecShutdownHash().

◆ ExecHashEstimate()

void ExecHashEstimate ( HashState node,
ParallelContext pcxt 
)

Definition at line 2760 of file nodeHash.c.

2761{
2762 size_t size;
2763
2764 /* don't need this if not instrumenting or no workers */
2765 if (!node->ps.instrument || pcxt->nworkers == 0)
2766 return;
2767
2768 size = mul_size(pcxt->nworkers, sizeof(HashInstrumentation));
2769 size = add_size(size, offsetof(SharedHashInfo, hinstrument));
2770 shm_toc_estimate_chunk(&pcxt->estimator, size);
2772}
#define shm_toc_estimate_chunk(e, sz)
Definition: shm_toc.h:51
#define shm_toc_estimate_keys(e, cnt)
Definition: shm_toc.h:53
Size add_size(Size s1, Size s2)
Definition: shmem.c:494
Size mul_size(Size s1, Size s2)
Definition: shmem.c:511
PlanState ps
Definition: execnodes.h:2817
shm_toc_estimator estimator
Definition: parallel.h:41
Instrumentation * instrument
Definition: execnodes.h:1175

References add_size(), ParallelContext::estimator, PlanState::instrument, mul_size(), ParallelContext::nworkers, HashState::ps, shm_toc_estimate_chunk, and shm_toc_estimate_keys.

Referenced by ExecParallelEstimate().

◆ ExecHashGetBucketAndBatch()

void ExecHashGetBucketAndBatch ( HashJoinTable  hashtable,
uint32  hashvalue,
int *  bucketno,
int *  batchno 
)

Definition at line 1959 of file nodeHash.c.

1963{
1964 uint32 nbuckets = (uint32) hashtable->nbuckets;
1965 uint32 nbatch = (uint32) hashtable->nbatch;
1966
1967 if (nbatch > 1)
1968 {
1969 *bucketno = hashvalue & (nbuckets - 1);
1970 *batchno = pg_rotate_right32(hashvalue,
1971 hashtable->log2_nbuckets) & (nbatch - 1);
1972 }
1973 else
1974 {
1975 *bucketno = hashvalue & (nbuckets - 1);
1976 *batchno = 0;
1977 }
1978}
uint32_t uint32
Definition: c.h:543
static uint32 pg_rotate_right32(uint32 word, int n)
Definition: pg_bitutils.h:422

References HashJoinTableData::log2_nbuckets, HashJoinTableData::nbatch, HashJoinTableData::nbuckets, and pg_rotate_right32().

Referenced by ExecHashIncreaseNumBatches(), ExecHashIncreaseNumBuckets(), ExecHashJoinImpl(), ExecHashRemoveNextSkewBucket(), ExecHashTableInsert(), ExecParallelHashIncreaseNumBuckets(), ExecParallelHashJoinPartitionOuter(), ExecParallelHashRepartitionFirst(), ExecParallelHashRepartitionRest(), ExecParallelHashTableInsert(), and ExecParallelHashTableInsertCurrentBatch().

◆ ExecHashGetSkewBucket()

int ExecHashGetSkewBucket ( HashJoinTable  hashtable,
uint32  hashvalue 
)

Definition at line 2554 of file nodeHash.c.

2555{
2556 int bucket;
2557
2558 /*
2559 * Always return INVALID_SKEW_BUCKET_NO if not doing skew optimization (in
2560 * particular, this happens after the initial batch is done).
2561 */
2562 if (!hashtable->skewEnabled)
2564
2565 /*
2566 * Since skewBucketLen is a power of 2, we can do a modulo by ANDing.
2567 */
2568 bucket = hashvalue & (hashtable->skewBucketLen - 1);
2569
2570 /*
2571 * While we have not hit a hole in the hashtable and have not hit the
2572 * desired bucket, we have collided with some other hash value, so try the
2573 * next bucket location.
2574 */
2575 while (hashtable->skewBucket[bucket] != NULL &&
2576 hashtable->skewBucket[bucket]->hashvalue != hashvalue)
2577 bucket = (bucket + 1) & (hashtable->skewBucketLen - 1);
2578
2579 /*
2580 * Found the desired bucket?
2581 */
2582 if (hashtable->skewBucket[bucket] != NULL)
2583 return bucket;
2584
2585 /*
2586 * There must not be any hashtable entry for this hash value.
2587 */
2589}
#define INVALID_SKEW_BUCKET_NO
Definition: hashjoin.h:120
HashSkewBucket ** skewBucket
Definition: hashjoin.h:317
uint32 hashvalue
Definition: hashjoin.h:115

References HashSkewBucket::hashvalue, INVALID_SKEW_BUCKET_NO, HashJoinTableData::skewBucket, HashJoinTableData::skewBucketLen, and HashJoinTableData::skewEnabled.

Referenced by ExecHashJoinImpl(), and MultiExecPrivateHash().

◆ ExecHashInitializeDSM()

void ExecHashInitializeDSM ( HashState node,
ParallelContext pcxt 
)

Definition at line 2779 of file nodeHash.c.

2780{
2781 size_t size;
2782
2783 /* don't need this if not instrumenting or no workers */
2784 if (!node->ps.instrument || pcxt->nworkers == 0)
2785 return;
2786
2787 size = offsetof(SharedHashInfo, hinstrument) +
2788 pcxt->nworkers * sizeof(HashInstrumentation);
2789 node->shared_info = (SharedHashInfo *) shm_toc_allocate(pcxt->toc, size);
2790
2791 /* Each per-worker area must start out as zeroes. */
2792 memset(node->shared_info, 0, size);
2793
2794 node->shared_info->num_workers = pcxt->nworkers;
2795 shm_toc_insert(pcxt->toc, node->ps.plan->plan_node_id,
2796 node->shared_info);
2797}
struct HashInstrumentation HashInstrumentation
void * shm_toc_allocate(shm_toc *toc, Size nbytes)
Definition: shm_toc.c:88
void shm_toc_insert(shm_toc *toc, uint64 key, void *address)
Definition: shm_toc.c:171
SharedHashInfo * shared_info
Definition: execnodes.h:2830
shm_toc * toc
Definition: parallel.h:44
Plan * plan
Definition: execnodes.h:1165
int plan_node_id
Definition: plannodes.h:227

References PlanState::instrument, SharedHashInfo::num_workers, ParallelContext::nworkers, PlanState::plan, Plan::plan_node_id, HashState::ps, HashState::shared_info, shm_toc_allocate(), shm_toc_insert(), and ParallelContext::toc.

Referenced by ExecParallelInitializeDSM().

◆ ExecHashInitializeWorker()

void ExecHashInitializeWorker ( HashState node,
ParallelWorkerContext pwcxt 
)

Definition at line 2804 of file nodeHash.c.

2805{
2806 SharedHashInfo *shared_info;
2807
2808 /* don't need this if not instrumenting */
2809 if (!node->ps.instrument)
2810 return;
2811
2812 /*
2813 * Find our entry in the shared area, and set up a pointer to it so that
2814 * we'll accumulate stats there when shutting down or rebuilding the hash
2815 * table.
2816 */
2817 shared_info = (SharedHashInfo *)
2818 shm_toc_lookup(pwcxt->toc, node->ps.plan->plan_node_id, false);
2819 node->hinstrument = &shared_info->hinstrument[ParallelWorkerNumber];
2820}
int ParallelWorkerNumber
Definition: parallel.c:115
void * shm_toc_lookup(shm_toc *toc, uint64 key, bool noError)
Definition: shm_toc.c:232
HashInstrumentation * hinstrument
Definition: execnodes.h:2837
HashInstrumentation hinstrument[FLEXIBLE_ARRAY_MEMBER]
Definition: execnodes.h:2808

References SharedHashInfo::hinstrument, HashState::hinstrument, PlanState::instrument, ParallelWorkerNumber, PlanState::plan, Plan::plan_node_id, HashState::ps, shm_toc_lookup(), and ParallelWorkerContext::toc.

Referenced by ExecParallelInitializeWorker().

◆ ExecHashRetrieveInstrumentation()

void ExecHashRetrieveInstrumentation ( HashState node)

Definition at line 2845 of file nodeHash.c.

2846{
2847 SharedHashInfo *shared_info = node->shared_info;
2848 size_t size;
2849
2850 if (shared_info == NULL)
2851 return;
2852
2853 /* Replace node->shared_info with a copy in backend-local memory. */
2854 size = offsetof(SharedHashInfo, hinstrument) +
2855 shared_info->num_workers * sizeof(HashInstrumentation);
2856 node->shared_info = palloc(size);
2857 memcpy(node->shared_info, shared_info, size);
2858}
void * palloc(Size size)
Definition: mcxt.c:1365

References SharedHashInfo::num_workers, palloc(), and HashState::shared_info.

Referenced by ExecParallelRetrieveInstrumentation().

◆ ExecHashTableCreate()

HashJoinTable ExecHashTableCreate ( HashState state)

Definition at line 445 of file nodeHash.c.

446{
447 Hash *node;
448 HashJoinTable hashtable;
449 Plan *outerNode;
450 size_t space_allowed;
451 int nbuckets;
452 int nbatch;
453 double rows;
454 int num_skew_mcvs;
455 int log2_nbuckets;
456 MemoryContext oldcxt;
457
458 /*
459 * Get information about the size of the relation to be hashed (it's the
460 * "outer" subtree of this node, but the inner relation of the hashjoin).
461 * Compute the appropriate size of the hash table.
462 */
463 node = (Hash *) state->ps.plan;
464 outerNode = outerPlan(node);
465
466 /*
467 * If this is shared hash table with a partial plan, then we can't use
468 * outerNode->plan_rows to estimate its size. We need an estimate of the
469 * total number of rows across all copies of the partial plan.
470 */
471 rows = node->plan.parallel_aware ? node->rows_total : outerNode->plan_rows;
472
473 ExecChooseHashTableSize(rows, outerNode->plan_width,
474 OidIsValid(node->skewTable),
475 state->parallel_state != NULL,
476 state->parallel_state != NULL ?
477 state->parallel_state->nparticipants - 1 : 0,
478 &space_allowed,
479 &nbuckets, &nbatch, &num_skew_mcvs);
480
481 /* nbuckets must be a power of 2 */
482 log2_nbuckets = pg_ceil_log2_32(nbuckets);
483 Assert(nbuckets == (1 << log2_nbuckets));
484
485 /*
486 * Initialize the hash table control block.
487 *
488 * The hashtable control block is just palloc'd from the executor's
489 * per-query memory context. Everything else should be kept inside the
490 * subsidiary hashCxt, batchCxt or spillCxt.
491 */
492 hashtable = palloc_object(HashJoinTableData);
493 hashtable->nbuckets = nbuckets;
494 hashtable->nbuckets_original = nbuckets;
495 hashtable->nbuckets_optimal = nbuckets;
496 hashtable->log2_nbuckets = log2_nbuckets;
497 hashtable->log2_nbuckets_optimal = log2_nbuckets;
498 hashtable->buckets.unshared = NULL;
499 hashtable->skewEnabled = false;
500 hashtable->skewBucket = NULL;
501 hashtable->skewBucketLen = 0;
502 hashtable->nSkewBuckets = 0;
503 hashtable->skewBucketNums = NULL;
504 hashtable->nbatch = nbatch;
505 hashtable->curbatch = 0;
506 hashtable->nbatch_original = nbatch;
507 hashtable->nbatch_outstart = nbatch;
508 hashtable->growEnabled = true;
509 hashtable->totalTuples = 0;
510 hashtable->partialTuples = 0;
511 hashtable->skewTuples = 0;
512 hashtable->innerBatchFile = NULL;
513 hashtable->outerBatchFile = NULL;
514 hashtable->spaceUsed = 0;
515 hashtable->spacePeak = 0;
516 hashtable->spaceAllowed = space_allowed;
517 hashtable->spaceUsedSkew = 0;
518 hashtable->spaceAllowedSkew =
519 hashtable->spaceAllowed * SKEW_HASH_MEM_PERCENT / 100;
520 hashtable->chunks = NULL;
521 hashtable->current_chunk = NULL;
522 hashtable->parallel_state = state->parallel_state;
523 hashtable->area = state->ps.state->es_query_dsa;
524 hashtable->batches = NULL;
525
526#ifdef HJDEBUG
527 printf("Hashjoin %p: initial nbatch = %d, nbuckets = %d\n",
528 hashtable, nbatch, nbuckets);
529#endif
530
531 /*
532 * Create temporary memory contexts in which to keep the hashtable working
533 * storage. See notes in executor/hashjoin.h.
534 */
536 "HashTableContext",
538
539 hashtable->batchCxt = AllocSetContextCreate(hashtable->hashCxt,
540 "HashBatchContext",
542
543 hashtable->spillCxt = AllocSetContextCreate(hashtable->hashCxt,
544 "HashSpillContext",
546
547 /* Allocate data that will live for the life of the hashjoin */
548
549 oldcxt = MemoryContextSwitchTo(hashtable->hashCxt);
550
551 if (nbatch > 1 && hashtable->parallel_state == NULL)
552 {
553 MemoryContext oldctx;
554
555 /*
556 * allocate and initialize the file arrays in hashCxt (not needed for
557 * parallel case which uses shared tuplestores instead of raw files)
558 */
559 oldctx = MemoryContextSwitchTo(hashtable->spillCxt);
560
561 hashtable->innerBatchFile = palloc0_array(BufFile *, nbatch);
562 hashtable->outerBatchFile = palloc0_array(BufFile *, nbatch);
563
564 MemoryContextSwitchTo(oldctx);
565
566 /* The files will not be opened until needed... */
567 /* ... but make sure we have temp tablespaces established for them */
569 }
570
571 MemoryContextSwitchTo(oldcxt);
572
573 if (hashtable->parallel_state)
574 {
575 ParallelHashJoinState *pstate = hashtable->parallel_state;
576 Barrier *build_barrier;
577
578 /*
579 * Attach to the build barrier. The corresponding detach operation is
580 * in ExecHashTableDetach. Note that we won't attach to the
581 * batch_barrier for batch 0 yet. We'll attach later and start it out
582 * in PHJ_BATCH_PROBE phase, because batch 0 is allocated up front and
583 * then loaded while hashing (the standard hybrid hash join
584 * algorithm), and we'll coordinate that using build_barrier.
585 */
586 build_barrier = &pstate->build_barrier;
587 BarrierAttach(build_barrier);
588
589 /*
590 * So far we have no idea whether there are any other participants,
591 * and if so, what phase they are working on. The only thing we care
592 * about at this point is whether someone has already created the
593 * SharedHashJoinBatch objects and the hash table for batch 0. One
594 * backend will be elected to do that now if necessary.
595 */
596 if (BarrierPhase(build_barrier) == PHJ_BUILD_ELECT &&
597 BarrierArriveAndWait(build_barrier, WAIT_EVENT_HASH_BUILD_ELECT))
598 {
599 pstate->nbatch = nbatch;
600 pstate->space_allowed = space_allowed;
601 pstate->growth = PHJ_GROWTH_OK;
602
603 /* Set up the shared state for coordinating batches. */
604 ExecParallelHashJoinSetUpBatches(hashtable, nbatch);
605
606 /*
607 * Allocate batch 0's hash table up front so we can load it
608 * directly while hashing.
609 */
610 pstate->nbuckets = nbuckets;
611 ExecParallelHashTableAlloc(hashtable, 0);
612 }
613
614 /*
615 * The next Parallel Hash synchronization point is in
616 * MultiExecParallelHash(), which will progress it all the way to
617 * PHJ_BUILD_RUN. The caller must not return control from this
618 * executor node between now and then.
619 */
620 }
621 else
622 {
623 /*
624 * Prepare context for the first-scan space allocations; allocate the
625 * hashbucket array therein, and set each bucket "empty".
626 */
628
629 hashtable->buckets.unshared = palloc0_array(HashJoinTuple, nbuckets);
630
631 /*
632 * Set up for skew optimization, if possible and there's a need for
633 * more than one batch. (In a one-batch join, there's no point in
634 * it.)
635 */
636 if (nbatch > 1)
637 ExecHashBuildSkewHash(state, hashtable, node, num_skew_mcvs);
638
639 MemoryContextSwitchTo(oldcxt);
640 }
641
642 return hashtable;
643}
void PrepareTempTablespaces(void)
Definition: tablespace.c:1331
int BarrierAttach(Barrier *barrier)
Definition: barrier.c:236
int BarrierPhase(Barrier *barrier)
Definition: barrier.c:265
bool BarrierArriveAndWait(Barrier *barrier, uint32 wait_event_info)
Definition: barrier.c:125
#define OidIsValid(objectId)
Definition: c.h:779
#define palloc_object(type)
Definition: fe_memutils.h:74
#define palloc0_array(type, count)
Definition: fe_memutils.h:77
@ PHJ_GROWTH_OK
Definition: hashjoin.h:233
#define PHJ_BUILD_ELECT
Definition: hashjoin.h:269
MemoryContext CurrentMemoryContext
Definition: mcxt.c:160
#define AllocSetContextCreate
Definition: memutils.h:129
#define ALLOCSET_DEFAULT_SIZES
Definition: memutils.h:160
static void ExecHashBuildSkewHash(HashState *hashstate, HashJoinTable hashtable, Hash *node, int mcvsToUse)
Definition: nodeHash.c:2402
static void ExecParallelHashJoinSetUpBatches(HashJoinTable hashtable, int nbatch)
Definition: nodeHash.c:3123
void ExecParallelHashTableAlloc(HashJoinTable hashtable, int batchno)
Definition: nodeHash.c:3288
static MemoryContext MemoryContextSwitchTo(MemoryContext context)
Definition: palloc.h:124
static uint32 pg_ceil_log2_32(uint32 num)
Definition: pg_bitutils.h:258
#define printf(...)
Definition: port.h:245
struct HashJoinTupleData ** unshared
Definition: hashjoin.h:311
HashMemoryChunk chunks
Definition: hashjoin.h:355
ParallelHashJoinBatchAccessor * batches
Definition: hashjoin.h:361
MemoryContext hashCxt
Definition: hashjoin.h:350
union HashJoinTableData::@110 buckets
double totalTuples
Definition: hashjoin.h:330
double partialTuples
Definition: hashjoin.h:331
ParallelHashJoinState * parallel_state
Definition: hashjoin.h:360
MemoryContext spillCxt
Definition: hashjoin.h:352
HashMemoryChunk current_chunk
Definition: hashjoin.h:358
Size spaceAllowedSkew
Definition: hashjoin.h:348
int * skewBucketNums
Definition: hashjoin.h:320
BufFile ** innerBatchFile
Definition: hashjoin.h:341
int log2_nbuckets_optimal
Definition: hashjoin.h:305
dsa_area * area
Definition: hashjoin.h:359
BufFile ** outerBatchFile
Definition: hashjoin.h:342
MemoryContext batchCxt
Definition: hashjoin.h:351
double skewTuples
Definition: hashjoin.h:332
Oid skewTable
Definition: plannodes.h:1411
Cardinality rows_total
Definition: plannodes.h:1418
Plan plan
Definition: plannodes.h:1402
ParallelHashGrowth growth
Definition: hashjoin.h:253
bool parallel_aware
Definition: plannodes.h:213
int plan_width
Definition: plannodes.h:207
Cardinality plan_rows
Definition: plannodes.h:205
Definition: regguts.h:323

References ALLOCSET_DEFAULT_SIZES, AllocSetContextCreate, HashJoinTableData::area, Assert(), BarrierArriveAndWait(), BarrierAttach(), BarrierPhase(), HashJoinTableData::batchCxt, HashJoinTableData::batches, HashJoinTableData::buckets, ParallelHashJoinState::build_barrier, HashJoinTableData::chunks, HashJoinTableData::curbatch, HashJoinTableData::current_chunk, CurrentMemoryContext, ExecChooseHashTableSize(), ExecHashBuildSkewHash(), ExecParallelHashJoinSetUpBatches(), ExecParallelHashTableAlloc(), HashJoinTableData::growEnabled, ParallelHashJoinState::growth, HashJoinTableData::hashCxt, HashJoinTableData::innerBatchFile, HashJoinTableData::log2_nbuckets, HashJoinTableData::log2_nbuckets_optimal, MemoryContextSwitchTo(), ParallelHashJoinState::nbatch, HashJoinTableData::nbatch, HashJoinTableData::nbatch_original, HashJoinTableData::nbatch_outstart, ParallelHashJoinState::nbuckets, HashJoinTableData::nbuckets, HashJoinTableData::nbuckets_optimal, HashJoinTableData::nbuckets_original, HashJoinTableData::nSkewBuckets, OidIsValid, HashJoinTableData::outerBatchFile, outerPlan, palloc0_array, palloc_object, Plan::parallel_aware, HashJoinTableData::parallel_state, HashJoinTableData::partialTuples, pg_ceil_log2_32(), PHJ_BUILD_ELECT, PHJ_GROWTH_OK, Hash::plan, Plan::plan_rows, Plan::plan_width, PrepareTempTablespaces(), printf, Hash::rows_total, SKEW_HASH_MEM_PERCENT, HashJoinTableData::skewBucket, HashJoinTableData::skewBucketLen, HashJoinTableData::skewBucketNums, HashJoinTableData::skewEnabled, Hash::skewTable, HashJoinTableData::skewTuples, ParallelHashJoinState::space_allowed, HashJoinTableData::spaceAllowed, HashJoinTableData::spaceAllowedSkew, HashJoinTableData::spacePeak, HashJoinTableData::spaceUsed, HashJoinTableData::spaceUsedSkew, HashJoinTableData::spillCxt, HashJoinTableData::totalTuples, and HashJoinTableData::unshared.

Referenced by ExecHashJoinImpl().

◆ ExecHashTableDestroy()

void ExecHashTableDestroy ( HashJoinTable  hashtable)

Definition at line 955 of file nodeHash.c.

956{
957 int i;
958
959 /*
960 * Make sure all the temp files are closed. We skip batch 0, since it
961 * can't have any temp files (and the arrays might not even exist if
962 * nbatch is only 1). Parallel hash joins don't use these files.
963 */
964 if (hashtable->innerBatchFile != NULL)
965 {
966 for (i = 1; i < hashtable->nbatch; i++)
967 {
968 if (hashtable->innerBatchFile[i])
969 BufFileClose(hashtable->innerBatchFile[i]);
970 if (hashtable->outerBatchFile[i])
971 BufFileClose(hashtable->outerBatchFile[i]);
972 }
973 }
974
975 /* Release working memory (batchCxt is a child, so it goes away too) */
976 MemoryContextDelete(hashtable->hashCxt);
977
978 /* And drop the control block */
979 pfree(hashtable);
980}
void BufFileClose(BufFile *file)
Definition: buffile.c:412
int i
Definition: isn.c:77
void pfree(void *pointer)
Definition: mcxt.c:1594
void MemoryContextDelete(MemoryContext context)
Definition: mcxt.c:469

References BufFileClose(), HashJoinTableData::hashCxt, i, HashJoinTableData::innerBatchFile, MemoryContextDelete(), HashJoinTableData::nbatch, HashJoinTableData::outerBatchFile, and pfree().

Referenced by ExecEndHashJoin(), and ExecReScanHashJoin().

◆ ExecHashTableDetach()

void ExecHashTableDetach ( HashJoinTable  hashtable)

Definition at line 3400 of file nodeHash.c.

3401{
3402 ParallelHashJoinState *pstate = hashtable->parallel_state;
3403
3404 /*
3405 * If we're involved in a parallel query, we must either have gotten all
3406 * the way to PHJ_BUILD_RUN, or joined too late and be in PHJ_BUILD_FREE.
3407 */
3408 Assert(!pstate ||
3410
3411 if (pstate && BarrierPhase(&pstate->build_barrier) == PHJ_BUILD_RUN)
3412 {
3413 int i;
3414
3415 /* Make sure any temporary files are closed. */
3416 if (hashtable->batches)
3417 {
3418 for (i = 0; i < hashtable->nbatch; ++i)
3419 {
3420 sts_end_write(hashtable->batches[i].inner_tuples);
3421 sts_end_write(hashtable->batches[i].outer_tuples);
3424 }
3425 }
3426
3427 /* If we're last to detach, clean up shared memory. */
3429 {
3430 /*
3431 * Late joining processes will see this state and give up
3432 * immediately.
3433 */
3435
3436 if (DsaPointerIsValid(pstate->batches))
3437 {
3438 dsa_free(hashtable->area, pstate->batches);
3439 pstate->batches = InvalidDsaPointer;
3440 }
3441 }
3442 }
3443 hashtable->parallel_state = NULL;
3444}
bool BarrierArriveAndDetach(Barrier *barrier)
Definition: barrier.c:203
void dsa_free(dsa_area *area, dsa_pointer dp)
Definition: dsa.c:841
#define InvalidDsaPointer
Definition: dsa.h:78
#define DsaPointerIsValid(x)
Definition: dsa.h:106
#define PHJ_BUILD_FREE
Definition: hashjoin.h:274
#define PHJ_BUILD_RUN
Definition: hashjoin.h:273
void sts_end_write(SharedTuplestoreAccessor *accessor)
void sts_end_parallel_scan(SharedTuplestoreAccessor *accessor)
SharedTuplestoreAccessor * outer_tuples
Definition: hashjoin.h:221
SharedTuplestoreAccessor * inner_tuples
Definition: hashjoin.h:220
dsa_pointer batches
Definition: hashjoin.h:248

References HashJoinTableData::area, Assert(), BarrierArriveAndDetach(), BarrierPhase(), ParallelHashJoinState::batches, HashJoinTableData::batches, ParallelHashJoinState::build_barrier, dsa_free(), DsaPointerIsValid, i, ParallelHashJoinBatchAccessor::inner_tuples, InvalidDsaPointer, HashJoinTableData::nbatch, ParallelHashJoinBatchAccessor::outer_tuples, HashJoinTableData::parallel_state, PHJ_BUILD_FREE, PHJ_BUILD_RUN, sts_end_parallel_scan(), and sts_end_write().

Referenced by ExecHashJoinReInitializeDSM(), and ExecShutdownHashJoin().

◆ ExecHashTableDetachBatch()

void ExecHashTableDetachBatch ( HashJoinTable  hashtable)

Definition at line 3308 of file nodeHash.c.

3309{
3310 if (hashtable->parallel_state != NULL &&
3311 hashtable->curbatch >= 0)
3312 {
3313 int curbatch = hashtable->curbatch;
3314 ParallelHashJoinBatch *batch = hashtable->batches[curbatch].shared;
3315 bool attached = true;
3316
3317 /* Make sure any temporary files are closed. */
3318 sts_end_parallel_scan(hashtable->batches[curbatch].inner_tuples);
3319 sts_end_parallel_scan(hashtable->batches[curbatch].outer_tuples);
3320
3321 /* After attaching we always get at least to PHJ_BATCH_PROBE. */
3324
3325 /*
3326 * If we're abandoning the PHJ_BATCH_PROBE phase early without having
3327 * reached the end of it, it means the plan doesn't want any more
3328 * tuples, and it is happy to abandon any tuples buffered in this
3329 * process's subplans. For correctness, we can't allow any process to
3330 * execute the PHJ_BATCH_SCAN phase, because we will never have the
3331 * complete set of match bits. Therefore we skip emitting unmatched
3332 * tuples in all backends (if this is a full/right join), as if those
3333 * tuples were all due to be emitted by this process and it has
3334 * abandoned them too.
3335 */
3336 if (BarrierPhase(&batch->batch_barrier) == PHJ_BATCH_PROBE &&
3337 !hashtable->batches[curbatch].outer_eof)
3338 {
3339 /*
3340 * This flag may be written to by multiple backends during
3341 * PHJ_BATCH_PROBE phase, but will only be read in PHJ_BATCH_SCAN
3342 * phase so requires no extra locking.
3343 */
3344 batch->skip_unmatched = true;
3345 }
3346
3347 /*
3348 * Even if we aren't doing a full/right outer join, we'll step through
3349 * the PHJ_BATCH_SCAN phase just to maintain the invariant that
3350 * freeing happens in PHJ_BATCH_FREE, but that'll be wait-free.
3351 */
3354 if (attached && BarrierArriveAndDetach(&batch->batch_barrier))
3355 {
3356 /*
3357 * We are not longer attached to the batch barrier, but we're the
3358 * process that was chosen to free resources and it's safe to
3359 * assert the current phase. The ParallelHashJoinBatch can't go
3360 * away underneath us while we are attached to the build barrier,
3361 * making this access safe.
3362 */
3364
3365 /* Free shared chunks and buckets. */
3366 while (DsaPointerIsValid(batch->chunks))
3367 {
3368 HashMemoryChunk chunk =
3369 dsa_get_address(hashtable->area, batch->chunks);
3370 dsa_pointer next = chunk->next.shared;
3371
3372 dsa_free(hashtable->area, batch->chunks);
3373 batch->chunks = next;
3374 }
3375 if (DsaPointerIsValid(batch->buckets))
3376 {
3377 dsa_free(hashtable->area, batch->buckets);
3378 batch->buckets = InvalidDsaPointer;
3379 }
3380 }
3381
3382 /*
3383 * Track the largest batch we've been attached to. Though each
3384 * backend might see a different subset of batches, explain.c will
3385 * scan the results from all backends to find the largest value.
3386 */
3387 hashtable->spacePeak =
3388 Max(hashtable->spacePeak,
3389 batch->size + sizeof(dsa_pointer_atomic) * hashtable->nbuckets);
3390
3391 /* Remember that we are not attached to a batch. */
3392 hashtable->curbatch = -1;
3393 }
3394}
bool BarrierArriveAndDetachExceptLast(Barrier *barrier)
Definition: barrier.c:213
static int32 next
Definition: blutils.c:224
void * dsa_get_address(dsa_area *area, dsa_pointer dp)
Definition: dsa.c:957
uint64 dsa_pointer
Definition: dsa.h:62
#define PHJ_BATCH_SCAN
Definition: hashjoin.h:281
#define PHJ_BATCH_PROBE
Definition: hashjoin.h:280
#define PHJ_BATCH_FREE
Definition: hashjoin.h:282
union HashMemoryChunkData::@109 next
dsa_pointer shared
Definition: hashjoin.h:138
ParallelHashJoinBatch * shared
Definition: hashjoin.h:209
dsa_pointer chunks
Definition: hashjoin.h:167
dsa_pointer buckets
Definition: hashjoin.h:164

References HashJoinTableData::area, Assert(), BarrierArriveAndDetach(), BarrierArriveAndDetachExceptLast(), BarrierPhase(), ParallelHashJoinBatch::batch_barrier, HashJoinTableData::batches, ParallelHashJoinBatch::buckets, ParallelHashJoinBatch::chunks, HashJoinTableData::curbatch, dsa_free(), dsa_get_address(), DsaPointerIsValid, ParallelHashJoinBatchAccessor::inner_tuples, InvalidDsaPointer, Max, HashJoinTableData::nbuckets, next, HashMemoryChunkData::next, ParallelHashJoinBatchAccessor::outer_eof, ParallelHashJoinBatchAccessor::outer_tuples, HashJoinTableData::parallel_state, PHJ_BATCH_FREE, PHJ_BATCH_PROBE, PHJ_BATCH_SCAN, HashMemoryChunkData::shared, ParallelHashJoinBatchAccessor::shared, ParallelHashJoinBatch::size, ParallelHashJoinBatch::skip_unmatched, HashJoinTableData::spacePeak, and sts_end_parallel_scan().

Referenced by ExecHashJoinReInitializeDSM(), ExecParallelHashJoinNewBatch(), ExecParallelPrepHashTableForUnmatched(), and ExecShutdownHashJoin().

◆ ExecHashTableInsert()

void ExecHashTableInsert ( HashJoinTable  hashtable,
TupleTableSlot slot,
uint32  hashvalue 
)

Definition at line 1748 of file nodeHash.c.

1751{
1752 bool shouldFree;
1753 MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot, &shouldFree);
1754 int bucketno;
1755 int batchno;
1756
1757 ExecHashGetBucketAndBatch(hashtable, hashvalue,
1758 &bucketno, &batchno);
1759
1760 /*
1761 * decide whether to put the tuple in the hash table or a temp file
1762 */
1763 if (batchno == hashtable->curbatch)
1764 {
1765 /*
1766 * put the tuple in hash table
1767 */
1768 HashJoinTuple hashTuple;
1769 int hashTupleSize;
1770 double ntuples = (hashtable->totalTuples - hashtable->skewTuples);
1771
1772 /* Create the HashJoinTuple */
1773 hashTupleSize = HJTUPLE_OVERHEAD + tuple->t_len;
1774 hashTuple = (HashJoinTuple) dense_alloc(hashtable, hashTupleSize);
1775
1776 hashTuple->hashvalue = hashvalue;
1777 memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
1778
1779 /*
1780 * We always reset the tuple-matched flag on insertion. This is okay
1781 * even when reloading a tuple from a batch file, since the tuple
1782 * could not possibly have been matched to an outer tuple before it
1783 * went into the batch file.
1784 */
1786
1787 /* Push it onto the front of the bucket's list */
1788 hashTuple->next.unshared = hashtable->buckets.unshared[bucketno];
1789 hashtable->buckets.unshared[bucketno] = hashTuple;
1790
1791 /*
1792 * Increase the (optimal) number of buckets if we just exceeded the
1793 * NTUP_PER_BUCKET threshold, but only when there's still a single
1794 * batch.
1795 */
1796 if (hashtable->nbatch == 1 &&
1797 ntuples > (hashtable->nbuckets_optimal * NTUP_PER_BUCKET))
1798 {
1799 /* Guard against integer overflow and alloc size overflow */
1800 if (hashtable->nbuckets_optimal <= INT_MAX / 2 &&
1801 hashtable->nbuckets_optimal * 2 <= MaxAllocSize / sizeof(HashJoinTuple))
1802 {
1803 hashtable->nbuckets_optimal *= 2;
1804 hashtable->log2_nbuckets_optimal += 1;
1805 }
1806 }
1807
1808 /* Account for space used, and back off if we've used too much */
1809 hashtable->spaceUsed += hashTupleSize;
1810 if (hashtable->spaceUsed > hashtable->spacePeak)
1811 hashtable->spacePeak = hashtable->spaceUsed;
1812 if (hashtable->spaceUsed +
1813 hashtable->nbuckets_optimal * sizeof(HashJoinTuple)
1814 > hashtable->spaceAllowed)
1815 ExecHashIncreaseNumBatches(hashtable);
1816 }
1817 else
1818 {
1819 /*
1820 * put the tuple into a temp file for later batches
1821 */
1822 Assert(batchno > hashtable->curbatch);
1824 hashvalue,
1825 &hashtable->innerBatchFile[batchno],
1826 hashtable);
1827 }
1828
1829 if (shouldFree)
1831}
MinimalTuple ExecFetchSlotMinimalTuple(TupleTableSlot *slot, bool *shouldFree)
Definition: execTuples.c:1881
#define HJTUPLE_MINTUPLE(hjtup)
Definition: hashjoin.h:91
void heap_free_minimal_tuple(MinimalTuple mtup)
Definition: heaptuple.c:1529
static void HeapTupleHeaderClearMatch(MinimalTupleData *tup)
Definition: htup_details.h:718
static void * dense_alloc(HashJoinTable hashtable, Size size)
Definition: nodeHash.c:2895
static void ExecHashIncreaseNumBatches(HashJoinTable hashtable)
Definition: nodeHash.c:1029
void ExecHashGetBucketAndBatch(HashJoinTable hashtable, uint32 hashvalue, int *bucketno, int *batchno)
Definition: nodeHash.c:1959
void ExecHashJoinSaveTuple(MinimalTuple tuple, uint32 hashvalue, BufFile **fileptr, HashJoinTable hashtable)
union HashJoinTupleData::@108 next
uint32 hashvalue
Definition: hashjoin.h:86
struct HashJoinTupleData * unshared
Definition: hashjoin.h:83

References Assert(), HashJoinTableData::buckets, HashJoinTableData::curbatch, dense_alloc(), ExecFetchSlotMinimalTuple(), ExecHashGetBucketAndBatch(), ExecHashIncreaseNumBatches(), ExecHashJoinSaveTuple(), HashJoinTupleData::hashvalue, heap_free_minimal_tuple(), HeapTupleHeaderClearMatch(), HJTUPLE_MINTUPLE, HJTUPLE_OVERHEAD, HashJoinTableData::innerBatchFile, HashJoinTableData::log2_nbuckets_optimal, MaxAllocSize, HashJoinTableData::nbatch, HashJoinTableData::nbuckets_optimal, HashJoinTupleData::next, NTUP_PER_BUCKET, HashJoinTableData::skewTuples, HashJoinTableData::spaceAllowed, HashJoinTableData::spacePeak, HashJoinTableData::spaceUsed, MinimalTupleData::t_len, HashJoinTableData::totalTuples, HashJoinTupleData::unshared, and HashJoinTableData::unshared.

Referenced by ExecHashJoinNewBatch(), and MultiExecPrivateHash().

◆ ExecHashTableReset()

void ExecHashTableReset ( HashJoinTable  hashtable)

Definition at line 2326 of file nodeHash.c.

2327{
2328 MemoryContext oldcxt;
2329 int nbuckets = hashtable->nbuckets;
2330
2331 /*
2332 * Release all the hash buckets and tuples acquired in the prior pass, and
2333 * reinitialize the context for a new pass.
2334 */
2335 MemoryContextReset(hashtable->batchCxt);
2336 oldcxt = MemoryContextSwitchTo(hashtable->batchCxt);
2337
2338 /* Reallocate and reinitialize the hash bucket headers. */
2339 hashtable->buckets.unshared = palloc0_array(HashJoinTuple, nbuckets);
2340
2341 hashtable->spaceUsed = 0;
2342
2343 MemoryContextSwitchTo(oldcxt);
2344
2345 /* Forget the chunks (the memory was freed by the context reset above). */
2346 hashtable->chunks = NULL;
2347}
void MemoryContextReset(MemoryContext context)
Definition: mcxt.c:400

References HashJoinTableData::batchCxt, HashJoinTableData::buckets, HashJoinTableData::chunks, MemoryContextReset(), MemoryContextSwitchTo(), HashJoinTableData::nbuckets, palloc0_array, HashJoinTableData::spaceUsed, and HashJoinTableData::unshared.

Referenced by ExecHashJoinNewBatch().

◆ ExecHashTableResetMatchFlags()

void ExecHashTableResetMatchFlags ( HashJoinTable  hashtable)

Definition at line 2354 of file nodeHash.c.

2355{
2356 HashJoinTuple tuple;
2357 int i;
2358
2359 /* Reset all flags in the main table ... */
2360 for (i = 0; i < hashtable->nbuckets; i++)
2361 {
2362 for (tuple = hashtable->buckets.unshared[i]; tuple != NULL;
2363 tuple = tuple->next.unshared)
2365 }
2366
2367 /* ... and the same for the skew buckets, if any */
2368 for (i = 0; i < hashtable->nSkewBuckets; i++)
2369 {
2370 int j = hashtable->skewBucketNums[i];
2371 HashSkewBucket *skewBucket = hashtable->skewBucket[j];
2372
2373 for (tuple = skewBucket->tuples; tuple != NULL; tuple = tuple->next.unshared)
2375 }
2376}
int j
Definition: isn.c:78
HashJoinTuple tuples
Definition: hashjoin.h:116

References HashJoinTableData::buckets, HeapTupleHeaderClearMatch(), HJTUPLE_MINTUPLE, i, j, HashJoinTableData::nbuckets, HashJoinTupleData::next, HashJoinTableData::nSkewBuckets, HashJoinTableData::skewBucket, HashJoinTableData::skewBucketNums, HashSkewBucket::tuples, HashJoinTupleData::unshared, and HashJoinTableData::unshared.

Referenced by ExecReScanHashJoin().

◆ ExecInitHash()

HashState * ExecInitHash ( Hash node,
EState estate,
int  eflags 
)

Definition at line 369 of file nodeHash.c.

370{
371 HashState *hashstate;
372
373 /* check for unsupported flags */
375
376 /*
377 * create state structure
378 */
379 hashstate = makeNode(HashState);
380 hashstate->ps.plan = (Plan *) node;
381 hashstate->ps.state = estate;
382 hashstate->ps.ExecProcNode = ExecHash;
383 /* delay building hashtable until ExecHashTableCreate() in executor run */
384 hashstate->hashtable = NULL;
385
386 /*
387 * Miscellaneous initialization
388 *
389 * create expression context for node
390 */
391 ExecAssignExprContext(estate, &hashstate->ps);
392
393 /*
394 * initialize child nodes
395 */
396 outerPlanState(hashstate) = ExecInitNode(outerPlan(node), estate, eflags);
397
398 /*
399 * initialize our result slot and type. No need to build projection
400 * because this node doesn't do projections.
401 */
403 hashstate->ps.ps_ProjInfo = NULL;
404
405 Assert(node->plan.qual == NIL);
406
407 /*
408 * Delay initialization of hash_expr until ExecInitHashJoin(). We cannot
409 * build the ExprState here as we don't yet know the join type we're going
410 * to be hashing values for and we need to know that before calling
411 * ExecBuildHash32Expr as the keep_nulls parameter depends on the join
412 * type.
413 */
414 hashstate->hash_expr = NULL;
415
416 return hashstate;
417}
PlanState * ExecInitNode(Plan *node, EState *estate, int eflags)
Definition: execProcnode.c:142
void ExecInitResultTupleSlotTL(PlanState *planstate, const TupleTableSlotOps *tts_ops)
Definition: execTuples.c:1988
const TupleTableSlotOps TTSOpsMinimalTuple
Definition: execTuples.c:86
void ExecAssignExprContext(EState *estate, PlanState *planstate)
Definition: execUtils.c:485
#define EXEC_FLAG_BACKWARD
Definition: executor.h:69
#define EXEC_FLAG_MARK
Definition: executor.h:70
static TupleTableSlot * ExecHash(PlanState *pstate)
Definition: nodeHash.c:90
#define makeNode(_type_)
Definition: nodes.h:161
#define NIL
Definition: pg_list.h:68
HashJoinTable hashtable
Definition: execnodes.h:2818
ExprState * hash_expr
Definition: execnodes.h:2819
EState * state
Definition: execnodes.h:1167
ProjectionInfo * ps_ProjInfo
Definition: execnodes.h:1205
ExecProcNodeMtd ExecProcNode
Definition: execnodes.h:1171
List * qual
Definition: plannodes.h:231

References Assert(), EXEC_FLAG_BACKWARD, EXEC_FLAG_MARK, ExecAssignExprContext(), ExecHash(), ExecInitNode(), ExecInitResultTupleSlotTL(), PlanState::ExecProcNode, HashState::hash_expr, HashState::hashtable, makeNode, NIL, outerPlan, outerPlanState, PlanState::plan, Hash::plan, HashState::ps, PlanState::ps_ProjInfo, Plan::qual, PlanState::state, and TTSOpsMinimalTuple.

Referenced by ExecInitNode().

◆ ExecParallelHashTableAlloc()

void ExecParallelHashTableAlloc ( HashJoinTable  hashtable,
int  batchno 
)

Definition at line 3288 of file nodeHash.c.

3289{
3290 ParallelHashJoinBatch *batch = hashtable->batches[batchno].shared;
3291 dsa_pointer_atomic *buckets;
3292 int nbuckets = hashtable->parallel_state->nbuckets;
3293 int i;
3294
3295 batch->buckets =
3296 dsa_allocate(hashtable->area, sizeof(dsa_pointer_atomic) * nbuckets);
3297 buckets = (dsa_pointer_atomic *)
3298 dsa_get_address(hashtable->area, batch->buckets);
3299 for (i = 0; i < nbuckets; ++i)
3301}
#define dsa_pointer_atomic_init
Definition: dsa.h:64
#define dsa_allocate(area, size)
Definition: dsa.h:109

References HashJoinTableData::area, HashJoinTableData::batches, ParallelHashJoinBatch::buckets, dsa_allocate, dsa_get_address(), dsa_pointer_atomic_init, i, InvalidDsaPointer, ParallelHashJoinState::nbuckets, HashJoinTableData::parallel_state, and ParallelHashJoinBatchAccessor::shared.

Referenced by ExecHashTableCreate(), and ExecParallelHashJoinNewBatch().

◆ ExecParallelHashTableInsert()

void ExecParallelHashTableInsert ( HashJoinTable  hashtable,
TupleTableSlot slot,
uint32  hashvalue 
)

Definition at line 1838 of file nodeHash.c.

1841{
1842 bool shouldFree;
1843 MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot, &shouldFree);
1844 dsa_pointer shared;
1845 int bucketno;
1846 int batchno;
1847
1848retry:
1849 ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno);
1850
1851 if (batchno == 0)
1852 {
1853 HashJoinTuple hashTuple;
1854
1855 /* Try to load it into memory. */
1858 hashTuple = ExecParallelHashTupleAlloc(hashtable,
1859 HJTUPLE_OVERHEAD + tuple->t_len,
1860 &shared);
1861 if (hashTuple == NULL)
1862 goto retry;
1863
1864 /* Store the hash value in the HashJoinTuple header. */
1865 hashTuple->hashvalue = hashvalue;
1866 memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
1868
1869 /* Push it onto the front of the bucket's list */
1870 ExecParallelHashPushTuple(&hashtable->buckets.shared[bucketno],
1871 hashTuple, shared);
1872 }
1873 else
1874 {
1875 size_t tuple_size = MAXALIGN(HJTUPLE_OVERHEAD + tuple->t_len);
1876
1877 Assert(batchno > 0);
1878
1879 /* Try to preallocate space in the batch if necessary. */
1880 if (hashtable->batches[batchno].preallocated < tuple_size)
1881 {
1882 if (!ExecParallelHashTuplePrealloc(hashtable, batchno, tuple_size))
1883 goto retry;
1884 }
1885
1886 Assert(hashtable->batches[batchno].preallocated >= tuple_size);
1887 hashtable->batches[batchno].preallocated -= tuple_size;
1888 sts_puttuple(hashtable->batches[batchno].inner_tuples, &hashvalue,
1889 tuple);
1890 }
1891 ++hashtable->batches[batchno].ntuples;
1892
1893 if (shouldFree)
1895}
#define PHJ_BUILD_HASH_INNER
Definition: hashjoin.h:271
static bool ExecParallelHashTuplePrealloc(HashJoinTable hashtable, int batchno, size_t size)
Definition: nodeHash.c:3560
static HashJoinTuple ExecParallelHashTupleAlloc(HashJoinTable hashtable, size_t size, dsa_pointer *shared)
Definition: nodeHash.c:2975
static void ExecParallelHashPushTuple(dsa_pointer_atomic *head, HashJoinTuple tuple, dsa_pointer tuple_shared)
Definition: nodeHash.c:3480
void sts_puttuple(SharedTuplestoreAccessor *accessor, void *meta_data, MinimalTuple tuple)
dsa_pointer_atomic * shared
Definition: hashjoin.h:313

References Assert(), BarrierPhase(), HashJoinTableData::batches, HashJoinTableData::buckets, ParallelHashJoinState::build_barrier, ExecFetchSlotMinimalTuple(), ExecHashGetBucketAndBatch(), ExecParallelHashPushTuple(), ExecParallelHashTupleAlloc(), ExecParallelHashTuplePrealloc(), HashJoinTupleData::hashvalue, heap_free_minimal_tuple(), HeapTupleHeaderClearMatch(), HJTUPLE_MINTUPLE, HJTUPLE_OVERHEAD, ParallelHashJoinBatchAccessor::inner_tuples, MAXALIGN, ParallelHashJoinBatchAccessor::ntuples, HashJoinTableData::parallel_state, PHJ_BUILD_HASH_INNER, ParallelHashJoinBatchAccessor::preallocated, HashJoinTableData::shared, sts_puttuple(), and MinimalTupleData::t_len.

Referenced by MultiExecParallelHash().

◆ ExecParallelHashTableInsertCurrentBatch()

void ExecParallelHashTableInsertCurrentBatch ( HashJoinTable  hashtable,
TupleTableSlot slot,
uint32  hashvalue 
)

Definition at line 1904 of file nodeHash.c.

1907{
1908 bool shouldFree;
1909 MinimalTuple tuple = ExecFetchSlotMinimalTuple(slot, &shouldFree);
1910 HashJoinTuple hashTuple;
1911 dsa_pointer shared;
1912 int batchno;
1913 int bucketno;
1914
1915 ExecHashGetBucketAndBatch(hashtable, hashvalue, &bucketno, &batchno);
1916 Assert(batchno == hashtable->curbatch);
1917 hashTuple = ExecParallelHashTupleAlloc(hashtable,
1918 HJTUPLE_OVERHEAD + tuple->t_len,
1919 &shared);
1920 hashTuple->hashvalue = hashvalue;
1921 memcpy(HJTUPLE_MINTUPLE(hashTuple), tuple, tuple->t_len);
1923 ExecParallelHashPushTuple(&hashtable->buckets.shared[bucketno],
1924 hashTuple, shared);
1925
1926 if (shouldFree)
1928}

References Assert(), HashJoinTableData::buckets, HashJoinTableData::curbatch, ExecFetchSlotMinimalTuple(), ExecHashGetBucketAndBatch(), ExecParallelHashPushTuple(), ExecParallelHashTupleAlloc(), HashJoinTupleData::hashvalue, heap_free_minimal_tuple(), HeapTupleHeaderClearMatch(), HJTUPLE_MINTUPLE, HJTUPLE_OVERHEAD, HashJoinTableData::shared, and MinimalTupleData::t_len.

Referenced by ExecParallelHashJoinNewBatch().

◆ ExecParallelHashTableSetCurrentBatch()

void ExecParallelHashTableSetCurrentBatch ( HashJoinTable  hashtable,
int  batchno 
)

Definition at line 3498 of file nodeHash.c.

3499{
3500 Assert(hashtable->batches[batchno].shared->buckets != InvalidDsaPointer);
3501
3502 hashtable->curbatch = batchno;
3503 hashtable->buckets.shared = (dsa_pointer_atomic *)
3504 dsa_get_address(hashtable->area,
3505 hashtable->batches[batchno].shared->buckets);
3506 hashtable->nbuckets = hashtable->parallel_state->nbuckets;
3507 hashtable->log2_nbuckets = pg_ceil_log2_32(hashtable->nbuckets);
3508 hashtable->current_chunk = NULL;
3510 hashtable->batches[batchno].at_least_one_chunk = false;
3511}
dsa_pointer current_chunk_shared
Definition: hashjoin.h:362

References HashJoinTableData::area, Assert(), ParallelHashJoinBatchAccessor::at_least_one_chunk, HashJoinTableData::batches, ParallelHashJoinBatch::buckets, HashJoinTableData::buckets, HashJoinTableData::curbatch, HashJoinTableData::current_chunk, HashJoinTableData::current_chunk_shared, dsa_get_address(), InvalidDsaPointer, HashJoinTableData::log2_nbuckets, ParallelHashJoinState::nbuckets, HashJoinTableData::nbuckets, HashJoinTableData::parallel_state, pg_ceil_log2_32(), ParallelHashJoinBatchAccessor::shared, and HashJoinTableData::shared.

Referenced by ExecParallelHashIncreaseNumBatches(), ExecParallelHashIncreaseNumBuckets(), ExecParallelHashJoinNewBatch(), and MultiExecParallelHash().

◆ ExecParallelPrepHashTableForUnmatched()

bool ExecParallelPrepHashTableForUnmatched ( HashJoinState hjstate)

Definition at line 2124 of file nodeHash.c.

2125{
2126 HashJoinTable hashtable = hjstate->hj_HashTable;
2127 int curbatch = hashtable->curbatch;
2128 ParallelHashJoinBatch *batch = hashtable->batches[curbatch].shared;
2129
2131
2132 /*
2133 * It would not be deadlock-free to wait on the batch barrier, because it
2134 * is in PHJ_BATCH_PROBE phase, and thus processes attached to it have
2135 * already emitted tuples. Therefore, we'll hold a wait-free election:
2136 * only one process can continue to the next phase, and all others detach
2137 * from this batch. They can still go any work on other batches, if there
2138 * are any.
2139 */
2141 {
2142 /* This process considers the batch to be done. */
2143 hashtable->batches[hashtable->curbatch].done = true;
2144
2145 /* Make sure any temporary files are closed. */
2146 sts_end_parallel_scan(hashtable->batches[curbatch].inner_tuples);
2147 sts_end_parallel_scan(hashtable->batches[curbatch].outer_tuples);
2148
2149 /*
2150 * Track largest batch we've seen, which would normally happen in
2151 * ExecHashTableDetachBatch().
2152 */
2153 hashtable->spacePeak =
2154 Max(hashtable->spacePeak,
2155 batch->size + sizeof(dsa_pointer_atomic) * hashtable->nbuckets);
2156 hashtable->curbatch = -1;
2157 return false;
2158 }
2159
2160 /* Now we are alone with this batch. */
2162
2163 /*
2164 * Has another process decided to give up early and command all processes
2165 * to skip the unmatched scan?
2166 */
2167 if (batch->skip_unmatched)
2168 {
2169 hashtable->batches[hashtable->curbatch].done = true;
2170 ExecHashTableDetachBatch(hashtable);
2171 return false;
2172 }
2173
2174 /* Now prepare the process local state, just as for non-parallel join. */
2176
2177 return true;
2178}
void ExecHashTableDetachBatch(HashJoinTable hashtable)
Definition: nodeHash.c:3308
void ExecPrepHashTableForUnmatched(HashJoinState *hjstate)
Definition: nodeHash.c:2103
HashJoinTable hj_HashTable
Definition: execnodes.h:2264

References Assert(), BarrierArriveAndDetachExceptLast(), BarrierPhase(), ParallelHashJoinBatch::batch_barrier, HashJoinTableData::batches, HashJoinTableData::curbatch, ParallelHashJoinBatchAccessor::done, ExecHashTableDetachBatch(), ExecPrepHashTableForUnmatched(), HashJoinState::hj_HashTable, ParallelHashJoinBatchAccessor::inner_tuples, Max, HashJoinTableData::nbuckets, ParallelHashJoinBatchAccessor::outer_tuples, PHJ_BATCH_PROBE, PHJ_BATCH_SCAN, ParallelHashJoinBatchAccessor::shared, ParallelHashJoinBatch::size, ParallelHashJoinBatch::skip_unmatched, HashJoinTableData::spacePeak, and sts_end_parallel_scan().

Referenced by ExecHashJoinImpl().

◆ ExecParallelScanHashBucket()

bool ExecParallelScanHashBucket ( HashJoinState hjstate,
ExprContext econtext 
)

Definition at line 2052 of file nodeHash.c.

2054{
2055 ExprState *hjclauses = hjstate->hashclauses;
2056 HashJoinTable hashtable = hjstate->hj_HashTable;
2057 HashJoinTuple hashTuple = hjstate->hj_CurTuple;
2058 uint32 hashvalue = hjstate->hj_CurHashValue;
2059
2060 /*
2061 * hj_CurTuple is the address of the tuple last returned from the current
2062 * bucket, or NULL if it's time to start scanning a new bucket.
2063 */
2064 if (hashTuple != NULL)
2065 hashTuple = ExecParallelHashNextTuple(hashtable, hashTuple);
2066 else
2067 hashTuple = ExecParallelHashFirstTuple(hashtable,
2068 hjstate->hj_CurBucketNo);
2069
2070 while (hashTuple != NULL)
2071 {
2072 if (hashTuple->hashvalue == hashvalue)
2073 {
2074 TupleTableSlot *inntuple;
2075
2076 /* insert hashtable's tuple into exec slot so ExecQual sees it */
2077 inntuple = ExecStoreMinimalTuple(HJTUPLE_MINTUPLE(hashTuple),
2078 hjstate->hj_HashTupleSlot,
2079 false); /* do not pfree */
2080 econtext->ecxt_innertuple = inntuple;
2081
2082 if (ExecQualAndReset(hjclauses, econtext))
2083 {
2084 hjstate->hj_CurTuple = hashTuple;
2085 return true;
2086 }
2087 }
2088
2089 hashTuple = ExecParallelHashNextTuple(hashtable, hashTuple);
2090 }
2091
2092 /*
2093 * no match
2094 */
2095 return false;
2096}
TupleTableSlot * ExecStoreMinimalTuple(MinimalTuple mtup, TupleTableSlot *slot, bool shouldFree)
Definition: execTuples.c:1635
static bool ExecQualAndReset(ExprState *state, ExprContext *econtext)
Definition: executor.h:546
static HashJoinTuple ExecParallelHashFirstTuple(HashJoinTable hashtable, int bucketno)
Definition: nodeHash.c:3450
static HashJoinTuple ExecParallelHashNextTuple(HashJoinTable hashtable, HashJoinTuple tuple)
Definition: nodeHash.c:3466
TupleTableSlot * ecxt_innertuple
Definition: execnodes.h:275
HashJoinTuple hj_CurTuple
Definition: execnodes.h:2268
ExprState * hashclauses
Definition: execnodes.h:2262
uint32 hj_CurHashValue
Definition: execnodes.h:2265
int hj_CurBucketNo
Definition: execnodes.h:2266
TupleTableSlot * hj_HashTupleSlot
Definition: execnodes.h:2270

References ExprContext::ecxt_innertuple, ExecParallelHashFirstTuple(), ExecParallelHashNextTuple(), ExecQualAndReset(), ExecStoreMinimalTuple(), HashJoinState::hashclauses, HashJoinTupleData::hashvalue, HashJoinState::hj_CurBucketNo, HashJoinState::hj_CurHashValue, HashJoinState::hj_CurTuple, HashJoinState::hj_HashTable, HashJoinState::hj_HashTupleSlot, and HJTUPLE_MINTUPLE.

Referenced by ExecHashJoinImpl().

◆ ExecParallelScanHashTableForUnmatched()

bool ExecParallelScanHashTableForUnmatched ( HashJoinState hjstate,
ExprContext econtext 
)

Definition at line 2263 of file nodeHash.c.

2265{
2266 HashJoinTable hashtable = hjstate->hj_HashTable;
2267 HashJoinTuple hashTuple = hjstate->hj_CurTuple;
2268
2269 for (;;)
2270 {
2271 /*
2272 * hj_CurTuple is the address of the tuple last returned from the
2273 * current bucket, or NULL if it's time to start scanning a new
2274 * bucket.
2275 */
2276 if (hashTuple != NULL)
2277 hashTuple = ExecParallelHashNextTuple(hashtable, hashTuple);
2278 else if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
2279 hashTuple = ExecParallelHashFirstTuple(hashtable,
2280 hjstate->hj_CurBucketNo++);
2281 else
2282 break; /* finished all buckets */
2283
2284 while (hashTuple != NULL)
2285 {
2287 {
2288 TupleTableSlot *inntuple;
2289
2290 /* insert hashtable's tuple into exec slot */
2291 inntuple = ExecStoreMinimalTuple(HJTUPLE_MINTUPLE(hashTuple),
2292 hjstate->hj_HashTupleSlot,
2293 false); /* do not pfree */
2294 econtext->ecxt_innertuple = inntuple;
2295
2296 /*
2297 * Reset temp memory each time; although this function doesn't
2298 * do any qual eval, the caller will, so let's keep it
2299 * parallel to ExecScanHashBucket.
2300 */
2301 ResetExprContext(econtext);
2302
2303 hjstate->hj_CurTuple = hashTuple;
2304 return true;
2305 }
2306
2307 hashTuple = ExecParallelHashNextTuple(hashtable, hashTuple);
2308 }
2309
2310 /* allow this loop to be cancellable */
2312 }
2313
2314 /*
2315 * no more unmatched tuples
2316 */
2317 return false;
2318}
#define ResetExprContext(econtext)
Definition: executor.h:650
static bool HeapTupleHeaderHasMatch(const MinimalTupleData *tup)
Definition: htup_details.h:706
#define CHECK_FOR_INTERRUPTS()
Definition: miscadmin.h:123

References CHECK_FOR_INTERRUPTS, ExprContext::ecxt_innertuple, ExecParallelHashFirstTuple(), ExecParallelHashNextTuple(), ExecStoreMinimalTuple(), HeapTupleHeaderHasMatch(), HashJoinState::hj_CurBucketNo, HashJoinState::hj_CurTuple, HashJoinState::hj_HashTable, HashJoinState::hj_HashTupleSlot, HJTUPLE_MINTUPLE, HashJoinTableData::nbuckets, and ResetExprContext.

Referenced by ExecHashJoinImpl().

◆ ExecPrepHashTableForUnmatched()

void ExecPrepHashTableForUnmatched ( HashJoinState hjstate)

Definition at line 2103 of file nodeHash.c.

2104{
2105 /*----------
2106 * During this scan we use the HashJoinState fields as follows:
2107 *
2108 * hj_CurBucketNo: next regular bucket to scan
2109 * hj_CurSkewBucketNo: next skew bucket (an index into skewBucketNums)
2110 * hj_CurTuple: last tuple returned, or NULL to start next bucket
2111 *----------
2112 */
2113 hjstate->hj_CurBucketNo = 0;
2114 hjstate->hj_CurSkewBucketNo = 0;
2115 hjstate->hj_CurTuple = NULL;
2116}
int hj_CurSkewBucketNo
Definition: execnodes.h:2267

References HashJoinState::hj_CurBucketNo, HashJoinState::hj_CurSkewBucketNo, and HashJoinState::hj_CurTuple.

Referenced by ExecHashJoinImpl(), and ExecParallelPrepHashTableForUnmatched().

◆ ExecReScanHash()

void ExecReScanHash ( HashState node)

Definition at line 2380 of file nodeHash.c.

2381{
2383
2384 /*
2385 * if chgParam of subnode is not null then plan will be re-scanned by
2386 * first ExecProcNode.
2387 */
2388 if (outerPlan->chgParam == NULL)
2390}
void ExecReScan(PlanState *node)
Definition: execAmi.c:77

References ExecReScan(), outerPlan, and outerPlanState.

Referenced by ExecReScan().

◆ ExecScanHashBucket()

bool ExecScanHashBucket ( HashJoinState hjstate,
ExprContext econtext 
)

Definition at line 1991 of file nodeHash.c.

1993{
1994 ExprState *hjclauses = hjstate->hashclauses;
1995 HashJoinTable hashtable = hjstate->hj_HashTable;
1996 HashJoinTuple hashTuple = hjstate->hj_CurTuple;
1997 uint32 hashvalue = hjstate->hj_CurHashValue;
1998
1999 /*
2000 * hj_CurTuple is the address of the tuple last returned from the current
2001 * bucket, or NULL if it's time to start scanning a new bucket.
2002 *
2003 * If the tuple hashed to a skew bucket then scan the skew bucket
2004 * otherwise scan the standard hashtable bucket.
2005 */
2006 if (hashTuple != NULL)
2007 hashTuple = hashTuple->next.unshared;
2008 else if (hjstate->hj_CurSkewBucketNo != INVALID_SKEW_BUCKET_NO)
2009 hashTuple = hashtable->skewBucket[hjstate->hj_CurSkewBucketNo]->tuples;
2010 else
2011 hashTuple = hashtable->buckets.unshared[hjstate->hj_CurBucketNo];
2012
2013 while (hashTuple != NULL)
2014 {
2015 if (hashTuple->hashvalue == hashvalue)
2016 {
2017 TupleTableSlot *inntuple;
2018
2019 /* insert hashtable's tuple into exec slot so ExecQual sees it */
2020 inntuple = ExecStoreMinimalTuple(HJTUPLE_MINTUPLE(hashTuple),
2021 hjstate->hj_HashTupleSlot,
2022 false); /* do not pfree */
2023 econtext->ecxt_innertuple = inntuple;
2024
2025 if (ExecQualAndReset(hjclauses, econtext))
2026 {
2027 hjstate->hj_CurTuple = hashTuple;
2028 return true;
2029 }
2030 }
2031
2032 hashTuple = hashTuple->next.unshared;
2033 }
2034
2035 /*
2036 * no match
2037 */
2038 return false;
2039}

References HashJoinTableData::buckets, ExprContext::ecxt_innertuple, ExecQualAndReset(), ExecStoreMinimalTuple(), HashJoinState::hashclauses, HashJoinTupleData::hashvalue, HashJoinState::hj_CurBucketNo, HashJoinState::hj_CurHashValue, HashJoinState::hj_CurSkewBucketNo, HashJoinState::hj_CurTuple, HashJoinState::hj_HashTable, HashJoinState::hj_HashTupleSlot, HJTUPLE_MINTUPLE, INVALID_SKEW_BUCKET_NO, HashJoinTupleData::next, HashJoinTableData::skewBucket, HashSkewBucket::tuples, HashJoinTupleData::unshared, and HashJoinTableData::unshared.

Referenced by ExecHashJoinImpl().

◆ ExecScanHashTableForUnmatched()

bool ExecScanHashTableForUnmatched ( HashJoinState hjstate,
ExprContext econtext 
)

Definition at line 2189 of file nodeHash.c.

2190{
2191 HashJoinTable hashtable = hjstate->hj_HashTable;
2192 HashJoinTuple hashTuple = hjstate->hj_CurTuple;
2193
2194 for (;;)
2195 {
2196 /*
2197 * hj_CurTuple is the address of the tuple last returned from the
2198 * current bucket, or NULL if it's time to start scanning a new
2199 * bucket.
2200 */
2201 if (hashTuple != NULL)
2202 hashTuple = hashTuple->next.unshared;
2203 else if (hjstate->hj_CurBucketNo < hashtable->nbuckets)
2204 {
2205 hashTuple = hashtable->buckets.unshared[hjstate->hj_CurBucketNo];
2206 hjstate->hj_CurBucketNo++;
2207 }
2208 else if (hjstate->hj_CurSkewBucketNo < hashtable->nSkewBuckets)
2209 {
2210 int j = hashtable->skewBucketNums[hjstate->hj_CurSkewBucketNo];
2211
2212 hashTuple = hashtable->skewBucket[j]->tuples;
2213 hjstate->hj_CurSkewBucketNo++;
2214 }
2215 else
2216 break; /* finished all buckets */
2217
2218 while (hashTuple != NULL)
2219 {
2221 {
2222 TupleTableSlot *inntuple;
2223
2224 /* insert hashtable's tuple into exec slot */
2225 inntuple = ExecStoreMinimalTuple(HJTUPLE_MINTUPLE(hashTuple),
2226 hjstate->hj_HashTupleSlot,
2227 false); /* do not pfree */
2228 econtext->ecxt_innertuple = inntuple;
2229
2230 /*
2231 * Reset temp memory each time; although this function doesn't
2232 * do any qual eval, the caller will, so let's keep it
2233 * parallel to ExecScanHashBucket.
2234 */
2235 ResetExprContext(econtext);
2236
2237 hjstate->hj_CurTuple = hashTuple;
2238 return true;
2239 }
2240
2241 hashTuple = hashTuple->next.unshared;
2242 }
2243
2244 /* allow this loop to be cancellable */
2246 }
2247
2248 /*
2249 * no more unmatched tuples
2250 */
2251 return false;
2252}

References HashJoinTableData::buckets, CHECK_FOR_INTERRUPTS, ExprContext::ecxt_innertuple, ExecStoreMinimalTuple(), HeapTupleHeaderHasMatch(), HashJoinState::hj_CurBucketNo, HashJoinState::hj_CurSkewBucketNo, HashJoinState::hj_CurTuple, HashJoinState::hj_HashTable, HashJoinState::hj_HashTupleSlot, HJTUPLE_MINTUPLE, j, HashJoinTableData::nbuckets, HashJoinTupleData::next, HashJoinTableData::nSkewBuckets, ResetExprContext, HashJoinTableData::skewBucket, HashJoinTableData::skewBucketNums, HashSkewBucket::tuples, HashJoinTupleData::unshared, and HashJoinTableData::unshared.

Referenced by ExecHashJoinImpl().

◆ ExecShutdownHash()

void ExecShutdownHash ( HashState node)

Definition at line 2830 of file nodeHash.c.

2831{
2832 /* Allocate save space if EXPLAIN'ing and we didn't do so already */
2833 if (node->ps.instrument && !node->hinstrument)
2835 /* Now accumulate data for the current (final) hash table */
2836 if (node->hinstrument && node->hashtable)
2838}
#define palloc0_object(type)
Definition: fe_memutils.h:75
void ExecHashAccumInstrumentation(HashInstrumentation *instrument, HashJoinTable hashtable)
Definition: nodeHash.c:2876

References ExecHashAccumInstrumentation(), HashState::hashtable, HashState::hinstrument, PlanState::instrument, palloc0_object, and HashState::ps.

Referenced by ExecShutdownNode_walker().

◆ MultiExecHash()

Node * MultiExecHash ( HashState node)

Definition at line 104 of file nodeHash.c.

105{
106 /* must provide our own instrumentation support */
107 if (node->ps.instrument)
109
110 if (node->parallel_state != NULL)
112 else
114
115 /* must provide our own instrumentation support */
116 if (node->ps.instrument)
118
119 /*
120 * We do not return the hash table directly because it's not a subtype of
121 * Node, and so would violate the MultiExecProcNode API. Instead, our
122 * parent Hashjoin node is expected to know how to fish it out of our node
123 * state. Ugly but not really worth cleaning up, since Hashjoin knows
124 * quite a bit more about Hash besides that.
125 */
126 return NULL;
127}
void InstrStartNode(Instrumentation *instr)
Definition: instrument.c:68
void InstrStopNode(Instrumentation *instr, double nTuples)
Definition: instrument.c:84
static void MultiExecParallelHash(HashState *node)
Definition: nodeHash.c:218
static void MultiExecPrivateHash(HashState *node)
Definition: nodeHash.c:137
struct ParallelHashJoinState * parallel_state
Definition: execnodes.h:2840

References HashState::hashtable, InstrStartNode(), InstrStopNode(), PlanState::instrument, MultiExecParallelHash(), MultiExecPrivateHash(), HashState::parallel_state, HashJoinTableData::partialTuples, and HashState::ps.

Referenced by MultiExecProcNode().