-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
Copy pathsink.go
480 lines (427 loc) · 13.1 KB
/
sink.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
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
// Copyright 2018 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt
package changefeedccl
import (
"context"
"net/url"
"strings"
"time"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/kvevent"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/bufalloc"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
)
// Sink is an abstraction for anything that a changefeed may emit into.
type Sink interface {
// Dial establishes connection to the sink.
Dial() error
// EmitRow enqueues a row message for asynchronous delivery on the sink. An
// error may be returned if a previously enqueued message has failed.
EmitRow(
ctx context.Context,
topic TopicDescriptor,
key, value []byte,
updated, mvcc hlc.Timestamp,
alloc kvevent.Alloc,
) error
// EmitResolvedTimestamp enqueues a resolved timestamp message for
// asynchronous delivery on every topic that has been seen by EmitRow. An
// error may be returned if a previously enqueued message has failed.
EmitResolvedTimestamp(ctx context.Context, encoder Encoder, resolved hlc.Timestamp) error
// Flush blocks until every message enqueued by EmitRow and
// EmitResolvedTimestamp has been acknowledged by the sink. If an error is
// returned, no guarantees are given about which messages have been
// delivered or not delivered.
Flush(ctx context.Context) error
// Close does not guarantee delivery of outstanding messages.
Close() error
}
// SinkWithTopics extends the Sink interface to include a method that returns
// the topics that a changefeed will emit to
type SinkWithTopics interface {
Sink
Topics() []string
}
func getSink(
ctx context.Context,
serverCfg *execinfra.ServerConfig,
feedCfg jobspb.ChangefeedDetails,
timestampOracle timestampLowerBoundOracle,
user username.SQLUsername,
jobID jobspb.JobID,
m metricsRecorder,
) (Sink, error) {
u, err := url.Parse(feedCfg.SinkURI)
if err != nil {
return nil, err
}
if scheme, ok := changefeedbase.NoLongerExperimental[u.Scheme]; ok {
u.Scheme = scheme
}
opts := changefeedbase.MakeStatementOptions(feedCfg.Opts)
// check that options are compatible with the given sink
validateOptionsAndMakeSink := func(sinkSpecificOpts map[string]struct{}, makeSink func() (Sink, error)) (Sink, error) {
err := validateSinkOptions(feedCfg.Opts, sinkSpecificOpts)
if err != nil {
return nil, err
}
return makeSink()
}
metricsBuilder := func(recordingRequired bool) metricsRecorder {
if recordingRequired {
return maybeWrapMetrics(ctx, m, serverCfg.ExternalIORecorder)
}
return m
}
newSink := func() (Sink, error) {
if feedCfg.SinkURI == "" {
return &bufferSink{metrics: m}, nil
}
encodingOpts, err := opts.GetEncodingOptions()
if err != nil {
return nil, err
}
switch {
case u.Scheme == changefeedbase.SinkSchemeNull:
nullIsAccounted := false
if knobs, ok := serverCfg.TestingKnobs.Changefeed.(*TestingKnobs); ok {
nullIsAccounted = knobs.NullSinkIsExternalIOAccounted
}
return makeNullSink(sinkURL{URL: u}, metricsBuilder(nullIsAccounted))
case u.Scheme == changefeedbase.SinkSchemeKafka:
return validateOptionsAndMakeSink(changefeedbase.KafkaValidOptions, func() (Sink, error) {
return makeKafkaSink(ctx, sinkURL{URL: u}, AllTargets(feedCfg), opts.GetKafkaConfigJSON(), metricsBuilder)
})
case isWebhookSink(u):
webhookOpts, err := opts.GetWebhookSinkOptions()
if err != nil {
return nil, err
}
return validateOptionsAndMakeSink(changefeedbase.WebhookValidOptions, func() (Sink, error) {
return makeWebhookSink(ctx, sinkURL{URL: u}, encodingOpts, webhookOpts,
defaultWorkerCount(), timeutil.DefaultTimeSource{}, metricsBuilder)
})
case isPubsubSink(u):
// TODO: add metrics to pubsubsink
return MakePubsubSink(ctx, u, encodingOpts, AllTargets(feedCfg))
case isCloudStorageSink(u):
return validateOptionsAndMakeSink(changefeedbase.CloudStorageValidOptions, func() (Sink, error) {
return makeCloudStorageSink(
ctx, sinkURL{URL: u}, serverCfg.NodeID.SQLInstanceID(), serverCfg.Settings, encodingOpts,
timestampOracle, serverCfg.ExternalStorageFromURI, user, metricsBuilder,
)
})
case u.Scheme == changefeedbase.SinkSchemeExperimentalSQL:
return validateOptionsAndMakeSink(changefeedbase.SQLValidOptions, func() (Sink, error) {
return makeSQLSink(sinkURL{URL: u}, sqlSinkTableName, AllTargets(feedCfg), metricsBuilder)
})
case u.Scheme == "":
return nil, errors.Errorf(`no scheme found for sink URL %q`, feedCfg.SinkURI)
default:
return nil, errors.Errorf(`unsupported sink: %s`, u.Scheme)
}
}
sink, err := newSink()
if err != nil {
return nil, err
}
if knobs, ok := serverCfg.TestingKnobs.Changefeed.(*TestingKnobs); ok && knobs.WrapSink != nil {
sink = knobs.WrapSink(sink, jobID)
}
if err := sink.Dial(); err != nil {
return nil, err
}
return sink, nil
}
func validateSinkOptions(opts map[string]string, sinkSpecificOpts map[string]struct{}) error {
for opt := range opts {
if _, ok := changefeedbase.CommonOptions[opt]; ok {
continue
}
if sinkSpecificOpts != nil {
if _, ok := sinkSpecificOpts[opt]; ok {
continue
}
}
return errors.Errorf("this sink is incompatible with option %s", opt)
}
return nil
}
// sinkURL is a helper struct which for "consuming" URL query
// parameters from the underlying URL.
type sinkURL struct {
*url.URL
q url.Values
}
func (u *sinkURL) consumeParam(p string) string {
if u.q == nil {
u.q = u.Query()
}
v := u.q.Get(p)
u.q.Del(p)
return v
}
func (u *sinkURL) addParam(p string, value string) {
if u.q == nil {
u.q = u.Query()
}
u.q.Add(p, value)
}
func (u *sinkURL) consumeBool(param string, dest *bool) (wasSet bool, err error) {
if paramVal := u.consumeParam(param); paramVal != "" {
wasSet, err := strToBool(paramVal, dest)
if err != nil {
return false, errors.Wrapf(err, "param %s must be a bool", param)
}
return wasSet, err
}
return false, nil
}
func (u *sinkURL) decodeBase64(param string, dest *[]byte) error {
// TODO(dan): There's a straightforward and unambiguous transformation
// between the base 64 encoding defined in RFC 4648 and the URL variant
// defined in the same RFC: simply replace all `+` with `-` and `/` with
// `_`. Consider always doing this for the user and accepting either
// variant.
val := u.consumeParam(param)
err := decodeBase64FromString(val, dest)
if err != nil {
return errors.Wrapf(err, `param %s must be base 64 encoded`, param)
}
return nil
}
func (u *sinkURL) remainingQueryParams() (res []string) {
for p := range u.q {
res = append(res, p)
}
return
}
func (u *sinkURL) String() string {
if u.q != nil {
// If we changed query params, re-encode them.
u.URL.RawQuery = u.q.Encode()
u.q = nil
}
return u.URL.String()
}
// errorWrapperSink delegates to another sink and marks all returned errors as
// retryable. During changefeed setup, we use the sink once without this to
// verify configuration, but in the steady state, no sink error should be
// terminal.
type errorWrapperSink struct {
wrapped Sink
}
// EmitRow implements Sink interface.
func (s errorWrapperSink) EmitRow(
ctx context.Context,
topic TopicDescriptor,
key, value []byte,
updated, mvcc hlc.Timestamp,
alloc kvevent.Alloc,
) error {
if err := s.wrapped.EmitRow(ctx, topic, key, value, updated, mvcc, alloc); err != nil {
return changefeedbase.MarkRetryableError(err)
}
return nil
}
// EmitResolvedTimestamp implements Sink interface.
func (s errorWrapperSink) EmitResolvedTimestamp(
ctx context.Context, encoder Encoder, resolved hlc.Timestamp,
) error {
if err := s.wrapped.EmitResolvedTimestamp(ctx, encoder, resolved); err != nil {
return changefeedbase.MarkRetryableError(err)
}
return nil
}
// Flush implements Sink interface.
func (s errorWrapperSink) Flush(ctx context.Context) error {
if err := s.wrapped.Flush(ctx); err != nil {
return changefeedbase.MarkRetryableError(err)
}
return nil
}
// Close implements Sink interface.
func (s errorWrapperSink) Close() error {
if err := s.wrapped.Close(); err != nil {
return changefeedbase.MarkRetryableError(err)
}
return nil
}
// Dial implements Sink interface.
func (s errorWrapperSink) Dial() error {
return s.wrapped.Dial()
}
// encDatumRowBuffer is a FIFO of `EncDatumRow`s.
//
// TODO(dan): There's some potential allocation savings here by reusing the same
// backing array.
type encDatumRowBuffer []rowenc.EncDatumRow
func (b *encDatumRowBuffer) IsEmpty() bool {
return b == nil || len(*b) == 0
}
func (b *encDatumRowBuffer) Push(r rowenc.EncDatumRow) {
*b = append(*b, r)
}
func (b *encDatumRowBuffer) Pop() rowenc.EncDatumRow {
ret := (*b)[0]
*b = (*b)[1:]
return ret
}
type bufferSink struct {
buf encDatumRowBuffer
alloc tree.DatumAlloc
scratch bufalloc.ByteAllocator
closed bool
metrics metricsRecorder
}
// EmitRow implements the Sink interface.
func (s *bufferSink) EmitRow(
ctx context.Context,
topic TopicDescriptor,
key, value []byte,
updated, mvcc hlc.Timestamp,
r kvevent.Alloc,
) error {
defer r.Release(ctx)
defer s.metrics.recordOneMessage()(mvcc, len(key)+len(value), sinkDoesNotCompress)
if s.closed {
return errors.New(`cannot EmitRow on a closed sink`)
}
s.buf.Push(rowenc.EncDatumRow{
{Datum: tree.DNull}, // resolved span
{Datum: s.getTopicDatum(topic)},
{Datum: s.alloc.NewDBytes(tree.DBytes(key))}, // key
{Datum: s.alloc.NewDBytes(tree.DBytes(value))}, // value
})
return nil
}
// EmitResolvedTimestamp implements the Sink interface.
func (s *bufferSink) EmitResolvedTimestamp(
ctx context.Context, encoder Encoder, resolved hlc.Timestamp,
) error {
if s.closed {
return errors.New(`cannot EmitResolvedTimestamp on a closed sink`)
}
defer s.metrics.recordResolvedCallback()()
var noTopic string
payload, err := encoder.EncodeResolvedTimestamp(ctx, noTopic, resolved)
if err != nil {
return err
}
s.scratch, payload = s.scratch.Copy(payload, 0 /* extraCap */)
s.buf.Push(rowenc.EncDatumRow{
{Datum: tree.DNull}, // resolved span
{Datum: tree.DNull}, // topic
{Datum: tree.DNull}, // key
{Datum: s.alloc.NewDBytes(tree.DBytes(payload))}, // value
})
return nil
}
// Flush implements the Sink interface.
func (s *bufferSink) Flush(_ context.Context) error {
defer s.metrics.recordFlushRequestCallback()()
return nil
}
// Close implements the Sink interface.
func (s *bufferSink) Close() error {
s.closed = true
return nil
}
// Dial implements the Sink interface.
func (s *bufferSink) Dial() error {
return nil
}
// TODO (zinger): Make this a tuple or array datum if it can be
// done without breaking backwards compatibility.
func (s *bufferSink) getTopicDatum(t TopicDescriptor) *tree.DString {
name, components := t.GetNameComponents()
strs := append([]string{string(name)}, components...)
return s.alloc.NewDString(tree.DString(strings.Join(strs, ".")))
}
type nullSink struct {
ticker *time.Ticker
metrics metricsRecorder
}
var _ Sink = (*nullSink)(nil)
func makeNullSink(u sinkURL, m metricsRecorder) (Sink, error) {
var pacer *time.Ticker
if delay := u.consumeParam(`delay`); delay != "" {
pace, err := time.ParseDuration(delay)
if err != nil {
return nil, err
}
pacer = time.NewTicker(pace)
}
return &nullSink{ticker: pacer, metrics: m}, nil
}
func (n *nullSink) pace(ctx context.Context) error {
if n.ticker != nil {
select {
case <-n.ticker.C:
return nil
case <-ctx.Done():
return ctx.Err()
}
}
return nil
}
// EmitRow implements Sink interface.
func (n *nullSink) EmitRow(
ctx context.Context,
topic TopicDescriptor,
key, value []byte,
updated, mvcc hlc.Timestamp,
r kvevent.Alloc,
) error {
defer r.Release(ctx)
defer n.metrics.recordOneMessage()(mvcc, len(key)+len(value), sinkDoesNotCompress)
if err := n.pace(ctx); err != nil {
return err
}
if log.V(2) {
log.Infof(ctx, "emitting row %s@%s", key, updated.String())
}
return nil
}
// EmitResolvedTimestamp implements Sink interface.
func (n *nullSink) EmitResolvedTimestamp(
ctx context.Context, encoder Encoder, resolved hlc.Timestamp,
) error {
defer n.metrics.recordResolvedCallback()()
if err := n.pace(ctx); err != nil {
return err
}
if log.V(2) {
log.Infof(ctx, "emitting resolved %s", resolved.String())
}
return nil
}
// Flush implements Sink interface.
func (n *nullSink) Flush(ctx context.Context) error {
defer n.metrics.recordFlushRequestCallback()()
if log.V(2) {
log.Info(ctx, "flushing")
}
return nil
}
// Close implements Sink interface.
func (n *nullSink) Close() error {
return nil
}
// Dial implements Sink interface.
func (n *nullSink) Dial() error {
return nil
}