-
Notifications
You must be signed in to change notification settings - Fork 124
/
Copy pathsource.go
442 lines (386 loc) · 18.3 KB
/
source.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
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
/*
Copyright 2022 The Numaproj 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 sources
import (
"context"
"fmt"
"os"
"strconv"
"time"
"go.uber.org/zap"
dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1"
"github.com/numaproj/numaflow/pkg/forwarder"
"github.com/numaproj/numaflow/pkg/isb"
jetstreamisb "github.com/numaproj/numaflow/pkg/isb/stores/jetstream"
redisisb "github.com/numaproj/numaflow/pkg/isb/stores/redis"
"github.com/numaproj/numaflow/pkg/isbsvc"
"github.com/numaproj/numaflow/pkg/metrics"
"github.com/numaproj/numaflow/pkg/sdkclient"
"github.com/numaproj/numaflow/pkg/sdkclient/serverinfo"
sourceclient "github.com/numaproj/numaflow/pkg/sdkclient/source"
"github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer"
"github.com/numaproj/numaflow/pkg/shared/callback"
jsclient "github.com/numaproj/numaflow/pkg/shared/clients/nats"
redisclient "github.com/numaproj/numaflow/pkg/shared/clients/redis"
"github.com/numaproj/numaflow/pkg/shared/logging"
sharedutil "github.com/numaproj/numaflow/pkg/shared/util"
"github.com/numaproj/numaflow/pkg/shuffle"
sourceforward "github.com/numaproj/numaflow/pkg/sources/forward"
"github.com/numaproj/numaflow/pkg/sources/generator"
"github.com/numaproj/numaflow/pkg/sources/http"
jetstreamsrc "github.com/numaproj/numaflow/pkg/sources/jetstream"
"github.com/numaproj/numaflow/pkg/sources/kafka"
"github.com/numaproj/numaflow/pkg/sources/nats"
"github.com/numaproj/numaflow/pkg/sources/sourcer"
"github.com/numaproj/numaflow/pkg/sources/transformer"
"github.com/numaproj/numaflow/pkg/sources/udsource"
"github.com/numaproj/numaflow/pkg/watermark/fetch"
"github.com/numaproj/numaflow/pkg/watermark/generic"
"github.com/numaproj/numaflow/pkg/watermark/generic/jetstream"
"github.com/numaproj/numaflow/pkg/watermark/publish"
"github.com/numaproj/numaflow/pkg/watermark/store"
"github.com/numaproj/numaflow/pkg/watermark/wmb"
)
type SourceProcessor struct {
ISBSvcType dfv1.ISBSvcType
VertexInstance *dfv1.VertexInstance
}
func (sp *SourceProcessor) Start(ctx context.Context) error {
var (
sourcePublisherStores, _ = store.BuildNoOpWatermarkStore()
sourceWmStores = make(map[string]store.WatermarkStore)
toVertexWatermarkStores = make(map[string]store.WatermarkStore)
log = logging.FromContext(ctx)
writersMap = make(map[string][]isb.BufferWriter)
srcTransformerGRPCClient *transformer.GRPCBasedTransformer
sourceReader sourcer.SourceReader
healthCheckers []metrics.HealthChecker
idleManager wmb.IdleManager
pipelineName = sp.VertexInstance.Vertex.Spec.PipelineName
vertexName = sp.VertexInstance.Vertex.Spec.Name
)
ctx, cancel := context.WithCancel(ctx)
defer cancel()
// watermark variables no-op initialization
// create a no op fetcher
fetchWatermark, _ := generic.BuildNoOpSourceWatermarkProgressors(sp.VertexInstance.Vertex.GetToBuffers())
// create no op publisher stores
for _, e := range sp.VertexInstance.Vertex.Spec.ToEdges {
toVertexWatermarkStores[e.To], _ = store.BuildNoOpWatermarkStore()
}
idleManager = wmb.NewNoOpIdleManager()
switch sp.ISBSvcType {
case dfv1.ISBSvcTypeRedis:
for _, e := range sp.VertexInstance.Vertex.Spec.ToEdges {
writeOpts := []redisclient.Option{
redisclient.WithBufferFullWritingStrategy(e.BufferFullWritingStrategy()),
}
if x := e.ToVertexLimits; x != nil && x.BufferMaxLength != nil {
writeOpts = append(writeOpts, redisclient.WithMaxLength(int64(*x.BufferMaxLength)))
}
if x := e.ToVertexLimits; x != nil && x.BufferUsageLimit != nil {
writeOpts = append(writeOpts, redisclient.WithBufferUsageLimit(float64(*x.BufferUsageLimit)/100))
}
partitionedBuffers := dfv1.GenerateBufferNames(sp.VertexInstance.Vertex.Namespace, sp.VertexInstance.Vertex.Spec.PipelineName, e.To, e.GetToVertexPartitionCount())
var bufferWriters []isb.BufferWriter
// create a writer for each partition.
for partitionIdx, partition := range partitionedBuffers {
group := partition + "-group"
redisClient := redisclient.NewInClusterRedisClient()
writer := redisisb.NewBufferWrite(ctx, redisClient, partition, group, int32(partitionIdx), writeOpts...)
bufferWriters = append(bufferWriters, writer)
}
writersMap[e.To] = bufferWriters
}
case dfv1.ISBSvcTypeJetStream:
// create a new NATS client pool
natsClientPool, err := jsclient.NewClientPool(ctx, jsclient.WithClientPoolSize(2))
if err != nil {
return fmt.Errorf("failed to create a new NATS client pool: %w", err)
}
defer natsClientPool.CloseAll()
for _, e := range sp.VertexInstance.Vertex.Spec.ToEdges {
writeOpts := []jetstreamisb.WriteOption{
jetstreamisb.WithBufferFullWritingStrategy(e.BufferFullWritingStrategy()),
}
if x := e.ToVertexLimits; x != nil && x.BufferMaxLength != nil {
writeOpts = append(writeOpts, jetstreamisb.WithMaxLength(int64(*x.BufferMaxLength)))
}
if x := e.ToVertexLimits; x != nil && x.BufferUsageLimit != nil {
writeOpts = append(writeOpts, jetstreamisb.WithBufferUsageLimit(float64(*x.BufferUsageLimit)/100))
}
var bufferWriters []isb.BufferWriter
partitionedBuffers := dfv1.GenerateBufferNames(sp.VertexInstance.Vertex.Namespace, sp.VertexInstance.Vertex.Spec.PipelineName, e.To, e.GetToVertexPartitionCount())
// create a writer for each partition.
for partitionIdx, partition := range partitionedBuffers {
streamName := isbsvc.JetStreamName(partition)
jetStreamClient := natsClientPool.NextAvailableClient()
writer, err := jetstreamisb.NewJetStreamBufferWriter(ctx, jetStreamClient, partition, streamName, streamName, int32(partitionIdx), writeOpts...)
if err != nil {
return err
}
bufferWriters = append(bufferWriters, writer)
}
writersMap[e.To] = bufferWriters
}
// created watermark related components only if watermark is enabled
// otherwise no op will be used
if !sp.VertexInstance.Vertex.Spec.Watermark.Disabled {
// build watermark stores for from vertex
sourceWmStores, err = jetstream.BuildFromVertexWatermarkStores(ctx, sp.VertexInstance, natsClientPool.NextAvailableClient())
if err != nil {
return fmt.Errorf("failed to build watermark stores: %w", err)
}
// create watermark fetcher using watermark stores of from vertex
fetchWatermark = fetch.NewSourceFetcher(ctx, sourceWmStores[sp.VertexInstance.Vertex.Name], fetch.WithIsSource(true))
// build watermark stores for to-vertex
toVertexWatermarkStores, err = jetstream.BuildToVertexWatermarkStores(ctx, sp.VertexInstance, natsClientPool.NextAvailableClient())
if err != nil {
return err
}
// build watermark stores for sourceReader (we publish twice for sourceReader)
sourcePublisherStores, err = jetstream.BuildSourcePublisherStores(ctx, sp.VertexInstance, natsClientPool.NextAvailableClient())
if err != nil {
return err
}
idleManager, _ = wmb.NewIdleManager(1, len(writersMap))
}
default:
return fmt.Errorf("unrecognized isb svc type %q", sp.ISBSvcType)
}
// Populate the shuffle function map
// we need to shuffle the messages, because we can have a reduce vertex immediately after a sourceReader vertex.
var toVertexPartitionMap = make(map[string]int)
shuffleFuncMap := make(map[string]*shuffle.Shuffle)
for _, edge := range sp.VertexInstance.Vertex.Spec.ToEdges {
if edge.GetToVertexPartitionCount() > 1 {
s := shuffle.NewShuffle(edge.To, edge.GetToVertexPartitionCount())
shuffleFuncMap[fmt.Sprintf("%s:%s", edge.From, edge.To)] = s
}
toVertexPartitionMap[edge.To] = edge.GetToVertexPartitionCount()
}
maxMessageSize := sharedutil.LookupEnvIntOr(dfv1.EnvGRPCMaxMessageSize, sdkclient.DefaultGRPCMaxMessageSize)
// if the sourceReader is a user-defined sourceReader, we create a gRPC client for it.
var udsGRPCClient *udsource.GRPCBasedUDSource
if sp.VertexInstance.Vertex.IsUDSource() {
// Wait for server info to be ready
serverInfo, err := serverinfo.SDKServerInfo(serverinfo.WithServerInfoFilePath(sdkclient.SourceServerInfoFile))
if err != nil {
return err
}
metrics.SDKInfo.WithLabelValues(dfv1.ComponentVertex, fmt.Sprintf("%s-%s", pipelineName, vertexName), string(serverinfo.ContainerTypeSourcer), serverInfo.Version, string(serverInfo.Language)).Set(1)
srcClient, err := sourceclient.New(ctx, serverInfo, sdkclient.WithMaxMessageSize(maxMessageSize))
if err != nil {
return fmt.Errorf("failed to create a new gRPC client: %w", err)
}
udsGRPCClient, err = udsource.NewUDSgRPCBasedUDSource(sp.VertexInstance, srcClient)
if err != nil {
return fmt.Errorf("failed to create gRPC client, %w", err)
}
// Readiness check
if err = udsGRPCClient.WaitUntilReady(ctx); err != nil {
return fmt.Errorf("failed on user-defined sourceReader readiness check, %w", err)
}
defer func() {
err = udsGRPCClient.CloseConn(ctx)
if err != nil {
log.Warnw("Failed to close gRPC client conn", zap.Error(err))
}
}()
healthCheckers = append(healthCheckers, udsGRPCClient)
}
var forwardOpts []sourceforward.Option
if x := sp.VertexInstance.Vertex.Spec.Limits; x != nil {
if x.ReadBatchSize != nil {
forwardOpts = append(forwardOpts, sourceforward.WithReadBatchSize(int64(*x.ReadBatchSize)))
}
}
if sp.VertexInstance.Vertex.HasUDTransformer() {
// Wait for server info to be ready
serverInfo, err := serverinfo.SDKServerInfo(serverinfo.WithServerInfoFilePath(sdkclient.SourceTransformerServerInfoFile))
if err != nil {
return err
}
metrics.SDKInfo.WithLabelValues(dfv1.ComponentVertex, fmt.Sprintf("%s-%s", pipelineName, vertexName), string(serverinfo.ContainerTypeSourcetransformer), serverInfo.Version, string(serverInfo.Language)).Set(1)
srcTransformerClient, err := sourcetransformer.New(ctx, serverInfo, sdkclient.WithMaxMessageSize(maxMessageSize))
if err != nil {
return fmt.Errorf("failed to create transformer gRPC client, %w", err)
}
srcTransformerGRPCClient = transformer.NewGRPCBasedTransformer(vertexName, srcTransformerClient)
// Close the connection when we are done
defer func() {
err = srcTransformerGRPCClient.CloseConn(ctx)
if err != nil {
log.Warnw("Failed to close transformer gRPC client conn", zap.Error(err))
}
}()
// Readiness check
if err = srcTransformerGRPCClient.WaitUntilReady(ctx); err != nil {
return fmt.Errorf("failed on user-defined transfomer readiness check, %w", err)
}
healthCheckers = append(healthCheckers, srcTransformerGRPCClient)
forwardOpts = append(forwardOpts, sourceforward.WithTransformer(srcTransformerGRPCClient))
}
sourceReader, err := sp.createSourceReader(ctx, udsGRPCClient)
if err != nil {
return fmt.Errorf("failed to create source, error: %w", err)
}
// create a source watermark publisher
sourceWmPublisher := publish.NewSourcePublish(ctx, pipelineName, vertexName, sourcePublisherStores, publish.WithDelay(sp.VertexInstance.Vertex.Spec.Watermark.GetMaxDelay()))
// if the callback is enabled, create a callback publisher
cbEnabled := sharedutil.LookupEnvBoolOr(dfv1.EnvCallbackEnabled, false)
if cbEnabled {
cbOpts := make([]callback.OptionFunc, 0)
cbUrl := os.Getenv(dfv1.EnvCallbackURL)
if cbUrl != "" {
cbOpts = append(cbOpts, callback.WithCallbackURL(cbUrl))
}
cbPublisher := callback.NewUploader(ctx, vertexName, pipelineName, cbOpts...)
forwardOpts = append(forwardOpts, sourceforward.WithCallbackUploader(cbPublisher))
}
// create source data forwarder
var sourceForwarder *sourceforward.DataForward
if sp.VertexInstance.Vertex.HasUDTransformer() {
sourceForwarder, err = sourceforward.NewDataForward(sp.VertexInstance, sourceReader, writersMap, sp.getTransformerGoWhereDecider(shuffleFuncMap), fetchWatermark, sourceWmPublisher, toVertexWatermarkStores, idleManager, forwardOpts...)
} else {
sourceForwarder, err = sourceforward.NewDataForward(sp.VertexInstance, sourceReader, writersMap, sp.getSourceGoWhereDecider(shuffleFuncMap), fetchWatermark, sourceWmPublisher, toVertexWatermarkStores, idleManager, forwardOpts...)
}
if err != nil {
return fmt.Errorf("failed to create source forwarder, error: %w", err)
}
metricsOpts := metrics.NewMetricsOptions(ctx, sp.VertexInstance.Vertex, healthCheckers, []isb.LagReader{sourceReader})
ms := metrics.NewMetricsServer(sp.VertexInstance.Vertex, metricsOpts...)
if shutdown, err := ms.Start(ctx); err != nil {
return fmt.Errorf("failed to start metrics server, error: %w", err)
} else {
defer func() { _ = shutdown(context.Background()) }()
}
log.Infow("Start processing source messages", zap.String("isbs", string(sp.ISBSvcType)), zap.Any("to", sp.VertexInstance.Vertex.GetToBuffers()))
stopped := sourceForwarder.Start()
select {
case <-ctx.Done(): // context cancelled case
log.Info("Context cancelled, stopping forwarder for partition...")
sourceForwarder.Stop()
if err := <-stopped; err != nil {
log.Errorw("Source forwarder stopped with error", zap.Error(err))
}
log.Info("Exited source forwarder...")
case err := <-stopped: // critical error case
if err != nil {
log.Errorw("Source forwarder stopped with error", zap.Error(err))
cancel()
}
}
// close all the sourceReader wm stores
for _, wmStore := range sourceWmStores {
_ = wmStore.Close()
}
// close all the to vertex wm stores
for _, ws := range toVertexWatermarkStores {
_ = ws.Close()
}
// close all the sourceReader publisher stores
_ = sourcePublisherStores.Close()
log.Info("Exited...")
return nil
}
// createSourceReader is used to send the sourcer information
func (sp *SourceProcessor) createSourceReader(ctx context.Context, udsGRPCClient *udsource.GRPCBasedUDSource) (sourcer.SourceReader, error) {
var readTimeout time.Duration
if l := sp.VertexInstance.Vertex.Spec.Limits; l != nil && l.ReadTimeout != nil {
readTimeout = l.ReadTimeout.Duration
}
src := sp.VertexInstance.Vertex.Spec.Source
if x := src.UDSource; x != nil && udsGRPCClient != nil {
return udsource.NewUserDefinedSource(ctx, sp.VertexInstance, udsGRPCClient, udsource.WithReadTimeout(readTimeout))
} else if x := src.Generator; x != nil {
return generator.NewMemGen(ctx, sp.VertexInstance, generator.WithReadTimeout(readTimeout))
} else if x := src.Kafka; x != nil {
return kafka.NewKafkaSource(ctx, sp.VertexInstance, kafka.NewConsumerHandler(dfv1.DefaultKafkaHandlerChannelSize), kafka.WithReadTimeOut(readTimeout), kafka.WithGroupName(x.ConsumerGroupName))
} else if x := src.HTTP; x != nil {
return http.NewHttpSource(ctx, sp.VertexInstance, http.WithReadTimeout(readTimeout))
} else if x := src.Nats; x != nil {
return nats.New(ctx, sp.VertexInstance, nats.WithReadTimeout(readTimeout))
} else if x := src.JetStream; x != nil {
return jetstreamsrc.New(ctx, sp.VertexInstance, jetstreamsrc.WithReadTimeout(readTimeout))
} else if x := src.Serving; x != nil {
return jetstreamsrc.New(ctx, sp.VertexInstance, jetstreamsrc.WithReadTimeout(readTimeout), jetstreamsrc.WithServingEnabled())
}
return nil, fmt.Errorf("invalid source spec")
}
func (sp *SourceProcessor) getSourceGoWhereDecider(shuffleFuncMap map[string]*shuffle.Shuffle) forwarder.GoWhere {
// create the conditional forwarder
conditionalForwarder := forwarder.GoWhere(func(keys []string, tags []string, msgId string) ([]forwarder.VertexBuffer, error) {
var result []forwarder.VertexBuffer
// Iterate through the edges
for _, edge := range sp.VertexInstance.Vertex.Spec.ToEdges {
// if the edge has more than one partition, shuffle the message
// else forward the message to the default partition
partitionIdx := isb.DefaultPartitionIdx
if edge.GetToVertexPartitionCount() > 1 {
edgeKey := edge.From + ":" + edge.To
if edge.ToVertexType == dfv1.VertexTypeReduceUDF { // Shuffle on keys
partitionIdx = shuffleFuncMap[edgeKey].ShuffleOnKeys(keys)
} else { // Shuffle on msgId
partitionIdx = shuffleFuncMap[edgeKey].ShuffleOnId(msgId)
}
}
result = append(result, forwarder.VertexBuffer{
ToVertexName: edge.To,
ToVertexPartitionIdx: partitionIdx,
})
}
return result, nil
})
return conditionalForwarder
}
func (sp *SourceProcessor) getTransformerGoWhereDecider(shuffleFuncMap map[string]*shuffle.Shuffle) forwarder.GoWhere {
// create the conditional forwarder
conditionalForwarder := forwarder.GoWhere(func(keys []string, tags []string, msgId string) ([]forwarder.VertexBuffer, error) {
var result []forwarder.VertexBuffer
// Drop message if it contains the special tag
if sharedutil.StringSliceContains(tags, dfv1.MessageTagDrop) {
metrics.UserDroppedMessages.With(map[string]string{
metrics.LabelVertex: sp.VertexInstance.Vertex.Spec.Name,
metrics.LabelPipeline: sp.VertexInstance.Vertex.Spec.PipelineName,
metrics.LabelVertexType: string(dfv1.VertexTypeSource),
metrics.LabelVertexReplicaIndex: strconv.Itoa(int(sp.VertexInstance.Replica)),
}).Inc()
return result, nil
}
// Iterate through the edges
for _, edge := range sp.VertexInstance.Vertex.Spec.ToEdges {
// Condition to proceed for forwarding message: No conditions on edge, or message tags match edge conditions
proceed := edge.Conditions == nil || edge.Conditions.Tags == nil || len(edge.Conditions.Tags.Values) == 0 || sharedutil.CompareSlice(edge.Conditions.Tags.GetOperator(), tags, edge.Conditions.Tags.Values)
if proceed {
// if the edge has more than one partition, shuffle the message
// else forward the message to the default partition
partitionIdx := isb.DefaultPartitionIdx
if edge.GetToVertexPartitionCount() > 1 {
edgeKey := edge.From + ":" + edge.To
if edge.ToVertexType == dfv1.VertexTypeReduceUDF { // Shuffle on keys
partitionIdx = shuffleFuncMap[edgeKey].ShuffleOnKeys(keys)
} else { // Shuffle on msgId
partitionIdx = shuffleFuncMap[edgeKey].ShuffleOnId(msgId)
}
}
result = append(result, forwarder.VertexBuffer{
ToVertexName: edge.To,
ToVertexPartitionIdx: partitionIdx,
})
}
}
return result, nil
})
return conditionalForwarder
}