forked from pingcap/ticdc
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathwriter.go
More file actions
405 lines (369 loc) · 12.9 KB
/
writer.go
File metadata and controls
405 lines (369 loc) · 12.9 KB
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
// Copyright 2025 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 cloudstorage
import (
"bytes"
"context"
"path"
"strconv"
"sync/atomic"
"time"
"github.com/pingcap/failpoint"
"github.com/pingcap/log"
"github.com/pingcap/ticdc/downstreamadapter/sink/metrics"
commonType "github.com/pingcap/ticdc/pkg/common"
"github.com/pingcap/ticdc/pkg/errors"
pmetrics "github.com/pingcap/ticdc/pkg/metrics"
"github.com/pingcap/ticdc/pkg/pdutil"
"github.com/pingcap/ticdc/pkg/sink/cloudstorage"
"github.com/pingcap/ticdc/pkg/sink/codec/common"
"github.com/pingcap/ticdc/utils/chann"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
// writer denotes a worker responsible for writing messages to cloud storage.
type writer struct {
// worker id
id int
changeFeedID commonType.ChangeFeedID
storage storage.ExternalStorage
config *cloudstorage.Config
// toBeFlushedCh contains a set of batchedTask waiting to be flushed to cloud storage.
toBeFlushedCh chan batchedTask
inputCh *chann.DrainableChann[eventFragment]
isClosed uint64
statistics *pmetrics.Statistics
filePathGenerator *cloudstorage.FilePathGenerator
metricWriteBytes prometheus.Gauge
metricFileCount prometheus.Gauge
metricWriteDuration prometheus.Observer
metricFlushDuration prometheus.Observer
metricsWorkerBusyRatio prometheus.Counter
}
func newWriter(
id int,
changefeedID commonType.ChangeFeedID,
storage storage.ExternalStorage,
config *cloudstorage.Config,
extension string,
inputCh *chann.DrainableChann[eventFragment],
statistics *pmetrics.Statistics,
) *writer {
d := &writer{
id: id,
changeFeedID: changefeedID,
storage: storage,
config: config,
inputCh: inputCh,
toBeFlushedCh: make(chan batchedTask, 64),
statistics: statistics,
filePathGenerator: cloudstorage.NewFilePathGenerator(changefeedID, config, storage, extension),
metricWriteBytes: metrics.CloudStorageWriteBytesGauge.
WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String()),
metricFileCount: metrics.CloudStorageFileCountGauge.
WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String()),
metricWriteDuration: metrics.CloudStorageWriteDurationHistogram.
WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String()),
metricFlushDuration: metrics.CloudStorageFlushDurationHistogram.
WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String()),
metricsWorkerBusyRatio: metrics.CloudStorageWorkerBusyRatio.
WithLabelValues(changefeedID.Keyspace(), changefeedID.ID().String(), strconv.Itoa(id)),
}
return d
}
// Run creates a set of background goroutines.
func (d *writer) Run(ctx context.Context) error {
eg, ctx := errgroup.WithContext(ctx)
eg.Go(func() error {
return d.flushMessages(ctx)
})
eg.Go(func() error {
return d.genAndDispatchTask(ctx, d.inputCh)
})
return eg.Wait()
}
// SetClock is used for unit test
func (d *writer) SetClock(pdClock pdutil.Clock) {
d.filePathGenerator.SetClock(pdClock)
}
// flushMessages flushed messages of active tables to cloud storage.
// active tables are those tables that have received events after the last flush.
func (d *writer) flushMessages(ctx context.Context) error {
var flushTimeSlice time.Duration
overseerDuration := d.config.FlushInterval * 2
overseerTicker := time.NewTicker(overseerDuration)
defer overseerTicker.Stop()
for {
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case <-overseerTicker.C:
d.metricsWorkerBusyRatio.Add(flushTimeSlice.Seconds())
flushTimeSlice = 0
case batchedTask := <-d.toBeFlushedCh:
if atomic.LoadUint64(&d.isClosed) == 1 {
return nil
}
start := time.Now()
for table, task := range batchedTask.batch {
if len(task.msgs) == 0 {
continue
}
// generate scheme.json file before generating the first data file if necessary
hasNewerSchemaVersion, err := d.filePathGenerator.CheckOrWriteSchema(ctx, table, task.tableInfo)
if err != nil {
log.Error("failed to write schema file to external storage",
zap.Int("workerID", d.id),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()),
zap.Error(err))
return errors.Trace(err)
}
// It is possible that a DML event is sent after a DDL event during dispatcher scheduling.
// We need to ignore such DML events, as they belong to a stale schema version.
if hasNewerSchemaVersion {
d.ignoreTableTask(task)
log.Warn("ignore messages belonging to an old schema version",
zap.Int("workerID", d.id),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()),
zap.String("schema", table.TableNameWithPhysicTableID.Schema),
zap.String("table", table.TableNameWithPhysicTableID.Table),
zap.Uint64("version", table.TableInfoVersion))
continue
}
// make sure that `generateDateStr()` is invoked ONLY once before
// generating data file path and index file path. Because we don't expect the index
// file is written to a different dir if date change happens between
// generating data and index file.
date := d.filePathGenerator.GenerateDateStr()
dataFilePath, err := d.filePathGenerator.GenerateDataFilePath(ctx, table, date)
if err != nil {
log.Error("failed to generate data file path",
zap.Int("workerID", d.id),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()),
zap.Error(err))
return errors.Trace(err)
}
indexFilePath, err := d.filePathGenerator.GenerateIndexFilePath(table, date)
if err != nil {
log.Error("failed to generate index file path",
zap.Int("workerID", d.id),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()),
zap.Error(err))
return errors.Trace(err)
}
// first write the data file to external storage.
err = d.writeDataFile(ctx, dataFilePath, indexFilePath, task)
if err != nil {
log.Error("failed to write data file to external storage",
zap.Int("workerID", d.id),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()),
zap.String("path", dataFilePath),
zap.Error(err))
return errors.Trace(err)
}
log.Debug("write file to storage success", zap.Int("workerID", d.id),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()),
zap.String("schema", table.TableNameWithPhysicTableID.Schema),
zap.String("table", table.TableNameWithPhysicTableID.Table),
zap.String("path", dataFilePath),
)
}
flushTimeSlice += time.Since(start)
}
}
}
func (d *writer) writeIndexFile(ctx context.Context, path, content string) error {
start := time.Now()
err := d.storage.WriteFile(ctx, path, []byte(content))
d.metricFlushDuration.Observe(time.Since(start).Seconds())
return err
}
func (d *writer) ignoreTableTask(task *singleTableTask) {
for _, msg := range task.msgs {
if msg.Callback != nil {
msg.Callback()
}
}
}
func (d *writer) writeDataFile(ctx context.Context, dataFilePath, indexFilePath string, task *singleTableTask) error {
var callbacks []func()
buf := bytes.NewBuffer(make([]byte, 0, task.size))
rowsCnt := 0
bytesCnt := int64(0)
// There is always only one message here in task.msgs
for _, msg := range task.msgs {
if msg.Key != nil && rowsCnt == 0 {
buf.Write(msg.Key)
bytesCnt += int64(len(msg.Key))
}
bytesCnt += int64(len(msg.Value))
rowsCnt += msg.GetRowsCount()
buf.Write(msg.Value)
callbacks = append(callbacks, msg.Callback)
}
if err := d.statistics.RecordBatchExecution(func() (int, int64, error) {
start := time.Now()
if d.config.FlushConcurrency <= 1 {
return rowsCnt, bytesCnt, d.storage.WriteFile(ctx, dataFilePath, buf.Bytes())
}
writer, inErr := d.storage.Create(ctx, dataFilePath, &storage.WriterOption{
Concurrency: d.config.FlushConcurrency,
})
if inErr != nil {
return 0, 0, inErr
}
if _, inErr = writer.Write(ctx, buf.Bytes()); inErr != nil {
return 0, 0, inErr
}
// We have to wait the writer to close to complete the upload
// If failed to close writer, some DMLs may not be upload successfully
if inErr = writer.Close(ctx); inErr != nil {
log.Error("failed to close writer", zap.Error(inErr),
zap.Int("workerID", d.id),
zap.Any("table", task.tableInfo.TableName),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()))
return 0, 0, inErr
}
d.metricFlushDuration.Observe(time.Since(start).Seconds())
return rowsCnt, bytesCnt, nil
}); err != nil {
return err
}
d.metricWriteBytes.Add(float64(bytesCnt))
d.metricFileCount.Add(1)
// then write the index file to external storage in the end.
// the file content is simply the last data file path
err := d.writeIndexFile(ctx, indexFilePath, path.Base(dataFilePath)+"\n")
if err != nil {
log.Error("failed to write index file to external storage",
zap.Int("workerID", d.id),
zap.String("keyspace", d.changeFeedID.Keyspace()),
zap.Stringer("changefeed", d.changeFeedID.ID()),
zap.String("path", indexFilePath),
zap.Error(err))
return errors.Trace(err)
}
for _, cb := range callbacks {
if cb != nil {
cb()
}
}
return nil
}
// genAndDispatchTask dispatches flush tasks in two conditions:
// 1. the flush interval exceeds the upper limit.
// 2. the file size exceeds the upper limit.
func (d *writer) genAndDispatchTask(ctx context.Context,
ch *chann.DrainableChann[eventFragment],
) error {
batchedTask := newBatchedTask()
ticker := time.NewTicker(d.config.FlushInterval)
defer ticker.Stop()
for {
// this failpoint is use to pass this ticker once
// to make writeEvent in the test case can write into the same file
failpoint.Inject("passTickerOnce", func() {
<-ticker.C
})
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case <-ticker.C:
if atomic.LoadUint64(&d.isClosed) == 1 {
return nil
}
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case d.toBeFlushedCh <- batchedTask:
log.Debug("flush task is emitted successfully when flush interval exceeds",
zap.Int("tablesLength", len(batchedTask.batch)))
batchedTask = newBatchedTask()
default:
}
case frag, ok := <-ch.Out():
if !ok || atomic.LoadUint64(&d.isClosed) == 1 {
return nil
}
batchedTask.handleSingleTableEvent(frag)
// if the file size exceeds the upper limit, emit the flush task containing the table
// as soon as possible.
table := frag.versionedTable
if batchedTask.batch[table].size >= uint64(d.config.FileSize) {
task := batchedTask.generateTaskByTable(table)
select {
case <-ctx.Done():
return errors.Trace(ctx.Err())
case d.toBeFlushedCh <- task:
log.Debug("flush task is emitted successfully when file size exceeds",
zap.Any("table", table),
zap.Int("eventsLenth", len(task.batch[table].msgs)))
}
}
}
}
}
func (d *writer) close() {
if !atomic.CompareAndSwapUint64(&d.isClosed, 0, 1) {
return
}
}
// batchedTask contains a set of singleTableTask.
// We batch message of different tables together to reduce the overhead of calling external storage API.
type batchedTask struct {
batch map[cloudstorage.VersionedTableName]*singleTableTask
}
// singleTableTask contains a set of messages belonging to the same table.
type singleTableTask struct {
size uint64
tableInfo *commonType.TableInfo
msgs []*common.Message
}
func newBatchedTask() batchedTask {
return batchedTask{
batch: make(map[cloudstorage.VersionedTableName]*singleTableTask),
}
}
func (t *batchedTask) handleSingleTableEvent(event eventFragment) {
table := event.versionedTable
if _, ok := t.batch[table]; !ok {
t.batch[table] = &singleTableTask{
size: 0,
tableInfo: event.event.TableInfo,
}
}
v := t.batch[table]
for _, msg := range event.encodedMsgs {
v.size += uint64(len(msg.Value))
}
v.msgs = append(v.msgs, event.encodedMsgs...)
}
func (t *batchedTask) generateTaskByTable(table cloudstorage.VersionedTableName) batchedTask {
v := t.batch[table]
if v == nil {
log.Panic("table not found in dml task", zap.Any("table", table), zap.Any("task", t))
}
delete(t.batch, table)
return batchedTask{
batch: map[cloudstorage.VersionedTableName]*singleTableTask{table: v},
}
}