forked from cockroachdb/cockroach
/
flow.go
323 lines (294 loc) · 8.97 KB
/
flow.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
// 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.
//
// Author: Radu Berinde (radu@cockroachlabs.com)
package distsql
import (
"sync"
"golang.org/x/net/context"
"github.com/cockroachdb/cockroach/internal/client"
"github.com/cockroachdb/cockroach/rpc"
"github.com/cockroachdb/cockroach/sql/parser"
"github.com/cockroachdb/cockroach/util/log"
"github.com/cockroachdb/cockroach/util/uuid"
"github.com/pkg/errors"
)
// StreamID identifies a stream that crosses machine boundaries. The identifier
// can only be used in the context of a specific flow.
type StreamID int
// LocalStreamID identifies a stream that is local to a flow. The identifier can
// only be used in the context of a specific flow.
type LocalStreamID int
// FlowID identifies a flow. It is most importantly used when setting up streams
// between nodes.
type FlowID struct {
uuid.UUID
}
// FlowCtx encompasses the contexts needed for various flow components.
type FlowCtx struct {
Context context.Context
id FlowID
evalCtx *parser.EvalContext
rpcCtx *rpc.Context
txn *client.Txn
}
type flowStatus int
// Flow status indicators.
const (
FlowNotStarted flowStatus = iota
FlowRunning
FlowFinished
)
// Flow represents a flow which consists of processors and streams.
type Flow struct {
FlowCtx
flowRegistry *flowRegistry
simpleFlowConsumer RowReceiver
waitGroup sync.WaitGroup
processors []processor
outboxes []*outbox
// inboundStreams are streams that receive data from other hosts, through
// the FlowStream API.
inboundStreams map[StreamID]RowReceiver
localStreams map[LocalStreamID]RowReceiver
status flowStatus
}
func newFlow(
flowCtx FlowCtx,
flowReg *flowRegistry,
simpleFlowConsumer RowReceiver,
) *Flow {
flowCtx.Context = log.WithLogTagStr(flowCtx.Context, "f", flowCtx.id.Short())
return &Flow{
FlowCtx: flowCtx,
flowRegistry: flowReg,
simpleFlowConsumer: simpleFlowConsumer,
status: FlowNotStarted,
}
}
// setupInboundStream adds a stream to the stream map (inboundStreams or
// localStreams).
func (f *Flow) setupInboundStream(spec StreamEndpointSpec, receiver RowReceiver) error {
if spec.Mailbox != nil {
if spec.Mailbox.SimpleResponse || spec.Mailbox.TargetAddr != "" {
return errors.Errorf("inbound stream has SimpleResponse or TargetAddr set")
}
sid := spec.Mailbox.StreamID
if _, found := f.inboundStreams[sid]; found {
return errors.Errorf("inbound stream %d has multiple consumers", sid)
}
if f.inboundStreams == nil {
f.inboundStreams = make(map[StreamID]RowReceiver)
}
if log.V(2) {
log.Infof(f.FlowCtx.Context, "Set up inbound stream %d", sid)
}
f.inboundStreams[sid] = receiver
return nil
}
sid := spec.LocalStreamID
if _, found := f.localStreams[sid]; found {
return errors.Errorf("local stream %d has multiple consumers", sid)
}
if f.localStreams == nil {
f.localStreams = make(map[LocalStreamID]RowReceiver)
}
f.localStreams[sid] = receiver
return nil
}
// setupOutStream sets up an output stream; if the stream is local, the
// RowChannel is looked up in the localStreams map; otherwise an outgoing
// mailbox is created.
func (f *Flow) setupOutStream(spec StreamEndpointSpec) (RowReceiver, error) {
if spec.Mailbox != nil {
if spec.Mailbox.SimpleResponse {
return f.simpleFlowConsumer, nil
}
outbox := newOutbox(&f.FlowCtx, spec.Mailbox.TargetAddr, f.id, spec.Mailbox.StreamID)
f.outboxes = append(f.outboxes, outbox)
return outbox, nil
}
sid := spec.LocalStreamID
rowChan, found := f.localStreams[sid]
if !found {
return nil, errors.Errorf("unconnected inbound stream %d", sid)
}
// Once we "connect" a stream, we set the value in the map to nil.
if rowChan == nil {
return nil, errors.Errorf("stream %d has multiple connections", sid)
}
f.localStreams[sid] = nil
return rowChan, nil
}
func (f *Flow) setupRouter(spec *OutputRouterSpec) (RowReceiver, error) {
streams := make([]RowReceiver, len(spec.Streams))
for i := range spec.Streams {
var err error
streams[i], err = f.setupOutStream(spec.Streams[i])
if err != nil {
return nil, err
}
}
return makeRouter(spec, streams)
}
func checkNumInOut(inputs []RowSource, outputs []RowReceiver, numIn, numOut int) error {
if len(inputs) != numIn {
return errors.Errorf("expected %d input(s), got %d", numIn, len(inputs))
}
if len(outputs) != numOut {
return errors.Errorf("expected %d output(s), got %d", numOut, len(outputs))
}
return nil
}
func (f *Flow) makeProcessor(ps *ProcessorSpec, inputs []RowSource) (processor, error) {
if len(ps.Output) != 1 {
return nil, errors.Errorf("only single-output processors supported")
}
outputs := make([]RowReceiver, len(ps.Output))
for i := range ps.Output {
var err error
outputs[i], err = f.setupRouter(&ps.Output[i])
if err != nil {
return nil, err
}
}
if ps.Core.Noop != nil {
if err := checkNumInOut(inputs, outputs, 1, 1); err != nil {
return nil, err
}
return newNoopProcessor(&f.FlowCtx, inputs[0], outputs[0]), nil
}
if ps.Core.TableReader != nil {
if err := checkNumInOut(inputs, outputs, 0, 1); err != nil {
return nil, err
}
return newTableReader(&f.FlowCtx, ps.Core.TableReader, outputs[0])
}
if ps.Core.JoinReader != nil {
if err := checkNumInOut(inputs, outputs, 1, 1); err != nil {
return nil, err
}
return newJoinReader(&f.FlowCtx, ps.Core.JoinReader, inputs[0], outputs[0])
}
if ps.Core.Sorter != nil {
if err := checkNumInOut(inputs, outputs, 1, 1); err != nil {
return nil, err
}
return newSorter(&f.FlowCtx, ps.Core.Sorter, inputs[0], outputs[0]), nil
}
return nil, errors.Errorf("unsupported processor %s", ps)
}
func (f *Flow) setupFlow(spec *FlowSpec) error {
// First step: setup the input synchronizers for all processors.
inputSyncs := make([][]RowSource, len(spec.Processors))
for pIdx, ps := range spec.Processors {
for _, is := range ps.Input {
if len(is.Streams) == 0 {
return errors.Errorf("input sync with no streams")
}
var sync RowSource
switch is.Type {
case InputSyncSpec_UNORDERED:
if len(is.Streams) == 1 {
rowChan := &RowChannel{}
rowChan.Init()
if err := f.setupInboundStream(is.Streams[0], rowChan); err != nil {
return err
}
sync = rowChan
} else {
mrc := &MultiplexedRowChannel{}
mrc.Init(len(is.Streams))
for _, s := range is.Streams {
if err := f.setupInboundStream(s, mrc); err != nil {
return err
}
}
sync = mrc
}
case InputSyncSpec_ORDERED:
// Ordered synchronizer: create a RowChannel for each input.
streams := make([]RowSource, len(is.Streams))
for i, s := range is.Streams {
rowChan := &RowChannel{}
rowChan.Init()
if err := f.setupInboundStream(s, rowChan); err != nil {
return err
}
streams[i] = rowChan
}
var err error
sync, err = makeOrderedSync(convertToColumnOrdering(is.Ordering), streams)
if err != nil {
return err
}
default:
return errors.Errorf("unsupported input sync type %s", is.Type)
}
inputSyncs[pIdx] = append(inputSyncs[pIdx], sync)
}
}
f.processors = make([]processor, len(spec.Processors))
for i := range spec.Processors {
var err error
f.processors[i], err = f.makeProcessor(&spec.Processors[i], inputSyncs[i])
if err != nil {
return err
}
}
return nil
}
func (f *Flow) getInboundStream(sid StreamID) (RowReceiver, error) {
// TODO(radu): detect if we connect to a stream multiple times.
recv, ok := f.inboundStreams[sid]
if !ok {
return nil, errors.Errorf("no inbound stream %d for flow %s", sid, f.id)
}
return recv, nil
}
// Start starts the flow (each processor runs in their own goroutine).
func (f *Flow) Start() {
f.status = FlowRunning
f.flowRegistry.RegisterFlow(f.id, f)
for _, o := range f.outboxes {
o.start(&f.waitGroup)
}
f.waitGroup.Add(len(f.processors))
for _, p := range f.processors {
go p.Run(&f.waitGroup)
}
}
// Wait waits for all the goroutines for this flow to exit.
func (f *Flow) Wait() {
f.waitGroup.Wait()
}
// Cleanup should be called when the flow completes (after all processors and
// mailboxes exited).
func (f *Flow) Cleanup() {
if f.status == FlowFinished {
panic("flow cleanup called twice")
}
if f.status != FlowNotStarted {
f.flowRegistry.UnregisterFlow(f.id)
}
f.status = FlowFinished
}
// RunSync runs the processors in the flow in order (serially), in the same
// context (no goroutines are spawned).
func (f *Flow) RunSync() {
for _, p := range f.processors {
p.Run(nil)
}
f.Cleanup()
}