github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/sql/opt/norm/rules/limit.opt (about) 1 # ============================================================================= 2 # limit.opt contains normalization rules for the Limit and Offset operators. 3 # ============================================================================= 4 5 # EliminateLimit discards a Limit operator if its constant limit is greater than 6 # or equal to the maximum number of rows that can be returned by the input. In 7 # this case, the Limit is just a no-op, because the rows are already limited. 8 [EliminateLimit, Normalize] 9 (Limit 10 $input:* 11 (Const $limit:*) & (LimitGeMaxRows $limit $input) 12 ) 13 => 14 $input 15 16 # EliminateOffset discards an Offset operator if its offset value is zero. 17 [EliminateOffset, Normalize] 18 (Offset $input:* (Const 0)) 19 => 20 $input 21 22 # PushLimitIntoProject pushes the Limit operator into its Project input. It is 23 # desirable to push the Limit operator as low in the query as possible, in order 24 # to minimize the number of rows that other operators need to process. 25 [PushLimitIntoProject, Normalize] 26 (Limit 27 (Project $input:* $projections:* $passthrough:*) 28 $limit:* 29 $ordering:* & 30 (OrderingCanProjectCols 31 $ordering 32 $cols:(OutputCols $input) 33 ) 34 ) 35 => 36 (Project 37 (Limit $input $limit (PruneOrdering $ordering $cols)) 38 $projections 39 $passthrough 40 ) 41 42 # PushOffsetIntoProject pushes the Offset operator into its Project input. It is 43 # desirable to push the Offset operator as low in the query as possible, in 44 # order to minimize the number of rows that other operators need to process. 45 [PushOffsetIntoProject, Normalize] 46 (Offset 47 (Project $input:* $projections:* $passthrough:*) 48 $offset:* 49 $ordering:* & 50 (OrderingCanProjectCols 51 $ordering 52 $cols:(OutputCols $input) 53 ) 54 ) 55 => 56 (Project 57 (Offset $input $offset (PruneOrdering $ordering $cols)) 58 $projections 59 $passthrough 60 ) 61 62 # PushLimitIntoOffset pushes the Limit operator into the offset. This should 63 # not have a negative impact but it would allow the use of the GenerateLimitedScans 64 # rule. 65 [PushLimitIntoOffset, Normalize] 66 (Limit 67 (Offset 68 $input:* 69 $offsetExpr:(Const $offset:* & (IsPositiveInt $offset)) 70 $offsetOrdering:* 71 ) 72 (Const $limit:* & (IsPositiveInt $limit)) 73 $limitOrdering:* & 74 (IsSameOrdering $offsetOrdering $limitOrdering) & 75 (CanAddConstInts $limit $offset) 76 ) 77 => 78 (Offset 79 (Limit $input (AddConstInts $offset $limit) $limitOrdering) 80 $offsetExpr 81 $offsetOrdering 82 ) 83 84 # PushLimitIntoOrdinality pushes the Limit operator into the Ordinality 85 # operator when the ordering associated with both operators allows it. 86 # 87 # Pushing the limit as far as possible down the tree shouldn't have negative 88 # effects, but will reduce the number of rows processed by operators higher up, 89 # and if the limit is pushed all the way down to a scan, the scan can be limited 90 # directly. 91 # 92 # In order to prevent this rule from affecting: 93 # 1. the set of rows kept by the limit, 94 # 2. the ordinals assigned to those rows by the ordinality, and 95 # 3. the final ordering of the rows, 96 # the new limit's ordering should be "extended" to imply the ordinality's 97 # ordering, so it is set to be an intersection of the original limit ordering 98 # and the ordinality's ordering (see OrderingChoice.Intersection). 99 [PushLimitIntoOrdinality, Normalize] 100 (Limit 101 (Ordinality $input:* $private:*) 102 $limit:* 103 $limitOrdering:* & 104 (OrderingCanProjectCols 105 $limitOrdering 106 (OutputCols $input) 107 ) & 108 (OrderingIntersects 109 (OrdinalityOrdering $private) 110 $limitOrdering 111 ) 112 ) 113 => 114 (Ordinality 115 (Limit 116 $input 117 $limit 118 (OrderingIntersection 119 (OrdinalityOrdering $private) 120 $limitOrdering 121 ) 122 ) 123 $private 124 ) 125 126 # PushLimitIntoLeftJoin pushes a Limit into the left input of a left join. Since 127 # the left join creates an output row for each left input row, we only need that 128 # many rows from that input. We can only do this if the limit ordering refers 129 # only to the left input columns. We also check that the cardinality of the left 130 # input is more than the limit, to prevent repeated applications of the rule. 131 [PushLimitIntoLeftJoin, Normalize] 132 (Limit 133 $input:(LeftJoin $left:* $right:* $on:* $private:*) 134 $limitExpr:(Const $limit:*) & 135 (IsPositiveInt $limit) & 136 ^(LimitGeMaxRows $limit $left) 137 $ordering:* & 138 (OrderingCanProjectCols 139 $ordering 140 $cols:(OutputCols $left) 141 ) 142 ) 143 => 144 (Limit 145 (LeftJoin 146 (Limit $left $limitExpr (PruneOrdering $ordering $cols)) 147 $right 148 $on 149 $private 150 ) 151 $limitExpr 152 $ordering 153 )