github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/colexec/mergejoiner_util.go (about) 1 // Copyright 2019 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 // circularGroupsBuffer is a struct designed to store the groups' slices for a 14 // given column. We know that there is a maximum number of possible groups per 15 // batch, so we can cap the buffer and make it circular. 16 type circularGroupsBuffer struct { 17 // startIdx indicates which index the next group to be processed is stored 18 // at. 19 startIdx int 20 // endIdx indicates the first empty slot within the buffer. 21 endIdx int 22 // nextColStartIdx is the index of the first group that belongs to the next 23 // column. 24 nextColStartIdx int 25 cap int 26 27 leftGroups []group 28 rightGroups []group 29 } 30 31 func makeGroupsBuffer(batchSize int) circularGroupsBuffer { 32 return circularGroupsBuffer{ 33 // The maximum number of possible groups per batch is achieved with FULL 34 // OUTER JOIN when no rows have matches, so there will be exactly 35 // batchSize x 2 groups. We add an additional element to the capacity in 36 // order to be able to distinguish between an "empty" (no groups) and a 37 // "full" (2*batchSize groups) buffers. 38 cap: 2*batchSize + 1, 39 // Since we have a circular buffer, it is possible for groups to wrap when 40 // cap is reached. Consider an example when batchSize = 3 and startIdx = 6 41 // when maximum number of groups is present: 42 // buffer = [1, 2, 3, 4, 5, x, 0] (integers denote different groups and 'x' 43 // stands for a garbage). 44 // When getGroups() is called, for ease of usage we need to return the 45 // buffer "flattened out", and in order to reduce allocation, we actually 46 // reserve 4*batchSize. In the example above we will copy the buffer as: 47 // buffer = [1, 2, 3, 4, 5, x, 0, 1, 2, 3, 4, 5] 48 // and will return buffer[6:12] when getGroups is called. 49 // The calculations for why 4*batchSize is sufficient: 50 // - the largest position in which the first group can be placed is 51 // 2*batchSize (cap field enforces that) 52 // - the largest number of groups to copy from the "physical" start of the 53 // buffer is 2*batchSize-1 54 // - adding those two numbers we arrive at 4*batchSize. 55 leftGroups: make([]group, 4*batchSize), 56 rightGroups: make([]group, 4*batchSize), 57 } 58 } 59 60 // reset sets the circular buffer state to groups that produce the maximal 61 // cross product, i.e. one maximal group on each side. 62 func (b *circularGroupsBuffer) reset(lStartIdx int, lEndIdx int, rStartIdx int, rEndIdx int) { 63 b.startIdx = 0 64 b.endIdx = 1 65 b.nextColStartIdx = 1 66 67 b.leftGroups[0] = group{lStartIdx, lEndIdx, 1, 0, false, false} 68 b.rightGroups[0] = group{rStartIdx, rEndIdx, 1, 0, false, false} 69 } 70 71 // nextGroupInCol returns whether or not there exists a next group in the 72 // current column, and sets the parameters to be the left and right groups 73 // corresponding to the next values in the buffer. 74 func (b *circularGroupsBuffer) nextGroupInCol(lGroup *group, rGroup *group) bool { 75 if b.startIdx == b.nextColStartIdx { 76 return false 77 } 78 idx := b.startIdx 79 b.startIdx++ 80 81 if b.startIdx >= b.cap { 82 b.startIdx -= b.cap 83 } 84 85 *lGroup = b.leftGroups[idx] 86 *rGroup = b.rightGroups[idx] 87 return true 88 } 89 90 // isLastGroupInCol returns whether the last group obtained via nextGroupInCol 91 // from the buffer is the last one for the column. 92 func (b *circularGroupsBuffer) isLastGroupInCol() bool { 93 return b.startIdx == b.nextColStartIdx 94 } 95 96 // addGroupsToNextCol appends a left and right group to the buffer. In an 97 // iteration of a column, these values are either processed in the next 98 // equality column or used to build the cross product. 99 func (b *circularGroupsBuffer) addGroupsToNextCol( 100 curLIdx int, lRunLength int, curRIdx int, rRunLength int, 101 ) { 102 b.leftGroups[b.endIdx] = group{ 103 rowStartIdx: curLIdx, 104 rowEndIdx: curLIdx + lRunLength, 105 numRepeats: rRunLength, 106 toBuild: lRunLength * rRunLength, 107 } 108 b.rightGroups[b.endIdx] = group{ 109 rowStartIdx: curRIdx, 110 rowEndIdx: curRIdx + rRunLength, 111 numRepeats: lRunLength, 112 toBuild: lRunLength * rRunLength, 113 } 114 b.endIdx++ 115 116 // Modulus on every step is more expensive than this check. 117 if b.endIdx >= b.cap { 118 b.endIdx -= b.cap 119 } 120 } 121 122 // addLeftUnmatchedGroup adds a left and right group to the buffer that 123 // correspond to an unmatched row from the left side in the case of LEFT OUTER, 124 // LEFT ANTI, or EXCEPT ALL joins. 125 func (b *circularGroupsBuffer) addLeftUnmatchedGroup(curLIdx int, curRIdx int) { 126 b.leftGroups[b.endIdx] = group{ 127 rowStartIdx: curLIdx, 128 rowEndIdx: curLIdx + 1, 129 numRepeats: 1, 130 toBuild: 1, 131 nullGroup: false, 132 unmatched: true, 133 } 134 b.rightGroups[b.endIdx] = group{ 135 rowStartIdx: curRIdx, 136 rowEndIdx: curRIdx + 1, 137 numRepeats: 1, 138 toBuild: 1, 139 nullGroup: true, 140 unmatched: false, 141 } 142 b.endIdx++ 143 144 // Modulus on every step is more expensive than this check. 145 if b.endIdx >= b.cap { 146 b.endIdx -= b.cap 147 } 148 } 149 150 // addRightOuterGroup adds a left and right group to the buffer that correspond 151 // to an unmatched row from the right side in the case of RIGHT OUTER JOIN. 152 func (b *circularGroupsBuffer) addRightOuterGroup(curLIdx int, curRIdx int) { 153 b.leftGroups[b.endIdx] = group{ 154 rowStartIdx: curLIdx, 155 rowEndIdx: curLIdx + 1, 156 numRepeats: 1, 157 toBuild: 1, 158 nullGroup: true, 159 unmatched: false, 160 } 161 b.rightGroups[b.endIdx] = group{ 162 rowStartIdx: curRIdx, 163 rowEndIdx: curRIdx + 1, 164 numRepeats: 1, 165 toBuild: 1, 166 nullGroup: false, 167 unmatched: true, 168 } 169 b.endIdx++ 170 171 // Modulus on every step is more expensive than this check. 172 if b.endIdx >= b.cap { 173 b.endIdx -= b.cap 174 } 175 } 176 177 // addLeftSemiGroup adds a left group to the buffer that corresponds to a run 178 // of tuples from the left side that all have a match on the right side. This 179 // should only be called after processing the last equality column, and this 180 // group will be used by the builder next. Note that we're not adding a right 181 // group here since tuples from the right are not outputted in LEFT SEMI nor 182 // INTERSECT ALL joins. 183 func (b *circularGroupsBuffer) addLeftSemiGroup(curLIdx int, lRunLength int) { 184 b.leftGroups[b.endIdx] = group{ 185 rowStartIdx: curLIdx, 186 rowEndIdx: curLIdx + lRunLength, 187 numRepeats: 1, 188 toBuild: lRunLength, 189 } 190 b.endIdx++ 191 192 // Modulus on every step is more expensive than this check. 193 if b.endIdx >= b.cap { 194 b.endIdx -= b.cap 195 } 196 } 197 198 // finishedCol is used to notify the circular buffer to update the indices 199 // representing the "window" of available values for the next column. 200 func (b *circularGroupsBuffer) finishedCol() { 201 b.startIdx = b.nextColStartIdx 202 b.nextColStartIdx = b.endIdx 203 } 204 205 // getGroups returns left and right slices of groups that are contiguous, which 206 // is a useful simplification for the build phase. 207 func (b *circularGroupsBuffer) getGroups() ([]group, []group) { 208 startIdx := b.startIdx 209 endIdx := b.endIdx 210 leftGroups, rightGroups := b.leftGroups, b.rightGroups 211 212 if endIdx < startIdx { 213 copy(leftGroups[b.cap:], leftGroups[:endIdx]) 214 copy(rightGroups[b.cap:], rightGroups[:endIdx]) 215 endIdx += b.cap 216 } 217 218 return leftGroups[startIdx:endIdx], rightGroups[startIdx:endIdx] 219 }