github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/docs/RFCS/20150806_gateway_batch.md (about) 1 - Feature Name: gateway_batch 2 - Status: completed 3 - Start Date: 2015-08-06 4 - RFC PR: [#1998](https://github.com/cockroachdb/cockroach/pull/1998), 5 [#2141](https://github.com/cockroachdb/cockroach/pull/2141), 6 [#2406](https://github.com/cockroachdb/cockroach/pull/2406) 7 - Cockroach Issue: [#2130](https://github.com/cockroachdb/cockroach/issues/2130) 8 9 # Summary 10 11 All requests processed at the client gateway to nodes should be Batch requests. 12 They should be split along range descriptor boundaries by the client gateways 13 prior to sending RPCs to the cluster, replacing the current multi-range logic. 14 This is the gateway portion of a major performance optimization: having Ranges 15 submit whole batches to Raft and execute them atomically within a single write 16 to RocksDB (which in turn allows most transactions to be optimized away). 17 18 # Motivation 19 20 This saves a lot of round-trips for the most common use cases and lays the 21 ground work for major server-side performance improvements. 22 23 # Detailed design 24 25 The proposition below assumes #1988. 26 27 `DistSender` is in charge of breaking a `Batch` up into pieces transparently. 28 That involves going through the batch, looking up the range descriptors for 29 each affected key range, and creating a batch for each each new recipient 30 `Range`. For the order in which they are sent, there's a caveat for 31 transactional batches: 32 33 In the (hopefully) most common case, requests contained within affect only one 34 `Range` and the `Batch` can be sent to that `Range` as a whole. If that is not 35 the case and we end up with several chunks of `Batch`, it's clear that the one 36 containing `EndTransaction` must be sent last. `DistSender` needs to be aware 37 of that. `TxnCoordSender` is changed to ensure that the list of intents from 38 the batch is added to `EndTransaction` in this case. 39 40 Schematically, `DistSender.Send()` would turn into something along the lines of 41 ```go 42 // SendBatch is called by (*TxnCoordSender).Send(). 43 func (ds *DistSender) SendBatch(b *roachpb.BatchRequest) (roachpb.BatchResponse, error) { 44 // Retry loop here. 45 // ... 46 { 47 // Chop up the batch. This is already in a retry loop. 48 var chopped []*roachpb.BatchRequest = ds.Split(b) 49 // `adapt` **must** be idempotent! It may be executed multiple times on 50 // retryable errors, in particular with stale descriptors. 51 // EndTransaction needs to go last; but TxnCoordSender has already 52 // inspected the batch and made sure that the list of intents is set. 53 b = adapt(b) 54 55 // Send the reordered batch requests one by one, much like ds.Send() 56 // ... 57 58 // Recombine the multi-range requests back into one. 59 // ... 60 } 61 return response 62 } 63 ``` 64 65 Another reasonable optimization, namely sending chunks of the batch in 66 parallel, could be carried out similarly: In this case, `adapt` would not only 67 operate on the slice, but return a `Plan` consisting of different stages, each 68 containing multiple chunks (extremes: each stage consisting of one one-element 69 batch = sequential querying as we do it now; one stage of `N` batches (to 70 different ranges) = completely parallel). Non-transactions would go completely 71 parallel, transactions are a bit more involved (going parallel might lay down a 72 lot of intents and do a lot of work for nothing) but will always have a last 73 stage with the chunk containing `EndTransaction`. 74 75 Since in many tests, `TxnCoordSender` has to talk to a `LocalSender` instead 76 of a `DistSender`, we may want to stick to the `client.Sender` interface though 77 and keep the signature `Send(roachpb.Call)`. 78 79 ## Transaction restarts/updates 80 81 Updating the transaction record on certain errors is currently a duty of 82 `TxnCoordSender`, but will have to be carried out by `Store`. Thus, 83 `(*TxnCoordSender).updateResponseTxn()` needs to move and is called in 84 `(*Store).ExecuteCmd()` instead. 85 86 ## Placeholders and Multi-Range requests 87 88 Some result types (notably `Scan`) implement a `Combinable` interface which 89 is used to glue together the subranges of the result contained on different 90 `Range`s. Combining those is easiest if the index of request in a `Batch` is 91 equal across all chunks. For any index with at least two non-placeholders requests, 92 these requests must implement `Combinable`. Similar considerations hold for 93 truncation, which is more complicated: once over limit, all future chunks of 94 the batch should turn that request into a placeholder to avoid querying 95 unnecessary further data. 96 97 Therefore, when we split a batch, each new batch has the same size as the 98 original batch, interspersed with placeholder values (`nil`?) as necessary. Each 99 request has the same index in the new batch as it had in the original batch. 100 Most indexes will have a non-nil value in exactly one batch, but a request that 101 spans multiple ranges will be represented in more than one. 102 103 ## Batch and Transaction wrapping 104 105 When a request arrives which is not a batch, wrap it in one appropriately. This 106 should be close to where `maybeBeginTxn` is today. `OpRequiresTxnError` should 107 be handled nearby, creating the transaction and retrying in-place. 108 109 ## Store/Replica 110 111 As discussed under [Unresolved Questions](#unresolved-questions), in an ideal 112 world we would propose batches to Raft. Since that is complex, the scope here 113 is less ambitious: 114 115 The main retry loop in `(*Store).ExecuteCmd()` gets an additional outer loop 116 which expands the Batch and deals with each request separately, aborting on 117 the first unhandled error. 118 119 This means that `Replica` will be blissfully unaware of this RFC. 120 121 ## Tracing 122 123 Tracing will need an update, but mostly content-wise. It prints the request 124 type in various locations, and those will all be `Batch` after this change. 125 126 # Drawbacks 127 128 * slight overhead for non-batch calls. It's not expected that there be many of those though. 129 130 # Alternatives 131 132 * different interfaces between `TxnCoordSender` and `DistSender` are possible. One could even argue that the order of `DistSender` and `TxnCoordSender` might be the wrong way around. 133 134 # Unresolved questions 135 136 ## Replica/Raft batches 137 138 A large reduction in network-related delay results if the replica proposes the 139 `BatchRequest`s into Raft without prior expansion for performance. This is 140 worthy of another RFC but here are some basic details. 141 142 It is going to require considerable refactoring around the command queue, and 143 timestamp and response caches which currently serialize those requests in 144 `add{Read,Write}Cmd`, including making sure requests in the same batch have the 145 same rules applied to them. 146 147 Applying such a batch would entail looping through its individual requests and 148 executing them in the same batch, stopping on the first error encountered, if 149 any. Such an error discards the `engine.Batch` as usual, but #1989 applies. 150 151 Further optimizations to this are possible: conflict resolution using this 152 approach could be inefficient: The first such conflict will have the whole batch 153 retry. It could be a good optimization to execute all commands unless "critical" 154 errors pop up (a `WriteIntentError` not being one of them).