github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/ddlsink/mq/mq_ddl_sink.go (about) 1 // Copyright 2022 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 mq 15 16 import ( 17 "context" 18 19 "github.com/pingcap/errors" 20 "github.com/pingcap/log" 21 "github.com/pingcap/tiflow/cdc/model" 22 "github.com/pingcap/tiflow/cdc/sink/ddlsink" 23 "github.com/pingcap/tiflow/cdc/sink/ddlsink/mq/ddlproducer" 24 "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/dispatcher" 25 "github.com/pingcap/tiflow/cdc/sink/dmlsink/mq/manager" 26 "github.com/pingcap/tiflow/cdc/sink/metrics" 27 "github.com/pingcap/tiflow/pkg/config" 28 "github.com/pingcap/tiflow/pkg/sink" 29 "github.com/pingcap/tiflow/pkg/sink/codec" 30 "github.com/pingcap/tiflow/pkg/sink/kafka" 31 "go.uber.org/zap" 32 ) 33 34 // DDLDispatchRule is the dispatch rule for DDL event. 35 type DDLDispatchRule int 36 37 const ( 38 // PartitionZero means the DDL event will be dispatched to partition 0. 39 // NOTICE: Only for canal and canal-json protocol. 40 PartitionZero DDLDispatchRule = iota 41 // PartitionAll means the DDL event will be broadcast to all the partitions. 42 PartitionAll 43 ) 44 45 func getDDLDispatchRule(protocol config.Protocol) DDLDispatchRule { 46 switch protocol { 47 case config.ProtocolCanal, config.ProtocolCanalJSON: 48 return PartitionZero 49 default: 50 } 51 return PartitionAll 52 } 53 54 // Assert Sink implementation 55 var _ ddlsink.Sink = (*DDLSink)(nil) 56 57 // DDLSink is a sink that sends DDL events to the MQ system. 58 type DDLSink struct { 59 // id indicates which processor (changefeed) this sink belongs to. 60 id model.ChangeFeedID 61 // protocol indicates the protocol used by this sink. 62 protocol config.Protocol 63 // eventRouter used to route events to the right topic and partition. 64 eventRouter *dispatcher.EventRouter 65 // topicManager used to manage topics. 66 // It is also responsible for creating topics. 67 topicManager manager.TopicManager 68 // encoderBuilder builds encoder for the sink. 69 encoderBuilder codec.RowEventEncoderBuilder 70 // producer used to send events to the MQ system. 71 // Usually it is a sync producer. 72 producer ddlproducer.DDLProducer 73 // statistics is used to record DDL metrics. 74 statistics *metrics.Statistics 75 // admin is used to query kafka cluster information. 76 admin kafka.ClusterAdminClient 77 } 78 79 func newDDLSink(ctx context.Context, 80 changefeedID model.ChangeFeedID, 81 producer ddlproducer.DDLProducer, 82 adminClient kafka.ClusterAdminClient, 83 topicManager manager.TopicManager, 84 eventRouter *dispatcher.EventRouter, 85 encoderBuilder codec.RowEventEncoderBuilder, 86 protocol config.Protocol, 87 ) *DDLSink { 88 return &DDLSink{ 89 id: changefeedID, 90 protocol: protocol, 91 eventRouter: eventRouter, 92 topicManager: topicManager, 93 encoderBuilder: encoderBuilder, 94 producer: producer, 95 statistics: metrics.NewStatistics(ctx, changefeedID, sink.RowSink), 96 admin: adminClient, 97 } 98 } 99 100 // WriteDDLEvent encodes the DDL event and sends it to the MQ system. 101 func (k *DDLSink) WriteDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { 102 encoder := k.encoderBuilder.Build() 103 msg, err := encoder.EncodeDDLEvent(ddl) 104 if err != nil { 105 return errors.Trace(err) 106 } 107 if msg == nil { 108 log.Info("Skip ddl event", zap.Uint64("commitTs", ddl.CommitTs), 109 zap.String("query", ddl.Query), 110 zap.String("protocol", k.protocol.String()), 111 zap.String("namespace", k.id.Namespace), 112 zap.String("changefeed", k.id.ID)) 113 return nil 114 } 115 116 topic := k.eventRouter.GetTopicForDDL(ddl) 117 partitionRule := getDDLDispatchRule(k.protocol) 118 log.Debug("Emit ddl event", 119 zap.Uint64("commitTs", ddl.CommitTs), 120 zap.String("query", ddl.Query), 121 zap.String("namespace", k.id.Namespace), 122 zap.String("changefeed", k.id.ID)) 123 // Notice: We must call GetPartitionNum here, 124 // which will be responsible for automatically creating topics when they don't exist. 125 // If it is not called here and kafka has `auto.create.topics.enable` turned on, 126 // then the auto-created topic will not be created as configured by ticdc. 127 partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) 128 if err != nil { 129 return errors.Trace(err) 130 } 131 132 if partitionRule == PartitionAll { 133 err = k.statistics.RecordDDLExecution(func() error { 134 return k.producer.SyncBroadcastMessage(ctx, topic, partitionNum, msg) 135 }) 136 return errors.Trace(err) 137 } 138 err = k.statistics.RecordDDLExecution(func() error { 139 return k.producer.SyncSendMessage(ctx, topic, 0, msg) 140 }) 141 return errors.Trace(err) 142 } 143 144 // WriteCheckpointTs sends the checkpoint ts to the MQ system. 145 func (k *DDLSink) WriteCheckpointTs(ctx context.Context, 146 ts uint64, tables []*model.TableInfo, 147 ) error { 148 encoder := k.encoderBuilder.Build() 149 msg, err := encoder.EncodeCheckpointEvent(ts) 150 if err != nil { 151 return errors.Trace(err) 152 } 153 if msg == nil { 154 return nil 155 } 156 // NOTICE: When there are no tables to replicate, 157 // we need to send checkpoint ts to the default topic. 158 // This will be compatible with the old behavior. 159 if len(tables) == 0 { 160 topic := k.eventRouter.GetDefaultTopic() 161 partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) 162 if err != nil { 163 return errors.Trace(err) 164 } 165 log.Debug("Emit checkpointTs to default topic", 166 zap.String("topic", topic), zap.Uint64("checkpointTs", ts)) 167 err = k.producer.SyncBroadcastMessage(ctx, topic, partitionNum, msg) 168 return errors.Trace(err) 169 } 170 var tableNames []model.TableName 171 for _, table := range tables { 172 tableNames = append(tableNames, table.TableName) 173 } 174 topics := k.eventRouter.GetActiveTopics(tableNames) 175 for _, topic := range topics { 176 partitionNum, err := k.topicManager.GetPartitionNum(ctx, topic) 177 if err != nil { 178 return errors.Trace(err) 179 } 180 err = k.producer.SyncBroadcastMessage(ctx, topic, partitionNum, msg) 181 if err != nil { 182 return errors.Trace(err) 183 } 184 } 185 return nil 186 } 187 188 // Close closes the sink. 189 func (k *DDLSink) Close() { 190 if k.producer != nil { 191 k.producer.Close() 192 } 193 if k.topicManager != nil { 194 k.topicManager.Close() 195 } 196 if k.admin != nil { 197 k.admin.Close() 198 } 199 }