github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cmd/pulsar-consumer/main.go (about) 1 // Copyright 2023 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package main 15 16 import ( 17 "context" 18 "fmt" 19 "math" 20 "net/url" 21 "os" 22 "os/signal" 23 "sort" 24 "strconv" 25 "strings" 26 "sync" 27 "sync/atomic" 28 "syscall" 29 "time" 30 31 "github.com/apache/pulsar-client-go/pulsar" 32 "github.com/apache/pulsar-client-go/pulsar/auth" 33 "github.com/pingcap/errors" 34 "github.com/pingcap/log" 35 "github.com/pingcap/tiflow/cdc/model" 36 "github.com/pingcap/tiflow/cdc/sink/ddlsink" 37 ddlsinkfactory "github.com/pingcap/tiflow/cdc/sink/ddlsink/factory" 38 eventsinkfactory "github.com/pingcap/tiflow/cdc/sink/dmlsink/factory" 39 "github.com/pingcap/tiflow/cdc/sink/tablesink" 40 sutil "github.com/pingcap/tiflow/cdc/sink/util" 41 cmdUtil "github.com/pingcap/tiflow/pkg/cmd/util" 42 "github.com/pingcap/tiflow/pkg/config" 43 "github.com/pingcap/tiflow/pkg/logutil" 44 "github.com/pingcap/tiflow/pkg/quotes" 45 "github.com/pingcap/tiflow/pkg/sink" 46 "github.com/pingcap/tiflow/pkg/sink/codec" 47 "github.com/pingcap/tiflow/pkg/sink/codec/canal" 48 "github.com/pingcap/tiflow/pkg/sink/codec/common" 49 tpulsar "github.com/pingcap/tiflow/pkg/sink/pulsar" 50 "github.com/pingcap/tiflow/pkg/spanz" 51 "github.com/pingcap/tiflow/pkg/util" 52 "github.com/pingcap/tiflow/pkg/version" 53 "github.com/spf13/cobra" 54 "go.uber.org/zap" 55 ) 56 57 // ConsumerOption represents the options of the pulsar consumer 58 type ConsumerOption struct { 59 address []string 60 topic string 61 62 protocol config.Protocol 63 enableTiDBExtension bool 64 65 // the replicaConfig of the changefeed which produce data to the kafka topic 66 replicaConfig *config.ReplicaConfig 67 68 logPath string 69 logLevel string 70 timezone string 71 ca, cert, key string 72 73 oauth2PrivateKey string 74 oauth2IssuerURL string 75 oauth2ClientID string 76 oauth2Scope string 77 oauth2Audience string 78 79 mtlsAuthTLSCertificatePath string 80 mtlsAuthTLSPrivateKeyPath string 81 82 downstreamURI string 83 partitionNum int 84 } 85 86 func newConsumerOption() *ConsumerOption { 87 return &ConsumerOption{ 88 protocol: config.ProtocolDefault, 89 } 90 } 91 92 // Adjust the consumer option by the upstream uri passed in parameters. 93 func (o *ConsumerOption) Adjust(upstreamURI *url.URL, configFile string) { 94 // the default value of partitionNum is 1 95 o.partitionNum = 1 96 97 o.topic = strings.TrimFunc(upstreamURI.Path, func(r rune) bool { 98 return r == '/' 99 }) 100 101 o.address = strings.Split(upstreamURI.Host, ",") 102 103 replicaConfig := config.GetDefaultReplicaConfig() 104 if configFile != "" { 105 err := cmdUtil.StrictDecodeFile(configFile, "pulsar consumer", replicaConfig) 106 if err != nil { 107 log.Panic("decode config file failed", zap.Error(err)) 108 } 109 } 110 o.replicaConfig = replicaConfig 111 112 s := upstreamURI.Query().Get("protocol") 113 if s != "" { 114 protocol, err := config.ParseSinkProtocolFromString(s) 115 if err != nil { 116 log.Panic("invalid protocol", zap.Error(err), zap.String("protocol", s)) 117 } 118 if !sutil.IsPulsarSupportedProtocols(protocol) { 119 log.Panic("unsupported protocol, pulsar sink currently only support these protocols: [canal-json, canal, maxwell]", 120 zap.String("protocol", s)) 121 } 122 o.protocol = protocol 123 } 124 125 s = upstreamURI.Query().Get("enable-tidb-extension") 126 if s != "" { 127 enableTiDBExtension, err := strconv.ParseBool(s) 128 if err != nil { 129 log.Panic("invalid enable-tidb-extension of upstream-uri") 130 } 131 if enableTiDBExtension { 132 if o.protocol != config.ProtocolCanalJSON && o.protocol != config.ProtocolAvro { 133 log.Panic("enable-tidb-extension only work with canal-json / avro") 134 } 135 } 136 o.enableTiDBExtension = enableTiDBExtension 137 } 138 139 log.Info("consumer option adjusted", 140 zap.String("configFile", configFile), 141 zap.String("address", strings.Join(o.address, ",")), 142 zap.String("topic", o.topic), 143 zap.Any("protocol", o.protocol), 144 zap.Bool("enableTiDBExtension", o.enableTiDBExtension)) 145 } 146 147 var ( 148 upstreamURIStr string 149 configFile string 150 consumerOption = newConsumerOption() 151 ) 152 153 func main() { 154 cmd := &cobra.Command{ 155 Use: "pulsar consumer", 156 Run: run, 157 } 158 // Flags for the root command 159 cmd.Flags().StringVar(&configFile, "config", "", "config file for changefeed") 160 cmd.Flags().StringVar(&upstreamURIStr, "upstream-uri", "", "pulsar uri") 161 cmd.Flags().StringVar(&consumerOption.downstreamURI, "downstream-uri", "", "downstream sink uri") 162 cmd.Flags().StringVar(&consumerOption.timezone, "tz", "System", "Specify time zone of pulsar consumer") 163 cmd.Flags().StringVar(&consumerOption.ca, "ca", "", "CA certificate path for pulsar SSL connection") 164 cmd.Flags().StringVar(&consumerOption.cert, "cert", "", "Certificate path for pulsar SSL connection") 165 cmd.Flags().StringVar(&consumerOption.key, "key", "", "Private key path for pulsar SSL connection") 166 cmd.Flags().StringVar(&consumerOption.logPath, "log-file", "cdc_pulsar_consumer.log", "log file path") 167 cmd.Flags().StringVar(&consumerOption.logLevel, "log-level", "info", "log file path") 168 cmd.Flags().StringVar(&consumerOption.oauth2PrivateKey, "oauth2-private-key", "", "oauth2 private key path") 169 cmd.Flags().StringVar(&consumerOption.oauth2IssuerURL, "oauth2-issuer-url", "", "oauth2 issuer url") 170 cmd.Flags().StringVar(&consumerOption.oauth2ClientID, "oauth2-client-id", "", "oauth2 client id") 171 cmd.Flags().StringVar(&consumerOption.oauth2Audience, "oauth2-scope", "", "oauth2 scope") 172 cmd.Flags().StringVar(&consumerOption.oauth2Audience, "oauth2-audience", "", "oauth2 audience") 173 cmd.Flags().StringVar(&consumerOption.mtlsAuthTLSCertificatePath, "auth-tls-certificate-path", "", "mtls certificate path") 174 cmd.Flags().StringVar(&consumerOption.mtlsAuthTLSPrivateKeyPath, "auth-tls-private-key-path", "", "mtls private key path") 175 if err := cmd.Execute(); err != nil { 176 fmt.Println(err) 177 } 178 } 179 180 func run(cmd *cobra.Command, args []string) { 181 err := logutil.InitLogger(&logutil.Config{ 182 Level: consumerOption.logLevel, 183 File: consumerOption.logPath, 184 }, 185 logutil.WithInitGRPCLogger(), 186 logutil.WithInitSaramaLogger(), 187 ) 188 if err != nil { 189 log.Error("init logger failed", zap.Error(err)) 190 return 191 } 192 193 version.LogVersionInfo("pulsar consumer") 194 195 upstreamURI, err := url.Parse(upstreamURIStr) 196 if err != nil { 197 log.Panic("invalid upstream-uri", zap.Error(err)) 198 } 199 scheme := strings.ToLower(upstreamURI.Scheme) 200 if !sink.IsPulsarScheme(scheme) { 201 log.Panic("invalid upstream-uri scheme, the scheme of upstream-uri must be pulsar schema", 202 zap.String("schema", scheme), 203 zap.String("upstreamURI", upstreamURIStr)) 204 } 205 206 consumerOption.Adjust(upstreamURI, configFile) 207 208 ctx, cancel := context.WithCancel(context.Background()) 209 consumer, err := NewConsumer(ctx, consumerOption) 210 if err != nil { 211 log.Panic("Error creating pulsar consumer", zap.Error(err)) 212 } 213 214 pulsarConsumer, client := NewPulsarConsumer(consumerOption) 215 defer client.Close() 216 defer pulsarConsumer.Close() 217 msgChan := pulsarConsumer.Chan() 218 219 wg := &sync.WaitGroup{} 220 wg.Add(1) 221 go func() { 222 defer wg.Done() 223 for { 224 select { 225 case <-ctx.Done(): 226 log.Info("terminating: context cancelled") 227 return 228 case consumerMsg := <-msgChan: 229 log.Debug(fmt.Sprintf("Received message msgId: %#v -- content: '%s'\n", 230 consumerMsg.ID(), 231 string(consumerMsg.Payload()))) 232 err := consumer.HandleMsg(consumerMsg.Message) 233 if err != nil { 234 log.Panic("Error consuming message", zap.Error(err)) 235 } 236 err = pulsarConsumer.AckID(consumerMsg.Message.ID()) 237 if err != nil { 238 log.Panic("Error ack message", zap.Error(err)) 239 } 240 } 241 } 242 }() 243 244 wg.Add(1) 245 go func() { 246 defer wg.Done() 247 if err := consumer.Run(ctx); err != nil { 248 if err != context.Canceled { 249 log.Panic("Error running consumer", zap.Error(err)) 250 } 251 } 252 }() 253 254 log.Info("TiCDC consumer up and running!...") 255 sigterm := make(chan os.Signal, 1) 256 signal.Notify(sigterm, syscall.SIGINT, syscall.SIGTERM) 257 select { 258 case <-ctx.Done(): 259 log.Info("terminating: context cancelled") 260 case <-sigterm: 261 log.Info("terminating: via signal") 262 } 263 cancel() 264 wg.Wait() 265 } 266 267 // NewPulsarConsumer creates a pulsar consumer 268 func NewPulsarConsumer(option *ConsumerOption) (pulsar.Consumer, pulsar.Client) { 269 var pulsarURL string 270 if len(option.ca) != 0 { 271 pulsarURL = "pulsar+ssl" + "://" + option.address[0] 272 } else { 273 pulsarURL = "pulsar" + "://" + option.address[0] 274 } 275 topicName := option.topic 276 subscriptionName := "pulsar-test-subscription" 277 278 clientOption := pulsar.ClientOptions{ 279 URL: pulsarURL, 280 Logger: tpulsar.NewPulsarLogger(log.L()), 281 } 282 if len(option.ca) != 0 { 283 clientOption.TLSTrustCertsFilePath = option.ca 284 clientOption.TLSCertificateFile = option.cert 285 clientOption.TLSKeyFilePath = option.key 286 } 287 288 var authentication pulsar.Authentication 289 if len(option.oauth2PrivateKey) != 0 { 290 authentication = pulsar.NewAuthenticationOAuth2(map[string]string{ 291 auth.ConfigParamIssuerURL: option.oauth2IssuerURL, 292 auth.ConfigParamAudience: option.oauth2Audience, 293 auth.ConfigParamKeyFile: option.oauth2PrivateKey, 294 auth.ConfigParamClientID: option.oauth2ClientID, 295 auth.ConfigParamScope: option.oauth2Scope, 296 auth.ConfigParamType: auth.ConfigParamTypeClientCredentials, 297 }) 298 log.Info("oauth2 authentication is enabled", zap.String("issuer url", option.oauth2IssuerURL)) 299 clientOption.Authentication = authentication 300 } 301 if len(option.mtlsAuthTLSCertificatePath) != 0 { 302 authentication = pulsar.NewAuthenticationTLS(option.mtlsAuthTLSCertificatePath, option.mtlsAuthTLSPrivateKeyPath) 303 log.Info("mtls authentication is enabled", 304 zap.String("cert", option.mtlsAuthTLSCertificatePath), 305 zap.String("key", option.mtlsAuthTLSPrivateKeyPath), 306 ) 307 clientOption.Authentication = authentication 308 } 309 310 client, err := pulsar.NewClient(clientOption) 311 if err != nil { 312 log.Fatal("can't create pulsar client", zap.Error(err)) 313 } 314 315 consumerConfig := pulsar.ConsumerOptions{ 316 Topic: topicName, 317 SubscriptionName: subscriptionName, 318 Type: pulsar.Exclusive, 319 SubscriptionInitialPosition: pulsar.SubscriptionPositionEarliest, 320 } 321 322 consumer, err := client.Subscribe(consumerConfig) 323 if err != nil { 324 log.Fatal("can't create pulsar consumer", zap.Error(err)) 325 } 326 return consumer, client 327 } 328 329 // partitionSinks maintained for each partition, it may sync data for multiple tables. 330 type partitionSinks struct { 331 tablesCommitTsMap sync.Map 332 tableSinksMap sync.Map 333 // resolvedTs record the maximum timestamp of the received event 334 resolvedTs uint64 335 } 336 337 // Consumer represents a local pulsar consumer 338 type Consumer struct { 339 eventGroups map[int64]*eventsGroup 340 ddlList []*model.DDLEvent 341 ddlListMu sync.Mutex 342 lastReceivedDDL *model.DDLEvent 343 ddlSink ddlsink.Sink 344 fakeTableIDGenerator *fakeTableIDGenerator 345 346 // sinkFactory is used to create table sink for each table. 347 sinkFactory *eventsinkfactory.SinkFactory 348 sinks []*partitionSinks 349 sinksMu sync.Mutex 350 351 // initialize to 0 by default 352 globalResolvedTs uint64 353 354 tz *time.Location 355 356 codecConfig *common.Config 357 358 option *ConsumerOption 359 } 360 361 // NewConsumer creates a new cdc pulsar consumer 362 // the consumer is responsible for consuming the data from the pulsar topic 363 // and write the data to the downstream. 364 func NewConsumer(ctx context.Context, o *ConsumerOption) (*Consumer, error) { 365 c := new(Consumer) 366 c.option = o 367 368 tz, err := util.GetTimezone(o.timezone) 369 if err != nil { 370 return nil, errors.Annotate(err, "can not load timezone") 371 } 372 config.GetGlobalServerConfig().TZ = o.timezone 373 c.tz = tz 374 375 c.fakeTableIDGenerator = &fakeTableIDGenerator{ 376 tableIDs: make(map[string]int64), 377 } 378 379 c.codecConfig = common.NewConfig(o.protocol) 380 c.codecConfig.EnableTiDBExtension = o.enableTiDBExtension 381 if c.codecConfig.Protocol == config.ProtocolAvro { 382 c.codecConfig.AvroEnableWatermark = true 383 } 384 385 c.sinks = make([]*partitionSinks, o.partitionNum) 386 ctx, cancel := context.WithCancel(ctx) 387 errChan := make(chan error, 1) 388 for i := 0; i < o.partitionNum; i++ { 389 c.sinks[i] = &partitionSinks{} 390 } 391 392 changefeedID := model.DefaultChangeFeedID("pulsar-consumer") 393 f, err := eventsinkfactory.New(ctx, changefeedID, o.downstreamURI, o.replicaConfig, errChan, nil) 394 if err != nil { 395 cancel() 396 return nil, errors.Trace(err) 397 } 398 c.sinkFactory = f 399 400 go func() { 401 err := <-errChan 402 if errors.Cause(err) != context.Canceled { 403 log.Error("error on running consumer", zap.Error(err)) 404 } else { 405 log.Info("consumer exited") 406 } 407 cancel() 408 }() 409 410 ddlSink, err := ddlsinkfactory.New(ctx, changefeedID, o.downstreamURI, o.replicaConfig) 411 if err != nil { 412 cancel() 413 return nil, errors.Trace(err) 414 } 415 c.ddlSink = ddlSink 416 c.eventGroups = make(map[int64]*eventsGroup) 417 return c, nil 418 } 419 420 type eventsGroup struct { 421 events []*model.RowChangedEvent 422 } 423 424 func newEventsGroup() *eventsGroup { 425 return &eventsGroup{ 426 events: make([]*model.RowChangedEvent, 0), 427 } 428 } 429 430 func (g *eventsGroup) Append(e *model.RowChangedEvent) { 431 g.events = append(g.events, e) 432 } 433 434 func (g *eventsGroup) Resolve(resolveTs uint64) []*model.RowChangedEvent { 435 sort.Slice(g.events, func(i, j int) bool { 436 return g.events[i].CommitTs < g.events[j].CommitTs 437 }) 438 439 i := sort.Search(len(g.events), func(i int) bool { 440 return g.events[i].CommitTs > resolveTs 441 }) 442 result := g.events[:i] 443 g.events = g.events[i:] 444 445 return result 446 } 447 448 // HandleMsg handles the message received from the pulsar consumer 449 func (c *Consumer) HandleMsg(msg pulsar.Message) error { 450 c.sinksMu.Lock() 451 sink := c.sinks[0] 452 c.sinksMu.Unlock() 453 if sink == nil { 454 panic("sink should initialized") 455 } 456 457 ctx := context.Background() 458 var ( 459 decoder codec.RowEventDecoder 460 err error 461 ) 462 463 switch c.codecConfig.Protocol { 464 case config.ProtocolCanalJSON: 465 decoder, err = canal.NewBatchDecoder(ctx, c.codecConfig, nil) 466 if err != nil { 467 return err 468 } 469 default: 470 log.Panic("Protocol not supported", zap.Any("Protocol", c.codecConfig.Protocol)) 471 } 472 if err != nil { 473 return errors.Trace(err) 474 } 475 476 if err := decoder.AddKeyValue([]byte(msg.Key()), msg.Payload()); err != nil { 477 log.Error("add key value to the decoder failed", zap.Error(err)) 478 return errors.Trace(err) 479 } 480 481 counter := 0 482 for { 483 tp, hasNext, err := decoder.HasNext() 484 if err != nil { 485 log.Panic("decode message key failed", zap.Error(err)) 486 } 487 if !hasNext { 488 break 489 } 490 491 counter++ 492 switch tp { 493 case model.MessageTypeDDL: 494 // for some protocol, DDL would be dispatched to all partitions, 495 // Consider that DDL a, b, c received from partition-0, the latest DDL is c, 496 // if we receive `a` from partition-1, which would be seemed as DDL regression, 497 // then cause the consumer panic, but it was a duplicate one. 498 // so we only handle DDL received from partition-0 should be enough. 499 // but all DDL event messages should be consumed. 500 ddl, err := decoder.NextDDLEvent() 501 if err != nil { 502 log.Panic("decode message value failed", 503 zap.ByteString("value", msg.Payload()), 504 zap.Error(err)) 505 } 506 log.Info("DDL event received", zap.Any("DDL", ddl)) 507 c.appendDDL(ddl) 508 case model.MessageTypeRow: 509 row, err := decoder.NextRowChangedEvent() 510 if err != nil { 511 log.Panic("decode message value failed", 512 zap.ByteString("value", msg.Payload()), 513 zap.Error(err)) 514 } 515 globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) 516 partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs) 517 if row.CommitTs <= globalResolvedTs || row.CommitTs <= partitionResolvedTs { 518 log.Warn("RowChangedEvent fallback row, ignore it", 519 zap.Uint64("commitTs", row.CommitTs), 520 zap.Uint64("globalResolvedTs", globalResolvedTs), 521 zap.Uint64("partitionResolvedTs", partitionResolvedTs), 522 zap.Int32("partition", msg.ID().PartitionIdx()), 523 zap.Any("row", row)) 524 // todo: mark the offset after the DDL is fully synced to the downstream mysql. 525 continue 526 } 527 var partitionID int64 528 if row.TableInfo.IsPartitionTable() { 529 partitionID = row.PhysicalTableID 530 } 531 // use schema, table and tableID to identify a table 532 tableID := c.fakeTableIDGenerator. 533 generateFakeTableID(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName(), partitionID) 534 row.TableInfo.TableName.TableID = tableID 535 536 group, ok := c.eventGroups[tableID] 537 if !ok { 538 group = newEventsGroup() 539 c.eventGroups[tableID] = group 540 } 541 group.Append(row) 542 case model.MessageTypeResolved: 543 ts, err := decoder.NextResolvedEvent() 544 if err != nil { 545 log.Panic("decode message value failed", 546 zap.ByteString("value", msg.Payload()), 547 zap.Error(err)) 548 } 549 550 globalResolvedTs := atomic.LoadUint64(&c.globalResolvedTs) 551 partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs) 552 if ts < globalResolvedTs || ts < partitionResolvedTs { 553 log.Warn("partition resolved ts fallback, skip it", 554 zap.Uint64("ts", ts), 555 zap.Uint64("partitionResolvedTs", partitionResolvedTs), 556 zap.Uint64("globalResolvedTs", globalResolvedTs), 557 zap.Int32("partition", msg.ID().PartitionIdx())) 558 continue 559 } 560 561 for tableID, group := range c.eventGroups { 562 events := group.Resolve(ts) 563 if len(events) == 0 { 564 continue 565 } 566 if _, ok := sink.tableSinksMap.Load(tableID); !ok { 567 log.Info("create table sink for consumer", zap.Any("tableID", tableID)) 568 tableSink := c.sinkFactory.CreateTableSinkForConsumer( 569 model.DefaultChangeFeedID("pulsar-consumer"), 570 spanz.TableIDToComparableSpan(tableID), 571 events[0].CommitTs) 572 573 log.Info("table sink created", zap.Any("tableID", tableID), 574 zap.Any("tableSink", tableSink.GetCheckpointTs())) 575 576 sink.tableSinksMap.Store(tableID, tableSink) 577 } 578 s, _ := sink.tableSinksMap.Load(tableID) 579 s.(tablesink.TableSink).AppendRowChangedEvents(events...) 580 commitTs := events[len(events)-1].CommitTs 581 lastCommitTs, ok := sink.tablesCommitTsMap.Load(tableID) 582 if !ok || lastCommitTs.(uint64) < commitTs { 583 sink.tablesCommitTsMap.Store(tableID, commitTs) 584 } 585 } 586 atomic.StoreUint64(&sink.resolvedTs, ts) 587 } 588 589 } 590 return nil 591 } 592 593 // append DDL wait to be handled, only consider the constraint among DDLs. 594 // for DDL a / b received in the order, a.CommitTs < b.CommitTs should be true. 595 func (c *Consumer) appendDDL(ddl *model.DDLEvent) { 596 c.ddlListMu.Lock() 597 defer c.ddlListMu.Unlock() 598 // DDL CommitTs fallback, just crash it to indicate the bug. 599 if c.lastReceivedDDL != nil && ddl.CommitTs < c.lastReceivedDDL.CommitTs { 600 log.Panic("DDL CommitTs < lastReceivedDDL.CommitTs", 601 zap.Uint64("commitTs", ddl.CommitTs), 602 zap.Uint64("lastReceivedDDLCommitTs", c.lastReceivedDDL.CommitTs), 603 zap.Any("DDL", ddl)) 604 } 605 606 // A rename tables DDL job contains multiple DDL events with same CommitTs. 607 // So to tell if a DDL is redundant or not, we must check the equivalence of 608 // the current DDL and the DDL with max CommitTs. 609 if ddl == c.lastReceivedDDL { 610 log.Info("ignore redundant DDL, the DDL is equal to ddlWithMaxCommitTs", 611 zap.Any("DDL", ddl)) 612 return 613 } 614 615 c.ddlList = append(c.ddlList, ddl) 616 log.Info("DDL event received", zap.Any("DDL", ddl)) 617 c.lastReceivedDDL = ddl 618 } 619 620 func (c *Consumer) getFrontDDL() *model.DDLEvent { 621 c.ddlListMu.Lock() 622 defer c.ddlListMu.Unlock() 623 if len(c.ddlList) > 0 { 624 return c.ddlList[0] 625 } 626 return nil 627 } 628 629 func (c *Consumer) popDDL() *model.DDLEvent { 630 c.ddlListMu.Lock() 631 defer c.ddlListMu.Unlock() 632 if len(c.ddlList) > 0 { 633 ddl := c.ddlList[0] 634 c.ddlList = c.ddlList[1:] 635 return ddl 636 } 637 return nil 638 } 639 640 func (c *Consumer) forEachSink(fn func(sink *partitionSinks) error) error { 641 c.sinksMu.Lock() 642 defer c.sinksMu.Unlock() 643 for _, sink := range c.sinks { 644 if err := fn(sink); err != nil { 645 return errors.Trace(err) 646 } 647 } 648 return nil 649 } 650 651 // getMinResolvedTs returns the minimum resolvedTs of all the partitionSinks 652 func (c *Consumer) getMinResolvedTs() (result uint64, err error) { 653 result = uint64(math.MaxUint64) 654 err = c.forEachSink(func(sink *partitionSinks) error { 655 a := atomic.LoadUint64(&sink.resolvedTs) 656 if a < result { 657 result = a 658 } 659 return nil 660 }) 661 return result, err 662 } 663 664 // Run the Consumer 665 func (c *Consumer) Run(ctx context.Context) error { 666 ticker := time.NewTicker(200 * time.Millisecond) 667 defer ticker.Stop() 668 for { 669 select { 670 case <-ctx.Done(): 671 return ctx.Err() 672 case <-ticker.C: 673 // 1. Get the minimum resolvedTs of all the partitionSinks 674 minResolvedTs, err := c.getMinResolvedTs() 675 if err != nil { 676 return errors.Trace(err) 677 } 678 679 // 2. check if there is a DDL event that can be executed 680 // if there is, execute it and update the minResolvedTs 681 nextDDL := c.getFrontDDL() 682 if nextDDL != nil { 683 log.Info("get nextDDL", zap.Any("DDL", nextDDL)) 684 } 685 if nextDDL != nil && minResolvedTs >= nextDDL.CommitTs { 686 // flush DMLs that commitTs <= todoDDL.CommitTs 687 if err := c.forEachSink(func(sink *partitionSinks) error { 688 return flushRowChangedEvents(ctx, sink, nextDDL.CommitTs) 689 }); err != nil { 690 return errors.Trace(err) 691 } 692 log.Info("begin to execute DDL", zap.Any("DDL", nextDDL)) 693 // all DMLs with commitTs <= todoDDL.CommitTs have been flushed to downstream, 694 // so we can execute the DDL now. 695 if err := c.ddlSink.WriteDDLEvent(ctx, nextDDL); err != nil { 696 return errors.Trace(err) 697 } 698 ddl := c.popDDL() 699 log.Info("DDL executed", zap.Any("DDL", ddl)) 700 minResolvedTs = ddl.CommitTs 701 } 702 703 // 3. Update global resolved ts 704 if c.globalResolvedTs > minResolvedTs { 705 log.Panic("global ResolvedTs fallback", 706 zap.Uint64("globalResolvedTs", c.globalResolvedTs), 707 zap.Uint64("minPartitionResolvedTs", minResolvedTs)) 708 } 709 710 if c.globalResolvedTs < minResolvedTs { 711 c.globalResolvedTs = minResolvedTs 712 } 713 714 // 4. flush all the DMLs that commitTs <= globalResolvedTs 715 if err := c.forEachSink(func(sink *partitionSinks) error { 716 return flushRowChangedEvents(ctx, sink, c.globalResolvedTs) 717 }); err != nil { 718 return errors.Trace(err) 719 } 720 } 721 } 722 } 723 724 // flushRowChangedEvents flushes all the DMLs that commitTs <= resolvedTs 725 // Note: This function is synchronous, it will block until all the DMLs are flushed. 726 func flushRowChangedEvents(ctx context.Context, sink *partitionSinks, resolvedTs uint64) error { 727 for { 728 select { 729 case <-ctx.Done(): 730 return ctx.Err() 731 default: 732 } 733 flushedResolvedTs := true 734 sink.tablesCommitTsMap.Range(func(key, value interface{}) bool { 735 tableID := key.(int64) 736 resolvedTs := model.NewResolvedTs(resolvedTs) 737 tableSink, ok := sink.tableSinksMap.Load(tableID) 738 if !ok { 739 log.Panic("Table sink not found", zap.Int64("tableID", tableID)) 740 } 741 if err := tableSink.(tablesink.TableSink).UpdateResolvedTs(resolvedTs); err != nil { 742 log.Error("Failed to update resolved ts", zap.Error(err)) 743 return false 744 } 745 if !tableSink.(tablesink.TableSink).GetCheckpointTs().EqualOrGreater(resolvedTs) { 746 flushedResolvedTs = false 747 } 748 return true 749 }) 750 if flushedResolvedTs { 751 return nil 752 } 753 } 754 } 755 756 type fakeTableIDGenerator struct { 757 tableIDs map[string]int64 758 currentTableID int64 759 mu sync.Mutex 760 } 761 762 func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 { 763 g.mu.Lock() 764 defer g.mu.Unlock() 765 key := quotes.QuoteSchema(schema, table) 766 if partition != 0 { 767 key = fmt.Sprintf("%s.`%d`", key, partition) 768 } 769 if tableID, ok := g.tableIDs[key]; ok { 770 return tableID 771 } 772 g.currentTableID++ 773 g.tableIDs[key] = g.currentTableID 774 return g.currentTableID 775 }