forked from asonawalla/gazette
/
pipeline.go
294 lines (259 loc) · 9.38 KB
/
pipeline.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
package broker
import (
"context"
"fmt"
"io"
"github.com/LiveRamp/gazette/v2/pkg/fragment"
pb "github.com/LiveRamp/gazette/v2/pkg/protocol"
log "github.com/sirupsen/logrus"
)
// pipeline is an in-flight write replication pipeline of a journal.
type pipeline struct {
pb.Header // Header of the pipeline.
spool fragment.Spool // Local, primary replication Spool.
returnCh chan<- fragment.Spool // |spool| return channel.
streams []pb.Journal_ReplicateClient // Established streams to each replication peer.
sendErrs []error // First error on send from each peer.
readBarrierCh chan struct{} // Coordinates hand-off of receive-side of the pipeline.
recvResp []pb.ReplicateResponse // Most recent response gathered from each peer.
recvErrs []error // First error on receive from each peer.
// readThroughRev, if set, indicates that a pipeline cannot be established
// until we have read through (and our Route reflects) this etcd revision.
readThroughRev int64
}
// newPipeline returns a new pipeline.
func newPipeline(ctx context.Context, hdr pb.Header, spool fragment.Spool, returnCh chan<- fragment.Spool, jc pb.JournalClient) *pipeline {
if hdr.Route.Primary == -1 {
panic("dial requires Route with Primary != -1")
}
var R = len(hdr.Route.Members)
var pln = &pipeline{
Header: hdr,
spool: spool,
returnCh: returnCh,
streams: make([]pb.Journal_ReplicateClient, R),
sendErrs: make([]error, R),
readBarrierCh: make(chan struct{}),
recvResp: make([]pb.ReplicateResponse, R),
recvErrs: make([]error, R),
}
close(pln.readBarrierCh)
for i := range pln.Route.Members {
if i == int(pln.Route.Primary) {
continue
}
pln.streams[i], pln.sendErrs[i] = jc.Replicate(
pb.WithDispatchRoute(ctx, pln.Route, pln.Route.Members[i]))
}
return pln
}
// synchronize all pipeline peers by scattering proposals and gathering peer
// responses. On disagreement, synchronize will iteratively update the proposal
// if it's possible to do so and reach agreement. If peers disagree on Etcd
// revision, synchronize will close the pipeline and set |readThroughRev|.
func (pln *pipeline) synchronize() error {
var proposal = pln.spool.Fragment.Fragment
for {
pln.scatter(&pb.ReplicateRequest{
Header: &pln.Header,
Journal: pln.spool.Journal,
Proposal: &proposal,
Acknowledge: true,
})
var rollToOffset, readThroughRev = pln.gatherSync(proposal)
var err = pln.recvErr()
if err == nil {
err = pln.sendErr()
}
if err != nil {
pln.shutdown(true)
return err
}
if rollToOffset != 0 {
// Update our |proposal| to roll forward to the new offset. Loop to try
// again. This time all peers should agree on the new Fragment.
proposal.Begin = rollToOffset
proposal.End = rollToOffset
proposal.Sum = pb.SHA1Sum{}
continue
}
if readThroughRev != 0 {
// Peer has a non-equivalent Route at a later etcd revision. Close the
// pipeline, and set its |readThroughRev| as an indication to other RPCs
// of the revision which must first be read through before attempting
// another pipeline.
pln.shutdown(false)
pln.readThroughRev = readThroughRev
}
return nil
}
}
// scatter asynchronously applies the ReplicateRequest to all replicas.
func (pln *pipeline) scatter(r *pb.ReplicateRequest) {
for i, s := range pln.streams {
if s != nil && pln.sendErrs[i] == nil {
if r.Header != nil {
// Copy and update to peer ProcessID.
r.Header = boxHeaderProcessID(*r.Header, pln.Route.Members[i])
}
// Send may return an io.EOF if the remote peer breaks the stream.
// We read the actual error in the gather() phase.
pln.sendErrs[i] = s.Send(r)
}
}
if i := pln.Route.Primary; pln.sendErrs[i] == nil {
var resp pb.ReplicateResponse
// Map an error into a |sendErr|.
// Status !OK is returned only on proposal mismatch, which cannot happen
// here as all proposals are derived from the Spool itself.
if resp, pln.sendErrs[i] = pln.spool.Apply(r, true); resp.Status != pb.Status_OK {
panic(resp.String())
}
}
}
// closeSend closes the send-side of all replica connections.
func (pln *pipeline) closeSend() {
// Apply a Spool commit which rolls back any partial content.
pln.spool.MustApply(&pb.ReplicateRequest{
Proposal: &pln.spool.Fragment.Fragment,
})
pln.returnCh <- pln.spool // Release ownership of Spool.
for i, s := range pln.streams {
if s != nil && pln.sendErrs[i] == nil {
pln.sendErrs[i] = s.CloseSend()
}
}
}
// sendErr returns the first encountered send-side error.
func (pln *pipeline) sendErr() error {
for i, err := range pln.sendErrs {
if err != nil {
return fmt.Errorf("send to %s: %s", &pln.Route.Members[i], err)
}
}
return nil
}
// barrier installs a new barrier in the pipeline. Clients should:
// * Invoke barrier after issuing all sent writes, and release the
// pipeline for other clients.
// * Block until |waitFor| is selectable.
// * Read expected responses from the pipeline.
// * Close |closeAfter|.
// By following this convention a pipeline can safely be passed among multiple
// clients, each performing writes followed by reads, while allowing for those
// writes and reads to happen concurrently.
func (pln *pipeline) barrier() (waitFor <-chan struct{}, closeAfter chan<- struct{}) {
waitFor, pln.readBarrierCh = pln.readBarrierCh, make(chan struct{})
closeAfter = pln.readBarrierCh
return
}
// gather synchronously receives a ReplicateResponse from all replicas.
func (pln *pipeline) gather() {
for i, s := range pln.streams {
if s != nil && pln.recvErrs[i] == nil {
pln.recvErrs[i] = s.RecvMsg(&pln.recvResp[i])
}
}
}
// gatherOK calls gather, and treats any non-OK response status as an error.
func (pln *pipeline) gatherOK() {
pln.gather()
for i, s := range pln.streams {
if s == nil || pln.recvErrs[i] != nil {
// Pass.
} else if pln.recvResp[i].Status != pb.Status_OK {
pln.recvErrs[i] = fmt.Errorf("unexpected !OK response: %s", &pln.recvResp[i])
}
}
}
// gatherSync calls gather, extracts and returns a peer-advertised future offset
// or etcd revision to read through relative to |proposal|, and treats any other
// non-OK response status as an error.
func (pln *pipeline) gatherSync(proposal pb.Fragment) (rollToOffset, readThroughRev int64) {
pln.gather()
for i, s := range pln.streams {
if s == nil || pln.recvErrs[i] != nil {
continue
}
switch resp := pln.recvResp[i]; resp.Status {
case pb.Status_OK:
// Pass.
case pb.Status_WRONG_ROUTE:
if !resp.Header.Route.Equivalent(&pln.Route) && resp.Header.Etcd.Revision > pln.Etcd.Revision {
// Peer has a non-equivalent Route at a later etcd revision.
if resp.Header.Etcd.Revision > readThroughRev {
readThroughRev = resp.Header.Etcd.Revision
}
} else {
pln.recvErrs[i] = fmt.Errorf("unexpected WRONG_ROUTE: %s", resp.Header)
}
case pb.Status_FRAGMENT_MISMATCH:
// If peer has an extant Spool at a greater offset, we must roll forward to it.
if (resp.Fragment.End > proposal.End) ||
// If the peer rolled its Spool to our offset, but does not have and
// therefore cannot extend Fragment content from [Begin, End), we
// must start a new Spool beginning at proposal.End.
(resp.Fragment.End == proposal.End && resp.Fragment.ContentLength() == 0) {
if resp.Fragment.End > rollToOffset {
rollToOffset = resp.Fragment.End
}
} else {
pln.recvErrs[i] = fmt.Errorf("unexpected FRAGMENT_MISMATCH: %s", resp.Fragment)
}
default:
pln.recvErrs[i] = fmt.Errorf("unexpected Status: %s", &resp)
}
}
return
}
// gatherEOF synchronously gathers expected EOFs from all replicas.
// An unexpected received message is treated as an error.
func (pln *pipeline) gatherEOF() {
for i, s := range pln.streams {
if s == nil || pln.recvErrs[i] != nil {
// Local spool placeholder, or the stream has already failed.
} else if msg, err := s.Recv(); err == io.EOF {
// Graceful stream closure.
} else if err != nil {
pln.recvErrs[i] = err
} else if pln.recvErrs[i] == nil && err == nil {
pln.recvErrs[i] = fmt.Errorf("unexpected response: %s", msg.String())
}
}
}
// recvErr returns the first encountered receive-side error.
func (pln *pipeline) recvErr() error {
for i, err := range pln.recvErrs {
if err != nil {
return fmt.Errorf("recv from %s: %s", &pln.Route.Members[i], err)
}
}
return nil
}
// shutdown performs a graceful, blocking shutdown of the pipeline.
func (pln *pipeline) shutdown(expectErr bool) {
var waitFor, closeAfter = pln.barrier()
if pln.closeSend(); !expectErr && pln.sendErr() != nil {
log.WithField("err", pln.sendErr()).Warn("tearing down pipeline: failed to closeSend")
}
<-waitFor
if pln.gatherEOF(); !expectErr && pln.recvErr() != nil {
log.WithField("err", pln.recvErr()).Warn("tearing down pipeline: failed to gatherEOF")
}
close(closeAfter)
}
// String is used to provide debugging output of a pipeline in a request trace.
func (pln *pipeline) String() string {
if pln == nil {
return "<nil>"
} else if pln.readThroughRev != 0 {
return fmt.Sprintf("readThroughRev<%d>", pln.readThroughRev)
}
return fmt.Sprintf("pipeline<header: %s, spool: %s>", &pln.Header, pln.spool.String())
}
func boxHeaderProcessID(hdr pb.Header, id pb.ProcessSpec_ID) *pb.Header {
var out = new(pb.Header)
*out = hdr
out.ProcessId = id
return out
}