github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/docs/RFCS/20150811_replica_batch.md (about) 1 - Feature Name: replica_batch 2 - Status: completed 3 - Start Date: 2015-08-11 4 - RFC PR: [#2340](https://github.com/cockroachdb/cockroach/pull/2340) 5 - Cockroach Issue: 6 7 # Summary 8 9 Assuming [#1998](https://github.com/cockroachdb/cockroach/pull/1998) in place, replace `roachpb.Request` by `roachpb.BatchRequest` throughout most of the main execution path starting in `Store.ExecuteCmd()`. 10 11 # Motivation 12 13 [#1998](https://github.com/cockroachdb/cockroach/pull/1998) introduces gateway 14 changes after the implementation of which only `BatchRequest` is received by 15 `(*Store).ExecuteCmd()`. The changes described here allow `BatchRequest` to 16 be submitted to Raft and executed in bulk, which should give significant 17 performance improvements, in particular due to the lower amount of Raft-related 18 round-trip delays. 19 20 # Detailed design 21 22 The required changes to the code are plenty. 23 24 The sections below follow the main request path and outline the necessary 25 changes in each. 26 27 ## Store.ExecuteCmd 28 29 This carries out 30 31 * request verification and clock updates, which generalize in a relatively 32 straightforward manner. 33 * the store retry loop, and particularly, write intent error handling. This 34 depends on the request which ran into the intent (and whether it's a read 35 or write), so `Replica` needs to return not only an error, but also, for 36 example, the associated index of the request in the `Batch`. 37 38 ## Replica.AddCmd 39 40 Control flow currently splits up into read, write and admin paths. For simplicity, 41 allowing `Admin` commands only as single elements of a `Batch`, we can keep the 42 admin path intact. Regarding the read/write path, there are two options: 43 44 * splitting the `Batch` into sub-batches which are completely read or write only. 45 This has the advantage of possibly less changes in the read and write paths, 46 but requires multiple Raft proposals when reads and writes mix (in the worst 47 case scenario, `len(Batch)-1` of them). Having to bookkeep multiple Raft 48 proposals for a single `Batch` is a disadvantage and raises questions about 49 atomicity and response cache handling. 50 * keeping the Batch whole, but merging `(*Replica).add{ReadOnly,Write}Cmd`. 51 The idea is that if we need to go through Raft (i.e. if the `Batch` contains 52 at least one write) anyway, we propose the whole `Batch` and satisfy the 53 reads through `Raft`. If the `Batch` is read-only, it executes directly. It 54 should be possible to refactor such that the code which executes reads is 55 shared. 56 57 Overall, option two seems preferable. As a byproduct, it would make `INCONSISTENT` 58 reads consistent for free when they're part of a mutating batch anyways, and 59 (almost) implement `CONSENSUS` reads. 60 61 ### Timestamp Cache and Command Queue 62 63 `(*Replica).{begin,end}Cmd` are changed to operate on `Batch` (instead of 64 `roachpb.RequestHeader`), obviating the `readOnly` flag (which is determined 65 from the request type). The entries are added to the command queue in bulk 66 so that overlaps are resolved gracefully: reading `[a,c)` and then writing 67 `b` should add `[a,b)` and `[b\x00,c)` for reading, and `b` for writing. 68 There is likely some potential for refactoring with `intersectIntents()`. 69 70 Timestamp cache handling is straightforward, except when commands within 71 the same `Batch` overlap: In that case, if the former is a read and the latter 72 a write, the latter command's timestamp must be moved past the former. 73 74 Note that there is some special-casing regarding the write timestamp cache with 75 `Transaction`s: transactional writes are still carried out even if they're 76 incompatible with prior writes' timestamps. This allows `Txn`s to write over 77 their own data, and to attempt to push in more cases. 78 79 ## (\*Replica).proposeRaftCommand 80 81 no noteworthy changes. 82 83 ## (\*Replica).processRaftCommand 84 85 `roachpb.ResponseWithError` changes to `roachpb.ResponsesWithError` which also 86 contains the index of the first error, if any (or, alternatively, by 87 convention the error occurred at index `len(rwe.Responses)`). 88 89 ## (\*Replica).applyRaftCommand 90 91 Returns `[]roachpb.Response`, one for each successfully executed request (in 92 `Batch` order). 93 94 ## (\*Replica).applyRaftCommandInBatch 95 96 same as `applyRaftCommand`. This actually unwinds the `Batch`, calling 97 `(*Replica).executeCmd` sequentially until done or an error occurs. 98 99 # Drawbacks 100 101 # Alternatives 102 103 # Unresolved questions