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

     1  # =============================================================================
     2  # window.opt contains normalization rules for the Window operator.
     3  # =============================================================================
     4  
     5  # TODO(justin): add a rule to best-effort collapse same ordering+partition
     6  # window functions, like in:
     7  # SELECT
     8  #     rank() OVER (PARTITION BY i), rank() OVER (PARTITION BY i, 1), rank() OVER (PARTITION BY i, 2)
     9  # FROM
    10  #     a
    11  
    12  # EliminateWindow removes a Window operator with no window functions (which can
    13  # occur via column pruning).
    14  [EliminateWindow, Normalize]
    15  (Window $input:* [])
    16  =>
    17  $input
    18  
    19  # ReduceWindowPartitionCols reduces a set of partition columns to a simpler form
    20  # using FDs. Window partition columns are redundant if they are functionally
    21  # determined by other window partition columns.
    22  [ReduceWindowPartitionCols, Normalize]
    23  (Window
    24      $input:*
    25      $fn:*
    26      $private:* &
    27          ^(ColsAreEmpty
    28              $redundantCols:(RedundantCols
    29                  $input
    30                  (WindowPartition $private)
    31              )
    32          )
    33  )
    34  =>
    35  (Window
    36      $input
    37      $fn
    38      (RemoveWindowPartitionCols $private $redundantCols)
    39  )
    40  
    41  # SimplifyWindowOrdering reduces an ordering to a simpler form using FDs.
    42  [SimplifyWindowOrdering, Normalize]
    43  (Window
    44      $input:*
    45      $fn:*
    46      $private:* & (CanSimplifyWindowOrdering $input $private)
    47  )
    48  =>
    49  (Window $input $fn (SimplifyWindowOrdering $input $private))
    50  
    51  # PushSelectIntoWindow pushes down a Select which can be satisfied by only the
    52  # functional closure of the columns being partitioned over. This is valid
    53  # because it's "all-or-nothing" - we only entirely eliminate a partition or
    54  # don't eliminate it at all.
    55  [PushSelectIntoWindow, Normalize]
    56  (Select
    57      (Window $input:* $fn:* $private:*)
    58      $filters:[
    59          ...
    60          $item:* &
    61              (ColsAreDeterminedBy
    62                  (OuterCols $item)
    63                  $partitionCols:(WindowPartition $private)
    64                  $input
    65              )
    66          ...
    67      ]
    68  )
    69  =>
    70  (Select
    71      (Window
    72          (Select
    73              $input
    74              (ExtractDeterminedConditions
    75                  $filters
    76                  $partitionCols
    77                  $input
    78              )
    79          )
    80          $fn
    81          $private
    82      )
    83      (ExtractUndeterminedConditions
    84          $filters
    85          $partitionCols
    86          $input
    87      )
    88  )
    89  
    90  # PushLimitIntoWindow moves a Limit below a Window when able. This is
    91  # all-or-nothing. Even if we could push the limit below *some* of the window
    92  # functions, if there are any we cannot, then we don't. This is because
    93  # computing additional window functions is not that expensive, and the
    94  # expensive part is doing the sorting and partitioning. Once exec supports
    95  # passing orderings through and does not require re-partitioning and re-sorting
    96  # of window functions, pushing past some-but-not-all of the window functions
    97  # might be profitable.
    98  # 
    99  # SELECT rank() OVER (ORDER BY c) FROM abc ORDER BY c LIMIT 10
   100  # => 
   101  # SELECT
   102  #     rank() OVER (ORDER BY c)
   103  # FROM
   104  #     (SELECT c FROM abc ORDER BY c LIMIT 10)
   105  # 
   106  # SELECT rank() OVER (PARTITION BY b ORDER BY c) FROM abc LIMIT 10
   107  # => 
   108  # SELECT
   109  #     rank() OVER (PARTITION BY b ORDER BY c)
   110  # FROM
   111  #     (SELECT b, c FROM abc ORDER BY b, c LIMIT 10)
   112  # 
   113  # First, we construct a "segmented ordering" consisting of the Window's
   114  # partition columns followed by its ordering columns (the relative positions of
   115  # the partition columns are arbitrary). This ordering is useful because it
   116  # performs the partitioning and then the ordering within each partition.  If
   117  # this ordering does not imply the Limit's ordering, we do not proceed.
   118  # 
   119  # Since we now know that the segmented ordering is stronger than the Limit's
   120  # ordering, it's safe to replace the limit's ordering with it.
   121  # 
   122  # The Limit having the segmented ordering means that there are three kinds of
   123  # partitions:
   124  #   1. those that are completely contained within the limited set of rows,
   125  #   2. those that are completely excluded from the set of rows, and
   126  #   3. *at most one* partition which is "cut off" partway through.
   127  # Including the window function's ordering in the Limit's ordering does not
   128  # matter for (1)- and (2)-style partitions (since the window function itself
   129  # will re-sort them), but for the (3)-style partition, we need to ensure that
   130  # the limit operator allows through a prefix of it, rather than an arbitrary
   131  # subset.
   132  # 
   133  # Finally, we require that every window function+frame pair being computed has
   134  # the "prefix-safe" property. A window function is prefix safe if it can be
   135  # correctly computed over only a prefix of a partition. For example, rank() has
   136  # this property because rows that come later in the ordering don't affect the
   137  # rank of the rows before, but avg()+UNBOUNDED {PRECEDING,FOLLOWING} doesn't,
   138  # because we must see the entire partition to compute the average over it.
   139  #
   140  # TODO(justin): Add a rule that translates a limit with an ordering on rank()
   141  # or dense_rank() into one using the ordering of the window function. This will
   142  # allow us to push down limits in cases like:
   143  #
   144  # SELECT rank() OVER (ORDER BY f) rnk FROM a ORDER BY rnk LIMIT 10
   145  # =>
   146  # SELECT rank() OVER (ORDER BY f) rnk FROM a ORDER BY f LIMIT 10
   147  # =>
   148  # SELECT rank() OVER (ORDER BY f) rnk FROM (SELECT * FROM a ORDER BY f LIMIT 10)
   149  [PushLimitIntoWindow, Normalize]
   150  (Limit
   151      (Window $input:* $fns:* & (AllArePrefixSafe $fns) $private:*)
   152      $limit:*
   153      $ordering:* &
   154          (OrderingSucceeded
   155              $newOrdering:(MakeSegmentedOrdering
   156                  $input
   157                  (WindowPartition $private)
   158                  (WindowOrdering $private)
   159                  $ordering
   160              )
   161          )
   162  )
   163  =>
   164  (Window
   165      (Limit $input $limit (DerefOrderingChoice $newOrdering))
   166      $fns
   167      $private
   168  )