github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/physicalplan/aggregator_funcs.go (about) 1 // Copyright 2016 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 physicalplan 12 13 import ( 14 "context" 15 16 "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" 17 "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" 18 "github.com/cockroachdb/cockroach/pkg/sql/types" 19 ) 20 21 // FinalStageInfo is a wrapper around an aggregation function performed 22 // in the final stage of distributed aggregations that allows us to specify the 23 // corresponding inputs from the local aggregations by their indices in the LocalStage. 24 type FinalStageInfo struct { 25 Fn execinfrapb.AggregatorSpec_Func 26 // Specifies the ordered slice of outputs from local aggregations to propagate 27 // as inputs to Fn. This must be ordered according to the underlying aggregate builtin 28 // arguments signature found in aggregate_builtins.go. 29 LocalIdxs []uint32 30 } 31 32 // DistAggregationInfo is a blueprint for planning distributed aggregations. It 33 // describes two stages - a local stage performs local aggregations wherever 34 // data is available and generates partial results, and a final stage aggregates 35 // the partial results from all data "partitions". 36 // 37 // The simplest example is SUM: the local stage computes the SUM of the items 38 // on each node, and a final stage SUMs those partial sums into a final sum. 39 // Similar functions are MIN, MAX, BOOL_AND, BOOL_OR. 40 // 41 // A less trivial example is COUNT: the local stage counts (COUNT), the final stage 42 // adds the counts (SUM_INT). 43 // 44 // A more complex example is AVG, for which we have to do *multiple* 45 // aggregations in each stage: we need to get a sum and a count, so the local 46 // stage does SUM and COUNT, and the final stage does SUM and SUM_INT. We also 47 // need an expression that takes these two values and generates the final AVG 48 // result. 49 type DistAggregationInfo struct { 50 // The local stage consists of one or more aggregations. All aggregations have 51 // the same input. 52 LocalStage []execinfrapb.AggregatorSpec_Func 53 54 // The final stage consists of one or more aggregations that take in an 55 // arbitrary number of inputs from the local stages. The inputs are ordered and 56 // mapped by the indices of the local aggregations in LocalStage (specified by 57 // LocalIdxs). 58 FinalStage []FinalStageInfo 59 60 // An optional rendering expression used to obtain the final result; required 61 // if there is more than one aggregation in each of the stages. 62 // 63 // Conceptually this is an expression that has access to the final stage 64 // results (via IndexedVars), to be run as the PostProcessing step of the 65 // final stage processor. However, there are some complications: 66 // - this structure is a blueprint for aggregating inputs of different 67 // types, and in some cases the expression may be different depending on 68 // the types (see AVG below). 69 // - we support combining multiple "top level" aggregations into the same 70 // processors, so the correct indexing of the input variables is not 71 // predetermined. 72 // 73 // Instead of defining a canonical non-typed expression and then tweaking it 74 // with visitors, we use a function that directly creates a typed expression 75 // on demand. The expression will refer to the final stage results using 76 // IndexedVars, with indices specified by varIdxs (1-1 mapping). 77 FinalRendering func(h *tree.IndexedVarHelper, varIdxs []int) (tree.TypedExpr, error) 78 } 79 80 // Convenient value for FinalStageInfo.LocalIdxs when there is only one aggregation 81 // function in each of the LocalStage and FinalStage. Otherwise, specify the explicit 82 // index corresponding to the local stage. 83 var passThroughLocalIdxs = []uint32{0} 84 85 // DistAggregationTable is DistAggregationInfo look-up table. Functions that 86 // don't have an entry in the table are not optimized with a local stage. 87 var DistAggregationTable = map[execinfrapb.AggregatorSpec_Func]DistAggregationInfo{ 88 execinfrapb.AggregatorSpec_ANY_NOT_NULL: { 89 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_ANY_NOT_NULL}, 90 FinalStage: []FinalStageInfo{ 91 { 92 Fn: execinfrapb.AggregatorSpec_ANY_NOT_NULL, 93 LocalIdxs: passThroughLocalIdxs, 94 }, 95 }, 96 }, 97 98 execinfrapb.AggregatorSpec_BIT_AND: { 99 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BIT_AND}, 100 FinalStage: []FinalStageInfo{ 101 { 102 Fn: execinfrapb.AggregatorSpec_BIT_AND, 103 LocalIdxs: passThroughLocalIdxs, 104 }, 105 }, 106 }, 107 108 execinfrapb.AggregatorSpec_BIT_OR: { 109 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BIT_OR}, 110 FinalStage: []FinalStageInfo{ 111 { 112 Fn: execinfrapb.AggregatorSpec_BIT_OR, 113 LocalIdxs: passThroughLocalIdxs, 114 }, 115 }, 116 }, 117 118 execinfrapb.AggregatorSpec_BOOL_AND: { 119 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BOOL_AND}, 120 FinalStage: []FinalStageInfo{ 121 { 122 Fn: execinfrapb.AggregatorSpec_BOOL_AND, 123 LocalIdxs: passThroughLocalIdxs, 124 }, 125 }, 126 }, 127 128 execinfrapb.AggregatorSpec_BOOL_OR: { 129 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_BOOL_OR}, 130 FinalStage: []FinalStageInfo{ 131 { 132 Fn: execinfrapb.AggregatorSpec_BOOL_OR, 133 LocalIdxs: passThroughLocalIdxs, 134 }, 135 }, 136 }, 137 138 execinfrapb.AggregatorSpec_COUNT: { 139 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_COUNT}, 140 FinalStage: []FinalStageInfo{ 141 { 142 Fn: execinfrapb.AggregatorSpec_SUM_INT, 143 LocalIdxs: passThroughLocalIdxs, 144 }, 145 }, 146 }, 147 148 execinfrapb.AggregatorSpec_COUNT_ROWS: { 149 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_COUNT_ROWS}, 150 FinalStage: []FinalStageInfo{ 151 { 152 Fn: execinfrapb.AggregatorSpec_SUM_INT, 153 LocalIdxs: passThroughLocalIdxs, 154 }, 155 }, 156 }, 157 158 execinfrapb.AggregatorSpec_MAX: { 159 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_MAX}, 160 FinalStage: []FinalStageInfo{ 161 { 162 Fn: execinfrapb.AggregatorSpec_MAX, 163 LocalIdxs: passThroughLocalIdxs, 164 }, 165 }, 166 }, 167 168 execinfrapb.AggregatorSpec_MIN: { 169 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_MIN}, 170 FinalStage: []FinalStageInfo{ 171 { 172 Fn: execinfrapb.AggregatorSpec_MIN, 173 LocalIdxs: passThroughLocalIdxs, 174 }, 175 }, 176 }, 177 178 execinfrapb.AggregatorSpec_SUM: { 179 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_SUM}, 180 FinalStage: []FinalStageInfo{ 181 { 182 Fn: execinfrapb.AggregatorSpec_SUM, 183 LocalIdxs: passThroughLocalIdxs, 184 }, 185 }, 186 }, 187 188 execinfrapb.AggregatorSpec_XOR_AGG: { 189 LocalStage: []execinfrapb.AggregatorSpec_Func{execinfrapb.AggregatorSpec_XOR_AGG}, 190 FinalStage: []FinalStageInfo{ 191 { 192 Fn: execinfrapb.AggregatorSpec_XOR_AGG, 193 LocalIdxs: passThroughLocalIdxs, 194 }, 195 }, 196 }, 197 198 // AVG is more tricky than the ones above; we need two intermediate values in 199 // the local and final stages: 200 // - the local stage accumulates the SUM and the COUNT; 201 // - the final stage sums these partial results (SUM and SUM_INT); 202 // - a final rendering then divides the two results. 203 // 204 // At a high level, this is analogous to rewriting AVG(x) as SUM(x)/COUNT(x). 205 execinfrapb.AggregatorSpec_AVG: { 206 LocalStage: []execinfrapb.AggregatorSpec_Func{ 207 execinfrapb.AggregatorSpec_SUM, 208 execinfrapb.AggregatorSpec_COUNT, 209 }, 210 FinalStage: []FinalStageInfo{ 211 { 212 Fn: execinfrapb.AggregatorSpec_SUM, 213 LocalIdxs: []uint32{0}, 214 }, 215 { 216 Fn: execinfrapb.AggregatorSpec_SUM_INT, 217 LocalIdxs: []uint32{1}, 218 }, 219 }, 220 FinalRendering: func(h *tree.IndexedVarHelper, varIdxs []int) (tree.TypedExpr, error) { 221 if len(varIdxs) < 2 { 222 panic("fewer than two final aggregation values passed into final render") 223 } 224 sum := h.IndexedVar(varIdxs[0]) 225 count := h.IndexedVar(varIdxs[1]) 226 227 expr := &tree.BinaryExpr{ 228 Operator: tree.Div, 229 Left: sum, 230 Right: count, 231 } 232 233 // There is no "FLOAT / INT" operator; cast the denominator to float in 234 // this case. Note that there is a "DECIMAL / INT" operator, so we don't 235 // need the same handling for that case. 236 if sum.ResolvedType().Family() == types.FloatFamily { 237 expr.Right = &tree.CastExpr{ 238 Expr: count, 239 Type: types.Float, 240 } 241 } 242 semaCtx := tree.MakeSemaContext() 243 semaCtx.IVarContainer = h.Container() 244 return expr.TypeCheck(context.TODO(), &semaCtx, types.Any) 245 }, 246 }, 247 248 // For VARIANCE/STDDEV the local stage consists of three aggregations, 249 // and the final stage aggregation uses all three values. 250 // respectively: 251 // - the local stage accumulates the SQRDIFF, SUM and the COUNT 252 // - the final stage calculates the FINAL_(VARIANCE|STDDEV) 253 // 254 // At a high level, this is analogous to rewriting VARIANCE(x) as 255 // SQRDIFF(x)/(COUNT(x) - 1) (and STDDEV(x) as sqrt(VARIANCE(x))). 256 execinfrapb.AggregatorSpec_VARIANCE: { 257 LocalStage: []execinfrapb.AggregatorSpec_Func{ 258 execinfrapb.AggregatorSpec_SQRDIFF, 259 execinfrapb.AggregatorSpec_SUM, 260 execinfrapb.AggregatorSpec_COUNT, 261 }, 262 // Instead of have a SUM_SQRDIFFS and SUM_INT (for COUNT) stage 263 // for VARIANCE (and STDDEV) then tailoring a FinalRendering 264 // stage specific to each, it is better to use a specific 265 // FINAL_(VARIANCE|STDDEV) aggregation stage: - For underlying 266 // Decimal results, it is not possible to reduce trailing zeros 267 // since the expression is wrapped in IndexVar. Taking the 268 // BinaryExpr Pow(0.5) for STDDEV would result in trailing 269 // zeros which is not ideal. 270 // TODO(richardwu): Consolidate FinalStage and FinalRendering: 271 // have one or the other 272 FinalStage: []FinalStageInfo{ 273 { 274 Fn: execinfrapb.AggregatorSpec_FINAL_VARIANCE, 275 LocalIdxs: []uint32{0, 1, 2}, 276 }, 277 }, 278 }, 279 280 execinfrapb.AggregatorSpec_STDDEV: { 281 LocalStage: []execinfrapb.AggregatorSpec_Func{ 282 execinfrapb.AggregatorSpec_SQRDIFF, 283 execinfrapb.AggregatorSpec_SUM, 284 execinfrapb.AggregatorSpec_COUNT, 285 }, 286 FinalStage: []FinalStageInfo{ 287 { 288 Fn: execinfrapb.AggregatorSpec_FINAL_STDDEV, 289 LocalIdxs: []uint32{0, 1, 2}, 290 }, 291 }, 292 }, 293 }