github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/docs/tech-notes/txn_coord_sender.md (about) 1 # Transactional interface between SQL and KV (and TxnCoordSender) 2 3 Original authors: knz, andrei 4 5 This tech note explains how the SQL/KV interface currently works, up 6 to the level of detail necessary to understand the processing of 7 batches, error handling and SQL savepoints, to understand bug findings 8 in this area and to participate in design discussions. 9 10 Table of contents: 11 12 - [Introduction](#Introduction) 13 - [client.Txn and RootTxns](#clientTxn-and-RootTxns) 14 - [LeafTxns and txn state repatriation](#LeafTxns-and-txn-state-repatriation) 15 - [client.Txn, meta and TxnCoordSender](#clientTxn-meta-and-TxnCoordSender) 16 - [Interceptors: between TxnCoordSender and DistSender](#Interceptors-between-TxnCoordSender-and-DistSender) 17 - [TxnCoordSender state](#TxnCoordSender-state) 18 - [Summary of the all-is-well path](#Summary-of-the-all-is-well-path) 19 - [Error handling in TxnCoordSender](#Error-handling-in-TxnCoordSender) 20 - [Error handling with LeafTxns](#Error-handling-with-LeafTxns) 21 - [Concurrency between root and leaf](#Concurrency-between-roof-and-leaf) 22 - [KV sequence numbers](#KV-sequence-numbers) 23 - [Seqnum consistency across TxnCoordSenders](#Seqnum-consistency-across-TxnCoordSenders) 24 25 ## Introduction 26 27 CockroachDB's at a high level is architected into the following layers: 28 29 1. SQL (incl. pgwire, SQL state machine, planning and execution) 30 2. Transactional (YOU ARE HERE) 31 3. Distribution (incl. range leasing, rebalancing) 32 4. Replication (incl. Raft, replica lifecycle) 33 5. Storage (incl. engines: RocksDB, Pebble) 34 35 This tech note pertains to level 2 in this list and especially the 36 boundary between levels 1 and 2. 37 38 Conceptually, the "transactional layer" offers an API to the SQL layer 39 which enables it to consider the CockroachDB cluster as a 40 transactional KV store. The API offers ways to "define a KV 41 transaction" and "execute KV operations" (and get results and errors), 42 and maintains the lifecycle of KV transaction objects. 43 44 In particular, it is responsible for a number of optimizations relating 45 to transient fault recovery (incl. implicit/automatic retries inside 46 that layer, invisible to SQL) and transaction conflict resolution 47 (incl. implicit/automatic txn reordering by updating timestamps). 48 49 Its other boundary, between levels 2 and 3 above, is another API 50 offered by level 3 called `DistSender`. That level also allows the 51 levels above it to "execute KV operations" but it has very little 52 logic for error recovery and does not maintain KV transaction state 53 itself. (In fact, the levels 3 and below are mostly state-less with 54 regards to SQL client connections. The transactional layer is the last 55 level that manages internal state on behalf of a single SQL client.) 56 57 The transactional layer's role is thus to *translate* requests coming 58 from above (SQL) to go below (DistSender), performing optimizations 59 and error recovery during that process. 60 61 Since the interactions are relatively complex, the remainder of the 62 tech note introduces the concepts incrementally. The explanations at 63 the beginning are thus somewhat inaccurate, merely providing an upramp 64 to understanding for the reader. 65 66 Interestingly, in the context of the transactional layer, the word 67 **"client"** designates the local SQL layer (e.g. a SQL executor or a 68 distsql processor), not the remote client app. Likewise, the word 69 **"server"** designates the local `DistSender` on each node, not the 70 CockroachDB node as a whole. This differs from the terminology in 71 each other layer. For example, in SQL: "client" = remote client app, 72 "server" = gateway or distsql execution server; in replication: 73 "client" = layer 2, "server" = leaseholder for some range. Depending 74 on the reader's own technical background, some caution will be needed 75 while reading the rest of this tech note. 76 77 ## client.Txn and RootTxns 78 79 The first two actors in this explanation are: 80 81 - the SQL executor, which organizes the state of the SQL transaction 82 and the sequencing of statements on the gateway. 83 - a transaction object called "RootTxn" (the name will be motivated 84 later), that exists on the SQL gateway, and which stores the "main" 85 state of the SQL/KV transaction - for example whether it's aborted, 86 waiting for a client-side retry, etc. 87 88 A simplified view of the interaction between the two is as follows: 89 90  91 92 - the SQL executor instantiates an object of Go type `*client.Txn` with 93 its type set to `RootTxn` (hence the name) 94 - during query execution, the SQL exec code (on the gateway, we ignore 95 distributed execution for now) uses the `Run()` method on that 96 object to run KV operations. 97 - "under the hood" the RootTxn translates the Run() calls into 98 BatchRequests into the cluster, and translates the 99 BatchResponses back into updates into the `client.Batch` object 100 provided by the SQL code. 101 - at the end of the SQL transaction (either commit or rollback, or 102 close on error), a call is made to the RootTxn to 103 finalize its state. 104 105 ## LeafTxns and txn state repatriation 106 107 When a query becomes distributed, we want other nodes to be able to 108 run KV operations "on behalf" of the main SQL transaction running on 109 the gateway. This needs the same txn ID, timestamp, list of write 110 intents, etc, so we can't just create a fresh new RootTxn on each node 111 where a distsql processor runs. 112 113 Instead, there is some new complexity, involving three new actors: 114 115 - one or more distSQL servers running on other nodes than the 116 gateway, which receive requests from the gateway to execute 117 work on behalf of a SQL session running there. 118 - distSQL units of work, called "flows", which are specified 119 to run some processing code and, relevant here, operate 120 using... 121 - ... another transaction object called "LeafTxn", which contains 122 a copy of many fields of the original RootTxn and is 123 able to run KV **read** operations. 124 125 This works as follows: 126 127  128 129 - the SQL executor instantiates the RootTxn as usual. 130 - when a distributed query is about to start, the distsql 131 execution code pulls out a struct from the RootTxn 132 called "LeafTxnInputState". This contains e.g. the txn ID, 133 timestamp and write intents as outlined above. 134 - the trimmed meta struct is sent along with the flow 135 request to a remote distsql server. 136 - on the other node, the distsql server instantiates the 137 `LeafTxn` object using the provided meta struct as input. 138 - the distsql processor(s) (e.g a table reader) then uses 139 the LeafTxn to run KV batches. 140 - when query execution completes, the distsql processor 141 extracts a similar state struct off the LeafTxn 142 called `LeafTxnFinalState` and the 143 result is repatriated on the gateway when the 144 flow is shut down. 145 - on the gateway, repatriated LeafTxn state structs 146 are merged into the RootTxn using `UpdateRootWithLeafFinalState()`. 147 - on the gateway, any error produced by a LeafTxn is also "ingested" 148 in the RootTxn to perform additional error recovery and clean-up, 149 using `UpdateStateOnRemoteRetryableErr()`. 150 151 Why do we need to bring back state from a LeafTxn into a RootTxn? 152 153 There are many uses for this data repatriation, not 154 all will be detailed further here. 155 156 One use which is good to explain why the repatriation is _necessary_ 157 is that of refresh spans: as KV reads are issued by the LeafTxn, it 158 populates a list of refresh spans. If we did not repatriate these 159 spans, then a subsequent txn conflict check would not detect that 160 reads performed by the LeafTxn are stale and incorrectly decide to 161 refresh the txn (bump its commit ts into the future and retry 162 automatically, instead of pushing the error back to the client). 163 164 Another use of repatriation that's not strictly necessary but is 165 nevertheless a useful optimization, is the case when the transaction 166 is aborted concurrently (e.g. if a deadlock was detected by another 167 txn). If the KV reads done on behalf of the LeafTxn detect the txn 168 record to become aborted, this new state will be repatriated and the 169 RootTxn will know that the entire KV txn has become aborted. This is 170 faster than letting the RootTxn discover this state later at the first 171 next KV operation launched on its behalf. 172 173 Related issues: 174 https://github.com/cockroachdb/cockroach/issues/41222 175 https://github.com/cockroachdb/cockroach/issues/41992 176 177 ## client.Txn, meta and TxnCoordSender 178 179 The two sections above used a simplified picture using 180 a single "transaction object". 181 182 In truth, the [type 183 `*client.Txn`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/internal/client/txn.go#L32) 184 is merely a thin facade for the SQL client. It contains, between other things: 185 186 - a type tag (RootTxn/LeafTxn) 187 - a reference of [interface type 188 `TxnSender`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/internal/client/sender.go#L57), 189 which abstracts the `Send()` operation to send batch requests to the 190 rest of the cluster. 191 192 In particular it does not contain the "main" txn payload including 193 commit timestamp, intents, etc. 194 195 Where is that payload then? Also, where are the refresh spans and 196 other in-flight txn properties stored? 197 198 The object referenced by `*client.Txn` is an instance of a coordinator 199 component called the "TxnCoordSender" of [type 200 `kv.TxnCoordSender`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_coord_sender.go#L104). 201 202 The TxnCoordSender (hereafter abbreviated TCS), as its name implies, 203 is in charge of maintaining the state of the txn at the top of the KV 204 layer, and is in charge of coordinating the distribution of KV batches 205 to layer underneath together with error handling, txn conflict 206 management, etc. 207 208 The TCS is also, itself, a rather thin data structure. 209 210 Its main payload is what the KV team actually calls the "txn object", 211 of [type 212 `roachpb.Transaction`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/roachpb/data.proto#L302), 213 which in turn also 214 [contains](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/roachpb/data.proto#L310) 215 a copy of the "txn meta" object, of [type 216 `enginepb.TxnMeta`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/storage/engine/enginepb/mvcc3.proto#L18). 217 218 The separation of purpose between `roachpb.Transaction` and 219 `enginepb.TxnMeta` is not further relevant in this RFC, and we will just 220 call them collectively "the txn object". 221 222 With this in place, the interaction goes roughly as follows: 223 224  225 226 The txn object is sent along in the header of every `BatchRequest` 227 produced by TCS while it processes a `client.Batch` from 228 SQL or other KV clients. This is passed along the 229 transaction/replication/storage boundaries and the low-level MVCC code in 230 storage has access to (a sufficient part of) the txn object during 231 processing of each single KV operation. 232 233 Additionally, the execution of low-level KV operations can _update_ 234 their copy of (parts of) the txn object. This will populate e.g. the 235 list of observed timestamps, used for later txn conflict resolution. 236 The resulting txn state then flows back to TCS in the 237 header of every `BatchResponse`. Upon receiving a BatchResponse, the 238 TCS *merges* the received txn object in the response with 239 the txn object it already has, using the `txn.Update()` method. 240 241 ## Interceptors: between TxnCoordSender and DistSender 242 243 The explanation above suggested that TCS sends 244 BatchRequests to "the cluster". 245 246 In truth, "the cluster" is the entry point of the distribution layer, 247 the overall architectural layer immediately under the transaction 248 layer in CockroachDb. Its entry point is an object called 249 [`DistSender`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/dist_sender.go), 250 of which there is one instance per node. 251 252 The interface between TCS and DistSender is an interface 253 called `client.Sender` which defines a method `Send(BatchRequest) 254 (BatchResponse, error)`. 255 256 So _conceptually_, we have something like this in the code: 257 258  259 260 However, there's a little more complexity hidden in there. If we had a 261 direct call from `TCS.Send()` into `DistSender.Send()`, 262 then a single blob of code in TCS itself would need to deal 263 with all the complexity of txn pipelining, parallel commits, etc. 264 265 To facilitate reasoning about the code and to ease maintenance, the 266 txn management logic is split away from TCS itself, and 267 across multiple other components arranged in a _pipeline_ placed between 268 TCS and DistSender. Each stage of this pipeline is called 269 an "interceptor" and responsible for a single aspect of txn 270 coordination. Each also contains additional local state. 271 272 Two example interceptors that happen to be relevant to this RFC are: 273 274 - the 275 [`txnSpanRefresher`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_interceptor_span_refresher.go#L103), 276 which contains and manages the read and write refresh spans already 277 mentioned above. 278 - the 279 [`txnSeqNumAllocator`](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_interceptor_seq_num_allocator.go#L58), 280 which assigns [sequence numbers](#KV-sequence-numbers) to individual KV 281 operations in batches. 282 283 Thus, in reality, the call stack looks more like this: 284 285  286 287 TCSs allocated for RootTxns use [the full pipeline of 288 interceptors (6 of them as of this 289 writing)](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_coord_sender.go#L529), 290 whereas LeafTxns, which only handle read requests, use [only a 291 subset](https://github.com/cockroachdb/cockroach/blob/a57647381a4714b48f6ec6dec0bf766eaa6746dd/pkg/kv/txn_coord_sender.go#L556). 292 293 ## TxnCoordSender state 294 295 The overall "current state" of a TCS is thus distributed 296 between various Go structs: 297 298 - the txn object (`roachpb.Transaction`), 299 - the set of its interceptors (each interceptor contains a portion of the txncoordsender state 300 sufficient and necessary for its local processing), 301 - its "own" very few fields, including a summary of the lifecycle of 302 the txn object called `txnState` (relevant to this RFC, we'll come 303 back to this later). 304 305 This overall state is a native Go struct and not a protobuf. However, 306 [as we've seen above](#LeafTxns-and-txn-state-repatriation) distributed execution needs to take the 307 "current state" of a RootTxn and carry it over to another node to 308 build a LeafTxn. 309 310 For this purpose, a separate protobuf message `LeafTxnInputState` is 311 defined. The TCS's `GetLeafTxnInputState()` method initially populates 312 it by asking every interceptor in turn to write its portion of the 313 state into it. 314 315 Conversely, when the state of a LeafTxn is repatriated and to be 316 "merged" into the RootTxn, the `UpdateRootFromLeafFinalState()` method 317 uses the `Update()` method on the `roachpb.Transaction` sub-object 318 (which merges the state of the txn object itself) then asks every 319 interceptor, in turn, to collect bits of state it may be interested to 320 merge in too. 321 322 For example, that's where the RootTxn's txnSpanRefresher interceptor 323 picks up the spans accumulated in the LeafTxn. 324 325 ## Summary of the all-is-well path 326 327 To summarize the previous sections, the SQL/KV interface 328 involves the following actors: 329 330 - a `client.Txn` object, which doesn't know much, other than... 331 - a reference to a `TCS`, which stores: 332 - (a copy of) the current txn object incl `roachpb.Transaction` and `enginepb.TxnMeta`, 333 - a set of txn interceptors, each with its own local state, 334 - at the end of the interceptor pipeline, a reference to the local node's `DistSender` object, 335 - a little additional TCS local state including a "txn status" field called `txnState`. 336 337 When a KV batch request arrives from SQL through `client.Txn`, it is 338 passed through TCS, the stack of interceptors, delivered to 339 DistSender, and the responses flow back up the same path. 340 341 Now on the next question: *What of errors?* 342 343 ## Error handling in TxnCoordSender 344 345 For simplicity in this section, we'll start with the simple 346 case of a RootTxn without any LeafTxn. 347 348 When an error is encountered either in DistSender or "underneath" 349 (remote replicas etc), it flows back through the interceptors back 350 into the TCS's `Send()` method. 351 352 Some interceptors peek into the error object and update their local 353 state. Some of them (like the `txnSpanRefresher`) fully absorb the 354 error to turn it into a non-error. 355 356 Additionally, some interceptors can generate errors of their own 357 either "on the way in" (towards DistSender), which causes a shortcut 358 to the return path; or "on the way out" (alongside a BatchResponse). 359 360 When `(TCS).Send()` receives an error from the chain 361 of interceptors, it then separates between 7 kinds of errors, currently 362 split into three groups: 363 364 - sub-group 1: *same-TCS recoverable* errors, which cause the TCS to 365 perform partial or full error recovery. 366 367 This group contains 3 kinds of errors: 368 369 1. *recoverable errors with in-place recovery*, where the TCS 370 will handle the error internally, then retry the operation 371 in a way that's invisible to the higher levels. 372 In this case, the txn object remains "live" and 373 its "identity" (ID, epoch) is unchanged. 374 375 For example, txn refreshes are processed automatically 376 in this way. 377 378 2. *recoverable errors with txn restart*, where the 379 TCS resets the txn object to a state where the 380 client (the SQL layer) can restart the operation, 381 or tell the client to attempt the operation again 382 (client-side retries). In this case, 383 the txn object remains "live" but its identity 384 (epoch) changes immediately. 385 386 Example sequence diagram in the case of a recoverable error with txn 387 restart: 388 389  390 391 3. *deferred retry errors*, where the TCS remember the error 392 has occurred but pretends the operation succeeded for the benefit 393 of the (SQL) client. The error is only reported at 394 the end of the SQL txn, where the client is requested to 395 perform a client-side retry. 396 397 This is currently used only for `WriteTooOldError`. 398 399 - sub-group 2: *new-TCS recoverable* errors, which cause the TCS to 400 become "trashed" (and unusable), but where the `*client.Txn` can 401 continue/restart with a new TCS: 402 403 1. *retryable transaction aborts* (`TransactionRetryWithProtoRefreshError` 404 with a `TransactionAbortedError` payload), which occurs when the KV 405 transaction gets aborted by some other transaction. This happens in case of 406 deadlock, or in case the coordinator fails to heartbeat the txn record for a 407 few seconds and another transaction is blocked on one of our intents. Faux 408 `TransactionAbortedErrors` can also happen for transactions that straddle a 409 lease transfer (the new leaseholder is not able to verify that the transaction 410 had not been aborted by someone else before the lease transfer because we lost 411 the information in the old timestamp cache). 412 413 For these errors, the TCS becomes unusable but the `*client.Txn` 414 immediately replaces the TCS by a fresh one, see 415 `UpdateStateOnRemoteRetryableErr()` and 416 `replaceSenderIfTxnAbortedLocked()` in `client/txn.go`. 417 418 - sub-group 3: *unrecoverable* errors, which cause both the TCS and 419 `*client.Txn` to become "trashed". 420 421 This group contains 3 kinds of errors: 422 423 1. *permanent transaction errors*, which 424 occurs when the transaction encounters a permanent unrecoverable 425 error typically due to client logic error (e.g. AOST read under GC). 426 427 2. *transient processing errors*, for which it is certain that 428 further processing is theoretically still possible after 429 the error occurs. For example, attempting to read data using 430 a historical timestamp that has already been garbage collected, 431 `CPut` condition failure, transient network error on the read path, etc. 432 433 3. *unhandled errors*, for which it is not certain that further 434 processing is safe or sound (or where we haven't yet proven that 435 it is). For example, "ambiguous result" errors, "store 436 unavailable" and internal assertion errors fall in this category. 437 438 When an unrecoverable error occurs, the TCS changes its `txnState` to 439 `txnError`. After this happens, any further attempt to use the 440 TCS will be rejected without even attempting further 441 processing. At the SQL level, this is then recognized as a forced txn 442 abort after which only ROLLBACK is accepted (or where COMMIT will 443 produce a "txn is aborted" error). 444 445 Example sequence diagram in the case of an unrecoverable error: 446 447  448 449 Summary table: 450 451 | Group | Error kind | Example | Current recovery | 452 |----------------------|------------------------------------|--------------------------------------------|-----------------------------------------------------------| 453 | same-TCS recoverable | recoverable with in-place recovery | `ReadWithinUncertaintyIntervalError` | internal auto retry, txn identity preserved | 454 | same-TCS recoverable | recoverable with txn restart | commit deadline exceeded error | tell client to retry, reset txn object to new epoch | 455 | same-TCS recoverable | deferred retry | transaction push, write too old (?) | store error state, reveal retry error only at commit time | 456 | new-TCS recoverable | retryable txn aborts | transaction aborted by concurrent txn | TCS becomes unusable but `client.Txn` can resume | 457 | unrecoverable | non-retryable txn aborts | read under GC threshold | hard fail, TCS becomes unusable | 458 | unrecoverable | transient processing errors | CPut condition failure | hard fail, TCS becomes unusable (see below) | 459 | unrecoverable | unhandled errors | store unavailable error, assertion failure | hard fail, TCS becomes unusable | 460 461 The keen reader may wonder why transient processing errors cause the txn 462 object and the TCS to become unusable. Indeed, there is no good reason 463 for that. It is actually poor design, as a SQL client may legitimately want 464 to continue using the txn object after detecting a logical error (eg 465 duplicate row) or transient error (eg network connection reset). **This 466 is to change with the introduction of savepoints.** 467 468 Another important aspect of "recoverable errors with txn restart" and 469 "retryable txn aborts", which will become more noteworthy below, is 470 that the txn object stored inside the TCS may become different "on the 471 way out" (back to client.Txn and the SQL layer) from what it was "on 472 the way in". It is currently the responsibility of the client (SQL 473 layer), which may have its own copy of the txn object, to pick up this 474 change. Cross-references on this point: 475 476 - `(*client.Txn) UpdateStateOnRemoteRetryableErr()` 477 - `(*DistSQLReceiver) Push()` -- `roachpb.ErrPriority(meta.Err) > roachpb.ErrPriority(r.resultWriter.Err())` 478 479 ## Some additional wisdom by Tobias 480 481 > Whenever we extract information from an error that counts as a read, 482 > we have to make sure that read is accounted for by the KV store. For 483 > example, a ConditionFailedError from a CPut is in fact a successful 484 > read; if this isn't accounted for properly at the KV layer (timestamp 485 > cache update), there will be anomalies (CPut has code to do exactly 486 > that here). I generally find it unfortunate that we're relying on 487 > errors to return what are really logical results, and I hope that we 488 > buy out of that as much as we can. For CPut, for example, we'd have 489 > `ConditionalPutResponse` carry a flag that tells us the actual value and 490 > whether a write was carried out. I suspect we're using errors for some 491 > of these only because errors eagerly halt the processing of the 492 > current batch. Continuing past errors generally needs a good amount 493 > of idempotency (for example, getting a timeout during a CPut and 494 > retrying the CPut without seqnos could read-your-own-write). We had no 495 > way of doing that prior to the intent history and seqnos. 496 497 > By the way, in case you haven't stumbled upon this yet, the txn span 498 > refresher (an interceptor inside TCS) has a horrible contract with 499 > `DistSender`, where DistSender returns a partially populated 500 > response batch on errors, from which the refresher then picks out 501 > spans for its refresh set. I'm wondering how this hasn't backfired 502 > yet. 503 504 ## Concurrency between root and leaf 505 506 Today, it is not valid (= KV/SQL protocol violation) to perform KV 507 operations using LeafTxns concurrently with a RootTxn, 508 or use multiple RootTxns for the same txn object side-by-side. 509 510 Note that while the SQL code is architected to take this restriction 511 into account, *it is not currently enforced on the KV side*. We 512 sometimes see bugs (eg #41222 / #41992) occuring because we do not 513 have infrastructure in place to detect violations of this restriction. 514 515 This restriction exists for 3 reasons, one of them actually invalid 516 (a flawed past understanding): 517 518 - KV writes must be uniquely identified, for txn pipelining. Since the 519 identification is currently performed using a single counter in the txn 520 object, there cannot be more than one TCS using this counter at a time. 521 522 Today only RootTxn can process KV writes, so this restricts 523 write concurrency to just 1 RootTxn. Even if LeafTxns had enough 524 complexity to process writes, concurrency would be limited by this 525 counter, until/unless we can guarantee that two separate TCSs 526 generate separate operation identifiers. 527 528 (For example, by combining sequence bits with a TCS ID.) 529 530 Tobias notes: 531 532 > I wonder if we also need their read/write spans to be 533 > non-overlapping. There's all sort of weird stuff if they do 534 > overlap, though maybe it's not actually illegal. Either way, not 535 > something we're going to do today or tomorrow. 536 537 - RootTxns update the txn object during error processing (see previous section). 538 539 If we let the RootTxn process operations and perform error recovery 540 while a LeafTxn is active, we'd need to answer difficult questions. 541 542 Consider the following sequence: 543 544  545 546 In this situation, at t1 the RootTxn handles a retriable error 547 by preparing the next client retry attempt via a new txn object, 548 then at later instant t2 is augmented by a LeafTxn whose 549 state was part of its "past" using the original txn object. 550 How to combine the states from the two "generations" of the txn object? 551 552 To avoid this situation altogether, any use of a LeafTxn 553 comes with a requirement to not use the RootTxn at all 554 while the LeafTxn is active. 555 556 - (Mistaken) Expectation that distributed reads are able to observe 557 concurrent writes on other nodes. 558 559 The KV read and write operations are mutually ordered using seqnums. 560 If we were to expect that a read is able to observe a write 561 performed on a separate node, it would be necessary to synchronize 562 seqnums across nodes for every KV write. This is neither practical 563 nor currently implemented. 564 565 This restriction currently mandates that there be no LeafTxn active 566 while KV writes are being processed by a RootTxn. 567 568 (The restriction is lifted by observing that the expectation is 569 invalid: PostgreSQL semantics require that all reads performed by a 570 mutation statement observe the state of the db prior to any 571 write. So there is no requirement of read-the-writes inside a single 572 SQL statement. The current crdb behavior actually is a bug, our 573 [current halloween 574 problem](https://github.com/cockroachdb/cockroach/issues/28842). 575 Since LeafTxns are re-generated across SQL statements, it's trivial 576 to get the right semantics without a restriction on LeafTxn/RootTxn 577 concurrency.) 578 579 The astute reader may wonder how distSQL deals with the requirement 580 that no LeafTxn be active while a RootTxn is active, or no RootTxn be 581 active while LeafTxns are active. To make this happen there is code in 582 the distsql planner to select whether to use _either_ multiple 583 LeafTxns, one per node / distsql processor, _or_ a single RootTxn, 584 shared by all distsql processors (and forces them to run on the 585 gateway, serially using a single goroutine) (also, this code has bugs. 586 See eg issues #41222 / #41992). 587 588 ## KV sequence numbers 589 590 At the SQL/KV interface, KV operations are associated with *sequence numbers* (seqnums): 591 592 - write operations generate new seqnums, which are stored inside write 593 intents. 594 - read operations operate "at" a particular seqnum: a MVCC read that 595 encounters an intent ignores the values written at later seqnums 596 and returns the most recent value prior to that seqnum instead. 597 - combined read/write operations, like CPut, operate their read part 598 at their write seqnum - 1. 599 600 Today the TCS (the component that receives KV request batches from 601 SQL) is responsible for generating seqnums. 602 603 The seqnum counter's current value is split between three locations: 604 605 - a local variable in one of the interceptors, called `txnSeqNumAllocator` inside the TCS; 606 - the `enginepb.TxnMeta` record, inside the `roachpb.Transaction` held inside the `LeafTxnInputState`. 607 - the `enginepb.TxnMeta` record, inside the `roachpb.Transaction` held inside the header of every executed KV batch. 608 609 These three values are synchronized as follows: 610 611 - The interceptor's counter is incremented for every KV write operation, 612 and the current counter value (with or without increment) is copied to 613 the `Sequence` field in the *request header* of every KV operation 614 flowing through the interceptor. This ensures that: 615 616 - every write gets a new sequence number. 617 - every read gets a copy of the seqnum of the last write. 618 - The `Sequence` field in the request header of individual KV 619 operations is also copied to same-name field in `TxnMeta` of the 620 batch header in certain circumstnaces (most notably by another later 621 interceptor, the `txnPipeliner`) for use during txn conflict 622 resolution and write reordering. 623 - When a TCS is instantiated from a LeafTxnInputState (e.g. forking a 624 RootTxn into a LeafTxn), the counter value from the TxnMeta inside the LeafTxnInputState 625 is copied into the interceptor. 626 - When a LeafTxnInputState is constructed from a TCS, the value is copied 627 from the interceptor. 628 629 Final note: the seqnum is scoped to a current txn epoch. When the 630 epoch field is incremented, the seqnum generator resets to 0. The 631 overall ordering of operation thus also needs to take the epoch into 632 account. 633 634 ## Seqnum consistency across TxnCoordSenders 635 636 The current code was designed with the assumption that a single TCS 637 can issue writes and assign new seqnums to requests. 638 639 Today the code is organized to use only a single RootTxn (and no LeafTxns) for 640 SQL statements that perform writes, so that anything that may 641 update the seqnum ends up running sequentially in a single goroutine. 642 643 It's interesting to consider how this would with LeafTxns if 644 we were to relax the restriction and allow multiple readers 645 with one writer. 646 647 The main mechanism that helps is that without writes, a TCS will 648 continue to assign the same seqnum to every read. A LeafTxn forked 649 from a RootTxn will thus continue to use the seqnum last generated by 650 the RootTxn before it was forked. 651 652 So if we have a SQL sequence like this: 653 654 1. UPDATE 655 2. SELECT 656 657 and the SELECT is distributed with LeafTxns, all the read requests 658 performed on its behalf by other nodes will use the last (epoch, seqnum) 659 generated for UPDATE and thus be able to "observe the writes". 660 661 The astute reader can then consider what happens for the UPDATE 662 itself. What if the UPDATE itself happens to be distributed, with some 663 LeafTxns on other nodes running the "read part" of the UPDATE, 664 and the RootTxn on the gateway issuing the KV operations? 665 666 Here it would also work, as follows: 667 668 - at the beginning of the UPDATE's execution, _before any writes have 669 been issued_, the UPDATE's LeafTxn are forked. This ensures that any 670 further distributed reads by the UPDATE will be using the last 671 (epoch, seqnum) generated by the statement _before_ the UPDATE. 672 - during the UPDATE's execution, the RootTxn increments its counter 673 to perform the mutation. This increase remains invisible 674 to the update's LeafTxns. 675 676 By ensuring that the read path only sees the writes prior to the 677 seqnum at the start of execution, it will be unaffected by subsequent 678 writes. This solves crdb's [current halloween 679 problem](https://github.com/cockroachdb/cockroach/issues/28842). 680