-
Notifications
You must be signed in to change notification settings - Fork 269
/
worker.go
327 lines (301 loc) · 9.96 KB
/
worker.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
// Copyright 2022 PingCAP, Inc.
//
// 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,
// See the License for the specific language governing permissions and
// limitations under the License.
package mq
import (
"context"
"time"
"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/cdc/sink/dmlsink"
"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dmlproducer"
"github.com/pingcap/tiflow/cdc/sink/metrics"
"github.com/pingcap/tiflow/cdc/sink/metrics/mq"
"github.com/pingcap/tiflow/cdc/sink/tablesink/state"
"github.com/pingcap/tiflow/pkg/chann"
"github.com/pingcap/tiflow/pkg/config"
"github.com/pingcap/tiflow/pkg/sink/codec"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
const (
// batchSize is the maximum size of the number of messages in a batch.
batchSize = 2048
// batchInterval is the interval of the worker to collect a batch of messages.
// It shouldn't be too large, otherwise it will lead to a high latency.
batchInterval = 15 * time.Millisecond
)
// mqEvent is the event of the mq worker.
// It carries the topic and partition information of the message.
type mqEvent struct {
key model.TopicPartitionKey
rowEvent *dmlsink.RowChangeCallbackableEvent
}
// worker will send messages to the DML producer on a batch basis.
type worker struct {
// changeFeedID indicates this sink belongs to which processor(changefeed).
changeFeedID model.ChangeFeedID
// protocol indicates the protocol used by this sink.
protocol config.Protocol
// msgChan caches the messages to be sent.
// It is an unbounded channel.
msgChan *chann.DrainableChann[mqEvent]
// ticker used to force flush the batched messages when the interval is reached.
ticker *time.Ticker
encoderGroup codec.EncoderGroup
// producer is used to send the messages to the Kafka broker.
producer dmlproducer.DMLProducer
// metricMQWorkerSendMessageDuration tracks the time duration cost on send messages.
metricMQWorkerSendMessageDuration prometheus.Observer
// metricMQWorkerBatchSize tracks each batch's size.
metricMQWorkerBatchSize prometheus.Observer
// metricMQWorkerBatchDuration tracks the time duration cost on batch messages.
metricMQWorkerBatchDuration prometheus.Observer
// statistics is used to record DML metrics.
statistics *metrics.Statistics
}
// newWorker creates a new flush worker.
func newWorker(
id model.ChangeFeedID,
protocol config.Protocol,
producer dmlproducer.DMLProducer,
encoderGroup codec.EncoderGroup,
statistics *metrics.Statistics,
) *worker {
w := &worker{
changeFeedID: id,
protocol: protocol,
msgChan: chann.NewAutoDrainChann[mqEvent](),
ticker: time.NewTicker(batchInterval),
encoderGroup: encoderGroup,
producer: producer,
metricMQWorkerSendMessageDuration: mq.WorkerSendMessageDuration.WithLabelValues(id.Namespace, id.ID),
metricMQWorkerBatchSize: mq.WorkerBatchSize.WithLabelValues(id.Namespace, id.ID),
metricMQWorkerBatchDuration: mq.WorkerBatchDuration.WithLabelValues(id.Namespace, id.ID),
statistics: statistics,
}
return w
}
// run starts a loop that keeps collecting, sorting and sending messages
// until it encounters an error or is interrupted.
func (w *worker) run(ctx context.Context) (retErr error) {
defer func() {
w.ticker.Stop()
log.Info("MQ sink worker exited", zap.Error(retErr),
zap.String("namespace", w.changeFeedID.Namespace),
zap.String("changefeed", w.changeFeedID.ID),
zap.String("protocol", w.protocol.String()),
)
}()
g, ctx := errgroup.WithContext(ctx)
g.Go(func() error {
return w.encoderGroup.Run(ctx)
})
g.Go(func() error {
if w.protocol.IsBatchEncode() {
return w.batchEncodeRun(ctx)
}
return w.nonBatchEncodeRun(ctx)
})
g.Go(func() error {
return w.sendMessages(ctx)
})
return g.Wait()
}
// nonBatchEncodeRun add events to the encoder group immediately.
func (w *worker) nonBatchEncodeRun(ctx context.Context) error {
log.Info("MQ sink non batch worker started",
zap.String("namespace", w.changeFeedID.Namespace),
zap.String("changefeed", w.changeFeedID.ID),
zap.String("protocol", w.protocol.String()),
)
for {
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case event, ok := <-w.msgChan.Out():
if !ok {
log.Warn("MQ sink flush worker channel closed",
zap.String("namespace", w.changeFeedID.Namespace),
zap.String("changefeed", w.changeFeedID.ID))
return nil
}
if event.rowEvent.GetTableSinkState() != state.TableSinkSinking {
event.rowEvent.Callback()
log.Debug("Skip event of stopped table",
zap.String("namespace", w.changeFeedID.Namespace),
zap.String("changefeed", w.changeFeedID.ID),
zap.Any("event", event))
continue
}
if err := w.encoderGroup.AddEvents(
ctx,
event.key,
event.rowEvent); err != nil {
return errors.Trace(err)
}
}
}
}
// batchEncodeRun collect messages into batch and add them to the encoder group.
func (w *worker) batchEncodeRun(ctx context.Context) (retErr error) {
log.Info("MQ sink batch worker started",
zap.String("namespace", w.changeFeedID.Namespace),
zap.String("changefeed", w.changeFeedID.ID),
zap.String("protocol", w.protocol.String()),
)
msgsBuf := make([]mqEvent, batchSize)
for {
start := time.Now()
msgCount, err := w.batch(ctx, msgsBuf, batchInterval)
if err != nil {
return errors.Trace(err)
}
if msgCount == 0 {
continue
}
w.metricMQWorkerBatchSize.Observe(float64(msgCount))
w.metricMQWorkerBatchDuration.Observe(time.Since(start).Seconds())
msgs := msgsBuf[:msgCount]
// Group messages by its TopicPartitionKey before adding them to the encoder group.
groupedMsgs := w.group(msgs)
for key, msg := range groupedMsgs {
if err := w.encoderGroup.AddEvents(ctx, key, msg...); err != nil {
return errors.Trace(err)
}
}
}
}
// batch collects a batch of messages from w.msgChan into buffer.
// It returns the number of messages collected.
// Note: It will block until at least one message is received.
func (w *worker) batch(
ctx context.Context, buffer []mqEvent, flushInterval time.Duration,
) (int, error) {
msgCount := 0
maxBatchSize := len(buffer)
// We need to receive at least one message or be interrupted,
// otherwise it will lead to idling.
select {
case <-ctx.Done():
return msgCount, ctx.Err()
case msg, ok := <-w.msgChan.Out():
if !ok {
log.Warn("MQ sink flush worker channel closed")
return msgCount, nil
}
if msg.rowEvent != nil {
w.statistics.ObserveRows(msg.rowEvent.Event)
buffer[msgCount] = msg
msgCount++
}
}
// Reset the ticker to start a new batching.
// We need to stop batching when the interval is reached.
w.ticker.Reset(flushInterval)
for {
select {
case <-ctx.Done():
return msgCount, ctx.Err()
case msg, ok := <-w.msgChan.Out():
if !ok {
log.Warn("MQ sink flush worker channel closed")
return msgCount, nil
}
if msg.rowEvent != nil {
w.statistics.ObserveRows(msg.rowEvent.Event)
buffer[msgCount] = msg
msgCount++
}
if msgCount >= maxBatchSize {
return msgCount, nil
}
case <-w.ticker.C:
return msgCount, nil
}
}
}
// group groups messages by its key.
func (w *worker) group(
msgs []mqEvent,
) map[model.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent {
groupedMsgs := make(map[model.TopicPartitionKey][]*dmlsink.RowChangeCallbackableEvent)
for _, msg := range msgs {
// Skip this event when the table is stopping.
if msg.rowEvent.GetTableSinkState() != state.TableSinkSinking {
msg.rowEvent.Callback()
log.Debug("Skip event of stopped table", zap.Any("event", msg.rowEvent))
continue
}
if _, ok := groupedMsgs[msg.key]; !ok {
groupedMsgs[msg.key] = make([]*dmlsink.RowChangeCallbackableEvent, 0)
}
groupedMsgs[msg.key] = append(groupedMsgs[msg.key], msg.rowEvent)
}
return groupedMsgs
}
func (w *worker) sendMessages(ctx context.Context) error {
ticker := time.NewTicker(15 * time.Second)
metric := codec.EncoderGroupOutputChanSizeGauge.
WithLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
defer func() {
ticker.Stop()
codec.EncoderGroupOutputChanSizeGauge.
DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
}()
var err error
outCh := w.encoderGroup.Output()
for {
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case <-ticker.C:
metric.Set(float64(len(outCh)))
case future, ok := <-outCh:
if !ok {
log.Warn("MQ sink encoder's output channel closed",
zap.String("namespace", w.changeFeedID.Namespace),
zap.String("changefeed", w.changeFeedID.ID))
return nil
}
if err = future.Ready(ctx); err != nil {
return errors.Trace(err)
}
for _, message := range future.Messages {
start := time.Now()
if err = w.statistics.RecordBatchExecution(func() (int, int64, error) {
message.SetPartitionKey(future.Key.PartitionKey)
if err := w.producer.AsyncSendMessage(
ctx,
future.Key.Topic,
future.Key.Partition,
message); err != nil {
return 0, 0, err
}
return message.GetRowsCount(), int64(message.Length()), nil
}); err != nil {
return err
}
w.metricMQWorkerSendMessageDuration.Observe(time.Since(start).Seconds())
}
}
}
}
func (w *worker) close() {
w.msgChan.CloseAndDrain()
w.producer.Close()
mq.WorkerSendMessageDuration.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
mq.WorkerBatchSize.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
mq.WorkerBatchDuration.DeleteLabelValues(w.changeFeedID.Namespace, w.changeFeedID.ID)
}