github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/kafka/factory.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 kafka 15 16 import ( 17 "context" 18 "time" 19 20 "github.com/IBM/sarama" 21 "github.com/pingcap/errors" 22 "github.com/pingcap/log" 23 "github.com/pingcap/tiflow/cdc/model" 24 cerror "github.com/pingcap/tiflow/pkg/errors" 25 "github.com/pingcap/tiflow/pkg/util" 26 "go.uber.org/zap" 27 ) 28 29 // Factory is used to produce all kafka components. 30 type Factory interface { 31 // AdminClient return a kafka cluster admin client 32 AdminClient(ctx context.Context) (ClusterAdminClient, error) 33 // SyncProducer creates a sync producer to writer message to kafka 34 SyncProducer(ctx context.Context) (SyncProducer, error) 35 // AsyncProducer creates an async producer to writer message to kafka 36 AsyncProducer(ctx context.Context, failpointCh chan error) (AsyncProducer, error) 37 // MetricsCollector returns the kafka metrics collector 38 MetricsCollector(role util.Role, adminClient ClusterAdminClient) MetricsCollector 39 } 40 41 // FactoryCreator defines the type of factory creator. 42 type FactoryCreator func(*Options, model.ChangeFeedID) (Factory, error) 43 44 // SyncProducer is the kafka sync producer 45 type SyncProducer interface { 46 // SendMessage produces a given message, and returns only when it either has 47 // succeeded or failed to produce. It will return the partition and the offset 48 // of the produced message, or an error if the message failed to produce. 49 SendMessage(ctx context.Context, 50 topic string, partitionNum int32, 51 key []byte, value []byte) error 52 53 // SendMessages produces a given set of messages, and returns only when all 54 // messages in the set have either succeeded or failed. Note that messages 55 // can succeed and fail individually; if some succeed and some fail, 56 // SendMessages will return an error. 57 SendMessages(ctx context.Context, 58 topic string, partitionNum int32, 59 key []byte, value []byte) error 60 61 // Close shuts down the producer; you must call this function before a producer 62 // object passes out of scope, as it may otherwise leak memory. 63 // You must call this before calling Close on the underlying client. 64 Close() 65 } 66 67 // AsyncProducer is the kafka async producer 68 type AsyncProducer interface { 69 // Close shuts down the producer and waits for any buffered messages to be 70 // flushed. You must call this function before a producer object passes out of 71 // scope, as it may otherwise leak memory. You must call this before process 72 // shutting down, or you may lose messages. You must call this before calling 73 // Close on the underlying client. 74 Close() 75 76 // AsyncSend is the input channel for the user to write messages to that they 77 // wish to send. 78 AsyncSend(ctx context.Context, topic string, 79 partition int32, key []byte, value []byte, 80 callback func()) error 81 82 // AsyncRunCallback process the messages that has sent to kafka, 83 // and run tha attached callback. the caller should call this 84 // method in a background goroutine 85 AsyncRunCallback(ctx context.Context) error 86 } 87 88 type saramaSyncProducer struct { 89 id model.ChangeFeedID 90 client sarama.Client 91 producer sarama.SyncProducer 92 } 93 94 func (p *saramaSyncProducer) SendMessage( 95 ctx context.Context, 96 topic string, partitionNum int32, 97 key []byte, value []byte, 98 ) error { 99 _, _, err := p.producer.SendMessage(&sarama.ProducerMessage{ 100 Topic: topic, 101 Key: sarama.ByteEncoder(key), 102 Value: sarama.ByteEncoder(value), 103 Partition: partitionNum, 104 }) 105 return err 106 } 107 108 func (p *saramaSyncProducer) SendMessages(ctx context.Context, 109 topic string, partitionNum int32, 110 key []byte, value []byte, 111 ) error { 112 msgs := make([]*sarama.ProducerMessage, partitionNum) 113 for i := 0; i < int(partitionNum); i++ { 114 msgs[i] = &sarama.ProducerMessage{ 115 Topic: topic, 116 Key: sarama.ByteEncoder(key), 117 Value: sarama.ByteEncoder(value), 118 Partition: int32(i), 119 } 120 } 121 return p.producer.SendMessages(msgs) 122 } 123 124 func (p *saramaSyncProducer) Close() { 125 go func() { 126 // We need to close it asynchronously. Otherwise, we might get stuck 127 // with an unhealthy(i.e. Network jitter, isolation) state of Kafka. 128 // Factory has a background thread to fetch and update the metadata. 129 // If we close the client synchronously, we might get stuck. 130 // Safety: 131 // * If the kafka cluster is running well, it will be closed as soon as possible. 132 // * If there is a problem with the kafka cluster, 133 // no data will be lost because this is a synchronous client. 134 // * There is a risk of goroutine leakage, but it is acceptable and our main 135 // goal is not to get stuck with the owner tick. 136 start := time.Now() 137 if err := p.client.Close(); err != nil { 138 log.Warn("Close Kafka DDL client with error", 139 zap.String("namespace", p.id.Namespace), 140 zap.String("changefeed", p.id.ID), 141 zap.Duration("duration", time.Since(start)), 142 zap.Error(err)) 143 } else { 144 log.Info("Kafka DDL client closed", 145 zap.String("namespace", p.id.Namespace), 146 zap.String("changefeed", p.id.ID), 147 zap.Duration("duration", time.Since(start))) 148 } 149 start = time.Now() 150 err := p.producer.Close() 151 if err != nil { 152 log.Error("Close Kafka DDL producer with error", 153 zap.String("namespace", p.id.Namespace), 154 zap.String("changefeed", p.id.ID), 155 zap.Duration("duration", time.Since(start)), 156 zap.Error(err)) 157 } else { 158 log.Info("Kafka DDL producer closed", 159 zap.String("namespace", p.id.Namespace), 160 zap.String("changefeed", p.id.ID), 161 zap.Duration("duration", time.Since(start))) 162 } 163 }() 164 } 165 166 type saramaAsyncProducer struct { 167 client sarama.Client 168 producer sarama.AsyncProducer 169 changefeedID model.ChangeFeedID 170 failpointCh chan error 171 } 172 173 func (p *saramaAsyncProducer) Close() { 174 go func() { 175 // We need to close it asynchronously. Otherwise, we might get stuck 176 // with an unhealthy(i.e. Network jitter, isolation) state of Kafka. 177 // Safety: 178 // * If the kafka cluster is running well, it will be closed as soon as possible. 179 // Also, we cancel all table pipelines before closed, so it's safe. 180 // * If there is a problem with the kafka cluster, it will shut down the client first, 181 // which means no more data will be sent because the connection to the broker is dropped. 182 // Also, we cancel all table pipelines before closed, so it's safe. 183 // * For Kafka Sink, duplicate data is acceptable. 184 // * There is a risk of goroutine leakage, but it is acceptable and our main 185 // goal is not to get stuck with the processor tick. 186 187 // `client` is mainly used by `asyncProducer` to fetch metadata and perform other related 188 // operations. When we close the `kafkaSaramaProducer`, 189 // there is no need for TiCDC to make sure that all buffered messages are flushed. 190 // Consider the situation where the broker is irresponsive. If the client were not 191 // closed, `asyncProducer.Close()` would waste a mount of time to try flush all messages. 192 // To prevent the scenario mentioned above, close the client first. 193 start := time.Now() 194 if err := p.client.Close(); err != nil { 195 log.Warn("Close kafka async producer client error", 196 zap.String("namespace", p.changefeedID.Namespace), 197 zap.String("changefeed", p.changefeedID.ID), 198 zap.Duration("duration", time.Since(start)), 199 zap.Error(err)) 200 } else { 201 log.Info("Close kafka async producer client success", 202 zap.String("namespace", p.changefeedID.Namespace), 203 zap.String("changefeed", p.changefeedID.ID), 204 zap.Duration("duration", time.Since(start))) 205 } 206 207 start = time.Now() 208 if err := p.producer.Close(); err != nil { 209 log.Warn("Close kafka async producer error", 210 zap.String("namespace", p.changefeedID.Namespace), 211 zap.String("changefeed", p.changefeedID.ID), 212 zap.Duration("duration", time.Since(start)), 213 zap.Error(err)) 214 } else { 215 log.Info("Close kafka async producer success", 216 zap.String("namespace", p.changefeedID.Namespace), 217 zap.String("changefeed", p.changefeedID.ID), 218 zap.Duration("duration", time.Since(start))) 219 } 220 }() 221 } 222 223 func (p *saramaAsyncProducer) AsyncRunCallback( 224 ctx context.Context, 225 ) error { 226 for { 227 select { 228 case <-ctx.Done(): 229 log.Info("async producer exit since context is done", 230 zap.String("namespace", p.changefeedID.Namespace), 231 zap.String("changefeed", p.changefeedID.ID)) 232 return errors.Trace(ctx.Err()) 233 case err := <-p.failpointCh: 234 log.Warn("Receive from failpoint chan in kafka DML producer", 235 zap.String("namespace", p.changefeedID.Namespace), 236 zap.String("changefeed", p.changefeedID.ID), 237 zap.Error(err)) 238 return errors.Trace(err) 239 case ack := <-p.producer.Successes(): 240 if ack != nil { 241 callback := ack.Metadata.(func()) 242 if callback != nil { 243 callback() 244 } 245 } 246 case err := <-p.producer.Errors(): 247 // We should not wrap a nil pointer if the pointer 248 // is of a subtype of `error` because Go would store the type info 249 // and the resulted `error` variable would not be nil, 250 // which will cause the pkg/error library to malfunction. 251 // See: https://go.dev/doc/faq#nil_error 252 if err == nil { 253 return nil 254 } 255 return cerror.WrapError(cerror.ErrKafkaAsyncSendMessage, err) 256 } 257 } 258 } 259 260 // AsyncSend is the input channel for the user to write messages to that they 261 // wish to send. 262 func (p *saramaAsyncProducer) AsyncSend(ctx context.Context, 263 topic string, 264 partition int32, 265 key []byte, 266 value []byte, 267 callback func(), 268 ) error { 269 msg := &sarama.ProducerMessage{ 270 Topic: topic, 271 Partition: partition, 272 Key: sarama.StringEncoder(key), 273 Value: sarama.ByteEncoder(value), 274 Metadata: callback, 275 } 276 select { 277 case <-ctx.Done(): 278 return errors.Trace(ctx.Err()) 279 case p.producer.Input() <- msg: 280 } 281 return nil 282 }