github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/colexec/hash_utils.go (about) 1 // Copyright 2020 The Cockroach Authors. 2 // 3 // Use of this software is governed by the Business Source License 4 // included in the file licenses/BSL.txt. 5 // 6 // As of the Change Date specified in that file, in accordance with 7 // the Business Source License, use of this software will be governed 8 // by the Apache License, Version 2.0, included in the file 9 // licenses/APL.txt. 10 11 package colexec 12 13 import ( 14 "context" 15 16 "github.com/cockroachdb/cockroach/pkg/col/coldata" 17 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 18 "github.com/cockroachdb/cockroach/pkg/sql/types" 19 ) 20 21 // initHash, rehash, and finalizeHash work together to compute the hash value 22 // for an individual key tuple which represents a row's equality columns. Since 23 // this key is a tuple of various types, rehash is used to apply a 24 // transformation on the resulting hash value based on an element of the key of 25 // a specified type. 26 // 27 // The combination of these three functions actually defines a hashing function 28 // family - changing the initial hash value will produce a "different" hash 29 // function. 30 // 31 // We currently use the same hash functions used by go's maps. 32 // TODO(asubiotto): Once https://go-review.googlesource.com/c/go/+/155118/ is 33 // in, we should use the public API. 34 35 // defaultInitHashValue is the default initValue to be used in initHash 36 // function. 37 const defaultInitHashValue = 1 38 39 var ( 40 uint64OneColumn []uint64 41 uint64TwoColumn []uint64 42 ) 43 44 func init() { 45 uint64OneColumn = make([]uint64, coldata.MaxBatchSize) 46 uint64TwoColumn = make([]uint64, coldata.MaxBatchSize) 47 for i := range uint64OneColumn { 48 uint64OneColumn[i] = 1 49 uint64TwoColumn[i] = 2 50 } 51 } 52 53 // rehash takes an element of a key (tuple representing a row of equality 54 // column values) at a given column and computes a new hash by applying a 55 // transformation to the existing hash. This function is generated by execgen, 56 // so it doesn't appear in this file. Look at hash_utils_tmpl.go for the source 57 // code. 58 // 59 // initHash initializes the hash value of each key to its initial state for 60 // rehashing purposes. 61 // NOTE: initValue *must* be non-zero. 62 func initHash(buckets []uint64, nKeys int, initValue uint64) { 63 switch initValue { 64 case 1: 65 for n := 0; n < nKeys; n += copy(buckets[n:], uint64OneColumn) { 66 } 67 case 2: 68 for n := 0; n < nKeys; n += copy(buckets[n:], uint64TwoColumn) { 69 } 70 default: 71 for i := 0; i < nKeys; i++ { 72 buckets[i] = initValue 73 } 74 } 75 } 76 77 // finalizeHash takes each key's hash value and applies a final transformation 78 // onto it so that it fits within numBuckets buckets. 79 func finalizeHash(buckets []uint64, nKeys int, numBuckets uint64) { 80 isPowerOfTwo := numBuckets&(numBuckets-1) == 0 81 if isPowerOfTwo { 82 for i := 0; i < nKeys; i++ { 83 // Since numBuckets is a power of 2, modulo numBuckets could be optimized 84 // into a bitwise operation which improves benchmark performance by 20%. 85 // In effect, the following code is equivalent to (but faster than): 86 // buckets[i] = buckets[i] % numBuckets 87 buckets[i] = buckets[i] & (numBuckets - 1) 88 } 89 } else { 90 for i := 0; i < nKeys; i++ { 91 buckets[i] = buckets[i] % numBuckets 92 } 93 } 94 } 95 96 // tupleHashDistributor is a helper struct that distributes tuples from batches 97 // according to the corresponding hashes. The "distribution" occurs by 98 // populating selection vectors which the caller needs to use accordingly. 99 type tupleHashDistributor struct { 100 // initHashValue is the value used to initialize the hash buckets. Different 101 // values can be used to define different hash functions. 102 initHashValue uint64 103 // buckets will contain the computed hash value of a group of columns with 104 // the same index in the current batch. 105 buckets []uint64 106 // selections stores the selection vectors that actually define how to 107 // distribute the tuples from the batch. 108 selections [][]int 109 // cancelChecker is used during the hashing of the rows to distribute to 110 // check for query cancellation. 111 cancelChecker CancelChecker 112 overloadHelper overloadHelper 113 datumAlloc sqlbase.DatumAlloc 114 } 115 116 func newTupleHashDistributor(initHashValue uint64, numOutputs int) *tupleHashDistributor { 117 selections := make([][]int, numOutputs) 118 for i := range selections { 119 selections[i] = make([]int, 0, coldata.BatchSize()) 120 } 121 return &tupleHashDistributor{ 122 initHashValue: initHashValue, 123 buckets: make([]uint64, coldata.BatchSize()), 124 selections: selections, 125 } 126 } 127 128 func (d *tupleHashDistributor) distribute( 129 ctx context.Context, b coldata.Batch, types []*types.T, hashCols []uint32, 130 ) [][]int { 131 n := b.Length() 132 initHash(d.buckets, n, d.initHashValue) 133 134 for _, i := range hashCols { 135 rehash(ctx, d.buckets, b.ColVec(int(i)), n, b.Selection(), d.cancelChecker, d.overloadHelper, &d.datumAlloc) 136 } 137 138 finalizeHash(d.buckets, n, uint64(len(d.selections))) 139 140 // Reset selections. 141 for i := 0; i < len(d.selections); i++ { 142 d.selections[i] = d.selections[i][:0] 143 } 144 145 // Build a selection vector for each output. 146 selection := b.Selection() 147 if selection != nil { 148 for i, selIdx := range selection[:n] { 149 outputIdx := d.buckets[i] 150 d.selections[outputIdx] = append(d.selections[outputIdx], selIdx) 151 } 152 } else { 153 for i := range d.buckets[:n] { 154 outputIdx := d.buckets[i] 155 d.selections[outputIdx] = append(d.selections[outputIdx], i) 156 } 157 } 158 return d.selections 159 } 160 161 // resetNumOutputs sets up the tupleHashDistributor to distribute the tuples 162 // to a different number of outputs. 163 func (d *tupleHashDistributor) resetNumOutputs(numOutputs int) { 164 if cap(d.selections) >= numOutputs { 165 d.selections = d.selections[:numOutputs] 166 return 167 } 168 d.selections = d.selections[:cap(d.selections)] 169 for len(d.selections) < numOutputs { 170 d.selections = append(d.selections, make([]int, 0, coldata.BatchSize())) 171 } 172 }