forked from cockroachdb/cockroach
-
Notifications
You must be signed in to change notification settings - Fork 0
/
flow_registry.go
399 lines (359 loc) · 12.9 KB
/
flow_registry.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
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
// Copyright 2016 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.
package distsqlrun
import (
"fmt"
"sync"
"time"
"golang.org/x/net/context"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
opentracing "github.com/opentracing/opentracing-go"
"github.com/pkg/errors"
)
// flowStreamDefaultTimeout is the amount of time incoming streams wait for a flow to
// be set up before erroring out.
const flowStreamDefaultTimeout time.Duration = 10 * time.Second
// inboundStreamInfo represents the endpoint where a data stream from another
// node connects to a flow. The external node initiates this process through a
// FlowStream RPC, which uses (*Flow).connectInboundStream() to associate the
// stream to a receiver to push rows to.
type inboundStreamInfo struct {
// receiver is the entity that will receive rows from another host, which is
// part of a processor (normally an input synchronizer).
//
// During a FlowStream RPC, rows are pushed to this entity using the
// RowReceiver interface.
receiver RowReceiver
connected bool
// if set, indicates that we waited too long for an inbound connection, or
// we don't want this stream to connect anymore due to flow cancellation.
cancelled bool
// finished is set if we have signaled that the stream is done transferring
// rows (to the flow's wait group).
finished bool
// waitGroup to signal on when finished.
waitGroup *sync.WaitGroup
}
// flowEntry is a structure associated with a (potential) flow.
type flowEntry struct {
// waitCh is set if one or more clients are waiting for the flow; the
// channel gets closed when the flow is registered.
waitCh chan struct{}
// refCount is used to allow multiple clients to wait for a flow - if the
// flow never shows up, the refCount is used to decide which client cleans
// up the entry.
refCount int
flow *Flow
// inboundStreams are streams that receive data from other hosts, through the
// FlowStream API. All fields in the inboundStreamInfos are protected by the
// flowRegistry mutex (except the receiver, whose methods can be called
// freely).
inboundStreams map[StreamID]*inboundStreamInfo
// streamTimer is a timer that fires after a timeout and verifies that all
// inbound streams have been connected.
streamTimer *time.Timer
}
// flowRegistry allows clients to look up flows by ID and to wait for flows to
// be registered. Multiple clients can wait concurrently for the same flow.
type flowRegistry struct {
syncutil.Mutex
// nodeID is the ID of the current node. Used for debugging.
nodeID roachpb.NodeID
// All fields in the flowEntry's are protected by the flowRegistry mutex,
// except flow, whose methods can be called freely.
flows map[FlowID]*flowEntry
// If true, the registry sends handshake messages whenever an inbound stream
// is connected. This can be turned off for compatibility with older node
// versions.
sendHandshakeMsg bool
}
type registryCompatibilityMode bool
const (
defaultCompatibility registryCompatibilityMode = false
extra1_0Compatibility registryCompatibilityMode = true
)
// makeFlowRegistry creates a new flowRegistry.
//
// nodeID is the ID of the current node. Used for debugging; pass 0 if you don't
// care.
func makeFlowRegistry(
nodeID roachpb.NodeID, compatibility registryCompatibilityMode,
) *flowRegistry {
fr := &flowRegistry{
nodeID: nodeID,
flows: make(map[FlowID]*flowEntry),
sendHandshakeMsg: compatibility != extra1_0Compatibility,
}
return fr
}
// getEntryLocked returns the flowEntry associated with the id. If the entry
// doesn't exist, one is created and inserted into the map.
// It should only be called while holding the mutex.
func (fr *flowRegistry) getEntryLocked(id FlowID) *flowEntry {
entry, ok := fr.flows[id]
if !ok {
entry = &flowEntry{}
fr.flows[id] = entry
}
return entry
}
// releaseEntryLocked decreases the refCount in the entry for the given id, and
// cleans up the entry if the refCount reaches 0.
// It should only be called while holding the mutex.
func (fr *flowRegistry) releaseEntryLocked(id FlowID) {
entry := fr.flows[id]
if entry.refCount > 1 {
entry.refCount--
} else {
if entry.refCount != 1 {
panic(fmt.Sprintf("invalid refCount: %d", entry.refCount))
}
delete(fr.flows, id)
}
}
// RegisterFlow makes a flow accessible to ConnectInboundStream. Any concurrent
// ConnectInboundStream calls that are waiting for this flow are woken up.
//
// It is expected that UnregisterFlow will be called at some point to remove the
// flow from the registry.
//
// inboundStreams are all the remote streams that will be connected into this
// flow. If any of them is not connected within timeout, errors are propagated.
// The inboundStreams are expected to have been initialized with their
// WaitGroups (the group should have been incremented). RegisterFlow takes
// responsibility for calling Done() on that WaitGroup; this responsibility will
// be forwarded forward by ConnectInboundStream. In case this method returns an
// error, the WaitGroup will be decremented.
func (fr *flowRegistry) RegisterFlow(
ctx context.Context,
id FlowID,
f *Flow,
inboundStreams map[StreamID]*inboundStreamInfo,
timeout time.Duration,
) (retErr error) {
fr.Lock()
defer fr.Unlock()
defer func() {
if retErr != nil {
for _, stream := range inboundStreams {
stream.waitGroup.Done()
}
}
}()
entry := fr.getEntryLocked(id)
if entry.flow != nil {
return errors.Errorf(
"flow already registered: current node ID: %d flowID: %d.\n"+
"Current flow: %+v\nExisting flow: %+v",
fr.nodeID, f.spec.FlowID, f.spec, entry.flow.spec)
}
// Take a reference that will be removed by UnregisterFlow.
entry.refCount++
entry.flow = f
entry.inboundStreams = inboundStreams
// If there are any waiters, wake them up by closing waitCh.
if entry.waitCh != nil {
close(entry.waitCh)
}
if len(inboundStreams) > 0 {
// Set up a function to time out inbound streams after a while.
entry.streamTimer = time.AfterFunc(timeout, func() {
fr.Lock()
defer fr.Unlock()
numTimedOut := 0
for streamID, is := range entry.inboundStreams {
if !is.connected && !is.cancelled {
is.cancelled = true
numTimedOut++
// We're giving up waiting for this inbound stream. Send an error to
// its consumer; the error will propagate and eventually drain all the
// processors.
is.receiver.Push(
nil, /* row */
ProducerMetadata{Err: errors.Errorf("no inbound stream connection")})
is.receiver.ProducerDone()
fr.finishInboundStreamLocked(id, streamID)
}
}
if numTimedOut != 0 {
// The span in the context might be finished by the time this runs. In
// principle, we could ForkCtxSpan() beforehand, but we don't want to
// create the extra span every time.
timeoutCtx := opentracing.ContextWithSpan(ctx, nil)
log.Errorf(
timeoutCtx,
"flow id:%s : %d inbound streams timed out after %s; propagated error throughout flow",
id,
numTimedOut,
timeout,
)
}
})
}
return nil
}
// UnregisterFlow removes a flow from the registry. Any subsequent
// ConnectInboundStream calls for the flow will fail to find it and time out.
func (fr *flowRegistry) UnregisterFlow(id FlowID) {
fr.Lock()
entry := fr.flows[id]
if entry.streamTimer != nil {
entry.streamTimer.Stop()
entry.streamTimer = nil
}
fr.releaseEntryLocked(id)
fr.Unlock()
}
// waitForFlowLocked waits until the flow with the given id gets registered -
// up to the given timeout - and returns the flowEntry. If the timeout elapses,
// returns nil. It should only be called while holding the mutex. The mutex is
// temporarily unlocked if we need to wait.
// It is illegal to call this if the flow is already connected.
func (fr *flowRegistry) waitForFlowLocked(
ctx context.Context, id FlowID, timeout time.Duration,
) *flowEntry {
entry := fr.getEntryLocked(id)
if entry.flow != nil {
log.Fatalf(ctx, "waitForFlowLocked called for a flow that's already registered: %d", id)
}
// Flow not registered (at least not yet).
// Set up a channel that gets closed when the flow shows up, or when the
// timeout elapses. The channel might have been created already if there are
// other waiters for the same id.
waitCh := entry.waitCh
if waitCh == nil {
waitCh = make(chan struct{})
entry.waitCh = waitCh
}
entry.refCount++
fr.Unlock()
select {
case <-waitCh:
case <-time.After(timeout):
case <-ctx.Done():
}
fr.Lock()
fr.releaseEntryLocked(id)
if entry.flow == nil {
return nil
}
return entry
}
// ConnectInboundStream finds the inboundStreamInfo for the given
// <flowID,streamID> pair and marks it as connected. It waits up to timeout for
// the stream to be registered with the registry. It also sends the handshake
// messages to the producer of the stream.
//
// stream is the inbound stream.
//
// It returns the Flow that the stream is connecting to, the receiver that the
// stream must push data to and a cleanup function that must be called to
// unregister the flow from the registry after all the data has been pushed.
//
// The cleanup function will decrement the flow's WaitGroup, so that Flow.Wait()
// is not blocked on this stream any more.
// In case an error is returned, the cleanup function is nil, the Flow is not
// considered connected and is not cleaned up.
func (fr *flowRegistry) ConnectInboundStream(
ctx context.Context,
flowID FlowID,
streamID StreamID,
stream DistSQL_FlowStreamServer,
timeout time.Duration,
) (_ *Flow, _ RowReceiver, _ func(), retErr error) {
fr.Lock()
defer fr.Unlock()
entry := fr.getEntryLocked(flowID)
if entry.flow == nil {
if fr.sendHandshakeMsg {
// Send the handshake message informing the producer that the consumer has
// not been scheduled yet. Another handshake will be sent below once the
// consumer has been connected.
deadline := timeutil.Now().Add(timeout)
if err := stream.Send(&ConsumerSignal{
Handshake: &ConsumerHandshake{
ConsumerScheduled: false,
ConsumerScheduleDeadline: &deadline,
Version: Version,
MinAcceptedVersion: MinAcceptedVersion,
},
}); err != nil {
// TODO(andrei): We failed to send a message to the producer; we'll return
// an error and leave this stream with connected == false so it times out
// later. We could call finishInboundStreamLocked() now so that the flow
// doesn't wait for the timeout and we could remember the error for the
// consumer if the consumer comes later, but I'm not sure what the best
// way to do that is. Similarly for the 2nd handshake message below,
// except there we already have the consumer and we can push the error.
return nil, nil, nil, err
}
}
entry = fr.waitForFlowLocked(ctx, flowID, timeout)
if entry == nil {
return nil, nil, nil, errors.Errorf("flow %s not found", flowID)
}
}
s, ok := entry.inboundStreams[streamID]
if !ok {
return nil, nil, nil, errors.Errorf("flow %s: no inbound stream %d", flowID, streamID)
}
if s.connected {
return nil, nil, nil, errors.Errorf("flow %s: inbound stream %d already connected", flowID, streamID)
}
if s.cancelled {
return nil, nil, nil, errors.Errorf("flow %s: inbound stream %d came too late", flowID, streamID)
}
// We now mark the stream as connected but, if an error happens later because
// the handshake fails, we reset the state; we want the stream to be
// considered timed out when the moment comes just as if this connection
// attempt never happened.
s.connected = true
defer func() {
if retErr != nil {
s.connected = false
}
}()
if fr.sendHandshakeMsg {
if err := stream.Send(&ConsumerSignal{
Handshake: &ConsumerHandshake{
ConsumerScheduled: true,
Version: Version,
MinAcceptedVersion: MinAcceptedVersion,
},
}); err != nil {
return nil, nil, nil, err
}
}
cleanup := func() {
fr.Lock()
fr.finishInboundStreamLocked(flowID, streamID)
fr.Unlock()
}
return entry.flow, s.receiver, cleanup, nil
}
func (fr *flowRegistry) finishInboundStreamLocked(fid FlowID, sid StreamID) {
flowEntry := fr.getEntryLocked(fid)
streamEntry := flowEntry.inboundStreams[sid]
if !streamEntry.connected && !streamEntry.cancelled {
panic("finising inbound stream that didn't connect or time out")
}
if streamEntry.finished {
panic("double finish")
}
streamEntry.finished = true
streamEntry.waitGroup.Done()
}