forked from gazette/core
/
broker.go
331 lines (288 loc) · 8.61 KB
/
broker.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
package journal
import (
"context"
"errors"
"io"
log "github.com/sirupsen/logrus"
"golang.org/x/net/trace"
"github.com/LiveRamp/gazette/pkg/metrics"
)
const (
kSpoolRollSize = 1 << 30
kCommitThreshold = 1 << 20
AppendOpBufferSize = 100
)
// BrokerConfig is used to periodically update Broker with updated
// cluster topology and replication configuration.
type BrokerConfig struct {
// Replica instances which should be involved in brokered transactions.
Replicas []Replicator
// Token representing the Broker's view of the current replication topology.
// Sent with replication requests and verified for consensus by each remote
// replica: for a transaction to succeed, all replicas must agree on the
// current |RouteToken|.
RouteToken
// Next offset of the next brokered write transaction. Also sent with
// replication requests and verifed for consensus by each remote replica:
// for a transaction to succeed, all replicas must agree on the |WriteHead|.
WriteHead int64
// Number of bytes written since the last spool roll.
writtenSinceRoll int64
}
// Broker is responsible for scattering journal writes to each replica, i.e.,
// brokering transactions.
type Broker struct {
journal Name
appendOps chan AppendOp
configUpdates chan BrokerConfig
config BrokerConfig
stop chan struct{}
}
func NewBroker(journal Name) *Broker {
b := &Broker{
journal: journal,
appendOps: make(chan AppendOp, AppendOpBufferSize),
configUpdates: make(chan BrokerConfig, 16),
stop: make(chan struct{}),
}
return b
}
// StartServingOps starts a loop to consume config updates and serves
// appends. Updates are always handled before appends.
func (b *Broker) StartServingOps(writeHead int64) *Broker {
b.config.WriteHead = writeHead
go b.loop()
return b
}
func (b *Broker) Append(op AppendOp) {
b.appendOps <- op
}
func (b *Broker) UpdateConfig(config BrokerConfig) {
b.configUpdates <- config
}
// Stop shuts down the broker. It blocks until all pending config updates and
// appends are handled.
func (b *Broker) Stop() {
close(b.appendOps)
close(b.configUpdates)
<-b.stop // Blocks until loop() exits.
}
func (b *Broker) loop() {
// The configUpdates and appendOps channels are set to nil to indicate they
// have been closed.
for b.configUpdates != nil || b.appendOps != nil {
// Consume available config updates prior to serving appends.
select {
case config, ok := <-b.configUpdates:
if ok {
b.onConfigUpdate(config)
continue
}
default:
}
select {
case config, ok := <-b.configUpdates:
if !ok {
b.configUpdates = nil
continue
}
b.onConfigUpdate(config)
continue
case op, ok := <-b.appendOps:
if !ok {
b.appendOps = nil
continue
}
if b.config.writtenSinceRoll > kSpoolRollSize {
b.config.writtenSinceRoll = 0
}
if writers, err := b.phaseOne(op.Context); err != nil {
op.Result <- AppendResult{Error: ErrReplicationFailed}
log.WithField("err", err).Warn("transaction failed (phase one)")
} else if err = b.phaseTwo(writers, op); err != nil {
log.WithField("err", err).Warn("transaction failed (phase two)")
}
}
}
log.WithField("journal", b.journal).Debug("broker exiting")
close(b.stop)
}
func (b *Broker) onConfigUpdate(config BrokerConfig) {
log.WithFields(log.Fields{"config": config, "journal": b.journal}).
Debug("updated config")
b.config.RouteToken = config.RouteToken
b.config.Replicas = config.Replicas
if config.WriteHead > b.config.WriteHead {
b.config.WriteHead = config.WriteHead
}
// We zero writtenSinceRoll so that replicas begin new spools after
// the route configuration changes.
b.config.writtenSinceRoll = 0
}
// Opens a write-stream with each replica for this transaction.
func (b *Broker) phaseOne(ctx context.Context) ([]WriteCommitter, error) {
if len(b.config.Replicas) == 0 {
return nil, errors.New("no configured replicas")
}
// Scatter replication request to each replica.
var results = make(chan ReplicateResult)
var args = ReplicateArgs{
Journal: b.journal,
RouteToken: b.config.RouteToken,
NewSpool: b.config.writtenSinceRoll == 0,
WriteHead: b.config.WriteHead,
// Replication requests are scoped to the lifecycle of the Broker, rather
// than the |ctx| of the initiating append request.
// TODO(johnny): The Broker should have its own cancel-able Context, used here.
Context: context.TODO(),
}
if tr, ok := trace.FromContext(ctx); ok {
tr.LazyPrintf("Broker.phaseOne request: %v", args)
}
for _, r := range b.config.Replicas {
r.Replicate(ReplicateOp{
ReplicateArgs: args,
Result: results,
})
}
// Gather responses.
var writers []WriteCommitter
var err error
for range b.config.Replicas {
var result = <-results
if tr, ok := trace.FromContext(ctx); ok {
tr.LazyPrintf("Broker.phaseOne result: %v", result)
}
if result.Error != nil {
if result.ErrorWriteHead > b.config.WriteHead {
b.config.WriteHead = result.ErrorWriteHead
}
err = result.Error
} else {
writers = append(writers, result.Writer)
}
}
// Require that all replicas accept the transaction.
if len(writers) != len(b.config.Replicas) {
scatterCommit(writers, 0) // Tell replicas to abort.
return nil, err
} else {
return writers, nil
}
}
func (b *Broker) phaseTwo(writers []WriteCommitter, op AppendOp) error {
var pending []AppendOp
var commitDelta int64
var readErr, writeErr error
var buf = make([]byte, 32*1024) // io.Copy's buffer size.
// Consume waiting AppendOps, streaming them to writers.
for {
var readSize int64
readSize, readErr, writeErr = streamToWriters(writers, op.Content, buf)
if readErr != nil {
op.Result <- AppendResult{Error: readErr}
} else {
// Only commit a complete read from a client.
commitDelta += readSize
pending = append(pending, op)
}
if tr, ok := trace.FromContext(op.Context); ok {
tr.LazyPrintf("Broker.phaseTwo read %d bytes; commit delta %d; readErr %v; writeErr %v",
readSize, commitDelta, readErr, writeErr)
}
// Break if any error occurred or we've reached a commit threshold.
if readErr != nil || writeErr != nil || commitDelta >= kCommitThreshold {
break
}
// Pop another append. Break if the channel blocks or closes.
var ok bool
select {
case op, ok = <-b.appendOps:
default:
ok = false
}
if !ok {
break
}
}
// If a write error occurred to any replica, roll back this transaction.
if writeErr != nil {
log.WithFields(log.Fields{"err": writeErr, "delta": commitDelta}).
Warn("aborting transaction due to replica write error")
commitDelta = 0
}
// Scatter / gather to close each writer in parallel.
// Retain a replica write error, if any occur.
var sawError = writeErr
var sawSuccess bool
var commitErrs = scatterCommit(writers, commitDelta)
for range writers {
if err := <-commitErrs; err != nil {
if sawError == nil {
sawError = err
}
log.WithFields(log.Fields{"err": err, "delta": commitDelta}).
Warn("reporting failure due to replica commit error")
} else {
sawSuccess = true
}
}
// The write head moves forward if at least one replica committed.
if sawSuccess {
b.config.WriteHead += commitDelta
b.config.writtenSinceRoll += int64(commitDelta)
metrics.CommittedBytesTotal.Add(float64(commitDelta))
metrics.CoalescedAppendsTotal.Add(float64(len(pending)))
}
if sawError != nil {
// At least one replica failed. The client must retry.
for _, p := range pending {
if tr, ok := trace.FromContext(p.Context); ok {
tr.LazyPrintf("Broker.phaseTwo abort: %v", sawError)
}
p.Result <- AppendResult{Error: ErrReplicationFailed}
}
return sawError
}
// The transaction was fully replicated. Notify client(s) of success and
// new write-head.
for _, p := range pending {
p.Result <- AppendResult{Error: nil, WriteHead: b.config.WriteHead}
}
return nil
}
func streamToWriters(dst []WriteCommitter, src io.Reader,
buf []byte) (written int64, readErr, writeErr error) {
for {
nr, er := src.Read(buf)
if nr > 0 {
for _, w := range dst {
nw, ew := w.Write(buf[0:nr])
if ew != nil {
return written, er, ew
}
if nr != nw {
return written, er, io.ErrShortWrite
}
}
written += int64(nr)
}
if er == io.EOF {
return written, nil, nil
}
if er != nil {
return written, er, nil
}
}
}
func scatterCommit(writers []WriteCommitter, delta int64) chan error {
// Buffer result channel to the number of writers, so goroutines
// will exit if caller never inspects results.
closeResults := make(chan error, len(writers))
for _, w := range writers {
go func(w WriteCommitter, delta int64) {
closeResults <- w.Commit(delta)
}(w, delta)
}
return closeResults
}