diff --git a/pkg/sql/colexec/colexechash/hashtable.go b/pkg/sql/colexec/colexechash/hashtable.go index bc2029afffe8..3784c20e1707 100644 --- a/pkg/sql/colexec/colexechash/hashtable.go +++ b/pkg/sql/colexec/colexechash/hashtable.go @@ -62,6 +62,15 @@ const ( HashTableDeletingProbeMode ) +// keyID encodes the ordinal of the corresponding tuple. +// +// For each tuple with the ordinal 'i' (the ordinal among all tuples in the +// hash table or within a single probing batch), keyID is calculated as: +// keyID = i + 1. +// +// keyID of 0 is reserved to indicate the end of the hash chain. +type keyID = uint64 + // hashChains describes the partitioning of a set of tuples into singly-linked // lists ("buckets") where all tuples in a list have the same hash. // @@ -70,17 +79,13 @@ const ( // First[bucket], and the following keyIDs of the tuples in the list can be // found using Next[keyID] traversal. Whenever keyID of 0 is encountered, the // end of the list has been reached. -// -// For each tuple with the ordinal 'i' (the ordinal among all tuples in the -// hash table or within a single probing batch), keyID is calculated as: -// keyID = i + 1. type hashChains struct { // First stores the first keyID of the tuple that resides in each bucket. // The tuple is the head of the corresponding hash chain. // // The length of this slice is equal to the number of buckets used in the // hash table at the moment. - First []uint64 + First []keyID // Next is a densely-packed list that stores the keyID of the next tuple in // the hash chain, where an ID of 0 is reserved to represent the end of the @@ -88,7 +93,7 @@ type hashChains struct { // // The length of this slice is equal to the number of tuples stored in the // hash table at the moment plus one (Next[0] is unused). - Next []uint64 + Next []keyID } // hashTableProbeBuffer stores the information related to the probing batch. @@ -146,7 +151,7 @@ type hashTableProbeBuffer struct { // ToCheckID[i] = Next[ToCheckID[i]]. // Whenever ToCheckID[i] becomes 0, there are no more matches for the ith // probing tuple. - ToCheckID []uint64 + ToCheckID []keyID // differs stores whether the probing tuple included in ToCheck differs // from the corresponding "candidate" tuple specified in ToCheckID. @@ -186,7 +191,7 @@ type hashTableProbeBuffer struct { // once the tuple is determined to not have duplicates with any tuples // already in the hash table. // See a comment on DistinctBuild for an example. - HeadID []uint64 + HeadID []keyID // HashBuffer stores the hash values of each tuple in the probing batch. It // will be dynamically updated when the HashTable is built in distinct mode. @@ -237,7 +242,7 @@ type HashTable struct { // the hash table that has the same value as the current key. The HeadID of // the key is the first key of that value found in the next linked list. // This field will be lazily populated by the prober. - Same []uint64 + Same []keyID // Visited represents whether each of the corresponding keys have been // touched by the prober. Visited []bool @@ -339,7 +344,7 @@ func NewHashTable( ht := &HashTable{ allocator: allocator, BuildScratch: hashChains{ - First: make([]uint64, initialNumHashBuckets), + First: make([]keyID, initialNumHashBuckets), }, Keys: make([]coldata.Vec, len(keyCols)), Vals: colexecutils.NewAppendOnlyBufferedBatch(allocator, sourceTypes, colsToStore), @@ -352,11 +357,11 @@ func NewHashTable( } if buildMode == HashTableDistinctBuildMode { - ht.ProbeScratch.First = make([]uint64, initialNumHashBuckets) + ht.ProbeScratch.First = make([]keyID, initialNumHashBuckets) // ht.BuildScratch.Next will be populated dynamically by appending to // it, but we need to make sure that the special keyID=0 (which // indicates the end of the hash chain) is always present. - ht.BuildScratch.Next = []uint64{0} + ht.BuildScratch.Next = []keyID{0} } ht.cancelChecker.Init(ctx) @@ -596,7 +601,7 @@ func (ht *HashTable) ComputeHashAndBuildChains(batch coldata.Batch) { batchLength := batch.Length() if cap(ht.ProbeScratch.Next) < batchLength+1 { - ht.ProbeScratch.Next = make([]uint64, batchLength+1) + ht.ProbeScratch.Next = make([]keyID, batchLength+1) } ht.ComputeBuckets(ht.ProbeScratch.Next[1:batchLength+1], ht.Keys, batchLength, batch.Selection()) ht.ProbeScratch.HashBuffer = append(ht.ProbeScratch.HashBuffer[:0], ht.ProbeScratch.Next[1:batchLength+1]...) @@ -630,7 +635,7 @@ func (ht *HashTable) ComputeHashAndBuildChains(batch coldata.Batch) { func (ht *HashTable) FindBuckets( batch coldata.Batch, keyCols []coldata.Vec, - first, next []uint64, + first, next []keyID, duplicatesChecker func([]coldata.Vec, uint64, []int) uint64, ) { batchLength := batch.Length() @@ -662,7 +667,7 @@ func (ht *HashTable) FindBuckets( func (ht *HashTable) RemoveDuplicates( batch coldata.Batch, keyCols []coldata.Vec, - first, next []uint64, + first, next []keyID, duplicatesChecker func([]coldata.Vec, uint64, []int) uint64, ) { ht.FindBuckets(batch, keyCols, first, next, duplicatesChecker) @@ -757,7 +762,7 @@ func (ht *HashTable) ComputeBuckets(buckets []uint64, keys []coldata.Vec, nKeys } // buildNextChains builds the hash map from the computed hash values. -func (ht *HashTable) buildNextChains(first, next []uint64, offset, batchSize uint64) { +func (ht *HashTable) buildNextChains(first, next []keyID, offset, batchSize uint64) { // The loop direction here is reversed to ensure that when we are building the // next chain for the probe table, the keyID in each equality chain inside // `next` is strictly in ascending order. This is crucial to ensure that when @@ -799,7 +804,7 @@ func (p *hashTableProbeBuffer) SetupLimitedSlices(length int, buildMode HashTabl // Note that we don't use maybeAllocate* methods below because ToCheckID and // ToCheck don't need to be zeroed out when reused. if cap(p.ToCheckID) < length { - p.ToCheckID = make([]uint64, length) + p.ToCheckID = make([]keyID, length) } else { p.ToCheckID = p.ToCheckID[:length] } @@ -812,7 +817,7 @@ func (p *hashTableProbeBuffer) SetupLimitedSlices(length int, buildMode HashTabl // FindNext determines the id of the next key inside the ToCheckID buckets for // each equality column key in ToCheck. -func (ht *HashTable) FindNext(next []uint64, nToCheck uint64) { +func (ht *HashTable) FindNext(next []keyID, nToCheck uint64) { for _, toCheck := range ht.ProbeScratch.ToCheck[:nToCheck] { ht.ProbeScratch.ToCheckID[toCheck] = next[ht.ProbeScratch.ToCheckID[toCheck]] }