github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/manager/kafka_manager.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 manager 15 16 import ( 17 "context" 18 "fmt" 19 "sync" 20 "time" 21 22 "github.com/pingcap/errors" 23 "github.com/pingcap/log" 24 "github.com/pingcap/tiflow/cdc/model" 25 cerror "github.com/pingcap/tiflow/pkg/errors" 26 "github.com/pingcap/tiflow/pkg/retry" 27 "github.com/pingcap/tiflow/pkg/sink/kafka" 28 "go.uber.org/zap" 29 ) 30 31 const ( 32 // metaRefreshInterval is the interval of refreshing metadata. 33 // We can't get the metadata too frequently, because it may cause 34 // the kafka cluster to be overloaded. Especially when there are 35 // many topics in the cluster or there are many TiCDC changefeeds. 36 metaRefreshInterval = 10 * time.Minute 37 ) 38 39 // kafkaTopicManager is a manager for kafka topics. 40 type kafkaTopicManager struct { 41 changefeedID model.ChangeFeedID 42 43 defaultTopic string 44 45 admin kafka.ClusterAdminClient 46 47 cfg *kafka.AutoCreateTopicConfig 48 49 topics sync.Map 50 51 metaRefreshTicker *time.Ticker 52 53 // cancel is used to cancel the background goroutine. 54 cancel context.CancelFunc 55 } 56 57 // NewKafkaTopicManager creates a new topic manager. 58 func NewKafkaTopicManager( 59 ctx context.Context, 60 defaultTopic string, 61 changefeedID model.ChangeFeedID, 62 admin kafka.ClusterAdminClient, 63 cfg *kafka.AutoCreateTopicConfig, 64 ) *kafkaTopicManager { 65 mgr := &kafkaTopicManager{ 66 defaultTopic: defaultTopic, 67 changefeedID: changefeedID, 68 admin: admin, 69 cfg: cfg, 70 metaRefreshTicker: time.NewTicker(metaRefreshInterval), 71 } 72 73 ctx, mgr.cancel = context.WithCancel(ctx) 74 // Background refresh metadata. 75 go mgr.backgroundRefreshMeta(ctx) 76 77 return mgr 78 } 79 80 // GetPartitionNum returns the number of partitions of the topic. 81 // It may also try to update the topics' information maintained by manager. 82 func (m *kafkaTopicManager) GetPartitionNum( 83 ctx context.Context, 84 topic string, 85 ) (int32, error) { 86 if partitions, ok := m.topics.Load(topic); ok { 87 return partitions.(int32), nil 88 } 89 90 // If the topic is not in the metadata, we try to create the topic. 91 partitionNum, err := m.CreateTopicAndWaitUntilVisible(ctx, topic) 92 if err != nil { 93 return 0, errors.Trace(err) 94 } 95 96 return partitionNum, nil 97 } 98 99 func (m *kafkaTopicManager) backgroundRefreshMeta(ctx context.Context) { 100 for { 101 select { 102 case <-ctx.Done(): 103 log.Info("Background refresh Kafka metadata goroutine exit.", 104 zap.String("namespace", m.changefeedID.Namespace), 105 zap.String("changefeed", m.changefeedID.ID), 106 ) 107 return 108 case <-m.metaRefreshTicker.C: 109 // We ignore the error here, because the error may be caused by the 110 // network problem, and we can try to get the metadata next time. 111 topicPartitionNums, _ := m.fetchAllTopicsPartitionsNum(ctx) 112 for topic, partitionNum := range topicPartitionNums { 113 m.tryUpdatePartitionsAndLogging(topic, partitionNum) 114 } 115 } 116 } 117 } 118 119 // tryUpdatePartitionsAndLogging try to update the partitions of the topic. 120 func (m *kafkaTopicManager) tryUpdatePartitionsAndLogging(topic string, partitions int32) { 121 oldPartitions, ok := m.topics.Load(topic) 122 if ok { 123 if oldPartitions.(int32) != partitions { 124 m.topics.Store(topic, partitions) 125 log.Info( 126 "update topic partition number", 127 zap.String("namespace", m.changefeedID.Namespace), 128 zap.String("changefeed", m.changefeedID.ID), 129 zap.String("topic", topic), 130 zap.Int32("oldPartitionNumber", oldPartitions.(int32)), 131 zap.Int32("newPartitionNumber", partitions), 132 ) 133 } 134 } else { 135 m.topics.Store(topic, partitions) 136 log.Info( 137 "store topic partition number", 138 zap.String("namespace", m.changefeedID.Namespace), 139 zap.String("changefeed", m.changefeedID.ID), 140 zap.String("topic", topic), 141 zap.Int32("partitionNumber", partitions), 142 ) 143 } 144 } 145 146 // fetchAllTopicsPartitionsNum fetches all topics' partitions number. 147 // The error returned by this method could be a transient error that is fixable by the underlying logic. 148 // When handling this error, please be cautious. 149 // If you simply throw the error to the caller, it may impact the robustness of your program. 150 func (m *kafkaTopicManager) fetchAllTopicsPartitionsNum( 151 ctx context.Context, 152 ) (map[string]int32, error) { 153 var topics []string 154 m.topics.Range(func(key, value any) bool { 155 topics = append(topics, key.(string)) 156 return true 157 }) 158 159 start := time.Now() 160 numPartitions, err := m.admin.GetTopicsPartitionsNum(ctx, topics) 161 if err != nil { 162 log.Warn( 163 "Kafka admin client describe topics failed", 164 zap.String("namespace", m.changefeedID.Namespace), 165 zap.String("changefeed", m.changefeedID.ID), 166 zap.Error(err), 167 zap.Duration("duration", time.Since(start)), 168 ) 169 return nil, err 170 } 171 172 // it may happen the following case: 173 // 1. user create the default topic with partition number set as 3 manually 174 // 2. set the partition-number as 2 in the sink-uri. 175 // in the such case, we should use 2 instead of 3 as the partition number. 176 _, ok := numPartitions[m.defaultTopic] 177 if ok { 178 numPartitions[m.defaultTopic] = m.cfg.PartitionNum 179 } 180 181 log.Info( 182 "Kafka admin client describe topics success", 183 zap.String("namespace", m.changefeedID.Namespace), 184 zap.String("changefeed", m.changefeedID.ID), 185 zap.Duration("duration", time.Since(start))) 186 187 return numPartitions, nil 188 } 189 190 // waitUntilTopicVisible is called after CreateTopic to make sure the topic 191 // can be safely written to. The reason is that it may take several seconds after 192 // CreateTopic returns success for all the brokers to become aware that the 193 // topics have been created. 194 // See https://kafka.apache.org/23/javadoc/org/apache/kafka/clients/admin/AdminClient.html 195 func (m *kafkaTopicManager) waitUntilTopicVisible( 196 ctx context.Context, 197 topicName string, 198 ) error { 199 topics := []string{topicName} 200 err := retry.Do(ctx, func() error { 201 start := time.Now() 202 // ignoreTopicError is set to false since we just create the topic, 203 // make sure the topic is visible. 204 meta, err := m.admin.GetTopicsMeta(ctx, topics, false) 205 if err != nil { 206 log.Warn(" topic not found, retry it", 207 zap.String("namespace", m.changefeedID.Namespace), 208 zap.String("changefeed", m.changefeedID.ID), 209 zap.Error(err), 210 zap.Duration("duration", time.Since(start)), 211 ) 212 return err 213 } 214 log.Info("topic found", 215 zap.String("namespace", m.changefeedID.Namespace), 216 zap.String("changefeed", m.changefeedID.ID), 217 zap.String("topic", topicName), 218 zap.Int32("partitionNumber", meta[topicName].NumPartitions), 219 zap.Duration("duration", time.Since(start))) 220 return nil 221 }, retry.WithBackoffBaseDelay(500), 222 retry.WithBackoffMaxDelay(1000), 223 retry.WithMaxTries(6), 224 ) 225 226 return err 227 } 228 229 // createTopic creates a topic with the given name 230 // and returns the number of partitions. 231 func (m *kafkaTopicManager) createTopic( 232 ctx context.Context, 233 topicName string, 234 ) (int32, error) { 235 if !m.cfg.AutoCreate { 236 return 0, cerror.ErrKafkaInvalidConfig.GenWithStack( 237 fmt.Sprintf("`auto-create-topic` is false, "+ 238 "and %s not found", topicName)) 239 } 240 241 start := time.Now() 242 err := m.admin.CreateTopic(ctx, &kafka.TopicDetail{ 243 Name: topicName, 244 NumPartitions: m.cfg.PartitionNum, 245 ReplicationFactor: m.cfg.ReplicationFactor, 246 }, false) 247 if err != nil { 248 log.Error( 249 "Kafka admin client create the topic failed", 250 zap.String("namespace", m.changefeedID.Namespace), 251 zap.String("changefeed", m.changefeedID.ID), 252 zap.String("topic", topicName), 253 zap.Int32("partitionNumber", m.cfg.PartitionNum), 254 zap.Int16("replicationFactor", m.cfg.ReplicationFactor), 255 zap.Error(err), 256 zap.Duration("duration", time.Since(start)), 257 ) 258 return 0, cerror.WrapError(cerror.ErrKafkaCreateTopic, err) 259 } 260 261 log.Info( 262 "Kafka admin client create the topic success", 263 zap.String("namespace", m.changefeedID.Namespace), 264 zap.String("changefeed", m.changefeedID.ID), 265 zap.String("topic", topicName), 266 zap.Int32("partitionNumber", m.cfg.PartitionNum), 267 zap.Int16("replicationFactor", m.cfg.ReplicationFactor), 268 zap.Duration("duration", time.Since(start)), 269 ) 270 m.tryUpdatePartitionsAndLogging(topicName, m.cfg.PartitionNum) 271 272 return m.cfg.PartitionNum, nil 273 } 274 275 // CreateTopicAndWaitUntilVisible wraps createTopic and waitUntilTopicVisible together. 276 func (m *kafkaTopicManager) CreateTopicAndWaitUntilVisible( 277 ctx context.Context, topicName string, 278 ) (int32, error) { 279 // If the topic is not in the cache, we try to get the metadata of the topic. 280 // ignoreTopicErr is set to true to ignore the error if the topic is not found, 281 // which means we should create the topic later. 282 topicDetails, err := m.admin.GetTopicsMeta(ctx, []string{topicName}, true) 283 if err != nil { 284 return 0, errors.Trace(err) 285 } 286 if detail, ok := topicDetails[topicName]; ok { 287 numPartition := detail.NumPartitions 288 if topicName == m.defaultTopic { 289 numPartition = m.cfg.PartitionNum 290 } 291 m.tryUpdatePartitionsAndLogging(topicName, numPartition) 292 return numPartition, nil 293 } 294 295 partitionNum, err := m.createTopic(ctx, topicName) 296 if err != nil { 297 return 0, errors.Trace(err) 298 } 299 300 err = m.waitUntilTopicVisible(ctx, topicName) 301 if err != nil { 302 return 0, errors.Trace(err) 303 } 304 305 return partitionNum, nil 306 } 307 308 // Close exits the background goroutine. 309 func (m *kafkaTopicManager) Close() { 310 m.cancel() 311 }