github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/opt/xform/rules/groupby.opt (about)

     1  # =============================================================================
     2  # groupby.opt contains exploration rules for the groupby operators
     3  # =============================================================================
     4  
     5  # ReplaceScalarMinMaxWithLimit replaces a min or max group by aggregation with a
     6  # limit 1 on an ordered set. This rule may result in a lower cost plan if the
     7  # aggregated column (e.g. the "x" in min(x)) is indexed.
     8  [ReplaceScalarMinMaxWithLimit, Explore]
     9  (ScalarGroupBy
    10      $input:*
    11      [
    12          (AggregationsItem
    13              $agg:(Min | Max $variable:(Variable $col:*))
    14              $aggPrivate:*
    15          )
    16      ]
    17      $groupingPrivate:* & (IsCanonicalGroupBy $groupingPrivate)
    18  )
    19  =>
    20  (ScalarGroupBy
    21      (Limit
    22          (Select
    23              $input
    24              [ (FiltersItem (IsNot $variable (Null (AnyType)))) ]
    25          )
    26          (IntConst 1)
    27          (MakeOrderingChoiceFromColumn (OpName $agg) $col)
    28      )
    29      [ (AggregationsItem (ConstAgg $variable) $aggPrivate) ]
    30      $groupingPrivate
    31  )
    32  
    33  # ReplaceMinWithLimit replaces a Min group-by expression with a limit 1, in the
    34  # case where the group-by returns zero or one groups (i.e. the grouping columns
    35  # are constant). In this situation, the group-by can be entirely replaced by a
    36  # project, similar to this:
    37  #
    38  #   SELECT w, min(k) FROM kw WHERE w = 5 GROUP BY w
    39  #   =>
    40  #   SELECT w, k FROM kw@wk WHERE w = 5 LIMIT 1
    41  #
    42  # This rule may result in a lower cost plan if the column min(k) is applied to
    43  # is indexed.
    44  #
    45  # NOTE: This rule is not fully symmetric with ReplaceMaxWithLimit. This is
    46  # because NULL values sort first in CRDB. This can interfere with the
    47  # calculation of the Min function, because NULL values need to be ignored unless
    48  # the group contains only NULL values (in which case the function returns NULL).
    49  # Therefore, this rule only works when the MIN column is NOT NULL, as only in
    50  # that case is one input row always sufficient to calculate MIN.
    51  [ReplaceMinWithLimit, Explore]
    52  (GroupBy
    53      $input:*
    54      $aggregations:[
    55              ...
    56              $item:(AggregationsItem
    57                  (Min $variable:(Variable $col:*))
    58              )
    59              ...
    60          ] &
    61          (IsColNotNull $col $input) &
    62          (OtherAggsAreConst $aggregations $item)
    63      $groupingPrivate:* &
    64          (IsCanonicalGroupBy $groupingPrivate) &
    65          (ColsAreConst (GroupingCols $groupingPrivate) $input)
    66  )
    67  =>
    68  (MakeProjectFromPassthroughAggs
    69      (Limit
    70          $input
    71          (IntConst 1)
    72          (MakeOrderingChoiceFromColumn Min $col)
    73      )
    74      $aggregations
    75  )
    76  
    77  # ReplaceMaxWithLimit is analogous to the ReplaceMinWithLimit rule, except that
    78  # it works when the Max function's operand can be NULL. See the header comment
    79  # for ReplaceMinWithLimit for more details.
    80  [ReplaceMaxWithLimit, Explore]
    81  (GroupBy
    82      $input:*
    83      $aggregations:[
    84              ...
    85              $item:(AggregationsItem
    86                  (Max $variable:(Variable $col:*))
    87              )
    88              ...
    89          ] &
    90          (OtherAggsAreConst $aggregations $item)
    91      $groupingPrivate:* &
    92          (IsCanonicalGroupBy $groupingPrivate) &
    93          (ColsAreConst (GroupingCols $groupingPrivate) $input)
    94  )
    95  =>
    96  (MakeProjectFromPassthroughAggs
    97      (Limit
    98          $input
    99          (IntConst 1)
   100          (MakeOrderingChoiceFromColumn Max $col)
   101      )
   102      $aggregations
   103  )
   104  
   105  # GenerateStreamingGroupBy creates variants of a GroupBy, DistinctOn, or
   106  # UpsertDistinctOn which require more specific orderings on the grouping
   107  # columns, using the interesting orderings property. When we have orderings on
   108  # grouping columns, we can execute aggregations in a streaming fashion which is
   109  # more efficient.
   110  [GenerateStreamingGroupBy, Explore]
   111  (GroupBy | DistinctOn | EnsureDistinctOn | UpsertDistinctOn
   112          | EnsureUpsertDistinctOn
   113      $input:*
   114      $aggs:*
   115      $private:* & (IsCanonicalGroupBy $private)
   116  )
   117  =>
   118  (GenerateStreamingGroupBy (OpName) $input $aggs $private)