Permalink
Newer
100644
1518 lines (1220 sloc)
66.4 KB
5
- RFC PR: [#6067](https://github.com/cockroachdb/cockroach/pull/6067)
6
- Cockroach Issue:
7
8
# Table of Contents
9
10
11
* [Table of Contents](#table-of-contents)
12
* [Summary](#summary)
13
* [Vocabulary](#vocabulary)
14
* [Motivation](#motivation)
15
* [Detailed design](#detailed-design)
16
* [Overview](#overview)
17
* [Logical model and logical plans](#logical-model-and-logical-plans)
18
* [Example 1](#example-1)
19
* [Example 2](#example-2)
20
* [Back propagation of ordering requirements](#back-propagation-of-ordering-requirements)
21
* [Example 3](#example-3)
22
* [Types of aggregators](#types-of-aggregators)
23
* [From logical to physical](#from-logical-to-physical)
24
* [Processors](#processors)
25
* [Joins](#joins)
26
* [Join-by-lookup](#join-by-lookup)
27
* [Stream joins](#stream-joins)
28
* [Inter-stream ordering](#inter-stream-ordering)
29
* [Execution infrastructure](#execution-infrastructure)
30
* [Creating a local plan: the ScheduleFlows RPC](#creating-a-local-plan-the-scheduleflows-rpc)
31
* [Local scheduling of flows](#local-scheduling-of-flows)
32
* [Mailboxes](#mailboxes)
33
* [On-the-fly flows setup](#on-the-fly-flows-setup)
34
* [Retiring flows](#retiring-flows)
35
* [Error handling](#error-handling)
36
* [A more complex example: Daily Promotion](#a-more-complex-example-daily-promotion)
37
* [Implementation strategy](#implementation-strategy)
38
* [Logical planning](#logical-planning)
39
* [Physical planning](#physical-planning)
40
* [Processor infrastructure and implementing processors](#processor-infrastructure-and-implementing-processors)
41
* [Joins](#joins-1)
42
* [Scheduling](#scheduling)
43
* [KV integration](#kv-integration)
44
* [Implementation notes](#implementation-notes)
45
* [Test infrastructure](#test-infrastructure)
46
* [Visualisation/tracing](#visualisationtracing)
47
* [Alternate approaches considered (and rejected)](#alternate-approaches-considered-and-rejected)
48
* [More logic in the KV layer](#more-logic-in-the-kv-layer)
49
* [Complexity](#complexity)
50
* [Applicability](#applicability)
51
* [SQL2SQL: Distributed SQL layer](#sql2sql-distributed-sql-layer)
52
* [Sample high-level query flows](#sample-high-level-query-flows)
53
* [Complexity](#complexity-1)
54
* [Applicability](#applicability-1)
55
* [Spark: Compiling SQL into a data-parallel language running on top of a distributed-execution runtime](#spark-compiling-sql-into-a-data-parallel-language-running-on-top-of-a-distributed-execution-runtime)
56
* [Sample program](#sample-program)
57
* [Complexity](#complexity-2)
58
* [Unresolved questions](#unresolved-questions)
59
60
61
# Summary
62
63
In this RFC we propose a general approach for distributing SQL processing and
64
moving computation closer to the data. The goal is to trigger an initial
65
discussion and not a complete detailed design.
66
67
## Vocabulary
68
69
- KV - the KV system in cockroach, defined by its key-value, range and batch API
70
- k/v - a key-value pair, usually used to refer to an entry in KV
71
- Node - machine in the cluster
72
- Client / Client-side - the SQL client
73
- Gateway node / Gateway-side - the cluster node to which the client SQL query is delivered first
74
- Leader node / Leader-side - the cluster node which resolves a KV operation and
75
has local access to the respective KV data
76
77
Most of the following text reads from the gateway-side perspective, where the query parsing and planning currently runs.
78
79
# Motivation
80
81
The desired improvements are listed below.
82
83
1. Remote-side filtering
84
85
When querying for a set of rows that match a filtering expression, we
86
currently query all the keys in certain ranges and process the filters after
87
receiving the data on the gateway node over the network. Instead, we want the
89
network traffic and related processing.
90
91
The remote-side filtering does not need to support full SQL expressions - it
92
can support a subset that includes common expressions (e.g. everything that
93
can be translated into expressions operating on strings) with the requesting
94
node applying a "final" filter.
95
96
2. Remote-side updates and deletes
97
98
For statements like `UPDATE .. WHERE` and `DELETE .. WHERE` we currently
99
perform a query, receive results at the gateway over the network, and then
100
perform the update or deletion there. This involves too many round-trips;
101
instead, we want the query and updates to happen on the node which has access
102
to the data.
103
104
Again, this does not need to cover all possible SQL expressions (we can keep
105
a working "slow" path for some cases). However, to cover the most important
106
queries we still need more than simple filtering expressions (`UPDATE`
107
commonly uses expressions and functions for calculating new values).
108
109
3. Distributed SQL operations
110
111
Currently SQL operations are processed by the entry node and thus their
112
performance does not scale with the size of the cluster. We want to be able to
113
distribute the processing on multiple nodes (parallelization for performance).
114
115
1. Distributed joins
116
117
In joins, we produce results by matching the values of certain columns
118
among multiple tables. One strategy for distributing this computation is
119
based on hashing: `K` nodes are chosen and each of the nodes with fast
120
access to the table data sends the results to the `K` nodes according to a
121
hash function computed on the join columns (guaranteeing that all results
122
with the same values on these columns go to the same node). Hash-joins are
123
employed e.g. by F1.
124
125
127
```sql
128
-- find all orders placed around the customer's birthday. Notice the
129
-- filtering needs to happen on the results. I've complicated the filtering
130
-- condition because a simple equality check could have been made part of
131
-- the join.
132
SELECT * FROM Customers c INNER JOIN Orders o ON c.ID = i.CustomerID
133
WHERE DayOfYear(c.birthday) - DayOfYear(o.date) < 7
134
```
137
138
When using `GROUP BY` we aggregate results according to a set of columns or
139
expressions and compute a function on each group of results. A strategy
140
similar to hash-joins can be employed to distribute the aggregation.
141
142
3. Distributed sorting
143
144
When ordering results, we want to be able to distribute the sorting effort.
146
results.
147
148
# Detailed design
149
150
## Overview
151
152
The proposed approach was originally inspired by [Sawzall][1] - a project by
153
Rob Pike et al. at Google that proposes a "shell" (high-level language
154
interpreter) to ease the exploitation of MapReduce. Its main innovation is a
155
concise syntax to define “local” processes that take a piece of local data and
156
emit zero or more results (these get translated to Map logic); then another
157
syntax which takes results from the local transformations and aggregates them
158
in different ways (this gets translated to Reduce logic). In a nutshell:
159
Sawzall = MapReduce + high-level syntax + new terminology (conveniently hiding
160
distributed computations behind a simple set of conceptual constructs).
161
162
We propose something somewhat similar, but targeting a different execution
163
model than MapReduce.
164
165
1. A predefined set of *aggregators*, performing functionality required by SQL.
166
Most aggregators are configurable, but not fully programmable.
167
2. One special aggregator, the 'evaluator', is programmable using a very simple
168
language, but is restricted to operating on one row of data at
169
a time.
170
3. A routing of the results of an aggregator to the next aggregator in the
171
query pipeline.
172
4. A logical model that allows for SQL to be compiled in a data-location-agnostic
173
way, but that captures enough information so that we can distribute the
174
computation.
175
176
Besides accumulating or aggregating data, the aggregators can feed their results
177
to another node or set of nodes, possibly as input for other programs. Finally,
178
aggregators with special functionality for batching up results and performing KV
179
commands are used to read data or make updates to the database.
180
181
The key idea is that we can map SQL to a well-defined logical model which we
182
then transform into a distributed execution plan.
183
184
[1]: http://research.google.com/archive/sawzall.html
185
186
## Logical model and logical plans
187
188
We compile SQL into a *logical plan* (similar on the surface to the current
189
`planNode` tree) which represents the abstract data flow through computation
190
stages. The logical plan is agnostic to the way data is partitioned and
191
distributed in the cluster; however, it contains enough information about the
192
structure of the planned computation to allow us to exploit data parallelism
193
later - in a subsequent phase, the logical plan will be converted into a
194
*physical plan*, which maps the abstract computation and data flow to concrete
195
data processors and communication channels between them.
196
197
The logical plan is made up **aggregators**. Each aggregator consumes an **input
198
stream** of rows (or more streams for joins, but let's leave that aside for now)
199
and produces an **output stream** of rows. Each row is a tuple of column values;
200
both the input and the output streams have a set schema. The schema is a set of
201
columns and types, with each row having a datum for each column. Again, we
202
emphasize that the streams are a logical concept and might not map to a single
203
data stream in the actual computation.
204
205
We introduce the concept of **grouping** to characterize a specific aspect of
206
the computation that happens inside an aggregator. The groups are defined based
207
on a **group key**, which is a subset of the columns in the input stream schema.
208
The computation that happens for each group is independent of the data in the
209
other groups, and the aggregator emits a concatenation of the results for all
210
the groups. The ordering between group results in the output stream is not
211
fixed - some aggregators may guarantee a certain ordering, others may not.
212
213
More precisely, we can define the computation in an aggregator using a function
214
`agg` that takes a sequence of input rows that are in a single group (same group
215
key) and produces a set of output rows. The output of an aggregator is
216
the concatenation of the outputs of `agg` on all the groups, in some order.
217
218
The grouping characteristic will be useful when we later decide how to
219
distribute the computation that is represented by an aggregator: since results
220
for each group are independent, different groups can be processed on different
221
nodes. The more groups we have, the better. At one end of the spectrum there are
222
single-group aggregators (group key is the empty set of columns - `Group key:
223
[]`, meaning everything is in the same group) which cannot be distributed. At
224
the other end there are no-grouping aggregators which can be parallelized
225
arbitrarily. Note that no-grouping aggregators are different than aggregators
226
where the group key is the full set of columns - the latter still requires rows
227
that are equal to be processed on a single node (this would be useful for an
228
aggregator implementing `DISTINCT` for example). An aggregator with no grouping
229
is a special but important case in which we are not aggregating multiple pieces
230
of data, but we may be filtering, transforming, or reordering individual pieces
231
of data.
232
233
Aggregators can make use of SQL expressions, evaluating them with various inputs
234
as part of their work. In particular, all aggregators can optionally use an
235
**output filter** expression - a boolean function that is used to discard
236
elements that would have otherwise been part of the output stream.
237
238
(Note: the alternative of restricting use of SQL expressions to only certain
239
aggregators was considered; that approach makes it much harder to support outer
240
joins, where the `ON` expression evaluation must be part of the internal join
241
logic and not just a filter on the output.)
242
244
"programmable" aggregator which processes the input stream sequentially (one
245
element at a time), potentially emitting output elements. This is an aggregator
246
with no grouping (group key is the full set of columns); the processing of each
247
row independent. An evaluator can be used, for example, to generate new values from
248
arbitrary expressions (like the `a+b` in `SELECT a+b FROM ..`); or to filter
249
rows according to a predicate.
250
251
Special **table reader** aggregators with no inputs are used as data sources; a
252
table reader can be configured to output only certain columns, as needed.
253
A special **final** aggregator with no outputs is used for the results of the
254
query/statement.
255
256
Some aggregators (final, limit) have an **ordering requirement** on the input
257
stream (a list of columns with corresponding ascending/descending requirements).
258
Some aggregators (like table readers) can guarantee a certain ordering on their
259
output stream, called an **ordering guarantee** (same as the `orderingInfo` in
260
the current code). All aggregators have an associated **ordering
261
characterization** function `ord(input_order) -> output_order` that maps
262
`input_order` (an ordering guarantee on the input stream) into `output_order`
263
(an ordering guarantee for the output stream) - meaning that if the rows in the
264
input stream are ordered according to `input_order`, then the rows in the output
265
stream will be ordered according to `output_order`.
266
267
The ordering guarantee of the table readers along with the characterization
268
functions can be used to propagate ordering information across the logical plan.
269
When there is a mismatch (an aggregator has an ordering requirement that is not
270
matched by a guarantee), we insert a **sorting aggregator** - this is a
271
non-grouping aggregator with output schema identical to the input schema that
272
reorders the elements in the input stream providing a certain output order
273
guarantee regardless of the input ordering. We can perform optimizations wrt
274
sorting at the logical plan level - we could potentially put the sorting
275
aggregator earlier in the pipeline, or split it into multiple nodes (one of
276
which performs preliminary sorting in an earlier stage).
277
278
To introduce the main types of aggregators we use of a simple query.
279
280
### Example 1
281
282
```sql
283
TABLE Orders (OId INT PRIMARY KEY, CId INT, Value DECIMAL, Date DATE)
284
285
SELECT CID, SUM(VALUE) FROM Orders
286
WHERE DATE > 2015
287
GROUP BY CID
288
ORDER BY 1 - SUM(Value)
289
```
290
291
This is a potential description of the aggregators and streams:
292
293
```
294
TABLE-READER src
295
Table: Orders
296
Table schema: Oid:INT, Cid:INT, Value:DECIMAL, Date:DATE
297
Output filter: (Date > 2015)
298
Output schema: Cid:INT, Value:DECIMAL
299
Ordering guarantee: Oid
300
301
AGGREGATOR summer
302
Input schema: Cid:INT, Value:DECIMAL
303
Output schema: Cid:INT, ValueSum:DECIMAL
304
Group Key: Cid
305
Ordering characterization: if input ordered by Cid, output ordered by Cid
306
308
Input schema: Cid:INT, ValueSum:DECIMAL
309
Output schema: SortVal:DECIMAL, Cid:INT, ValueSum:DECIMAL
310
Ordering characterization:
311
ValueSum -> ValueSum and -SortVal
312
Cid,ValueSum -> Cid,ValueSum and Cid,-SortVal
313
ValueSum,Cid -> ValueSum,Cid and -SortVal,Cid
314
SQL Expressions: E(x:INT) INT = (1 - x)
315
Code {
316
EMIT E(ValueSum), CId, ValueSum
317
}
318
319
AGGREGATOR final:
320
Input schema: SortVal:DECIMAL, Cid:INT, ValueSum:DECIMAL
321
Input ordering requirement: SortVal
322
Group Key: []
323
324
Composition: src -> summer -> sortval -> final
325
```
326
327
Note that the logical description does not include sorting aggregators. This
328
preliminary plan will lead to a full logical plan when we propagate ordering
329
information. We will have to insert a sorting aggregator before `final`:
330
```
331
src -> summer -> sortval -> sort(OrderSum) -> final
332
```
333
Each arrow is a logical stream. This is the complete logical plan.
334
335
In this example we only had one option for the sorting aggregator. Let's look at
336
another example.
337
338
339
### Example 2
340
341
```sql
342
TABLE People (Age INT, NetWorth DECIMAL, ...)
343
344
SELECT Age, Sum(NetWorth) FROM v GROUP BY AGE ORDER BY AGE
345
```
346
347
Preliminary logical plan description:
348
```
349
TABLE-READER src
350
Table: People
351
Table schema: Age:INT, NetWorth:DECIMAL
352
Output schema: Age:INT, NetWorth:DECIMAL
353
Ordering guarantee: XXX // will consider different cases later
354
355
AGGREGATOR summer
356
Input schema: Age:INT, NetWorth:DECIMAL
357
Output schema: Age:INT, NetWorthSum:DECIMAL
358
Group Key: Age
359
Ordering characterization: if input ordered by Age, output ordered by Age
360
361
AGGREGATOR final:
362
Input schema: Age:INT, NetWorthSum:DECIMAL
363
Input ordering requirement: Age
364
Group Key: []
365
366
Composition: src -> summer -> final
367
```
368
369
The `summer` aggregator can perform aggregation in two ways - if the input is
370
not ordered by Age it will use an unordered map with one entry per `Age` and the
371
results will be output in arbitrary order; if the input is ordered by `Age` it can
372
aggregate on one age at a time and it will emit the results in age order.
373
374
Let's take two cases:
375
376
1. src is ordered by `Age` (we use a covering index on `Age`)
377
378
In this case, when we propagate the ordering
379
information we will notice that `summer` preserves ordering by age and we
380
won't need to add sorting aggregators.
381
382
2. src is not ordered by anything
383
384
In this case, summer will not have any output ordering guarantees and we will
385
need to add a sorting aggregator before `final`:
386
```
387
src -> summer -> sort(Age) -> final
388
```
389
We could also use the fact that `summer` would preserve the order by `Age`
390
and put the sorting aggregator before `summer`:
391
```
392
src -> sort(Age) -> summer -> final
394
We would choose between these two logical plans.
395
396
There is also the possibility that `summer` uses an ordered map, in which case
397
it will always output the results in age order; that would mean we are always in
398
case 1 above, regardless of the ordering of `src`.
399
400
### Back propagation of ordering requirements
401
403
stream along with an order preservation guarantee to avoid sorting. The
404
preliminary logical plan will try to preserve ordering as much as possible to
405
minimize any additional sorting.
406
407
However, in some cases preserving ordering might come with some cost; some
408
aggregators could be configured to either preserve ordering or not. To avoid
409
preserving ordering unnecessarily, after the sorting aggregators are put in
410
place we post-process the logical plan to relax the ordering on the streams
411
wherever possible. Specifically, we inspect each logical stream (in reverse
412
topological order) and check if removing its ordering still yields a correct
413
plan; this results in a back-propagation of the ordering requirements.
414
415
To recap, the logical planning has three stages:
416
1. preliminary logical plan, with ordering preserved as much as possible and no
417
sort nodes,
418
2. order-satisfying logical plan, with sort nodes added as necessary,
419
3. final logical plan, with ordering requirements relaxed where possible.
420
421
### Example 3
422
423
```sql
424
TABLE v (Name STRING, Age INT, Account INT)
425
426
SELECT COUNT(DISTINCT(account)) FROM v
427
WHERE age > 10 and age < 30
428
GROUP BY age HAVING MIN(Name) > 'k'
429
```
430
431
```
432
TABLE-READER src
433
Table: v
434
Table schema: Name:STRING, Age:INT, Account:INT
435
Filter: (Age > 10 AND Age < 30)
436
Output schema: Name:STRING, Age:INT, Account:INT
437
Ordering guarantee: Name
438
439
AGGREGATOR countdistinctmin
440
Input schema: Name:String, Age:INT, Account:INT
441
Group Key: Age
442
Group results: distinct count as AcctCount:INT
443
MIN(Name) as MinName:STRING
444
Output filter: (MinName > 'k')
445
Output schema: AcctCount:INT
446
Ordering characterization: if input ordered by Age, output ordered by Age
447
448
AGGREGATOR final:
449
Input schema: AcctCount:INT
450
Input ordering requirement: none
451
Group Key: []
452
453
Composition: src -> countdistinctmin -> final
454
```
455
456
### Types of aggregators
457
458
- `TABLE READER` is a special aggregator, with no input stream. It's configured
459
with spans of a table or index and the schema that it needs to read.
460
Like every other aggregator, it can be configured with a programmable output
461
filter.
462
- `EVALUATOR` is a fully programmable no-grouping aggregator. It runs a "program"
463
on each individual row. The evaluator can drop the row, or modify it
464
arbitrarily.
465
- `JOIN` performs a join on two streams, with equality constraints between
466
certain columns. The aggregator is grouped on the columns that are
467
constrained to be equal. See [Stream joins](#stream-joins).
469
input stream. It can do so by performing (potentially remote) KV reads, or by
470
setting up remote flows. See [Join-by-lookup](#join-by-lookup) and
471
[On-the-fly flows setup](#on-the-fly-flows-setup).
472
- `MUTATE` performs insertions/deletions/updates to KV. See section TODO.
473
- `SET OPERATION` takes several inputs and performs set arithmetic on them
474
(union, difference).
475
- `AGGREGATOR` is the one that does "aggregation" in the SQL sense. It groups
476
rows and computes an aggregate for each group. The group is configured using
477
the group key. `AGGREGATOR` can be configured with one or more aggregation
478
functions:
479
- `SUM`
480
- `COUNT`
481
- `COUNT DISTINCT`
482
- `DISTINCT`
483
484
`AGGREGATOR`'s output schema consists of the group key, plus a configurable
487
values that are not ultimately outputted).
488
- `SORT` sorts the input according to a configurable set of columns. Note that
489
this is a no-grouping aggregator, hence it can be distributed arbitrarily to
490
the data producers. This means, of course, that it doesn't produce a global
491
ordering, instead it just guarantees an intra-stream ordering on each
492
physical output streams). The global ordering, when needed, is achieved by an
493
input synchronizer of a grouped processor (such as `LIMIT` or `FINAL`).
494
- `LIMIT` is a single-group aggregator that stops after reading so many input
495
rows.
496
- `INTENT-COLLECTOR` is a single-group aggregator, scheduled on the gateway,
497
that receives all the intents generated by a `MUTATE` and keeps track of them
498
in memory until the transaction is committed.
499
- `FINAL` is a single-group aggregator, scheduled on the gateway, that collects
500
the results of the query. This aggregator will be hooked up to the pgwire
501
connection to the client.
502
503
## From logical to physical
504
505
To distribute the computation that was described in terms of aggregators and
506
logical streams, we use the following facts:
507
508
- for any aggregator, groups can be partitioned into subsets and processed in
509
parallel, as long as all processing for a group happens on a single node.
510
511
- the ordering characterization of an aggregator applies to *any* input stream
512
with a certain ordering; it is useful even when we have multiple parallel
513
instances of computation for that logical node: if the physical input streams
514
in all the parallel instances are ordered according to the logical input
515
stream guarantee (in the logical plan), the physical output streams in all
516
instances will have the output guarantee of the logical output stream. If at
517
some later stage these streams are merged into a single stream (merge-sorted,
518
i.e. with the ordering properly maintained), that physical stream will have
519
the correct ordering - that of the corresponding logical stream.
520
521
- aggregators with empty group keys (`limit`, `final`) must have their final
522
processing on a single node (they can however have preliminary distributed
523
stages).
524
525
So each logical aggregator can correspond to multiple distributed instances, and
526
each logical stream can correspond to multiple physical streams **with the same
527
ordering guarantees**.
528
529
We can distribute using a few simple rules:
530
531
- table readers have multiple instances, split according to the ranges; each
532
instance is processed by the raft leader of the relevant ranges and is the
533
start of a physical stream.
534
535
- streams continue in parallel through programs. When an aggregator is reached,
536
the streams can be redistributed to an arbitrary number of instances using
537
hashing on the group key. Aggregators with empty group keys will have a
538
single physical instance, and the input streams are merged according to the
539
desired ordering. As mentioned above, each physical stream will be already
540
ordered (because they all correspond to an ordered logical stream).
541
542
- sorting aggregators apply to each physical stream corresponding to the
543
logical stream it is sorting. A sort aggregator by itself will *not* result
544
in coalescing results into a single node. This is implicit from the fact that
546
547
It is important to note that correctly distributing the work along range
548
boundaries is not necessary for correctness - if a range gets split or moved
549
while we are planning the query, it will not cause incorrect results. Some key
550
reads might be slower because they actually happen remotely, but as long as
551
*most of the time, most of the keys* are read locally this should not be a
552
problem.
553
554
Assume that we run the Example 1 query on a **Gateway** node and the table has
555
data that on two nodes **A** and **B** (i.e. these two nodes are masters for all
556
the relevant range). The logical plan is:
557
558
```
559
TABLE-READER src
560
Table: Orders
561
Table schema: Oid:INT, Cid:INT, Value:DECIMAL, Date:DATE
562
Output filter: (Date > 2015)
563
Output schema: Cid:INT, Value:DECIMAL
564
Ordering guarantee: Oid
565
566
AGGREGATOR summer
567
Input schema: Cid:INT, Value:DECIMAL
568
Output schema: Cid:INT, ValueSum:DECIMAL
569
Group Key: Cid
570
Ordering characterization: if input ordered by Cid, output ordered by Cid
571
573
Input schema: Cid:INT, ValueSum:DECIMAL
574
Output schema: SortVal:DECIMAL, Cid:INT, ValueSum:DECIMAL
575
Ordering characterization: if input ordered by [Cid,]ValueSum[,Cid], output ordered by [Cid,]-ValueSum[,Cid]
576
SQL Expressions: E(x:INT) INT = (1 - x)
577
Code {
578
EMIT E(ValueSum), CId, ValueSum
579
}
580
```
581
583
584
This logical plan above could be instantiated as the following physical plan:
585
586

587
588
Each box in the physical plan is a *processor*:
589
- `src` is a table reader and performs KV Get operations and forms rows; it is
590
programmed to read the spans that belong to the respective node. It evaluates
591
the `Date > 2015` filter before outputting rows.
592
- `summer-stage1` is the first stage of the `summer` aggregator; its purpose is
593
to do the aggregation it can do locally and distribute the partial results to
594
the `summer-stage2` processes, such that all values for a certain group key
595
(`CId`) reach the same process (by hashing `CId` to one of two "buckets").
596
- `summer-stage2` performs the actual sum and outputs the index (`CId`) and
597
corresponding sum.
598
- `sortval` calculates and emits the additional `SortVal` value, along with the
599
`CId` and `ValueSum`
600
- `sort` sorts the stream according to `SortVal`
601
- `final` merges the two input streams of data to produce the final sorted
602
result.
603
604
Note that the second stage of the `summer` aggregator doesn't need to run on the
605
same nodes; for example, an alternate physical plan could use a single stage 2
606
processor:
607
608

609
610
The processors always form a directed acyclic graph.
611
612
### Processors
613
614
Processors are generally made up of three components:
615
617
618
1. The *input synchronizer* merges the input streams into a single stream of
619
data. Types:
620
* single-input (pass-through)
621
* unsynchronized: passes rows from all input streams, arbitrarily
622
interleaved.
623
* ordered: the input physical streams have an ordering guarantee (namely the
625
to interleave the streams so that the merged stream has the same guarantee.
626
627
2. The *data processor* core implements the data transformation or aggregation
628
logic (and in some cases performs KV operations).
629
630
3. The *output router* splits the data processor's output to multiple streams;
631
types:
632
* single-output (pass-through)
633
* mirror: every row is sent to all output streams
634
* hashing: each row goes to a single output stream, chosen according
635
to a hash function applied on certain elements of the data tuples.
636
* by range: the router is configured with range information (relating to a
638
the respective ranges (useful for `JoinReader` nodes (taking index values
639
to the node responsible for the PK) and `INSERT` (taking new rows to their
641
642
## Joins
643
644
### Join-by-lookup
645
646
The join-by-lookup method involves receiving data from one table and looking up
647
corresponding rows from another table. It is typically used for joining an index
648
with the table, but they can be used for any join in the right circumstances,
649
e.g. joining a small number of rows from one table ON the primary key of another
650
table. We introduce a variant of `TABLE-READER` which has an input stream. Each
651
element of the input stream drives a point-lookup in another table or index,
652
with a corresponding value in the output. Internally the aggregator performs
653
lookups in batches, the way we already do it today. Example:
654
655
```sql
656
TABLE t (k INT PRIMARY KEY, u INT, v INT, INDEX(u))
657
SELECT k, u, v FROM t WHERE u >= 1 AND u <= 5
658
```
659
Logical plan:
660
```
661
TABLE-READER indexsrc
662
Table: t@u, span /1-/6
663
Output schema: k:INT, u:INT
664
Output ordering: u
665
666
JOIN-READER pksrc
667
Table: t
668
Input schema: k:INT, u:INT
669
Output schema: k:INT, u:INT, v:INT
670
Ordering characterization: preserves any ordering on k/u
671
672
AGGREGATOR final
673
Input schema: k:INT, u:INT, v:INT
674
675
indexsrc -> pksrc -> final
676
```
677
678
Example of when this can be used for a join:
679
```
680
TABLE t1 (k INT PRIMARY KEY, v INT, INDEX(v))
681
TABLE t2 (k INT PRIMARY KEY, w INT)
682
SELECT t1.k, t1.v, t2.w FROM t1 INNER JOIN t2 ON t1.k = t2.k WHERE t1.v >= 1 AND t1.v <= 5
683
```
684
685
Logical plan:
686
```
687
TABLE-READER t1src
688
Table: t1@v, span /1-/6
689
Output schema: k:INT, v:INT
690
Output ordering: v
691
692
JOIN-READER t2src
693
Table: t2
694
Input schema: k:INT, v:INT
695
Output schema: k:INT, v:INT, w:INT
696
Ordering characterization: preserves any ordering on k
697
698
AGGREGATOR final
699
Input schema: k:INT, u:INT, v:INT
700
701
t1src -> t2src -> final
702
```
703
704
Note that `JOIN-READER` has the capability of "plumbing" through an input column
705
to the output (`v` in this case). In the case of index joins, this is only
706
useful to skip reading or decoding the values for `v`; but in the general case
707
it is necessary to pass through columns from the first table.
708
709
In terms of the physical implementation of `JOIN-READER`, there are two possibilities:
710
711
1. it can perform the KV queries (in batches) from the node it is receiving the
712
physical input stream from; the output stream continues on the same node.
713
714
This is simple but involves round-trips between the node and the range
716
717
2. it can use routers-by-range to route each input to an instance of
718
`JOIN-READER` on the node for the respective range of `t2`; the flow of data
719
continues on that node.
720
721
This avoids the round-trip but is problematic because we may be setting up
722
flows on too many nodes (for a large table, many/all nodes in the cluster
723
could hold ranges). To implement this effectively, we require the ability to
724
set up the flows "lazily" (as needed), only when we actually find a row that
725
needs to go through a certain flow. This strategy can be particularly useful
726
when the ordering of `t1` and `t2` are correlated (e.g. t1 could be ordered
727
by a date, `t2` could be ordered by an implicit primary key).
728
729
730
Even with this optimization, it would be wasteful if we involve too many
731
remote nodes but we only end up doing a handful of queries. We can
732
investigate a hybrid approach where we batch some results and depending on
733
how many we have and how many different ranges/nodes they span, we choose
734
between the two strategies.
735
736
737
### Stream joins
738
739
The join aggregator performs a join on two streams, with equality constraints
740
between certain columns. The aggregator is grouped on the columns that are
741
constrained to be equal.
742
743
```
744
TABLE People (First STRING, Last STRING, Age INT)
745
TABLE Applications (College STRING PRIMARY KEY, First STRING, Last STRING)
746
SELECT College, Last, First, Age FROM People INNER JOIN Applications ON First, Last
747
748
TABLE-READER src1
749
Table: People
750
Output Schema: First:STRING, Last:STRING, Age:INT
751
Output Ordering: none
752
753
TABLE_READER src2
754
Table: Applications
755
Output Schema: College:STRING, First:STRING, Last:STRING
756
Output Ordering: none
757
758
JOIN AGGREGATOR join
759
Input schemas:
760
1: First:STRING, Last:STRING, Age:INT
761
2: College:STRING, First:STRING, Last:STRING
762
Output schema: First:STRING, Last:STRING, Age:INT, College:STRING
763
Group key: (1.First, 1.Last) = (2.First, 2.Last) // we need to get the group key from either stream
764
Order characterization: no order preserved // could also preserve the order of one of the streams
765
766
AGGREGATOR final
767
Ordering requirement: none
768
Input schema: First:STRING, Last:STRING, Age:INT, College:STRING
769
```
770
771

772
773
At the heart of the physical implementation of the stream join aggregators sits
774
the join processor which (in general) puts all the rows from one stream in a
775
hash map and then processes the other stream. If both streams are ordered by the
776
group key, it can perform a merge-join which requires less memory.
777
778
779
Using the same join processor implementation, we can implement different
780
distribution strategies depending how we set up the physical streams and
781
routers:
782
783
- the routers can distribute each row to one of multiple join processors based
784
on a hash on the elements for the group key; this ensures that all elements
785
in a group reach the same instance, achieving a hash-join. An example
786
physical plan:
787
788

789
790
- the routers can *duplicate* all rows from the physical streams for one table
791
and distribute copies to all processor instances; the streams for the other
792
table get processed on their respective nodes. This strategy is useful when
793
we are joining a big table with a small table, and can be particularly useful
794
for subqueries, e.g. `SELECT ... WHERE ... AND x IN (SELECT ...)`.
795
796
For the query above, if we expect few results from `src2`, this plan would
797
be more efficient:
798
799

801
The difference in this case is that the streams for the first table stay on
802
the same node, and the routers after the `src2` table readers are configured
805
806
## Inter-stream ordering
807
808
**This is a feature that relates to implementing certain optimizations, but does
809
not alter the structure of logical or physical plans. It will not be part of the
810
initial implementation but we keep it in mind for potential use at a later
811
point.**
812
813
Consider this example:
814
```sql
815
TABLE t (k INT PRIMARY KEY, v INT)
816
SELECT k, v FROM t WHERE k + v > 10 ORDER BY k
817
```
818
819
This is a simple plan:
820
821
```
822
READER src
823
Table: t
824
Output filter: (k + v > 10)
825
Output schema: k:INT, v:INT
826
Ordering guarantee: k
827
828
AGGREGATOR final:
829
Input schema: k:INT, v:INT
830
Input ordering requirement: k
831
Group Key: []
832
833
Composition: src -> final
834
```
835
836
Now let's say that the table spans two ranges on two different nodes - one range
837
for keys `k <= 10` and one range for keys `k > 10`. In the physical plan we
838
would have two streams starting from two readers; the streams get merged into a
839
single stream before `final`. But in this case, we know that *all* elements in
840
one stream are ordered before *all* elements in the other stream - we say that
841
we have an **inter-stream ordering**. We can be more efficient when merging
842
(before `final`): we simply read all elements from the first stream and then all
843
elements from the second stream. Moreover, we would also know that the reader
844
and other processors for the second stream don't need to be scheduled until the
845
first stream is consumed, which is useful information for scheduling the query.
846
In particular, this is important when we have a query with `ORDER BY` and
847
`LIMIT`: the limit would be represented by an aggregator with a single group,
848
with physical streams merging at that point; knowledge of the inter-stream
849
ordering would allow us to potentially satisfy the limit by only reading from
850
one range.
851
852
We add the concept of inter-physical-stream ordering to the logical plan - it is
853
a property of a logical stream (even though it refers to multiple physical
854
streams that could be associated with that logical stream). We annotate all
855
aggregators with an **inter-stream ordering characterization function** (similar
856
to the ordering characterization described above, which can be thought of as
857
"intra-stream" ordering). The inter-stream ordering function maps an input
858
ordering to an output ordering, with the meaning that if the physical streams
859
that are inputs to distributed instances of that aggregator have the
860
inter-stream ordering described by input, then the output streams have the given
861
output ordering.
862
863
Like the intra-stream ordering information, we can propagate the inter-stream
864
ordering information starting from the table readers onward. The streams coming
865
out of a table reader have an inter-stream order if the spans each reader "works
866
on" have a total order (this is always the case if each table reader is
867
associated to a separate range).
868
869
The information can be used to apply the optimization above - if a logical
870
stream has an appropriate associated inter-stream ordering, the merging of the
871
physical streams can happen by reading the streams sequentially. The same
872
information can be used for scheduling optimizations (such as scheduling table
873
readers that eventually feed into a limit sequentially instead of
874
concurrently).
875
876
## Execution infrastructure
877
878
Once a physical plan has been generated, the system needs to divvy it up
879
between the nodes and send it around for execution. Each node is responsible
880
for locally scheduling data processors and input synchronizers. Nodes also need
881
to be able to communicate with each other for connecting output routers to
882
input synchronizers. In particular, a streaming interface is needed for
883
connecting these actors. To avoid paying extra synchronization costs, the
884
execution environment providing all these needs to be flexible enough so that
885
different nodes can start executing their pieces in isolation, without any
886
orchestration from the gateway besides the initial request to schedule a part
887
of the plan.
888
889
### Creating a local plan: the `ScheduleFlows` RPC
890
891
Distributed execution starts with the gateway making a request to every node
892
that's supposed to execute part of the plan asking the node to schedule the
893
sub-plan(s) it's responsible for (modulo "on-the-fly" flows, see below). A node
894
might be responsible for multiple disparate pieces of the overall DAG. Let's
895
call each of them a *flow*. A flow is described by the sequence of physical
896
plan nodes in it, the connections between them (input synchronizers, output
897
routers) plus identifiers for the input streams of the top node in the plan and
898
the output streams of the (possibly multiple) bottom nodes. A node might be
899
responsible for multiple heterogeneous flows. More commonly, when a node is the
900
lease holder for multiple ranges from the same table involved in the query, it will
901
be responsible for a set of homogeneous flows, one per range, all starting with
902
a `TableReader` processor. In the beginning, we'll coalesce all these
903
`TableReader`s into one, configured with all the spans to be read across all
904
the ranges local to the node. This means that we'll lose the inter-stream
905
ordering (since we've turned everything into a single stream). Later on we
906
might move to having one `TableReader` per range, so that we can schedule
907
multiple of them to run in parallel.
908
909
A node therefore implements a `ScheduleFlows` RPC which takes a set of flows,
910
sets up the input and output mailboxes (see below), creates the local
911
processors and starts their execution. We might imagine at some point
912
implementing admission control for flows at the node boundary, in which case
913
the RPC response would have the option to push back on the volume of work
914
that's being requested.
915
916
### Local scheduling of flows
917
918
The simplest way to schedule the different processors locally on a node is
919
concurrently: each data processor, synchronizer and router can be run as a
920
goroutine, with channels between them. The channels can be buffered to
921
synchronize producers and consumers to a controllable degree.
922
923
### Mailboxes
924
925
Flows on different nodes communicate with each other over GRPC streams. To
926
allow the producer and the consumer to start at different times,
927
`ScheduleFlows` creates named mailboxes for all the input and output streams.
928
These message boxes will hold some number of tuples in an internal queue until
929
a GRPC stream is established for transporting them. From that moment on, GRPC
931
A GRPC stream is established by the consumer using the `StreamMailbox` RPC,
932
taking a mailbox id (the same one that's been already used in the flows passed
933
to `ScheduleFlows`). This call might arrive to a node even before the
934
corresponding `ScheduleFlows` arrives. In this case, the mailbox is created on
935
the fly, in the hope that the `ScheduleFlows` will follow soon. If that doesn't
936
happen within a timeout, the mailbox is retired.
938
If we move to a multiple `TableReader`s/flows per node, we'd still want one
939
single output mailbox for all the homogeneous flows (if a node has 1mil ranges,
940
we don't want 1mil mailboxes/streams). At that point we might want to add
941
tagging to the different streams entering the mailbox, so that the inter-stream
942
ordering property can still be used by the consumer.
943
944
A diagram of a simple query using mailboxes for its execution:
946
947
### On-the-fly flows setup
948
949
In a couple of cases, we don't want all the flows to be setup from the get-go.
950
`PointLookup` and `Mutate` generally start on a few ranges and then send data
951
to arbitrary nodes. The amount of data to be sent to each node will often be
952
very small (e.g. `PointLookup` might perform a handful of lookups in total on
953
table *A*, so we don't want to set up receivers for those lookups on all nodes
954
containing ranges for table *A*. Instead, the physical plan will contain just
955
one processor, making the `PointLookup` aggregator single-stage; this node can
956
chose whether to perform KV operations directly to do the lookups (for ranges
957
with few lookups), or setup remote flows on the fly using the `ScheduleFlows`
958
RPC for ranges with tons of lookups. In this case, the idea is to push a bunch
959
of the computation to the data, so the flow passed to `ScheduleFlows` will be a
960
copy of the physical nodes downstream of the aggregator, including filtering
961
and aggregation. We imagine the processor will take the decision to move to
962
this heavywight process once it sees that it's batching a lot of lookups for
963
the same range.
964
965
## Retiring flows
966
967
Processors and mailboxes needs to be destroyed at some point. This happens
968
under a number of circumstances:
969
970
1. A processor retires when it receives a sentinel on all of its input streams
971
and has outputted the last tuple (+ a sentinel) on all of its output
972
streams.
973
2. A processor retires once either one of its input or output streams is
974
closed. This can be used by a consumer to inform its producers that it's
975
gotten all the data it needs.
976
3. An input mailbox retires once it has put the sentinel on the wire or once
977
its GRPC stream is closed remotely.
978
4. An output mailbox retires once it has passed on the sentinel to the reader,
979
which it does once all of its input channels are closed (remember that an
980
output mailbox may receive input from many channels, one per member of a
981
homogeneous flow family). It also retires if its GRPC stream is closed
982
remotely.
983
5. `TableReader` retires once it has delivered the last tuple in its range (+ a
984
sentinel).
985
986
### Error handling
987
988
At least initially, the plan is to have no error recovery (anything goes wrong
989
during execution, the query fails and the transaction is rolled back).
990
The only concern is releasing all resources taken by the plan nodes.
991
This can be done by propagating an error signal when any GRPC stream is
992
closed abruptly.
993
Similarly, cancelling a running query can be done by asking the `FINAL` processor
994
to close its input channel. This close will propagate backwards to all plan nodes.
995
996
997
# A more complex example: Daily Promotion
998
999
Let's draw a possible logical and physical plan for a more complex query. The
1000
point of the query is to help with a promotion that goes out daily, targeting
1001
customers that have spent over $1000 in the last year. We'll insert into the
1002
`DailyPromotion` table rows representing each such customer and the sum of her
1003
recent orders.
1004
1005
```SQL
1006
TABLE DailyPromotion (
1007
Email TEXT,
1008
Name TEXT,
1009
OrderCount INT
1010
)
1011
1012
TABLE Customers (
1013
CustomerID INT PRIMARY KEY,
1024
INDEX date (Date)
1025
)
1026
1027
INSERT INTO DailyPromotion
1028
(SELECT c.Email, c.Name, os.OrderCount FROM
1029
Customers AS c
1030
INNER JOIN
1031
(SELECT CustomerID, COUNT(*) as OrderCount FROM Orders
1032
WHERE Date >= '2015-01-01'
1033
GROUP BY CustomerID HAVING SUM(Value) >= 1000) AS os
1034
ON c.CustomerID = os.CustomerID)
1035
```
1036
1037
Logical plan:
1038
1039
```
1040
TABLE-READER orders-by-date
1042
Input schema: Date: Datetime, OrderID: INT
1043
Output schema: Cid:INT, Value:DECIMAL
1044
Output filter: None (the filter has been turned into a scan range)
1045
Intra-stream ordering characterization: Date
1046
Inter-stream ordering characterization: Date
1047
1048
JOIN-READER orders
1049
Table: Orders
1050
Input schema: Oid:INT, Date:DATETIME
1051
Output filter: None
1052
Output schema: Cid:INT, Date:DATETIME, Value:INT
1053
// TODO: The ordering characterizations aren't necessary in this example
1054
// and we might get better performance if we remove it and let the aggregator
1055
// emit results out of order. Update after the section on backpropagation of
1056
// ordering requirements.
1058
Inter-stream ordering characterization: Oid
1059
1060
AGGREGATOR count-and-sum
1061
Input schema: CustomerID:INT, Value:INT
1062
Aggregation: SUM(Value) as sumval:INT
1063
COUNT(*) as OrderCount:INT
1064
Group key: CustomerID
1065
Output schema: CustomerID:INT, OrderCount:INT
1066
Output filter: sumval >= 1000
1067
Intra-stream ordering characterization: None
1068
Inter-stream ordering characterization: None
1069
1070
JOIN-READER customers
1071
Table: Customers
1072
Input schema: CustomerID:INT, OrderCount: INT
1073
Output schema: e-mail: TEXT, Name: TEXT, OrderCount: INT
1074
Output filter: None
1075
// TODO: The ordering characterizations aren't necessary in this example
1076
// and we might get better performance if we remove it and let the aggregator
1077
// emit results out of order. Update after the section on backpropagation of
1078
// ordering requirements.
1080
Inter-stream ordering characterization: same as input
1081
1082
INSERT inserter
1083
Table: DailyPromotion
1084
Input schema: email: TEXT, name: TEXT, OrderCount: INT
1085
Table schema: email: TEXT, name: TEXT, OrderCount: INT
1086
1087
INTENT-COLLECTOR intent-collector
1088
Group key: []
1089
Input schema: k: TEXT, v: TEXT
1090
1091
AGGREGATOR final:
1097
order-by-date -> orders -> count-and-sum -> customers -> inserter -> intent-collector
1098
\-> final (sum)
1099
```
1100
1101
A possible physical plan:
1102

1103
1104
# Implementation strategy
1105
1106
There are five streams of work. We keep in mind two initial milestones to track
1107
the extent of progress we must achieve within each stream:
1108
- Milestone M1: offloading filters to remote side
1109
- Milestone M2: offloading updates to remote side
1110
1111
### Logical planning
1112
1113
Building a logical plan for a statement involves many aspects:
1114
- index selection
1115
- query optimization
1116
- choosing between various strategies for sorting, aggregation
1117
- choosing between join strategies
1118
1119
This is a very big area where we will see a long tail of improvements over time.
1120
However, we can start with a basic implementation based on the existing code.
1121
For a while we can use a hybrid approach where we implement table reading and
1122
filtering using the new framework and make the results accessible via a
1123
`planNode` so we can use the existing code for everything else. This would be
1124
sufficient for M1. The next stage would be implementing the mutation aggregators
1125
and refactoring the existing code to allow using them (enabling M2).
1126
1127
### Physical planning
1128
1129
A lot of the decisions in the physical planning stage are "forced" - table
1130
readers are distributed according to ranges, and much of the physical planning
1131
follows from that.
1132
1133
One place where physical planning involves difficult decisions is when
1134
distributing the second stage of an aggregation or join - we could set up any
1135
number of "buckets" (and subsequent flows) on any nodes. E.g. see the `summer`
1136
example. Fortunately there is a simple strategy we can start with - use as many
1137
buckets as input flows and distribute them among the same nodes. This strategy
1138
scales well with the query size: if a query draws data from a single node, we
1139
will do all the aggregation on that node; if a query draws data from many nodes,
1141
1142
We will also support configuring things to minimize the distribution - gettting
1143
everything back on the single gateway node as quickly as possible. This will be
1144
useful to compare with the current "everything on the gateway" approach; it is
1145
also a conservative step that might avoid some problems when distributing
1146
queries between too many nodes.
1147
1148
A "stage 2" refinement would be detecting when a computation (and subsequent
1149
stages) might be cheap enough to not need distribution and automatically switch
1150
to performing the aggregation on the gateway node. Further improvements
1151
(statistics based) can be investigated later.
1152
1153
We should add extended SQL syntax to allow the query author to control some of
1154
these parameters, even if only as a development/testing facility that we don't
1155
advertise.
1156
1157
### Processor infrastructure and implementing processors
1158
1159
This involves building the framework within which we can run processors and
1160
implementing the various processors we need. We can start with the table readers
1161
(enough for M1). If necessary, this work stream can advance faster than the
1162
logical planning stream - we can build processors even if the logical plan isn't
1163
using them yet (as long as there is a good testing framework in place); we can
1164
also potentially use the implementations internally, hidden behind a `planNode`
1165
interface and running non-distributed.
1166
1167
##### Joins
1168
1169
A tangential work stream is to make progress toward supporting joins (initially
1170
non-distributed). This involves building the processor that will be at the heart
1171
of the hash join implementation and integrating that code with the current
1172
`planNode` tree.
1173
1174
### Scheduling
1175
1176
The problem of efficiently queuing and scheduling processors will also involve a
1177
long tail of improvements. But we can start with a basic implementation using
1178
simple strategies:
1179
- the queue ordering is by transactions; we don't order individual processors
1180
- limit the number of transactions for which we run processors at any one time;
1181
we can also limit the total number of processors running at any one time, as
1182
long as we allow all the processors needed for at least one txn
1183
- the txn queuing order is a function of the txn timestamp and its priority,
1184
allowing the nodes to automatically agree on the relative ordering of
1185
transactions, eliminating deadlock scenarios (example of a deadlock: txn A
1186
has some processors running on node 1, and some processors on node 2 that are
1187
queued behind running processors of txn B; and txn B also has some processors
1188
that are queued behind txn A on node 1)
1189
1190
We won't need anything fancier in this area to reach M1 and M2.
1191
1192
### KV integration
1193
1194
We do not propose introducing any new KV Get/Put APIs. The current APIs are to
1195
be used; we simply rely on the fact that when run from the lease holder node they will
1196
be faster as the work they do is local.
1197
1198
However, we still require some integration with the KV layer:
1199
1200
1. Range information lookup
1201
1202
At the physical planning stage we need to break up key spans into ranges and
1203
determine who is the lease holder for each range. We may also use range info at the
1204
logical planning phase to help estimate table sizes (for index selection,
1205
join order, etc). The KV layer already has a range cache that maintains this
1206
information, but we will need to make changes to be more aggressive in terms
1207
of how much information we maintain, and how we invalidate/update it.
1208
1209
2. Distributed reads
1210
1211
There is very little in the KV layer that needs to change to allow
1212
distributed reads - they are currently prevented only because of a fix
1213
involving detecting aborted transactions (PR #5323).
1214
1215
3. Distributed writes
1216
1217
The txn coordinator currently keeps track of all the modified keys or key
1218
ranges. The new sql distribution layer is designed to allow funneling the
1219
modified key information back to the gateway node (which acts as the txn
1220
coordinator). There will need to be some integration here, to allow us to
1221
pass this information to the KV layer. There are also likely various cases
1222
where checking for error cases must be relaxed.
1223
1224
The details of all these need to be further investigated. Only 1 and 2 are
1225
required for M1; 3 is required for M2.
1226
1227
Another potential improvement is fixing the impedance mismatch between the
1228
`TableReader`, which produces a stream, and the underlying KV range reads,
1229
which do batch reading. Eventually we'll need a streaming reading interface for
1230
range reads, although in the beginning we can use what we have.
1231
1232
## Implementation notes
1233
1234
Capturing various notes and suggestions.
1235
1236
#### Test infrastructure
1237
1238
Either extend the logictest framework to allow specification of additional
1239
system attributes, or create new framework. We must have tests that can control
1240
various settings (number of nodes, range distribution etc) and examine the
1241
resulting query plans.
1242
1243
#### Visualisation/tracing
1244
1245
Detailed information about logical and physical plans must be available, as well
1246
as detailed traces for all phases of queries, including execution timings,
1247
stats, etc.
1248
1249
At the SQL we will have to present data, plans in textual representations. One
1250
idea to help with visualisation is to build a small web page where we can paste
1251
a textual representation to get a graphical display.
1252
1253
# Alternate approaches considered (and rejected)
1254
1255
We outline a few different approaches we considered but eventually decided
1256
against.
1257
1258
## More logic in the KV layer
1259
1260
In this approach we would build more intelligence in the KV layer. It would
1261
understand rows, and it would be able to process expressions (either SQL
1262
expressions, or some kind of simplified language, e.g. string based).
1263
1264
### Complexity
1265
1266
Most of the complexity of this approach is around building APIs and support for
1267
expressions. For full SQL expressions, we would need a KV-level language that
1268
is able to read and modify SQL values without being part of the SQL layer. This
1269
would mean a compiler able to translate SQL expressions to programs in a
1270
KV-level VM that perform the SQL-to-bytes and bytes-to-SQL translations
1271
explicitly (i.e. translate/migrate our data encoding routines from Go to that
1272
KV-level VM's instructions).
1273
1274
### Applicability
1275
1276
The applicability of this technique is limited: it would work well for
1277
filtering and possibly for remote-side updates, but it is hard to imagine
1278
building the logic necessary for distributed SQL operations (joins,
1279
aggregation) into the KV layer.
1280
1281
It seems that if we want to meet all described goals, we need to make use of a
1282
smarter approach. With this in mind, expending any effort toward this approach
1283
seems wasteful at this point in time. We may want to implement some of these
1284
ideas in the future if it helps make things more efficient, but for now we
1285
should focus on initial steps towards a more encompassing solution.
1286
1287
1288
## SQL2SQL: Distributed SQL layer
1289
1290
In this approach we would build a distributed SQL layer, where the SQL layer of
1291
a node can make requests to the SQL layer of any other node. The SQL layer
1292
would "peek" into the range information in the KV layer to decide how to split
1293
the workload so that data is processed by the respective raft range lease holders.
1294
Achieving a correct distribution to range lease holders would not be necessary for
1296
layer to synchronize with range splits/merges or lease holdership changes during an
1297
SQL operation.
1298
1299
1300
### Sample high-level query flows
1301
1302
Sample flow of a “simple” query (select or update with filter):
1303
1304
| **Node A** | **Node B** | **Node C** | **Node D** |
1305
|--------------------------------------------|--------------|------------|------------|
1306
| Receives statement | | | |
1307
| Finds that the table data spans three ranges on **B**, **C**, **D** | | |
1308
| Sends scan requests to **B**, **C**, **D** | | | |
1309
| | Starts scan (w/filtering, updates) | Starts scan (w/filtering, updates) | Starts scan (w/filtering, updates) |
1310
| | Sends results back to **A** | Sends results back to **A** | Sends results back to **A** |
1311
| Aggregates and returns results. | | | |
1312
1313
Sample flow for a hash-join:
1314
1315
| **Node A** | **Node B** | **Node C** | **Node D** |
1316
|--------------------------------------------|--------------|------------|------------|
1317
| Receives statement | | | |
1318
| Finds that the table data spans three ranges on **B**, **C**, **D** | | |
1319
| Sets up 3 join buckets on B, C, D | | | |
1320
| | Expects join data for bucket 0 | Expects join data for bucket 1 | Expects join data for bucket 2 |
1321
| Sends scan requests to **B**, **C**, **D** | | | |
1322
| | Starts scan (w/ filtering). Results are sent to the three buckets in batches | Starts scan (w/ filtering) Results are sent to the three buckets in batches | Starts scan (w/ filtering). Results are sent to the three buckets in batches |
1323
| | Tells **A** scan is finished | Tells **A** scan is finished | Tells **A** scan is finished |
1324
| Sends finalize requests to the buckets | | | |
1325
| | Sends bucket data to **A** | Sends bucket data to **A** | Sends bucket data to **A** |
1326
| Returns results | | | |
1327
1328
### Complexity
1329
1330
We would need to build new infrastructure and APIs for SQL-to-SQL. The APIs
1331
would need to support SQL expressions, either as SQL strings (which requires
1332
each node to re-parse expressions) or a more efficient serialization of ASTs.
1333
1334
The APIs also need to include information about what key ranges the request
1335
should be restricted to (so that a node processes the keys that it is lease holder
1336
for - or at least was, at the time when we started the operation). Since tables
1337
can span many raft ranges, this information can include a large number of
1338
disjoint key ranges.
1339
1340
The design should not be rigid on the assumption that for any key there is a
1341
single node with "fast" access to that key. In the future we may implement
1342
consensus algorithms like EPaxos which allow operations to happen directly on
1343
the replicas, giving us multiple choices for how to distribute an operation.
1344
1345
Finally, the APIs must be designed to allow overlap between processing, network
1346
transfer, and storage operations - it should be possible to stream results
1347
before all of them are available (F1 goes as far as streaming results
1348
out-of-order as they become available from storage).
1349
1350
### Applicability
1351
1352
This general approach can be used for distributed SQL operations as well as
1353
remote-side filtering and updates. The main drawback of this approach is that it
1354
is very general and not prescriptive on how to build reusable pieces of
1355
functionality. It is not clear how we could break apart the work in modular
1356
pieces, and it has the potential of evolving into a monster of unmanageable
1357
complexity.
1358
1359
1360
## Spark: Compiling SQL into a data-parallel language running on top of a distributed-execution runtime
1361
1362
The idea here is to introduce a new system - an execution environment for
1363
distributed computation. The computations use a programming model like M/R, or
1364
more pipeline stuff - Spark, or Google's [Dataflow][1] (parts of it are an
1365
Apache project that can run on top of other execution environments - e.g.
1367
1368
In these models, you think about arrays of data, or maps on which you can
1369
operate in parallel. The storage for these is distributed. And all you do is
1370
operation on these arrays or maps - sort them, group them by key, transform
1371
them, filter them. You can also operate on pairs of these datasets to do joins.
1372
1373
These models try to have *a)* smart compilers that do symbolic execution, e.g.
1374
fuse as many operations together as possible - `map(f, map(g, dataset)) == map(f
1375
● g, dataset)` and *b)* dynamic runtimes. The runtimes probably look at operations
1376
after their input have been at least partially computed and decide which nodes
1377
participate in this current operation based on who has the input and who needs
1378
the output. And maybe some of this work has already been done for us in one of
1379
these open source projects.
1380
1381
The idea would be to compile SQL into this sort of language, considering that we
1382
start execution with one big sorted map as a dataset, and run it.If the
1383
execution environment is good, it takes advantage of the data topology. This is
1384
different from "distributed sql" because *a)* the execution environment is
1385
dynamic, so you don't need to come up with an execution plan up front that says
1386
what node is gonna issues what command to what other node and *b)* data can be
1387
pushed from one node to another, not just pulled.
1388
1389
We can start small - no distributed runtime, just filtering for `SELECTS` and
1390
filtering with side effects for `UPDATE, DELETE, INSERT FROM SELECT`. But we
1391
build this outside of KV; we build it on top of KV (these programs call into KV,
1392
as opposed to KV calling a filtering callback for every k/v or row).
1393
1394
[1]: https://cloud.google.com/dataflow/model/programming-model
1395
1396
### Sample program
1397
1398
Here's a quick sketch of a program that does remote-side filtering and deletion
1399
for a table with an index.
1400
1401
Written in a language for (what I imagine to be) Spark-like parallel operations.
1402
The code is pretty tailored to this particular table and this particular query
1403
(which is a good thing). The idea of the exercise is to see if it'd be feasible
1404
at all to generate such a thing, assuming we had a way to execute it.
1405
1406
The language has some data types, notably maps and tuples, besides the
1407
distributed maps that the computation is based on. It interfaces with KV through
1408
some `builtin::` functions.
1409
1410
It starts with a map corresponding to the KV map, and then munges and aggregates
1411
the keys to form a map of "rows", and then generates the delete KV commands.
1412
1413
The structure of the computation would stay the same and the code would be a lot
1414
shorter if it weren't tailored to this particular table, and instead it used
1415
generic built-in functions to encode and decode primary key keys and index keys.
1416
1417
```sql
1418
TABLE t (
1419
id1 int
1420
id2 string
1421
a string
1422
b int DEFAULT NULL
1423
1424
PRIMARY KEY id1, id2
1425
INDEX foo (id1, b)
1426
)
1427
1428
DELETE FROM t WHERE id1 >= 100 AND id2 < 200 AND len(id2) == 5 and b == 77
1429
```
1430
1431
```go
1432
func runQuery() {
1433
// raw => Map<string,string>. The key is a primary key string - table id, id1,
1434
// id2, col id. This map is also sorted.
1435
raw = builtin::readRange("t/primary_key/100", "t/primary_key/200")
1436
1437
// m1 => Map<(int, string), (int, string)>. This map is also sorted because the
1438
// input is sorted and the function maintains sorting.
1439
m1 = Map(raw, transformPK).
1440
1441
// Now build something resembling SQL rows. Since m1 is sorted, ReduceByKey is
1442
// a simple sequential scan of m1.
1443
// m2 => Map<(int, string), Map<colId, val>>. These are the rows.
1445
1446
// afterFilter => Map<(int, string), Map<colId, val>>. Like m2, but only the rows that passed the filter
1447
afterFilter = Map(m2, filter)
1448
1449
// now we batch up all delete commands, for the primary key (one KV command
1450
// per SQL column), and for the indexes (one KV command per SQL row)
1451
Map(afterFilter, deletePK)
1452
Map(afterFilter, deleteIndexFoo)
1453
1454
// return the number of rows affected
1455
return len(afterFilter)
1456
}
1457
1458
func transformPK(k, v) {
1459
#pragma maintainsSort // important, keys remain sorted. So future
1460
// reduce-by-key operations are efficient
1461
id1, id2, colId = breakPrimaryKey(k)
1462
return (key: {id1, id2}, value: (colId, v))
1463
}
1464
1465
func breakPrimaryKey(k) {
1466
// remove table id and the col_id
1467
tableId, remaining = consumeInt(k)
1468
id1, remaining = consumeInt(remaining)
1469
id2, remaining = consumeInt(remaining)
1470
colId = consumeInt(remaining)
1471
returns (id1, id2, colId)
1472
}
1473
1474
func BuildColMap(k, val) {
1475
colId, originalVal = val // unpack
1476
a, remaining = consumeInt(originalVal)
1477
b, remaining = consumeString(remaining)
1478
// output produces a result. Can appear 0, 1 or more times.
1479
output (k, {'colId': colId, 'a': a, 'b': b})
1480
}
1481
1482
func Filter(k, v) {
1483
// id1 >= 100 AND id2 < 200 AND len(id2) == 5 and b == 77
1484
id1, id2 = k
1485
if len(id2) == 5 && v.getWithDefault('b', NULL) == 77 {
1486
output (k, v)
1487
}
1488
// if filter doesn't pass, we don't output anything
1489
}
1490
1491
1492
func deletePK(k, v) {
1493
id1, id2 = k
1494
// delete KV row for column a
1495
builtIn::delete(makePK(id1, id2, 'a'))
1496
// delete KV row for column b, if it exists
1497
if v.hasKey('b') {
1498
builtIn::delete(makePK(id1, id2, 'b'))
1499
}
1500
}
1501
1502
func deleteIndexFoo(k, v) {
1503
id1, id2 = k
1504
b = v.getWithDefault('b', NULL)
1506
builtIn::delete(makeIndex(id1, b))
1507
}
1508
```
1509
1510
### Complexity
1511
1512
This approach involves building the most machinery; it is probably overkill
1513
unless we want to use that machinery in other ways than SQL.
1514
1515
# Unresolved questions
1516
1517
The question of what unresolved questions there are is, as of yet, unresolved.