-
Notifications
You must be signed in to change notification settings - Fork 269
/
main.go
966 lines (853 loc) · 27.7 KB
/
main.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
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
// Copyright 2020 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 main
import (
"context"
"database/sql"
"errors"
"flag"
"fmt"
"math"
"net/http"
_ "net/http/pprof"
"net/url"
"os"
"os/signal"
"runtime/debug"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"syscall"
"time"
"github.com/IBM/sarama"
"github.com/google/uuid"
cerror "github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/cdc/sink/ddlsink"
ddlsinkfactory "github.com/pingcap/tiflow/cdc/sink/ddlsink/factory"
eventsinkfactory "github.com/pingcap/tiflow/cdc/sink/dmlsink/factory"
"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher"
"github.com/pingcap/tiflow/cdc/sink/tablesink"
cmdUtil "github.com/pingcap/tiflow/pkg/cmd/util"
"github.com/pingcap/tiflow/pkg/config"
"github.com/pingcap/tiflow/pkg/filter"
"github.com/pingcap/tiflow/pkg/logutil"
"github.com/pingcap/tiflow/pkg/quotes"
"github.com/pingcap/tiflow/pkg/security"
"github.com/pingcap/tiflow/pkg/sink/codec"
"github.com/pingcap/tiflow/pkg/sink/codec/avro"
"github.com/pingcap/tiflow/pkg/sink/codec/canal"
"github.com/pingcap/tiflow/pkg/sink/codec/common"
"github.com/pingcap/tiflow/pkg/sink/codec/open"
"github.com/pingcap/tiflow/pkg/sink/codec/simple"
"github.com/pingcap/tiflow/pkg/spanz"
"github.com/pingcap/tiflow/pkg/util"
"github.com/pingcap/tiflow/pkg/version"
"go.uber.org/zap"
)
func newConsumerOption() *consumerOption {
return &consumerOption{
version: "2.4.0",
maxMessageBytes: math.MaxInt64,
maxBatchSize: math.MaxInt64,
}
}
type consumerOption struct {
address []string
version string
topic string
partitionNum int32
groupID string
maxMessageBytes int
maxBatchSize int
protocol config.Protocol
codecConfig *common.Config
// the replicaConfig of the changefeed which produce data to the kafka topic
replicaConfig *config.ReplicaConfig
logPath string
logLevel string
timezone string
ca, cert, key string
downstreamURI string
// avro schema registry uri should be set if the encoding protocol is avro
schemaRegistryURI string
// upstreamTiDBDSN is the dsn of the upstream TiDB cluster
upstreamTiDBDSN string
enableProfiling bool
}
// Adjust the consumer option by the upstream uri passed in parameters.
func (o *consumerOption) Adjust(upstreamURI *url.URL, configFile string) error {
s := upstreamURI.Query().Get("version")
if s != "" {
o.version = s
}
o.topic = strings.TrimFunc(upstreamURI.Path, func(r rune) bool {
return r == '/'
})
o.address = strings.Split(upstreamURI.Host, ",")
s = upstreamURI.Query().Get("partition-num")
if s == "" {
partition, err := getPartitionNum(o.address, o.topic)
if err != nil {
log.Panic("can not get partition number", zap.String("topic", o.topic), zap.Error(err))
}
o.partitionNum = partition
} else {
c, err := strconv.ParseInt(s, 10, 32)
if err != nil {
log.Panic("invalid partition-num of upstream-uri")
}
o.partitionNum = int32(c)
}
s = upstreamURI.Query().Get("max-message-bytes")
if s != "" {
c, err := strconv.Atoi(s)
if err != nil {
log.Panic("invalid max-message-bytes of upstream-uri")
}
o.maxMessageBytes = c
}
s = upstreamURI.Query().Get("max-batch-size")
if s != "" {
c, err := strconv.Atoi(s)
if err != nil {
log.Panic("invalid max-batch-size of upstream-uri")
}
o.maxBatchSize = c
}
s = upstreamURI.Query().Get("protocol")
if s == "" {
log.Panic("cannot found the protocol from the sink url")
}
protocol, err := config.ParseSinkProtocolFromString(s)
if err != nil {
log.Panic("invalid protocol", zap.Error(err), zap.String("protocol", s))
}
o.protocol = protocol
replicaConfig := config.GetDefaultReplicaConfig()
replicaConfig.Sink.Protocol = util.AddressOf(protocol.String())
if configFile != "" {
err = cmdUtil.StrictDecodeFile(configFile, "kafka consumer", replicaConfig)
if err != nil {
return cerror.Trace(err)
}
if _, err = filter.VerifyTableRules(replicaConfig.Filter); err != nil {
return cerror.Trace(err)
}
}
o.replicaConfig = replicaConfig
o.codecConfig = common.NewConfig(protocol)
if err = o.codecConfig.Apply(upstreamURI, o.replicaConfig); err != nil {
return cerror.Trace(err)
}
if protocol == config.ProtocolAvro {
o.codecConfig.AvroEnableWatermark = true
}
log.Info("consumer option adjusted",
zap.String("configFile", configFile),
zap.String("address", strings.Join(o.address, ",")),
zap.String("version", o.version),
zap.String("topic", o.topic),
zap.Int32("partitionNum", o.partitionNum),
zap.String("groupID", o.groupID),
zap.Int("maxMessageBytes", o.maxMessageBytes),
zap.Int("maxBatchSize", o.maxBatchSize),
zap.String("upstreamURI", upstreamURI.String()))
return nil
}
func main() {
debug.SetMemoryLimit(14 * 1024 * 1024 * 1024)
consumerOption := newConsumerOption()
var (
upstreamURIStr string
configFile string
)
groupID := fmt.Sprintf("ticdc_kafka_consumer_%s", uuid.New().String())
flag.StringVar(&configFile, "config", "", "config file for changefeed")
flag.StringVar(&upstreamURIStr, "upstream-uri", "", "Kafka uri")
flag.StringVar(&consumerOption.downstreamURI, "downstream-uri", "", "downstream sink uri")
flag.StringVar(&consumerOption.schemaRegistryURI, "schema-registry-uri", "", "schema registry uri")
flag.StringVar(&consumerOption.upstreamTiDBDSN, "upstream-tidb-dsn", "", "upstream TiDB DSN")
flag.StringVar(&consumerOption.groupID, "consumer-group-id", groupID, "consumer group id")
flag.StringVar(&consumerOption.logPath, "log-file", "cdc_kafka_consumer.log", "log file path")
flag.StringVar(&consumerOption.logLevel, "log-level", "info", "log file path")
flag.StringVar(&consumerOption.timezone, "tz", "System", "Specify time zone of Kafka consumer")
flag.StringVar(&consumerOption.ca, "ca", "", "CA certificate path for Kafka SSL connection")
flag.StringVar(&consumerOption.cert, "cert", "", "Certificate path for Kafka SSL connection")
flag.StringVar(&consumerOption.key, "key", "", "Private key path for Kafka SSL connection")
flag.BoolVar(&consumerOption.enableProfiling, "enable-profiling", false, "enable pprof profiling")
flag.Parse()
err := logutil.InitLogger(&logutil.Config{
Level: consumerOption.logLevel,
File: consumerOption.logPath,
},
logutil.WithInitGRPCLogger(),
logutil.WithInitSaramaLogger(),
)
if err != nil {
log.Error("init logger failed", zap.Error(err))
return
}
version.LogVersionInfo("kafka consumer")
upstreamURI, err := url.Parse(upstreamURIStr)
if err != nil {
log.Panic("invalid upstream-uri", zap.Error(err))
}
scheme := strings.ToLower(upstreamURI.Scheme)
if scheme != "kafka" {
log.Panic("invalid upstream-uri scheme, the scheme of upstream-uri must be `kafka`",
zap.String("upstreamURI", upstreamURIStr))
}
err = consumerOption.Adjust(upstreamURI, configFile)
if err != nil {
log.Panic("adjust consumer option failed", zap.Error(err))
}
///**
// * Construct a new Sarama configuration.
// * The Kafka cluster version has to be defined before the consumer/producer is initialized.
// */
config, err := newSaramaConfig(consumerOption)
if err != nil {
log.Panic("Error creating sarama config", zap.Error(err))
}
err = waitTopicCreated(consumerOption.address, consumerOption.topic, config)
if err != nil {
log.Panic("wait topic created failed", zap.Error(err))
}
ctx, cancel := context.WithCancel(context.Background())
consumer, err := NewConsumer(ctx, consumerOption)
if err != nil {
log.Panic("Error creating consumer", zap.Error(err))
}
client, err := sarama.NewConsumerGroup(consumerOption.address, consumerOption.groupID, config)
if err != nil {
log.Panic("Error creating consumer group client", zap.Error(err))
}
var wg sync.WaitGroup
if consumerOption.enableProfiling {
wg.Add(1)
go func() {
defer wg.Done()
if err := http.ListenAndServe(":6060", nil); err != nil {
log.Panic("Error starting pprof", zap.Error(err))
}
}()
}
wg.Add(1)
go func() {
defer wg.Done()
for {
// `consume` should be called inside an infinite loop, when a
// server-side rebalance happens, the consumer session will need to be
// recreated to get the new claims
if err := client.Consume(ctx, strings.Split(consumerOption.topic, ","), consumer); err != nil {
log.Panic("Error from consumer", zap.Error(err))
}
// check if context was cancelled, signaling that the consumer should stop
if ctx.Err() != nil {
return
}
consumer.ready = make(chan bool)
}
}()
go func() {
if err := consumer.Run(ctx); err != nil {
if err != context.Canceled {
log.Panic("Error running consumer", zap.Error(err))
}
}
}()
<-consumer.ready // wait till the consumer has been set up
log.Info("TiCDC consumer up and running!...")
sigterm := make(chan os.Signal, 1)
signal.Notify(sigterm, syscall.SIGINT, syscall.SIGTERM)
select {
case <-ctx.Done():
log.Info("terminating: context cancelled")
case <-sigterm:
log.Info("terminating: via signal")
}
cancel()
wg.Wait()
if err = client.Close(); err != nil {
log.Panic("Error closing client", zap.Error(err))
}
}
func getPartitionNum(address []string, topic string) (int32, error) {
saramaConfig := sarama.NewConfig()
// get partition number or create topic automatically
admin, err := sarama.NewClusterAdmin(address, saramaConfig)
if err != nil {
return 0, cerror.Trace(err)
}
topics, err := admin.ListTopics()
if err != nil {
return 0, cerror.Trace(err)
}
err = admin.Close()
if err != nil {
return 0, cerror.Trace(err)
}
topicDetail, exist := topics[topic]
if !exist {
return 0, cerror.Errorf("can not find topic %s", topic)
}
log.Info("get partition number of topic",
zap.String("topic", topic),
zap.Int32("partitionNum", topicDetail.NumPartitions))
return topicDetail.NumPartitions, nil
}
func waitTopicCreated(address []string, topic string, cfg *sarama.Config) error {
admin, err := sarama.NewClusterAdmin(address, cfg)
if err != nil {
return cerror.Trace(err)
}
defer admin.Close()
for i := 0; i <= 30; i++ {
topics, err := admin.ListTopics()
if err != nil {
return cerror.Trace(err)
}
if _, ok := topics[topic]; ok {
return nil
}
log.Info("wait the topic created", zap.String("topic", topic))
time.Sleep(1 * time.Second)
}
return cerror.Errorf("wait the topic(%s) created timeout", topic)
}
func newSaramaConfig(o *consumerOption) (*sarama.Config, error) {
config := sarama.NewConfig()
version, err := sarama.ParseKafkaVersion(o.version)
if err != nil {
return nil, cerror.Trace(err)
}
config.ClientID = "ticdc_kafka_sarama_consumer"
config.Version = version
config.Metadata.Retry.Max = 10000
config.Metadata.Retry.Backoff = 500 * time.Millisecond
config.Consumer.Retry.Backoff = 500 * time.Millisecond
config.Consumer.Offsets.Initial = sarama.OffsetOldest
if len(o.ca) != 0 {
config.Net.TLS.Enable = true
config.Net.TLS.Config, err = (&security.Credential{
CAPath: o.ca,
CertPath: o.cert,
KeyPath: o.key,
}).ToTLSConfig()
if err != nil {
return nil, cerror.Trace(err)
}
}
return config, err
}
// partitionSinks maintained for each partition, it may sync data for multiple tables.
type partitionSinks struct {
tablesCommitTsMap sync.Map
tableSinksMap sync.Map
// resolvedTs record the maximum timestamp of the received event
resolvedTs uint64
}
// Consumer represents a Sarama consumer group consumer
type Consumer struct {
ready chan bool
ddlList []*model.DDLEvent
ddlListMu sync.Mutex
ddlWithMaxCommitTs *model.DDLEvent
ddlSink ddlsink.Sink
fakeTableIDGenerator *fakeTableIDGenerator
// sinkFactory is used to create table sink for each table.
sinkFactory *eventsinkfactory.SinkFactory
sinks []*partitionSinks
sinksMu sync.Mutex
eventRouter *dispatcher.EventRouter
option *consumerOption
upstreamTiDB *sql.DB
}
// NewConsumer creates a new cdc kafka consumer
func NewConsumer(ctx context.Context, o *consumerOption) (*Consumer, error) {
c := new(Consumer)
c.option = o
tz, err := util.GetTimezone(o.timezone)
if err != nil {
return nil, cerror.Annotate(err, "can not load timezone")
}
config.GetGlobalServerConfig().TZ = o.timezone
o.codecConfig.TimeZone = tz
c.fakeTableIDGenerator = &fakeTableIDGenerator{
tableIDs: make(map[string]int64),
}
if o.codecConfig.LargeMessageHandle.HandleKeyOnly() {
db, err := openDB(ctx, o.upstreamTiDBDSN)
if err != nil {
return nil, err
}
c.upstreamTiDB = db
}
eventRouter, err := dispatcher.NewEventRouter(o.replicaConfig, o.protocol, o.topic, "kafka")
if err != nil {
return nil, cerror.Trace(err)
}
c.eventRouter = eventRouter
c.sinks = make([]*partitionSinks, o.partitionNum)
ctx, cancel := context.WithCancel(ctx)
errChan := make(chan error, 1)
for i := 0; i < int(o.partitionNum); i++ {
c.sinks[i] = &partitionSinks{}
}
changefeedID := model.DefaultChangeFeedID("kafka-consumer")
f, err := eventsinkfactory.New(ctx, changefeedID, o.downstreamURI, o.replicaConfig, errChan, nil)
if err != nil {
cancel()
return nil, cerror.Trace(err)
}
c.sinkFactory = f
go func() {
err := <-errChan
if !errors.Is(cerror.Cause(err), context.Canceled) {
log.Error("error on running consumer", zap.Error(err))
} else {
log.Info("consumer exited")
}
cancel()
}()
ddlSink, err := ddlsinkfactory.New(ctx, changefeedID, o.downstreamURI, o.replicaConfig)
if err != nil {
cancel()
return nil, cerror.Trace(err)
}
c.ddlSink = ddlSink
c.ready = make(chan bool)
return c, nil
}
// Setup is run at the beginning of a new session, before ConsumeClaim
func (c *Consumer) Setup(sarama.ConsumerGroupSession) error {
// Mark the c as ready
close(c.ready)
return nil
}
// Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited
func (c *Consumer) Cleanup(sarama.ConsumerGroupSession) error {
return nil
}
type eventsGroup struct {
events []*model.RowChangedEvent
}
func newEventsGroup() *eventsGroup {
return &eventsGroup{
events: make([]*model.RowChangedEvent, 0),
}
}
func (g *eventsGroup) Append(e *model.RowChangedEvent) {
g.events = append(g.events, e)
}
func (g *eventsGroup) Resolve(resolveTs uint64) []*model.RowChangedEvent {
sort.Slice(g.events, func(i, j int) bool {
return g.events[i].CommitTs < g.events[j].CommitTs
})
i := sort.Search(len(g.events), func(i int) bool {
return g.events[i].CommitTs > resolveTs
})
result := g.events[:i]
g.events = g.events[i:]
return result
}
// ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
func (c *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
partition := claim.Partition()
c.sinksMu.Lock()
sink := c.sinks[partition]
c.sinksMu.Unlock()
if sink == nil {
panic("sink should initialized")
}
ctx := context.Background()
var (
decoder codec.RowEventDecoder
err error
)
switch c.option.protocol {
case config.ProtocolOpen, config.ProtocolDefault:
decoder, err = open.NewBatchDecoder(ctx, c.option.codecConfig, c.upstreamTiDB)
case config.ProtocolCanalJSON:
decoder, err = canal.NewBatchDecoder(ctx, c.option.codecConfig, c.upstreamTiDB)
if err != nil {
return err
}
case config.ProtocolAvro:
schemaM, err := avro.NewConfluentSchemaManager(ctx, c.option.schemaRegistryURI, nil)
if err != nil {
return cerror.Trace(err)
}
decoder = avro.NewDecoder(c.option.codecConfig, schemaM, c.option.topic)
case config.ProtocolSimple:
decoder, err = simple.NewDecoder(ctx, c.option.codecConfig, c.upstreamTiDB)
default:
log.Panic("Protocol not supported", zap.Any("Protocol", c.option.protocol))
}
if err != nil {
return cerror.Trace(err)
}
log.Info("start consume claim",
zap.String("topic", claim.Topic()), zap.Int32("partition", partition),
zap.Int64("initialOffset", claim.InitialOffset()), zap.Int64("highWaterMarkOffset", claim.HighWaterMarkOffset()))
eventGroups := make(map[int64]*eventsGroup)
for message := range claim.Messages() {
if err = decoder.AddKeyValue(message.Key, message.Value); err != nil {
log.Error("add key value to the decoder failed", zap.Error(err))
return cerror.Trace(err)
}
counter := 0
for {
tp, hasNext, err := decoder.HasNext()
if err != nil {
log.Panic("decode message key failed", zap.Error(err))
}
if !hasNext {
break
}
counter++
// If the message containing only one event exceeds the length limit, CDC will allow it and issue a warning.
if len(message.Key)+len(message.Value) > c.option.maxMessageBytes && counter > 1 {
log.Panic("kafka max-messages-bytes exceeded",
zap.Int("max-message-bytes", c.option.maxMessageBytes),
zap.Int("receivedBytes", len(message.Key)+len(message.Value)))
}
switch tp {
case model.MessageTypeDDL:
// for some protocol, DDL would be dispatched to all partitions,
// Consider that DDL a, b, c received from partition-0, the latest DDL is c,
// if we receive `a` from partition-1, which would be seemed as DDL regression,
// then cause the consumer panic, but it was a duplicate one.
// so we only handle DDL received from partition-0 should be enough.
// but all DDL event messages should be consumed.
ddl, err := decoder.NextDDLEvent()
if err != nil {
log.Panic("decode message value failed",
zap.ByteString("value", message.Value),
zap.Error(err))
}
if simple, ok := decoder.(*simple.Decoder); ok {
cachedEvents := simple.GetCachedEvents()
for _, row := range cachedEvents {
tableID := row.PhysicalTableID
group, ok := eventGroups[tableID]
if !ok {
group = newEventsGroup()
eventGroups[tableID] = group
}
group.Append(row)
}
}
// the Query maybe empty if using simple protocol, it's comes from `bootstrap` event.
if partition != 0 || ddl.Query == "" {
continue
}
partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs)
if ddl.CommitTs < partitionResolvedTs {
log.Panic("DDL event commit-ts less than the resolved ts",
zap.Int32("partition", partition),
zap.Int64("offset", message.Offset),
zap.Uint64("partitionResolvedTs", partitionResolvedTs),
zap.Uint64("commitTs", ddl.CommitTs),
zap.String("DDL", ddl.Query))
}
atomic.StoreUint64(&sink.resolvedTs, ddl.CommitTs)
log.Info("partition resolved ts updated by the DDL event",
zap.Int32("partition", partition),
zap.Int64("offset", message.Offset),
zap.Uint64("oldResolvedTs", partitionResolvedTs),
zap.Uint64("resolvedTs", ddl.CommitTs))
c.appendDDL(ddl)
case model.MessageTypeRow:
row, err := decoder.NextRowChangedEvent()
if err != nil {
log.Panic("decode message value failed",
zap.ByteString("value", message.Value),
zap.Error(err))
}
// when using simple protocol, the row may be nil, since it's table info not received yet,
// it's cached in the decoder, so just continue here.
if c.option.protocol == config.ProtocolSimple && row == nil {
continue
}
target, _, err := c.eventRouter.GetPartitionForRowChange(row, c.option.partitionNum)
if err != nil {
return cerror.Trace(err)
}
if partition != target {
log.Panic("RowChangedEvent dispatched to wrong partition",
zap.Int32("partition", partition),
zap.Int64("offset", message.Offset),
zap.Int32("obtained", partition),
zap.Int32("expected", target),
zap.Any("row", row),
)
}
partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs)
if row.CommitTs < partitionResolvedTs {
log.Panic("RowChangedEvent commit-ts less than the resolved ts",
zap.Int32("partition", partition),
zap.Int64("offset", message.Offset),
zap.Uint64("commitTs", row.CommitTs),
zap.Uint64("partitionResolvedTs", partitionResolvedTs),
zap.Any("row", row))
}
tableID := row.PhysicalTableID
// simple protocol decoder should have set the table id already.
if c.option.protocol != config.ProtocolSimple {
var partitionID int64
if row.TableInfo.IsPartitionTable() {
partitionID = row.PhysicalTableID
}
tableID = c.fakeTableIDGenerator.
generateFakeTableID(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName(), partitionID)
row.TableInfo.TableName.TableID = tableID
}
group, ok := eventGroups[tableID]
if !ok {
group = newEventsGroup()
eventGroups[tableID] = group
}
group.Append(row)
case model.MessageTypeResolved:
ts, err := decoder.NextResolvedEvent()
if err != nil {
log.Panic("decode message value failed",
zap.ByteString("value", message.Value),
zap.Error(err))
}
partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs)
if ts < partitionResolvedTs {
log.Panic("partition resolved ts fallback",
zap.Int32("partition", partition),
zap.Int64("offset", message.Offset),
zap.Uint64("ts", ts),
zap.Uint64("partitionResolvedTs", partitionResolvedTs))
}
atomic.StoreUint64(&sink.resolvedTs, ts)
for tableID, group := range eventGroups {
events := group.Resolve(ts)
if len(events) == 0 {
continue
}
if _, ok := sink.tableSinksMap.Load(tableID); !ok {
sink.tableSinksMap.Store(tableID, c.sinkFactory.CreateTableSinkForConsumer(
model.DefaultChangeFeedID("kafka-consumer"),
spanz.TableIDToComparableSpan(tableID),
events[0].CommitTs,
))
}
s, _ := sink.tableSinksMap.Load(tableID)
s.(tablesink.TableSink).AppendRowChangedEvents(events...)
commitTs := events[len(events)-1].CommitTs
lastCommitTs, ok := sink.tablesCommitTsMap.Load(tableID)
if !ok || lastCommitTs.(uint64) < commitTs {
sink.tablesCommitTsMap.Store(tableID, commitTs)
}
}
}
session.MarkMessage(message, "")
}
if counter > c.option.maxBatchSize {
log.Panic("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", c.option.maxBatchSize),
zap.Int("actual-batch-size", counter))
}
}
return nil
}
// append DDL wait to be handled, only consider the constraint among DDLs.
// for DDL a / b received in the order, a.CommitTs < b.CommitTs should be true.
func (c *Consumer) appendDDL(ddl *model.DDLEvent) {
c.ddlListMu.Lock()
defer c.ddlListMu.Unlock()
// DDL CommitTs fallback, just crash it to indicate the bug.
if c.ddlWithMaxCommitTs != nil && ddl.CommitTs < c.ddlWithMaxCommitTs.CommitTs {
log.Warn("DDL CommitTs < maxCommitTsDDL.CommitTs",
zap.Uint64("commitTs", ddl.CommitTs),
zap.Uint64("maxCommitTs", c.ddlWithMaxCommitTs.CommitTs),
zap.String("DDL", ddl.Query))
return
}
// A rename tables DDL job contains multiple DDL events with same CommitTs.
// So to tell if a DDL is redundant or not, we must check the equivalence of
// the current DDL and the DDL with max CommitTs.
if ddl == c.ddlWithMaxCommitTs {
log.Info("ignore redundant DDL, the DDL is equal to ddlWithMaxCommitTs",
zap.Uint64("commitTs", ddl.CommitTs), zap.String("DDL", ddl.Query))
return
}
c.ddlList = append(c.ddlList, ddl)
log.Info("DDL event received", zap.Uint64("commitTs", ddl.CommitTs), zap.String("DDL", ddl.Query))
c.ddlWithMaxCommitTs = ddl
}
func (c *Consumer) getFrontDDL() *model.DDLEvent {
c.ddlListMu.Lock()
defer c.ddlListMu.Unlock()
if len(c.ddlList) > 0 {
return c.ddlList[0]
}
return nil
}
func (c *Consumer) popDDL() *model.DDLEvent {
c.ddlListMu.Lock()
defer c.ddlListMu.Unlock()
if len(c.ddlList) > 0 {
ddl := c.ddlList[0]
c.ddlList = c.ddlList[1:]
return ddl
}
return nil
}
func (c *Consumer) forEachSink(fn func(sink *partitionSinks) error) error {
c.sinksMu.Lock()
defer c.sinksMu.Unlock()
for _, sink := range c.sinks {
if err := fn(sink); err != nil {
return cerror.Trace(err)
}
}
return nil
}
func (c *Consumer) getMinPartitionResolvedTs() (result uint64, err error) {
result = uint64(math.MaxUint64)
err = c.forEachSink(func(sink *partitionSinks) error {
a := atomic.LoadUint64(&sink.resolvedTs)
if a < result {
result = a
}
return nil
})
return result, err
}
// Run the Consumer
func (c *Consumer) Run(ctx context.Context) error {
ticker := time.NewTicker(100 * time.Millisecond)
defer ticker.Stop()
var globalResolvedTs uint64
for {
select {
case <-ctx.Done():
return ctx.Err()
case <-ticker.C:
}
minPartitionResolvedTs, err := c.getMinPartitionResolvedTs()
if err != nil {
return cerror.Trace(err)
}
// handle DDL
todoDDL := c.getFrontDDL()
if todoDDL != nil && todoDDL.CommitTs <= minPartitionResolvedTs {
// flush DMLs
if err = c.forEachSink(func(sink *partitionSinks) error {
return syncFlushRowChangedEvents(ctx, sink, todoDDL.CommitTs)
}); err != nil {
return cerror.Trace(err)
}
// DDL can be executed, do it first.
if err = c.ddlSink.WriteDDLEvent(ctx, todoDDL); err != nil {
return cerror.Trace(err)
}
c.popDDL()
if todoDDL.CommitTs < minPartitionResolvedTs {
log.Info("update minPartitionResolvedTs by DDL",
zap.Uint64("minPartitionResolvedTs", minPartitionResolvedTs),
zap.String("DDL", todoDDL.Query))
}
minPartitionResolvedTs = todoDDL.CommitTs
}
if globalResolvedTs > minPartitionResolvedTs {
log.Panic("global ResolvedTs fallback",
zap.Uint64("globalResolvedTs", globalResolvedTs),
zap.Uint64("minPartitionResolvedTs", minPartitionResolvedTs))
}
if globalResolvedTs < minPartitionResolvedTs {
globalResolvedTs = minPartitionResolvedTs
}
if err = c.forEachSink(func(sink *partitionSinks) error {
return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs)
}); err != nil {
return cerror.Trace(err)
}
}
}
func syncFlushRowChangedEvents(ctx context.Context, sink *partitionSinks, resolvedTs uint64) error {
for {
select {
case <-ctx.Done():
return ctx.Err()
default:
}
flushedResolvedTs := true
sink.tablesCommitTsMap.Range(func(key, value interface{}) bool {
tableID := key.(int64)
resolvedTs := model.NewResolvedTs(resolvedTs)
tableSink, ok := sink.tableSinksMap.Load(tableID)
if !ok {
log.Panic("Table sink not found", zap.Int64("tableID", tableID))
}
if err := tableSink.(tablesink.TableSink).UpdateResolvedTs(resolvedTs); err != nil {
log.Error("Failed to update resolved ts", zap.Error(err))
return false
}
checkpoint := tableSink.(tablesink.TableSink).GetCheckpointTs()
if !checkpoint.EqualOrGreater(resolvedTs) {
flushedResolvedTs = false
}
return true
})
if flushedResolvedTs {
return nil
}
}
}
type fakeTableIDGenerator struct {
tableIDs map[string]int64
currentTableID int64
mu sync.Mutex
}
func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 {
g.mu.Lock()
defer g.mu.Unlock()
key := quotes.QuoteSchema(schema, table)
if partition != 0 {
key = fmt.Sprintf("%s.`%d`", key, partition)
}
if tableID, ok := g.tableIDs[key]; ok {
return tableID
}
g.currentTableID++
g.tableIDs[key] = g.currentTableID
return g.currentTableID
}
func openDB(ctx context.Context, dsn string) (*sql.DB, error) {
db, err := sql.Open("mysql", dsn)
if err != nil {
log.Error("open db failed", zap.Error(err))
return nil, cerror.Trace(err)
}
db.SetMaxOpenConns(10)
db.SetMaxIdleConns(10)
db.SetConnMaxLifetime(10 * time.Minute)
ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
defer cancel()
if err = db.PingContext(ctx); err != nil {
log.Error("ping db failed", zap.String("dsn", dsn), zap.Error(err))
return nil, cerror.Trace(err)
}
log.Info("open db success", zap.String("dsn", dsn))
return db, nil
}