github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cmd/kafka-consumer/main.go (about)

     1  // Copyright 2020 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  	"database/sql"
    19  	"errors"
    20  	"flag"
    21  	"fmt"
    22  	"math"
    23  	"net/http"
    24  	_ "net/http/pprof"
    25  	"net/url"
    26  	"os"
    27  	"os/signal"
    28  	"runtime/debug"
    29  	"sort"
    30  	"strconv"
    31  	"strings"
    32  	"sync"
    33  	"sync/atomic"
    34  	"syscall"
    35  	"time"
    36  
    37  	"github.com/IBM/sarama"
    38  	"github.com/google/uuid"
    39  	cerror "github.com/pingcap/errors"
    40  	"github.com/pingcap/log"
    41  	"github.com/pingcap/tiflow/cdc/model"
    42  	"github.com/pingcap/tiflow/cdc/sink/ddlsink"
    43  	ddlsinkfactory "github.com/pingcap/tiflow/cdc/sink/ddlsink/factory"
    44  	eventsinkfactory "github.com/pingcap/tiflow/cdc/sink/dmlsink/factory"
    45  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher"
    46  	"github.com/pingcap/tiflow/cdc/sink/tablesink"
    47  	cmdUtil "github.com/pingcap/tiflow/pkg/cmd/util"
    48  	"github.com/pingcap/tiflow/pkg/config"
    49  	"github.com/pingcap/tiflow/pkg/filter"
    50  	"github.com/pingcap/tiflow/pkg/logutil"
    51  	"github.com/pingcap/tiflow/pkg/quotes"
    52  	"github.com/pingcap/tiflow/pkg/security"
    53  	"github.com/pingcap/tiflow/pkg/sink/codec"
    54  	"github.com/pingcap/tiflow/pkg/sink/codec/avro"
    55  	"github.com/pingcap/tiflow/pkg/sink/codec/canal"
    56  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    57  	"github.com/pingcap/tiflow/pkg/sink/codec/open"
    58  	"github.com/pingcap/tiflow/pkg/sink/codec/simple"
    59  	"github.com/pingcap/tiflow/pkg/spanz"
    60  	"github.com/pingcap/tiflow/pkg/util"
    61  	"github.com/pingcap/tiflow/pkg/version"
    62  	"go.uber.org/zap"
    63  )
    64  
    65  func newConsumerOption() *consumerOption {
    66  	return &consumerOption{
    67  		version: "2.4.0",
    68  
    69  		maxMessageBytes: math.MaxInt64,
    70  		maxBatchSize:    math.MaxInt64,
    71  	}
    72  }
    73  
    74  type consumerOption struct {
    75  	address      []string
    76  	version      string
    77  	topic        string
    78  	partitionNum int32
    79  	groupID      string
    80  
    81  	maxMessageBytes int
    82  	maxBatchSize    int
    83  
    84  	protocol    config.Protocol
    85  	codecConfig *common.Config
    86  
    87  	// the replicaConfig of the changefeed which produce data to the kafka topic
    88  	replicaConfig *config.ReplicaConfig
    89  
    90  	logPath       string
    91  	logLevel      string
    92  	timezone      string
    93  	ca, cert, key string
    94  
    95  	downstreamURI string
    96  
    97  	// avro schema registry uri should be set if the encoding protocol is avro
    98  	schemaRegistryURI string
    99  
   100  	// upstreamTiDBDSN is the dsn of the upstream TiDB cluster
   101  	upstreamTiDBDSN string
   102  
   103  	enableProfiling bool
   104  }
   105  
   106  // Adjust the consumer option by the upstream uri passed in parameters.
   107  func (o *consumerOption) Adjust(upstreamURI *url.URL, configFile string) error {
   108  	s := upstreamURI.Query().Get("version")
   109  	if s != "" {
   110  		o.version = s
   111  	}
   112  	o.topic = strings.TrimFunc(upstreamURI.Path, func(r rune) bool {
   113  		return r == '/'
   114  	})
   115  	o.address = strings.Split(upstreamURI.Host, ",")
   116  
   117  	s = upstreamURI.Query().Get("partition-num")
   118  	if s == "" {
   119  		partition, err := getPartitionNum(o.address, o.topic)
   120  		if err != nil {
   121  			log.Panic("can not get partition number", zap.String("topic", o.topic), zap.Error(err))
   122  		}
   123  		o.partitionNum = partition
   124  	} else {
   125  		c, err := strconv.ParseInt(s, 10, 32)
   126  		if err != nil {
   127  			log.Panic("invalid partition-num of upstream-uri")
   128  		}
   129  		o.partitionNum = int32(c)
   130  	}
   131  
   132  	s = upstreamURI.Query().Get("max-message-bytes")
   133  	if s != "" {
   134  		c, err := strconv.Atoi(s)
   135  		if err != nil {
   136  			log.Panic("invalid max-message-bytes of upstream-uri")
   137  		}
   138  		o.maxMessageBytes = c
   139  	}
   140  
   141  	s = upstreamURI.Query().Get("max-batch-size")
   142  	if s != "" {
   143  		c, err := strconv.Atoi(s)
   144  		if err != nil {
   145  			log.Panic("invalid max-batch-size of upstream-uri")
   146  		}
   147  		o.maxBatchSize = c
   148  	}
   149  
   150  	s = upstreamURI.Query().Get("protocol")
   151  	if s == "" {
   152  		log.Panic("cannot found the protocol from the sink url")
   153  	}
   154  	protocol, err := config.ParseSinkProtocolFromString(s)
   155  	if err != nil {
   156  		log.Panic("invalid protocol", zap.Error(err), zap.String("protocol", s))
   157  	}
   158  	o.protocol = protocol
   159  
   160  	replicaConfig := config.GetDefaultReplicaConfig()
   161  	replicaConfig.Sink.Protocol = util.AddressOf(protocol.String())
   162  	if configFile != "" {
   163  		err = cmdUtil.StrictDecodeFile(configFile, "kafka consumer", replicaConfig)
   164  		if err != nil {
   165  			return cerror.Trace(err)
   166  		}
   167  		if _, err = filter.VerifyTableRules(replicaConfig.Filter); err != nil {
   168  			return cerror.Trace(err)
   169  		}
   170  	}
   171  	o.replicaConfig = replicaConfig
   172  
   173  	o.codecConfig = common.NewConfig(protocol)
   174  	if err = o.codecConfig.Apply(upstreamURI, o.replicaConfig); err != nil {
   175  		return cerror.Trace(err)
   176  	}
   177  	if protocol == config.ProtocolAvro {
   178  		o.codecConfig.AvroEnableWatermark = true
   179  	}
   180  
   181  	log.Info("consumer option adjusted",
   182  		zap.String("configFile", configFile),
   183  		zap.String("address", strings.Join(o.address, ",")),
   184  		zap.String("version", o.version),
   185  		zap.String("topic", o.topic),
   186  		zap.Int32("partitionNum", o.partitionNum),
   187  		zap.String("groupID", o.groupID),
   188  		zap.Int("maxMessageBytes", o.maxMessageBytes),
   189  		zap.Int("maxBatchSize", o.maxBatchSize),
   190  		zap.String("upstreamURI", upstreamURI.String()))
   191  	return nil
   192  }
   193  
   194  func main() {
   195  	debug.SetMemoryLimit(14 * 1024 * 1024 * 1024)
   196  
   197  	consumerOption := newConsumerOption()
   198  
   199  	var (
   200  		upstreamURIStr string
   201  		configFile     string
   202  	)
   203  
   204  	groupID := fmt.Sprintf("ticdc_kafka_consumer_%s", uuid.New().String())
   205  
   206  	flag.StringVar(&configFile, "config", "", "config file for changefeed")
   207  
   208  	flag.StringVar(&upstreamURIStr, "upstream-uri", "", "Kafka uri")
   209  	flag.StringVar(&consumerOption.downstreamURI, "downstream-uri", "", "downstream sink uri")
   210  	flag.StringVar(&consumerOption.schemaRegistryURI, "schema-registry-uri", "", "schema registry uri")
   211  	flag.StringVar(&consumerOption.upstreamTiDBDSN, "upstream-tidb-dsn", "", "upstream TiDB DSN")
   212  	flag.StringVar(&consumerOption.groupID, "consumer-group-id", groupID, "consumer group id")
   213  	flag.StringVar(&consumerOption.logPath, "log-file", "cdc_kafka_consumer.log", "log file path")
   214  	flag.StringVar(&consumerOption.logLevel, "log-level", "info", "log file path")
   215  	flag.StringVar(&consumerOption.timezone, "tz", "System", "Specify time zone of Kafka consumer")
   216  	flag.StringVar(&consumerOption.ca, "ca", "", "CA certificate path for Kafka SSL connection")
   217  	flag.StringVar(&consumerOption.cert, "cert", "", "Certificate path for Kafka SSL connection")
   218  	flag.StringVar(&consumerOption.key, "key", "", "Private key path for Kafka SSL connection")
   219  	flag.BoolVar(&consumerOption.enableProfiling, "enable-profiling", false, "enable pprof profiling")
   220  	flag.Parse()
   221  
   222  	err := logutil.InitLogger(&logutil.Config{
   223  		Level: consumerOption.logLevel,
   224  		File:  consumerOption.logPath,
   225  	},
   226  		logutil.WithInitGRPCLogger(),
   227  		logutil.WithInitSaramaLogger(),
   228  	)
   229  	if err != nil {
   230  		log.Error("init logger failed", zap.Error(err))
   231  		return
   232  	}
   233  
   234  	version.LogVersionInfo("kafka consumer")
   235  
   236  	upstreamURI, err := url.Parse(upstreamURIStr)
   237  	if err != nil {
   238  		log.Panic("invalid upstream-uri", zap.Error(err))
   239  	}
   240  	scheme := strings.ToLower(upstreamURI.Scheme)
   241  	if scheme != "kafka" {
   242  		log.Panic("invalid upstream-uri scheme, the scheme of upstream-uri must be `kafka`",
   243  			zap.String("upstreamURI", upstreamURIStr))
   244  	}
   245  
   246  	err = consumerOption.Adjust(upstreamURI, configFile)
   247  	if err != nil {
   248  		log.Panic("adjust consumer option failed", zap.Error(err))
   249  	}
   250  
   251  	///**
   252  	// * Construct a new Sarama configuration.
   253  	// * The Kafka cluster version has to be defined before the consumer/producer is initialized.
   254  	// */
   255  	config, err := newSaramaConfig(consumerOption)
   256  	if err != nil {
   257  		log.Panic("Error creating sarama config", zap.Error(err))
   258  	}
   259  	err = waitTopicCreated(consumerOption.address, consumerOption.topic, config)
   260  	if err != nil {
   261  		log.Panic("wait topic created failed", zap.Error(err))
   262  	}
   263  
   264  	ctx, cancel := context.WithCancel(context.Background())
   265  	consumer, err := NewConsumer(ctx, consumerOption)
   266  	if err != nil {
   267  		log.Panic("Error creating consumer", zap.Error(err))
   268  	}
   269  
   270  	client, err := sarama.NewConsumerGroup(consumerOption.address, consumerOption.groupID, config)
   271  	if err != nil {
   272  		log.Panic("Error creating consumer group client", zap.Error(err))
   273  	}
   274  
   275  	var wg sync.WaitGroup
   276  	if consumerOption.enableProfiling {
   277  		wg.Add(1)
   278  		go func() {
   279  			defer wg.Done()
   280  			if err := http.ListenAndServe(":6060", nil); err != nil {
   281  				log.Panic("Error starting pprof", zap.Error(err))
   282  			}
   283  		}()
   284  	}
   285  
   286  	wg.Add(1)
   287  	go func() {
   288  		defer wg.Done()
   289  		for {
   290  			// `consume` should be called inside an infinite loop, when a
   291  			// server-side rebalance happens, the consumer session will need to be
   292  			// recreated to get the new claims
   293  			if err := client.Consume(ctx, strings.Split(consumerOption.topic, ","), consumer); err != nil {
   294  				log.Panic("Error from consumer", zap.Error(err))
   295  			}
   296  			// check if context was cancelled, signaling that the consumer should stop
   297  			if ctx.Err() != nil {
   298  				return
   299  			}
   300  			consumer.ready = make(chan bool)
   301  		}
   302  	}()
   303  
   304  	go func() {
   305  		if err := consumer.Run(ctx); err != nil {
   306  			if err != context.Canceled {
   307  				log.Panic("Error running consumer", zap.Error(err))
   308  			}
   309  		}
   310  	}()
   311  
   312  	<-consumer.ready // wait till the consumer has been set up
   313  	log.Info("TiCDC consumer up and running!...")
   314  
   315  	sigterm := make(chan os.Signal, 1)
   316  	signal.Notify(sigterm, syscall.SIGINT, syscall.SIGTERM)
   317  	select {
   318  	case <-ctx.Done():
   319  		log.Info("terminating: context cancelled")
   320  	case <-sigterm:
   321  		log.Info("terminating: via signal")
   322  	}
   323  	cancel()
   324  	wg.Wait()
   325  	if err = client.Close(); err != nil {
   326  		log.Panic("Error closing client", zap.Error(err))
   327  	}
   328  }
   329  
   330  func getPartitionNum(address []string, topic string) (int32, error) {
   331  	saramaConfig := sarama.NewConfig()
   332  	// get partition number or create topic automatically
   333  	admin, err := sarama.NewClusterAdmin(address, saramaConfig)
   334  	if err != nil {
   335  		return 0, cerror.Trace(err)
   336  	}
   337  	topics, err := admin.ListTopics()
   338  	if err != nil {
   339  		return 0, cerror.Trace(err)
   340  	}
   341  	err = admin.Close()
   342  	if err != nil {
   343  		return 0, cerror.Trace(err)
   344  	}
   345  	topicDetail, exist := topics[topic]
   346  	if !exist {
   347  		return 0, cerror.Errorf("can not find topic %s", topic)
   348  	}
   349  	log.Info("get partition number of topic",
   350  		zap.String("topic", topic),
   351  		zap.Int32("partitionNum", topicDetail.NumPartitions))
   352  	return topicDetail.NumPartitions, nil
   353  }
   354  
   355  func waitTopicCreated(address []string, topic string, cfg *sarama.Config) error {
   356  	admin, err := sarama.NewClusterAdmin(address, cfg)
   357  	if err != nil {
   358  		return cerror.Trace(err)
   359  	}
   360  	defer admin.Close()
   361  	for i := 0; i <= 30; i++ {
   362  		topics, err := admin.ListTopics()
   363  		if err != nil {
   364  			return cerror.Trace(err)
   365  		}
   366  		if _, ok := topics[topic]; ok {
   367  			return nil
   368  		}
   369  		log.Info("wait the topic created", zap.String("topic", topic))
   370  		time.Sleep(1 * time.Second)
   371  	}
   372  	return cerror.Errorf("wait the topic(%s) created timeout", topic)
   373  }
   374  
   375  func newSaramaConfig(o *consumerOption) (*sarama.Config, error) {
   376  	config := sarama.NewConfig()
   377  
   378  	version, err := sarama.ParseKafkaVersion(o.version)
   379  	if err != nil {
   380  		return nil, cerror.Trace(err)
   381  	}
   382  
   383  	config.ClientID = "ticdc_kafka_sarama_consumer"
   384  	config.Version = version
   385  
   386  	config.Metadata.Retry.Max = 10000
   387  	config.Metadata.Retry.Backoff = 500 * time.Millisecond
   388  	config.Consumer.Retry.Backoff = 500 * time.Millisecond
   389  	config.Consumer.Offsets.Initial = sarama.OffsetOldest
   390  
   391  	if len(o.ca) != 0 {
   392  		config.Net.TLS.Enable = true
   393  		config.Net.TLS.Config, err = (&security.Credential{
   394  			CAPath:   o.ca,
   395  			CertPath: o.cert,
   396  			KeyPath:  o.key,
   397  		}).ToTLSConfig()
   398  		if err != nil {
   399  			return nil, cerror.Trace(err)
   400  		}
   401  	}
   402  
   403  	return config, err
   404  }
   405  
   406  // partitionSinks maintained for each partition, it may sync data for multiple tables.
   407  type partitionSinks struct {
   408  	tablesCommitTsMap sync.Map
   409  	tableSinksMap     sync.Map
   410  	// resolvedTs record the maximum timestamp of the received event
   411  	resolvedTs uint64
   412  }
   413  
   414  // Consumer represents a Sarama consumer group consumer
   415  type Consumer struct {
   416  	ready chan bool
   417  
   418  	ddlList              []*model.DDLEvent
   419  	ddlListMu            sync.Mutex
   420  	ddlWithMaxCommitTs   *model.DDLEvent
   421  	ddlSink              ddlsink.Sink
   422  	fakeTableIDGenerator *fakeTableIDGenerator
   423  
   424  	// sinkFactory is used to create table sink for each table.
   425  	sinkFactory *eventsinkfactory.SinkFactory
   426  	sinks       []*partitionSinks
   427  	sinksMu     sync.Mutex
   428  
   429  	eventRouter *dispatcher.EventRouter
   430  
   431  	option *consumerOption
   432  
   433  	upstreamTiDB *sql.DB
   434  }
   435  
   436  // NewConsumer creates a new cdc kafka consumer
   437  func NewConsumer(ctx context.Context, o *consumerOption) (*Consumer, error) {
   438  	c := new(Consumer)
   439  	c.option = o
   440  
   441  	tz, err := util.GetTimezone(o.timezone)
   442  	if err != nil {
   443  		return nil, cerror.Annotate(err, "can not load timezone")
   444  	}
   445  	config.GetGlobalServerConfig().TZ = o.timezone
   446  	o.codecConfig.TimeZone = tz
   447  
   448  	c.fakeTableIDGenerator = &fakeTableIDGenerator{
   449  		tableIDs: make(map[string]int64),
   450  	}
   451  
   452  	if o.codecConfig.LargeMessageHandle.HandleKeyOnly() {
   453  		db, err := openDB(ctx, o.upstreamTiDBDSN)
   454  		if err != nil {
   455  			return nil, err
   456  		}
   457  		c.upstreamTiDB = db
   458  	}
   459  
   460  	eventRouter, err := dispatcher.NewEventRouter(o.replicaConfig, o.protocol, o.topic, "kafka")
   461  	if err != nil {
   462  		return nil, cerror.Trace(err)
   463  	}
   464  	c.eventRouter = eventRouter
   465  
   466  	c.sinks = make([]*partitionSinks, o.partitionNum)
   467  	ctx, cancel := context.WithCancel(ctx)
   468  	errChan := make(chan error, 1)
   469  
   470  	for i := 0; i < int(o.partitionNum); i++ {
   471  		c.sinks[i] = &partitionSinks{}
   472  	}
   473  
   474  	changefeedID := model.DefaultChangeFeedID("kafka-consumer")
   475  	f, err := eventsinkfactory.New(ctx, changefeedID, o.downstreamURI, o.replicaConfig, errChan, nil)
   476  	if err != nil {
   477  		cancel()
   478  		return nil, cerror.Trace(err)
   479  	}
   480  	c.sinkFactory = f
   481  
   482  	go func() {
   483  		err := <-errChan
   484  		if !errors.Is(cerror.Cause(err), context.Canceled) {
   485  			log.Error("error on running consumer", zap.Error(err))
   486  		} else {
   487  			log.Info("consumer exited")
   488  		}
   489  		cancel()
   490  	}()
   491  
   492  	ddlSink, err := ddlsinkfactory.New(ctx, changefeedID, o.downstreamURI, o.replicaConfig)
   493  	if err != nil {
   494  		cancel()
   495  		return nil, cerror.Trace(err)
   496  	}
   497  	c.ddlSink = ddlSink
   498  	c.ready = make(chan bool)
   499  	return c, nil
   500  }
   501  
   502  // Setup is run at the beginning of a new session, before ConsumeClaim
   503  func (c *Consumer) Setup(sarama.ConsumerGroupSession) error {
   504  	// Mark the c as ready
   505  	close(c.ready)
   506  	return nil
   507  }
   508  
   509  // Cleanup is run at the end of a session, once all ConsumeClaim goroutines have exited
   510  func (c *Consumer) Cleanup(sarama.ConsumerGroupSession) error {
   511  	return nil
   512  }
   513  
   514  type eventsGroup struct {
   515  	events []*model.RowChangedEvent
   516  }
   517  
   518  func newEventsGroup() *eventsGroup {
   519  	return &eventsGroup{
   520  		events: make([]*model.RowChangedEvent, 0),
   521  	}
   522  }
   523  
   524  func (g *eventsGroup) Append(e *model.RowChangedEvent) {
   525  	g.events = append(g.events, e)
   526  }
   527  
   528  func (g *eventsGroup) Resolve(resolveTs uint64) []*model.RowChangedEvent {
   529  	sort.Slice(g.events, func(i, j int) bool {
   530  		return g.events[i].CommitTs < g.events[j].CommitTs
   531  	})
   532  
   533  	i := sort.Search(len(g.events), func(i int) bool {
   534  		return g.events[i].CommitTs > resolveTs
   535  	})
   536  	result := g.events[:i]
   537  	g.events = g.events[i:]
   538  
   539  	return result
   540  }
   541  
   542  // ConsumeClaim must start a consumer loop of ConsumerGroupClaim's Messages().
   543  func (c *Consumer) ConsumeClaim(session sarama.ConsumerGroupSession, claim sarama.ConsumerGroupClaim) error {
   544  	partition := claim.Partition()
   545  	c.sinksMu.Lock()
   546  	sink := c.sinks[partition]
   547  	c.sinksMu.Unlock()
   548  	if sink == nil {
   549  		panic("sink should initialized")
   550  	}
   551  
   552  	ctx := context.Background()
   553  	var (
   554  		decoder codec.RowEventDecoder
   555  		err     error
   556  	)
   557  
   558  	switch c.option.protocol {
   559  	case config.ProtocolOpen, config.ProtocolDefault:
   560  		decoder, err = open.NewBatchDecoder(ctx, c.option.codecConfig, c.upstreamTiDB)
   561  	case config.ProtocolCanalJSON:
   562  		decoder, err = canal.NewBatchDecoder(ctx, c.option.codecConfig, c.upstreamTiDB)
   563  		if err != nil {
   564  			return err
   565  		}
   566  	case config.ProtocolAvro:
   567  		schemaM, err := avro.NewConfluentSchemaManager(ctx, c.option.schemaRegistryURI, nil)
   568  		if err != nil {
   569  			return cerror.Trace(err)
   570  		}
   571  		decoder = avro.NewDecoder(c.option.codecConfig, schemaM, c.option.topic)
   572  	case config.ProtocolSimple:
   573  		decoder, err = simple.NewDecoder(ctx, c.option.codecConfig, c.upstreamTiDB)
   574  	default:
   575  		log.Panic("Protocol not supported", zap.Any("Protocol", c.option.protocol))
   576  	}
   577  	if err != nil {
   578  		return cerror.Trace(err)
   579  	}
   580  
   581  	log.Info("start consume claim",
   582  		zap.String("topic", claim.Topic()), zap.Int32("partition", partition),
   583  		zap.Int64("initialOffset", claim.InitialOffset()), zap.Int64("highWaterMarkOffset", claim.HighWaterMarkOffset()))
   584  
   585  	eventGroups := make(map[int64]*eventsGroup)
   586  	for message := range claim.Messages() {
   587  		if err = decoder.AddKeyValue(message.Key, message.Value); err != nil {
   588  			log.Error("add key value to the decoder failed", zap.Error(err))
   589  			return cerror.Trace(err)
   590  		}
   591  
   592  		counter := 0
   593  		for {
   594  			tp, hasNext, err := decoder.HasNext()
   595  			if err != nil {
   596  				log.Panic("decode message key failed", zap.Error(err))
   597  			}
   598  			if !hasNext {
   599  				break
   600  			}
   601  
   602  			counter++
   603  			// If the message containing only one event exceeds the length limit, CDC will allow it and issue a warning.
   604  			if len(message.Key)+len(message.Value) > c.option.maxMessageBytes && counter > 1 {
   605  				log.Panic("kafka max-messages-bytes exceeded",
   606  					zap.Int("max-message-bytes", c.option.maxMessageBytes),
   607  					zap.Int("receivedBytes", len(message.Key)+len(message.Value)))
   608  			}
   609  
   610  			switch tp {
   611  			case model.MessageTypeDDL:
   612  				// for some protocol, DDL would be dispatched to all partitions,
   613  				// Consider that DDL a, b, c received from partition-0, the latest DDL is c,
   614  				// if we receive `a` from partition-1, which would be seemed as DDL regression,
   615  				// then cause the consumer panic, but it was a duplicate one.
   616  				// so we only handle DDL received from partition-0 should be enough.
   617  				// but all DDL event messages should be consumed.
   618  				ddl, err := decoder.NextDDLEvent()
   619  				if err != nil {
   620  					log.Panic("decode message value failed",
   621  						zap.ByteString("value", message.Value),
   622  						zap.Error(err))
   623  				}
   624  
   625  				if simple, ok := decoder.(*simple.Decoder); ok {
   626  					cachedEvents := simple.GetCachedEvents()
   627  					for _, row := range cachedEvents {
   628  						tableID := row.PhysicalTableID
   629  						group, ok := eventGroups[tableID]
   630  						if !ok {
   631  							group = newEventsGroup()
   632  							eventGroups[tableID] = group
   633  						}
   634  						group.Append(row)
   635  					}
   636  				}
   637  
   638  				// the Query maybe empty if using simple protocol, it's comes from `bootstrap` event.
   639  				if partition != 0 || ddl.Query == "" {
   640  					continue
   641  				}
   642  
   643  				partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs)
   644  				if ddl.CommitTs < partitionResolvedTs {
   645  					log.Panic("DDL event commit-ts less than the resolved ts",
   646  						zap.Int32("partition", partition),
   647  						zap.Int64("offset", message.Offset),
   648  						zap.Uint64("partitionResolvedTs", partitionResolvedTs),
   649  						zap.Uint64("commitTs", ddl.CommitTs),
   650  						zap.String("DDL", ddl.Query))
   651  				}
   652  				atomic.StoreUint64(&sink.resolvedTs, ddl.CommitTs)
   653  				log.Info("partition resolved ts updated by the DDL event",
   654  					zap.Int32("partition", partition),
   655  					zap.Int64("offset", message.Offset),
   656  					zap.Uint64("oldResolvedTs", partitionResolvedTs),
   657  					zap.Uint64("resolvedTs", ddl.CommitTs))
   658  				c.appendDDL(ddl)
   659  			case model.MessageTypeRow:
   660  				row, err := decoder.NextRowChangedEvent()
   661  				if err != nil {
   662  					log.Panic("decode message value failed",
   663  						zap.ByteString("value", message.Value),
   664  						zap.Error(err))
   665  				}
   666  				// when using simple protocol, the row may be nil, since it's table info not received yet,
   667  				// it's cached in the decoder, so just continue here.
   668  				if c.option.protocol == config.ProtocolSimple && row == nil {
   669  					continue
   670  				}
   671  				target, _, err := c.eventRouter.GetPartitionForRowChange(row, c.option.partitionNum)
   672  				if err != nil {
   673  					return cerror.Trace(err)
   674  				}
   675  				if partition != target {
   676  					log.Panic("RowChangedEvent dispatched to wrong partition",
   677  						zap.Int32("partition", partition),
   678  						zap.Int64("offset", message.Offset),
   679  						zap.Int32("obtained", partition),
   680  						zap.Int32("expected", target),
   681  						zap.Any("row", row),
   682  					)
   683  				}
   684  
   685  				partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs)
   686  				if row.CommitTs < partitionResolvedTs {
   687  					log.Panic("RowChangedEvent commit-ts less than the resolved ts",
   688  						zap.Int32("partition", partition),
   689  						zap.Int64("offset", message.Offset),
   690  						zap.Uint64("commitTs", row.CommitTs),
   691  						zap.Uint64("partitionResolvedTs", partitionResolvedTs),
   692  						zap.Any("row", row))
   693  				}
   694  
   695  				tableID := row.PhysicalTableID
   696  				// simple protocol decoder should have set the table id already.
   697  				if c.option.protocol != config.ProtocolSimple {
   698  					var partitionID int64
   699  					if row.TableInfo.IsPartitionTable() {
   700  						partitionID = row.PhysicalTableID
   701  					}
   702  					tableID = c.fakeTableIDGenerator.
   703  						generateFakeTableID(row.TableInfo.GetSchemaName(), row.TableInfo.GetTableName(), partitionID)
   704  					row.TableInfo.TableName.TableID = tableID
   705  				}
   706  				group, ok := eventGroups[tableID]
   707  				if !ok {
   708  					group = newEventsGroup()
   709  					eventGroups[tableID] = group
   710  				}
   711  
   712  				group.Append(row)
   713  			case model.MessageTypeResolved:
   714  				ts, err := decoder.NextResolvedEvent()
   715  				if err != nil {
   716  					log.Panic("decode message value failed",
   717  						zap.ByteString("value", message.Value),
   718  						zap.Error(err))
   719  				}
   720  
   721  				partitionResolvedTs := atomic.LoadUint64(&sink.resolvedTs)
   722  				if ts < partitionResolvedTs {
   723  					log.Panic("partition resolved ts fallback",
   724  						zap.Int32("partition", partition),
   725  						zap.Int64("offset", message.Offset),
   726  						zap.Uint64("ts", ts),
   727  						zap.Uint64("partitionResolvedTs", partitionResolvedTs))
   728  				}
   729  				atomic.StoreUint64(&sink.resolvedTs, ts)
   730  
   731  				for tableID, group := range eventGroups {
   732  					events := group.Resolve(ts)
   733  					if len(events) == 0 {
   734  						continue
   735  					}
   736  					if _, ok := sink.tableSinksMap.Load(tableID); !ok {
   737  						sink.tableSinksMap.Store(tableID, c.sinkFactory.CreateTableSinkForConsumer(
   738  							model.DefaultChangeFeedID("kafka-consumer"),
   739  							spanz.TableIDToComparableSpan(tableID),
   740  							events[0].CommitTs,
   741  						))
   742  					}
   743  					s, _ := sink.tableSinksMap.Load(tableID)
   744  					s.(tablesink.TableSink).AppendRowChangedEvents(events...)
   745  					commitTs := events[len(events)-1].CommitTs
   746  					lastCommitTs, ok := sink.tablesCommitTsMap.Load(tableID)
   747  					if !ok || lastCommitTs.(uint64) < commitTs {
   748  						sink.tablesCommitTsMap.Store(tableID, commitTs)
   749  					}
   750  				}
   751  			}
   752  			session.MarkMessage(message, "")
   753  		}
   754  
   755  		if counter > c.option.maxBatchSize {
   756  			log.Panic("Open Protocol max-batch-size exceeded", zap.Int("max-batch-size", c.option.maxBatchSize),
   757  				zap.Int("actual-batch-size", counter))
   758  		}
   759  	}
   760  
   761  	return nil
   762  }
   763  
   764  // append DDL wait to be handled, only consider the constraint among DDLs.
   765  // for DDL a / b received in the order, a.CommitTs < b.CommitTs should be true.
   766  func (c *Consumer) appendDDL(ddl *model.DDLEvent) {
   767  	c.ddlListMu.Lock()
   768  	defer c.ddlListMu.Unlock()
   769  	// DDL CommitTs fallback, just crash it to indicate the bug.
   770  	if c.ddlWithMaxCommitTs != nil && ddl.CommitTs < c.ddlWithMaxCommitTs.CommitTs {
   771  		log.Warn("DDL CommitTs < maxCommitTsDDL.CommitTs",
   772  			zap.Uint64("commitTs", ddl.CommitTs),
   773  			zap.Uint64("maxCommitTs", c.ddlWithMaxCommitTs.CommitTs),
   774  			zap.String("DDL", ddl.Query))
   775  		return
   776  	}
   777  
   778  	// A rename tables DDL job contains multiple DDL events with same CommitTs.
   779  	// So to tell if a DDL is redundant or not, we must check the equivalence of
   780  	// the current DDL and the DDL with max CommitTs.
   781  	if ddl == c.ddlWithMaxCommitTs {
   782  		log.Info("ignore redundant DDL, the DDL is equal to ddlWithMaxCommitTs",
   783  			zap.Uint64("commitTs", ddl.CommitTs), zap.String("DDL", ddl.Query))
   784  		return
   785  	}
   786  
   787  	c.ddlList = append(c.ddlList, ddl)
   788  	log.Info("DDL event received", zap.Uint64("commitTs", ddl.CommitTs), zap.String("DDL", ddl.Query))
   789  	c.ddlWithMaxCommitTs = ddl
   790  }
   791  
   792  func (c *Consumer) getFrontDDL() *model.DDLEvent {
   793  	c.ddlListMu.Lock()
   794  	defer c.ddlListMu.Unlock()
   795  	if len(c.ddlList) > 0 {
   796  		return c.ddlList[0]
   797  	}
   798  	return nil
   799  }
   800  
   801  func (c *Consumer) popDDL() *model.DDLEvent {
   802  	c.ddlListMu.Lock()
   803  	defer c.ddlListMu.Unlock()
   804  	if len(c.ddlList) > 0 {
   805  		ddl := c.ddlList[0]
   806  		c.ddlList = c.ddlList[1:]
   807  		return ddl
   808  	}
   809  	return nil
   810  }
   811  
   812  func (c *Consumer) forEachSink(fn func(sink *partitionSinks) error) error {
   813  	c.sinksMu.Lock()
   814  	defer c.sinksMu.Unlock()
   815  	for _, sink := range c.sinks {
   816  		if err := fn(sink); err != nil {
   817  			return cerror.Trace(err)
   818  		}
   819  	}
   820  	return nil
   821  }
   822  
   823  func (c *Consumer) getMinPartitionResolvedTs() (result uint64, err error) {
   824  	result = uint64(math.MaxUint64)
   825  	err = c.forEachSink(func(sink *partitionSinks) error {
   826  		a := atomic.LoadUint64(&sink.resolvedTs)
   827  		if a < result {
   828  			result = a
   829  		}
   830  		return nil
   831  	})
   832  	return result, err
   833  }
   834  
   835  // Run the Consumer
   836  func (c *Consumer) Run(ctx context.Context) error {
   837  	ticker := time.NewTicker(100 * time.Millisecond)
   838  	defer ticker.Stop()
   839  
   840  	var globalResolvedTs uint64
   841  	for {
   842  		select {
   843  		case <-ctx.Done():
   844  			return ctx.Err()
   845  		case <-ticker.C:
   846  		}
   847  
   848  		minPartitionResolvedTs, err := c.getMinPartitionResolvedTs()
   849  		if err != nil {
   850  			return cerror.Trace(err)
   851  		}
   852  
   853  		// handle DDL
   854  		todoDDL := c.getFrontDDL()
   855  		if todoDDL != nil && todoDDL.CommitTs <= minPartitionResolvedTs {
   856  			// flush DMLs
   857  			if err = c.forEachSink(func(sink *partitionSinks) error {
   858  				return syncFlushRowChangedEvents(ctx, sink, todoDDL.CommitTs)
   859  			}); err != nil {
   860  				return cerror.Trace(err)
   861  			}
   862  
   863  			// DDL can be executed, do it first.
   864  			if err = c.ddlSink.WriteDDLEvent(ctx, todoDDL); err != nil {
   865  				return cerror.Trace(err)
   866  			}
   867  			c.popDDL()
   868  
   869  			if todoDDL.CommitTs < minPartitionResolvedTs {
   870  				log.Info("update minPartitionResolvedTs by DDL",
   871  					zap.Uint64("minPartitionResolvedTs", minPartitionResolvedTs),
   872  					zap.String("DDL", todoDDL.Query))
   873  			}
   874  			minPartitionResolvedTs = todoDDL.CommitTs
   875  		}
   876  
   877  		if globalResolvedTs > minPartitionResolvedTs {
   878  			log.Panic("global ResolvedTs fallback",
   879  				zap.Uint64("globalResolvedTs", globalResolvedTs),
   880  				zap.Uint64("minPartitionResolvedTs", minPartitionResolvedTs))
   881  		}
   882  
   883  		if globalResolvedTs < minPartitionResolvedTs {
   884  			globalResolvedTs = minPartitionResolvedTs
   885  		}
   886  
   887  		if err = c.forEachSink(func(sink *partitionSinks) error {
   888  			return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs)
   889  		}); err != nil {
   890  			return cerror.Trace(err)
   891  		}
   892  	}
   893  }
   894  
   895  func syncFlushRowChangedEvents(ctx context.Context, sink *partitionSinks, resolvedTs uint64) error {
   896  	for {
   897  		select {
   898  		case <-ctx.Done():
   899  			return ctx.Err()
   900  		default:
   901  		}
   902  		flushedResolvedTs := true
   903  		sink.tablesCommitTsMap.Range(func(key, value interface{}) bool {
   904  			tableID := key.(int64)
   905  			resolvedTs := model.NewResolvedTs(resolvedTs)
   906  			tableSink, ok := sink.tableSinksMap.Load(tableID)
   907  			if !ok {
   908  				log.Panic("Table sink not found", zap.Int64("tableID", tableID))
   909  			}
   910  			if err := tableSink.(tablesink.TableSink).UpdateResolvedTs(resolvedTs); err != nil {
   911  				log.Error("Failed to update resolved ts", zap.Error(err))
   912  				return false
   913  			}
   914  			checkpoint := tableSink.(tablesink.TableSink).GetCheckpointTs()
   915  			if !checkpoint.EqualOrGreater(resolvedTs) {
   916  				flushedResolvedTs = false
   917  			}
   918  			return true
   919  		})
   920  		if flushedResolvedTs {
   921  			return nil
   922  		}
   923  	}
   924  }
   925  
   926  type fakeTableIDGenerator struct {
   927  	tableIDs       map[string]int64
   928  	currentTableID int64
   929  	mu             sync.Mutex
   930  }
   931  
   932  func (g *fakeTableIDGenerator) generateFakeTableID(schema, table string, partition int64) int64 {
   933  	g.mu.Lock()
   934  	defer g.mu.Unlock()
   935  	key := quotes.QuoteSchema(schema, table)
   936  	if partition != 0 {
   937  		key = fmt.Sprintf("%s.`%d`", key, partition)
   938  	}
   939  	if tableID, ok := g.tableIDs[key]; ok {
   940  		return tableID
   941  	}
   942  	g.currentTableID++
   943  	g.tableIDs[key] = g.currentTableID
   944  	return g.currentTableID
   945  }
   946  
   947  func openDB(ctx context.Context, dsn string) (*sql.DB, error) {
   948  	db, err := sql.Open("mysql", dsn)
   949  	if err != nil {
   950  		log.Error("open db failed", zap.Error(err))
   951  		return nil, cerror.Trace(err)
   952  	}
   953  
   954  	db.SetMaxOpenConns(10)
   955  	db.SetMaxIdleConns(10)
   956  	db.SetConnMaxLifetime(10 * time.Minute)
   957  
   958  	ctx, cancel := context.WithTimeout(ctx, 5*time.Second)
   959  	defer cancel()
   960  	if err = db.PingContext(ctx); err != nil {
   961  		log.Error("ping db failed", zap.String("dsn", dsn), zap.Error(err))
   962  		return nil, cerror.Trace(err)
   963  	}
   964  	log.Info("open db success", zap.String("dsn", dsn))
   965  	return db, nil
   966  }