github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/opt/norm/rules/groupby.opt (about) 1 # ============================================================================= 2 # groupby.opt contains normalization rules for the GroupBy operator. 3 # ============================================================================= 4 5 # ConvertGroupByToDistinct converts a GroupBy operator that has no aggregations 6 # to an equivalent DistinctOn operator. 7 [ConvertGroupByToDistinct, Normalize] 8 (GroupBy $input:* $aggregations:[] $groupingPrivate:*) 9 => 10 (DistinctOn $input $aggregations $groupingPrivate) 11 12 # EliminateGroupByProject discards a nested Project operator that is only 13 # removing columns from its input (and not synthesizing new ones). That's 14 # something the GroupBy operators can do on their own. 15 # 16 # Note: EliminateGroupByProject should be located above 17 # EliminateJoinUnderGroupByLeft so that it can remove any interfering Projects. 18 [EliminateGroupByProject, Normalize] 19 (GroupBy | ScalarGroupBy | DistinctOn | EnsureDistinctOn 20 | UpsertDistinctOn | EnsureUpsertDistinctOn 21 $input:(Project $innerInput:*) & 22 (ColsAreSubset 23 (OutputCols $input) 24 (OutputCols $innerInput) 25 ) 26 $aggregations:* 27 $groupingPrivate:* 28 ) 29 => 30 ((OpName) $innerInput $aggregations $groupingPrivate) 31 32 # EliminateJoinUnderGroupByLeft removes a Join operator and its right input if 33 # it can be proven that the removal does not affect the output of the parent 34 # grouping operator. This is the case if: 35 # 36 # 1. Only columns from the left input are being used by the grouping operator. 37 # 38 # 2. It can be proven that removal of the Join does not affect the result of the 39 # grouping operator's aggregate functions. 40 # 41 # 3. The OrderingChoice of the grouping operator can be expressed with only 42 # columns from the left input. Or in other words, at least one column in 43 # every ordering group is one of the left output columns. 44 # 45 # Condition #2 is only true when the following are all true: 46 # 47 # 1. All left rows are included in the output of the join. See the comment above 48 # filtersMatchAllLeftRows in multiplicity_builder.go for more information on 49 # when this is the case. 50 # 2. Either the join does not duplicate any left rows, or the join duplicates 51 # left rows but the grouping operator's aggregate functions ignore duplicate 52 # values. See the comment above filtersMatchLeftRowsAtMostOnce in 53 # multiplicity_builder.go for more information on when rows are duplicated. 54 # 3. The join does not null-extend the left columns. 55 # 56 # EliminateJoinUnderGroupByLeft should stay at the top of the file so that it 57 # has a chance to fire before rules like EliminateDistinctOn that might prevent 58 # matching. 59 [EliminateJoinUnderGroupByLeft, Normalize] 60 (GroupBy | ScalarGroupBy | DistinctOn 61 $input:(InnerJoin | LeftJoin $left:*) 62 $aggs:* 63 $private:(GroupingPrivate $groupingCols:* $ordering:*) & 64 (OrderingCanProjectCols 65 $ordering 66 $leftCols:(OutputCols $left) 67 ) & 68 (ColsAreSubset 69 (UnionCols 70 $groupingCols 71 (AggregationOuterCols $aggs) 72 ) 73 $leftCols 74 ) & 75 (CanEliminateJoinUnderGroupByLeft $input $aggs) 76 ) 77 => 78 ((OpName) 79 $left 80 $aggs 81 (MakeGrouping 82 $groupingCols 83 (PruneOrdering $ordering $leftCols) 84 ) 85 ) 86 87 # EliminateJoinUnderGroupByRight is symmetric with 88 # EliminateJoinUnderGroupByLeft, except that it only matches on InnerJoins. 89 [EliminateJoinUnderGroupByRight, Normalize] 90 (GroupBy | ScalarGroupBy | DistinctOn 91 $input:(InnerJoin * $right:*) 92 $aggs:* 93 $private:(GroupingPrivate $groupingCols:* $ordering:*) & 94 (OrderingCanProjectCols 95 $ordering 96 $rightCols:(OutputCols $right) 97 ) & 98 (ColsAreSubset 99 (UnionCols 100 $groupingCols 101 (AggregationOuterCols $aggs) 102 ) 103 $rightCols 104 ) & 105 (CanEliminateJoinUnderGroupByRight $input $aggs) 106 ) 107 => 108 ((OpName) 109 $right 110 $aggs 111 (MakeGrouping 112 $groupingCols 113 (PruneOrdering $ordering $rightCols) 114 ) 115 ) 116 117 # EliminateDistinct discards a DistinctOn operator that is eliminating duplicate 118 # rows by using grouping columns that are statically known to form a strong key. 119 # By definition, a strong key does not allow duplicate values, so the GroupBy is 120 # redundant and can be eliminated. 121 # 122 # Since a DistinctOn operator can serve as a projection operator, we need to 123 # replace it with a Project so that the correct columns are projected. The 124 # project itself may be eliminated later by other rules. 125 [EliminateDistinct, Normalize] 126 (DistinctOn | EnsureDistinctOn 127 $input:* 128 $aggs:* 129 $groupingPrivate:* & 130 (ColsAreStrictKey (GroupingCols $groupingPrivate) $input) 131 ) 132 => 133 (Project $input [] (GroupingOutputCols $groupingPrivate $aggs)) 134 135 # ReduceGroupingCols eliminates redundant grouping columns from the GroupBy 136 # operator and replaces them by ConstAgg aggregate functions. A grouping 137 # column is redundant if it is functionally determined by the other grouping 138 # columns. If that's true, then its value must be constant within a group. 139 # Therefore, it has no effect on the grouping and can instead be represented as 140 # an ConstAgg aggregate, since all rows in the group have the same value for 141 # that column. 142 # 143 # Note: Doesn't match EnsureDistinctOn because test cases were too difficult to 144 # find. If a test case for EnsureDistinctOn is found, it should be added to the 145 # match pattern. 146 [ReduceGroupingCols, Normalize] 147 (GroupBy | DistinctOn 148 $input:* 149 $aggregations:* 150 $groupingPrivate:* & 151 ^(ColsAreEmpty 152 $redundantCols:(RedundantCols 153 $input 154 (GroupingCols $groupingPrivate) 155 ) 156 ) 157 ) 158 => 159 ((OpName) 160 $input 161 (AppendAggCols $aggregations ConstAgg $redundantCols) 162 (RemoveGroupingCols $groupingPrivate $redundantCols) 163 ) 164 165 # ReduceNotNullGroupingCols is similar to ReduceGroupingCols, but with the 166 # additional restriction that nullable columns cannot be removed from the set of 167 # grouping columns. This is because the UpsertDistinctOn operator treats NULL 168 # values as not equal to one another, and therefore will not group them 169 # together. Since removing a grouping column is equivalent to grouping all 170 # values of that column together, this would be incorrect in the case where all 171 # input rows are NULL for that column: 172 # 173 # SELECT c FROM t WHERE c IS NULL 174 # 175 [ReduceNotNullGroupingCols, Normalize] 176 (UpsertDistinctOn | EnsureUpsertDistinctOn 177 $input:* 178 $aggregations:* 179 $groupingPrivate:* & 180 ^(ColsAreEmpty 181 $redundantCols:(IntersectionCols 182 (RedundantCols 183 $input 184 (GroupingCols $groupingPrivate) 185 ) 186 (NotNullCols $input) 187 ) 188 ) 189 ) 190 => 191 ((OpName) 192 $input 193 (AppendAggCols $aggregations ConstAgg $redundantCols) 194 (RemoveGroupingCols $groupingPrivate $redundantCols) 195 ) 196 197 # EliminateAggDistinctForKeys eliminates unnecessary AggDistinct modifiers when 198 # it is known that the aggregation argument is unique within each group. 199 [EliminateAggDistinctForKeys, Normalize] 200 (GroupBy | ScalarGroupBy 201 $input:* & (HasStrictKey $input) 202 $aggregations:[ 203 ... 204 $item:(AggregationsItem (AggDistinct $agg:*)) 205 ... 206 ] 207 $groupingPrivate:* & 208 (CanRemoveAggDistinctForKeys 209 $input 210 $groupingPrivate 211 $agg 212 ) 213 ) 214 => 215 ((OpName) 216 $input 217 (ReplaceAggregationsItem $aggregations $item $agg) 218 $groupingPrivate 219 ) 220 221 # EliminateAggFilteredDistinctForKeys is similar to EliminateAggDistinctForKeys, 222 # except that it works when an AggFilter operator is also present. 223 [EliminateAggFilteredDistinctForKeys, Normalize] 224 (GroupBy | ScalarGroupBy 225 $input:* & (HasStrictKey $input) 226 $aggregations:[ 227 ... 228 $item:(AggregationsItem 229 (AggFilter (AggDistinct $agg:*) $filter:*) 230 ) 231 ... 232 ] 233 $groupingPrivate:* & 234 (CanRemoveAggDistinctForKeys 235 $input 236 $groupingPrivate 237 $agg 238 ) 239 ) 240 => 241 ((OpName) 242 $input 243 (ReplaceAggregationsItem 244 $aggregations 245 $item 246 (AggFilter $agg $filter) 247 ) 248 $groupingPrivate 249 ) 250 251 # EliminateDistinctNoColumns eliminates a distinct operator with no grouping 252 # columns, replacing it with a projection and a LIMIT 1. For example: 253 # SELECT DISTINCT ON (a) a, b FROM ab WHERE a=1 254 # is equivalent to: 255 # SELECT a, b FROM ab WHERE a=1 LIMIT 1 256 # 257 # Note that this rule does not apply to EnsureDistinctOn or 258 # EnsureUpsertDistinctOn, since they will raise an error if there are duplicate 259 # rows. 260 [EliminateDistinctNoColumns, Normalize] 261 (DistinctOn | UpsertDistinctOn 262 $input:* 263 $aggregations:* 264 $groupingPrivate:* & (HasNoGroupingCols $groupingPrivate) 265 ) 266 => 267 (ConstructProjectionFromDistinctOn 268 (Limit 269 $input 270 (IntConst 1) 271 (GroupingInputOrdering $groupingPrivate) 272 ) 273 (MakeEmptyColSet) 274 $aggregations 275 ) 276 277 # EliminateEnsureDistinctNoColumns is similar to EliminateDistinctNoColumns, 278 # except that Max1Row will raise an error if there are no grouping columns and 279 # the input has more than one row. No grouping columns means there is at most 280 # one group. And the Max1Row operator is needed to raise an error if that group 281 # has more than one row, which is a requirement of the EnsureDistinct and 282 # EnsureUpsertDistinct operators. 283 [EliminateEnsureDistinctNoColumns, Normalize] 284 (EnsureDistinctOn | EnsureUpsertDistinctOn 285 $input:* 286 $aggregations:* 287 $groupingPrivate:* & (HasNoGroupingCols $groupingPrivate) 288 ) 289 => 290 (ConstructProjectionFromDistinctOn 291 (Max1Row $input (ErrorOnDup $groupingPrivate)) 292 (MakeEmptyColSet) 293 $aggregations 294 ) 295 296 # EliminateDistinctOnValues eliminates a distinct operator that has a constant 297 # input Values operator that is already distinct with respect to the grouping 298 # columns. The Values operator may be the immediate input, or it may be wrapped 299 # by Select, Project, LeftJoin, and/or other operators. These are common 300 # patterns that are generated by the optbuilder's upsert construction code, 301 # which must ensure the same row cannot be updated twice. See the comment for 302 # UpsertDistinctOn for more detail on NullsAreDistinct behavior. 303 # 304 # Note: Doesn't match EnsureDistinctOn because test cases were too difficult to 305 # find. If a test case for EnsureDistinctOn is found, it should be added to the 306 # match pattern. 307 [EliminateDistinctOnValues, Normalize] 308 (DistinctOn | UpsertDistinctOn | EnsureUpsertDistinctOn 309 $input:* 310 $aggregations:* 311 $groupingPrivate:* & 312 (AreValuesDistinct 313 $input 314 (GroupingCols $groupingPrivate) 315 (NullsAreDistinct $groupingPrivate) 316 ) 317 ) 318 => 319 (ConstructProjectionFromDistinctOn 320 $input 321 (GroupingCols $groupingPrivate) 322 $aggregations 323 ) 324 325 # PushAggDistinctIntoScalarGroupBy pushes an aggregate function DISTINCT 326 # modifier into the input of the ScalarGroupBy operator. This allows the 327 # optimizer to take advantage of an index on the column(s) subject to the 328 # DISTINCT operation. PushAggDistinctIntoScalarGroupBy can match any single 329 # aggregate function, including those that have multiple input arguments. 330 [PushAggDistinctIntoScalarGroupBy, Normalize] 331 (ScalarGroupBy 332 $input:* 333 $aggregations:[ 334 $item:(AggregationsItem (AggDistinct $agg:*) $aggColID:*) 335 ] 336 $groupingPrivate:* 337 ) 338 => 339 (ScalarGroupBy 340 (DistinctOn 341 $input 342 [] 343 (MakeGrouping 344 (ExtractAggInputColumns $agg) 345 (EmptyOrdering) 346 ) 347 ) 348 [ (AggregationsItem $agg $aggColID) ] 349 $groupingPrivate 350 ) 351 352 # PushAggFilterIntoScalarGroupBy pushes an aggregate function FILTER 353 # modifier into the input of the ScalarGroupBy operator. This allows the 354 # optimizer to take advantage of an index on the column(s) subject to the 355 # FILTER operation. PushAggFilterIntoScalarGroupBy can match any single 356 # aggregate function, including those that have multiple input arguments. 357 [PushAggFilterIntoScalarGroupBy, Normalize] 358 (ScalarGroupBy 359 $input:* 360 $aggregations:[ 361 $item:(AggregationsItem 362 (AggFilter $agg:* $condition:*) 363 $aggColID:* 364 ) 365 ] 366 $groupingPrivate:* 367 ) 368 => 369 (ScalarGroupBy 370 (Select $input [ (FiltersItem $condition) ]) 371 [ (AggregationsItem $agg $aggColID) ] 372 $groupingPrivate 373 ) 374 375 # ConvertCountToCountRows replaces a Count operator performed on a non-null 376 # expression with a CountRows operator. CountRows is significantly faster to 377 # execute than Count. 378 [ConvertCountToCountRows, Normalize] 379 (GroupBy | ScalarGroupBy 380 $input:* 381 $aggregations:[ 382 ... 383 $item:(AggregationsItem (Count $arg:*)) & 384 (ExprIsNeverNull $arg (NotNullCols $input)) 385 ... 386 ] 387 $groupingPrivate:* 388 ) 389 => 390 ((OpName) 391 $input 392 (ReplaceAggregationsItem $aggregations $item (CountRows)) 393 $groupingPrivate 394 ) 395 396 # FoldGroupingOperators folds two grouping operators into one equivalent 397 # operator. As an example, the following pairs of queries are equivalent: 398 # 399 # SELECT sum(t) FROM (SELECT sum(b) FROM ab GROUP BY a) AS g(t); 400 # SELECT sum(b) FROM ab; 401 # 402 # SELECT max(t) FROM (SELECT max(b) FROM ab GROUP BY a) AS g(t); 403 # SELECT max(b) FROM ab; 404 # 405 # SELECT sum_int(t) FROM (SELECT count(b) FROM ab GROUP BY a) AS g(t); 406 # SELECT count(b) FROM ab; 407 # 408 # This transformation is possible when the following conditions are met: 409 # 410 # 1. All of the outer aggregates are aggregating on the output columns of the 411 # inner aggregates. 412 # 2. All of the inner-outer aggregate pairs can be replaced with an equivalent 413 # single aggregate. (See the AggregatesCanMerge comment in operator.go). 414 # 3. The grouping columns of the inner operator functionally determine the 415 # grouping columns of the outer operator according to the functional 416 # dependencies of the input of the inner operator. 417 # 4. Both grouping operators are unordered. 418 # 419 # Why is it sufficient for the inner grouping columns to functionally determine 420 # the outer grouping columns? 421 # * Duplicate values in the determinant ("from" side) imply duplicate values in 422 # the dependent ("to" side). 423 # * Grouping on the determinant will not remove unique values from the 424 # determinant. Therefore, the grouping will not remove unique values from the 425 # dependent, by the properties of functional dependencies. 426 # * Grouping on the dependent will simply reduce the dependent to its unique 427 # values. 428 # * Therefore, grouping on the dependent produces the same final groups as 429 # grouping on the dependent after grouping on the determinant. 430 # * Conditions #2 and #4 guarantee that the aggregates produce the same result 431 # regardless of how the grouping is accomplished, as long as the same groups 432 # result in the end. 433 # 434 # Take the following table as an example: 435 # 436 # r a b 437 # ----- 438 # 1 4 3 439 # 2 4 3 440 # 3 2 3 441 # 4 2 3 442 # 5 6 5 443 # 6 6 5 444 # 445 # Its functional dependencies: key(r), r-->(a, b), a-->(b) 446 # 447 # Here are some examples of possible groupings taking the sum over the "r" 448 # column: 449 # 450 # Grouping by a: SUM(1, 2), SUM(3, 4), SUM(5, 6) 451 # Grouping by b: SUM(1, 2, 3, 4), SUM(5, 6) 452 # Grouping by a then b: SUM(SUM(1, 2), SUM(3, 4)), SUM(SUM(5, 6)) 453 # 454 # Rows can always be grouped together by subsequent groupings, but they can 455 # never be "ungrouped". Grouping on a does not group any rows together that 456 # would not also be grouped by b. 457 # 458 # This situation is rare in direct SQL queries, but can arise when composing 459 # views and queries. 460 [FoldGroupingOperators, Normalize] 461 (GroupBy | ScalarGroupBy 462 (GroupBy 463 $innerInput:* 464 $innerAggs:* 465 $innerGrouping:* & (IsUnorderedGrouping $innerGrouping) 466 ) 467 $outerAggs:* 468 $outerGrouping:* & 469 (IsUnorderedGrouping $outerGrouping) & 470 (ColsAreDeterminedBy 471 $outerGroupingCols:(GroupingCols $outerGrouping) 472 (GroupingCols $innerGrouping) 473 $innerInput 474 ) & 475 (CanMergeAggs $innerAggs $outerAggs) 476 ) 477 => 478 ((OpName) 479 $innerInput 480 (MergeAggs $innerAggs $outerAggs) 481 (MakeGrouping $outerGroupingCols (EmptyOrdering)) 482 )