github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/docs/RFCS/20170602_distsql_limit.md (about) 1 - Feature Name: Limit handling in DistSQL 2 - Status: in-progress 3 - Start Date: 2017-06-02 4 - Authors: Radu Berinde 5 - RFC PR: [#16287](https://github.com/cockroachdb/cockroach/pull/16287) 6 - Cockroach Issue: (one or more # from the issue tracker) 7 8 # Summary 9 10 This RFC proposes a set of mechanisms for efficiently dealing with `LIMIT` in 11 DistSQL. 12 13 # Motivation 14 15 First, we describe the current state of things. 16 17 ### Local SQL ### 18 19 There are two types of limits that can apply to a planNode: 20 - a *hard limit* is one where we know we only need **X** rows from a node. 21 Example: `SELECT * FROM t LIMIT 10` 22 - a *soft limit* is one where we have only an optimistic estimation for how 23 many rows we need. Notable cases which lead to soft limits: 24 25 * limits with filters. `SELECT * FROM t WHERE color = 'red' LIMIT 10`. Here 26 the limit of 10 at the scanNode is soft because we may need more rows if 27 some don't pass the filter. 28 29 * partial orders: `SELECT * FROM t ORDER BY pk1,val1 LIMIT 10`: we need to 30 sort but we already have a partial order (let's assume the table is ordered 31 by `pk1,pk2`). If our sort can take advantage of the partial order (and 32 only sort groups of rows that are equal on `pk1`), we only need to read 10 33 rows plus any rows that have the same `pk1` value as the 10th row. This 34 would ideally lead to a soft limit of 10 or slightly more on the 35 `scanNode`. 36 37 *Note: The DistSQL sorter is aware of partial orders but the local SQL 38 sortNode is not, so this is not actually implemented yet.* 39 40 How are the limits actually handled at the scanNode level? The row fetcher 41 receives a "hint" about how many rows we need. If we have a hard limit of **X**, 42 this is simply **X**; if we have a soft limit of **Y** this is currently **2Y**. 43 In the future the multiplier here needs to be determined based on statistics 44 (e.g. an estimation of the selectivity of a filter). The row fetcher calculates 45 the maximum number of keys per row, based on the family configuration (or simply 46 1 key per row for secondary indexes) and sizes the first KV batch accordingly. 47 The scanNode returns all rows before scanning another batch. In the case of hard 48 limits, we never need to scan another batch. In the case of a soft limit, we 49 could; if that happens, the second batch is significantly larger. The third 50 and subsequent batches have the maximum batch size (like unlimited scans). The 51 growth of the batch sizes is a trade-off between performance in "good" scenarios 52 (when we are close to the soft limit) and in "bad" scenarios (if the soft limit 53 estimation turns out to be way off). 54 55 ### Distributed SQL ### 56 57 DistSQL supports limits semantically: it can run any query with limits and it 58 will produce the correct results. However, there are hurdles in implementing the 59 kind of optimizations explained above, which can lead to dramatic performance 60 degradation in many cases: 61 - DistSQL processors (a processor is roughly one of potentially multiple 62 instances corresponding to a planNode) output rows to streams, in parallel 63 with the consumers of those streams. We have flow control on streams - 64 provided by buffered channels on local streams, and by gRPC on remote 65 streams; the gRPC window sizes are quite large, necessary for high 66 throughput/high latency links. These buffering effects can cause a processor 67 to emit more rows than needed before the consumer even has a chance to 68 receive any of them. There is no sequencing step where a processor can stop 69 and wait and see if the consumer needs more rows. The TableReaders use the 70 same row fetching mechanism explained above, which means we will almost 71 always read a second batch even if we don't need to. 72 - there are multiple instances of TableReaders for a given table, each working 73 on the ranges held by that node. Without a special way to plan limits, we 74 would read **X** rows from every node. 75 - the planning process itself has to go through all the ranges in a table, 76 which may be comparatively expensive if we only need to read a few rows 77 (especially as we scale up in cluster and table size). 78 79 As a consequence, by default we don't use DistSQL for statements which involve 80 limits that get propagated all the way to the scanNode. Note that there are 81 useful classes of statements that have limits at a higher level but where we 82 can't put an (even optimistic) limit on scanNodes; for example 83 `SELECT * FROM t ORDER BY v LIMIT 10`, where we have to read the entire table in 84 all cases. These classes of queries are run via DistSQL. 85 86 The goal for this RFC is to find ways to work around the hurdles above to 87 achieve efficiency comparable with local SQL in all limit cases. 88 Specifically: 89 - read no more than **X** rows in hard limit cases, and only involve the 90 respective hosts in the computation; 91 - don't execute more batches than needed in soft limit cases. 92 93 In addition, we want our solution to work well for the partial ordering case 94 described above. 95 96 The main focus here is not performance in terms of the latency of a single 97 query, but avoiding wasted work (which generally translates to better 98 overall throughput). 99 100 # Detailed design 101 102 The proposal has two main aspects: new ways to plan limit queries, and a 103 mechanism for limiting batches for soft limits. 104 105 ## Planning ## 106 107 We have two proposals here: one is simple and should be implemented in the 108 short-term; the other is more complex and should be considered in the long 109 (medium?) term. 110 111 ### Short-term proposal ### 112 113 The simple proposal is to not distribute TableReaders for limited scans. 114 Instead, plan a single TableReader on the host that holds the first range to 115 scan. In the best case, we will only need to scan that first range. In the worst 116 case, we may end up reading the entire table, and many of the reads will be 117 remote reads. 118 119 We believe that this solution is good for many common cases. It is at least as 120 good as local SQL for queries that only read rows from a single range. It also 121 has the advantage that we only need to resolve one range during planning. The 122 main problem is that in bad cases the table data will be "routed" through that 123 first node; this can be very bad if that first node happens to be in a far away 124 zone. This will need to be addressed as we become more optimized for widely 125 distributed scenarios. 126 127 There is also a possibility that the node holding the first range will become a 128 hotspot if we have many similar queries running concurrently. We may need the 129 planning process to be sensitive to load and minimize the amount of computation 130 happening on that node if it is overloaded. 131 132 Note that there will be a cut-off for what we consider a "limited scan" - a very 133 large `LIMIT` will be treated like an unlimited scan. 134 135 #### When order is not required #### 136 137 In cases like `SELECT * FROM t LIMIT 10`, we can return any 10 rows; so we are 138 free to read ranges in any order. An optimization to the proposed solution is to 139 place the single TableReader on one of the nodes that have many range leases for 140 the table, and read from those ranges first. In widely distributed clusters, we 141 can always choose among the nodes that are close to the gateway. We may need the 142 node choice to be sensitive to load; e.g. choose the least loaded node that has 143 some range leases; or choose randomly according to a weighted distribution where 144 each node's weight is proportional to the number of range leases and inversely 145 proportional with load. 146 147 ### Long-term proposal ### 148 149 The second proposal is to build a new *planning synchronizer*. A synchronizer is 150 an existing concept in DistSQL: it merges multiple input streams into a single 151 stream that a processor can use. This new synchronizer will operate as follows: 152 - it initially has one input stream (normally connected to a TableReader 153 configured to read the first block of ranges held by a node); 154 - once this stream completes (i.e. producer has no more rows), the synchronizer 155 (which is pre-configured with a list of spans) resolves the spans into the 156 next batch of ranges, and sets up a processor on the corresponding node; 157 - the synchronizer can proceed in this manner, setting up a new processor when 158 the last one completes; or it can do more advanced things like set up new 159 processors before the last stream completes, or set up multiple new 160 processors at the same time. It can monitor the number of rows passing 161 through in relation to the limit to make these decisions. 162 163 Implementing this will require infrastructure for augmenting running flows with 164 new processors (or running multiple flows per node for the same query). Note 165 that we already need something along those lines for tolerating dying hosts. 166 167 This solution comes with a planning question: on which node do we put this 168 planning synchronizer? A reasonable first implementation is to always use the 169 gateway node. More advanced decisions are possible here though (e.g. if the 170 table is confined to a zone, the synchronizer should be in that 171 zone). 172 173 #### When order is not required #### 174 175 When order does not matter, we order the streams so that we start with the 176 closer nodes first, and among them with nodes that have more ranges. 177 178 ## Soft limit mechanism ## 179 180 In addition to the new planning methods, we introduce a mechanism that allows 181 processors to "pause" their work until the consumer tells us that it needs more 182 data. This works as follows: 183 - once a TableReader with a soft limit sends all the rows in the current KV 184 batch, it sends a special control message that means "I may have more rows, 185 but I'm waiting for you to tell me you need them". 186 - a processor that receives this message can send back a control message 187 informing the producer that more rows are needed. 188 - all processors need to be able to handle this message and forward it to their 189 consumers as necessary. In many cases the processors can forward this to 190 their consumer and then react to the consumer's request for more rows. In 191 other cases (e.g. sorting with no partial order, aggregation), the processor 192 will always need to request more rows (this can also be used as a simple but 193 correct "default" implementation to allow incremental implementation. 194 195 Note that we already have infrastructure to send control messages over processor 196 streams (in both directions). 197 198 One thing that we will need for implementing this is to enrich the RowFetcher 199 interface to give the caller hints about batch boundaries. This is useful for 200 local SQL as well: index-joins would benefit from aligning primary table batch 201 boundaries with index batch boundaries. 202 203 # Drawbacks 204 205 # Alternatives 206 207 #### When order is not required #### 208 209 In cases like `SELECT * FROM t LIMIT 10`, we could start processors on all nodes 210 that have ranges and stop the flow after the first 10 rows are received. This 211 may achieve better latency but it will have a (probably larger) negative impact 212 on overall throughput - we would waste a lot of work reading rows that we throw 213 away, and we set up operations on more nodes than we have to. We may consider 214 this kind of plan for some cases (perhaps tunable by a knob). 215 216 # Unresolved questions