github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/kafka/v2/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 v2 15 16 import ( 17 "context" 18 "crypto/tls" 19 "strings" 20 "time" 21 22 "github.com/jcmturner/gokrb5/v8/client" 23 "github.com/jcmturner/gokrb5/v8/config" 24 "github.com/jcmturner/gokrb5/v8/keytab" 25 "github.com/pingcap/log" 26 "github.com/pingcap/tiflow/cdc/model" 27 "github.com/pingcap/tiflow/pkg/errors" 28 "github.com/pingcap/tiflow/pkg/security" 29 pkafka "github.com/pingcap/tiflow/pkg/sink/kafka" 30 "github.com/pingcap/tiflow/pkg/util" 31 "github.com/segmentio/kafka-go" 32 "github.com/segmentio/kafka-go/sasl" 33 "github.com/segmentio/kafka-go/sasl/plain" 34 "github.com/segmentio/kafka-go/sasl/scram" 35 "go.uber.org/zap" 36 ) 37 38 type factory struct { 39 // transport is used to contact kafka cluster and also maintain the `metadata cache` 40 // it's shared by the admin client and producers to keep the cache the same to make 41 // sure that the newly created topics can be found by the both. 42 transport *kafka.Transport 43 changefeedID model.ChangeFeedID 44 options *pkafka.Options 45 46 writer *kafka.Writer 47 } 48 49 // NewFactory returns a factory implemented based on kafka-go 50 func NewFactory( 51 options *pkafka.Options, 52 changefeedID model.ChangeFeedID, 53 ) (pkafka.Factory, error) { 54 transport, err := newTransport(options) 55 if err != nil { 56 return nil, errors.Trace(err) 57 } 58 return &factory{ 59 transport: transport, 60 changefeedID: changefeedID, 61 options: options, 62 writer: &kafka.Writer{}, 63 }, nil 64 } 65 66 func newClient(brokerEndpoints []string, transport *kafka.Transport) *kafka.Client { 67 return &kafka.Client{ 68 Addr: kafka.TCP(brokerEndpoints...), 69 // todo: make this configurable 70 Timeout: 10 * time.Second, 71 Transport: transport, 72 } 73 } 74 75 func newTransport(o *pkafka.Options) (*kafka.Transport, error) { 76 mechanism, err := completeSASLConfig(o) 77 if err != nil { 78 return nil, err 79 } 80 tlsConfig, err := completeSSLConfig(o) 81 if err != nil { 82 return nil, err 83 } 84 return &kafka.Transport{ 85 SASL: mechanism, 86 TLS: tlsConfig, 87 DialTimeout: o.DialTimeout, 88 ClientID: o.ClientID, 89 }, nil 90 } 91 92 func completeSSLConfig(options *pkafka.Options) (*tls.Config, error) { 93 if options.EnableTLS { 94 tlsConfig := &tls.Config{ 95 MinVersion: tls.VersionTLS12, 96 NextProtos: []string{"h2", "http/1.1"}, 97 } 98 99 // for SSL encryption with self-signed CA certificate, we reassign the 100 // config.Net.TLS.Config using the relevant credential files. 101 if options.Credential != nil && options.Credential.IsTLSEnabled() { 102 tlsConfig, err := options.Credential.ToTLSConfig() 103 return tlsConfig, errors.Trace(err) 104 } 105 106 tlsConfig.InsecureSkipVerify = options.InsecureSkipVerify 107 return tlsConfig, nil 108 } 109 return nil, nil 110 } 111 112 func completeSASLConfig(o *pkafka.Options) (sasl.Mechanism, error) { 113 if o.SASL != nil && o.SASL.SASLMechanism != "" { 114 switch o.SASL.SASLMechanism { 115 case pkafka.SASLTypeSCRAMSHA256, pkafka.SASLTypeSCRAMSHA512, pkafka.SASLTypePlaintext: 116 if strings.EqualFold(string(o.SASL.SASLMechanism), pkafka.SASLTypeSCRAMSHA256) { 117 mechanism, err := scram.Mechanism(scram.SHA256, 118 o.SASL.SASLUser, o.SASL.SASLPassword) 119 return mechanism, errors.Trace(err) 120 } else if strings.EqualFold(string(o.SASL.SASLMechanism), pkafka.SASLTypeSCRAMSHA512) { 121 mechanism, err := scram.Mechanism(scram.SHA512, 122 o.SASL.SASLUser, o.SASL.SASLPassword) 123 return mechanism, errors.Trace(err) 124 } else { 125 return plain.Mechanism{ 126 Username: o.SASL.SASLUser, 127 Password: o.SASL.SASLPassword, 128 }, nil 129 } 130 case pkafka.SASLTypeGSSAPI: 131 cfg, err := config.Load(o.SASL.GSSAPI.KerberosConfigPath) 132 if err != nil { 133 return nil, errors.Trace(err) 134 } 135 var clnt *client.Client 136 switch o.SASL.GSSAPI.AuthType { 137 case security.UserAuth: 138 clnt = client.NewWithPassword(o.SASL.GSSAPI.Username, o.SASL.GSSAPI.Realm, 139 o.SASL.GSSAPI.Password, cfg, 140 client.DisablePAFXFAST(o.SASL.GSSAPI.DisablePAFXFAST)) 141 case security.KeyTabAuth: 142 ktab, err := keytab.Load(o.SASL.GSSAPI.KeyTabPath) 143 if err != nil { 144 return nil, errors.Trace(err) 145 } 146 clnt = client.NewWithKeytab(o.SASL.GSSAPI.Username, o.SASL.GSSAPI.Realm, ktab, cfg, 147 client.DisablePAFXFAST(o.SASL.GSSAPI.DisablePAFXFAST)) 148 } 149 err = clnt.Login() 150 if err != nil { 151 return nil, errors.Trace(err) 152 } 153 return Gokrb5v8(&gokrb5v8ClientImpl{clnt}, 154 o.SASL.GSSAPI.ServiceName), nil 155 156 case pkafka.SASLTypeOAuth: 157 return nil, errors.ErrKafkaInvalidConfig.GenWithStack( 158 "OAuth is not yet supported in Kafka sink v2") 159 } 160 } 161 return nil, nil 162 } 163 164 func (f *factory) newWriter(async bool) *kafka.Writer { 165 w := &kafka.Writer{ 166 Addr: kafka.TCP(f.options.BrokerEndpoints...), 167 Balancer: newManualPartitioner(), 168 Transport: f.transport, 169 ReadTimeout: f.options.ReadTimeout, 170 WriteTimeout: f.options.WriteTimeout, 171 // For kafka cluster with a bad network condition, 172 // do not waste too much time to prevent long time blocking. 173 MaxAttempts: 2, 174 WriteBackoffMin: 10 * time.Millisecond, 175 RequiredAcks: kafka.RequiredAcks(f.options.RequiredAcks), 176 BatchBytes: int64(f.options.MaxMessageBytes), 177 Async: async, 178 } 179 f.writer = w 180 compression := strings.ToLower(strings.TrimSpace(f.options.Compression)) 181 switch compression { 182 case "none": 183 case "gzip": 184 w.Compression = kafka.Gzip 185 case "snappy": 186 w.Compression = kafka.Snappy 187 case "lz4": 188 w.Compression = kafka.Lz4 189 case "zstd": 190 w.Compression = kafka.Zstd 191 default: 192 log.Warn("Unsupported compression algorithm", 193 zap.String("namespace", f.changefeedID.Namespace), 194 zap.String("changefeed", f.changefeedID.ID), 195 zap.String("compression", f.options.Compression)) 196 } 197 log.Info("Kafka producer uses "+f.options.Compression+" compression algorithm", 198 zap.String("namespace", f.changefeedID.Namespace), 199 zap.String("changefeed", f.changefeedID.ID)) 200 return w 201 } 202 203 func (f *factory) AdminClient(_ context.Context) (pkafka.ClusterAdminClient, error) { 204 return newClusterAdminClient(f.options.BrokerEndpoints, f.transport, f.changefeedID), nil 205 } 206 207 // SyncProducer creates a sync producer to writer message to kafka 208 func (f *factory) SyncProducer(_ context.Context) (pkafka.SyncProducer, error) { 209 w := f.newWriter(false) 210 // set batch size to 1 to make sure the message is sent immediately 211 w.BatchTimeout = time.Millisecond 212 w.BatchSize = 1 213 return &syncWriter{ 214 w: w, 215 changefeedID: f.changefeedID, 216 }, nil 217 } 218 219 // AsyncProducer creates an async producer to writer message to kafka 220 func (f *factory) AsyncProducer( 221 ctx context.Context, 222 failpointCh chan error, 223 ) (pkafka.AsyncProducer, error) { 224 w := f.newWriter(true) 225 // assume each message is 1KB, 226 // and set batch timeout to 5ms to avoid waste too much time on waiting for messages. 227 w.BatchTimeout = 5 * time.Millisecond 228 w.BatchSize = int(w.BatchBytes / 1024) 229 aw := &asyncWriter{ 230 w: w, 231 changefeedID: f.changefeedID, 232 failpointCh: failpointCh, 233 errorsChan: make(chan error, 1), 234 } 235 236 w.Completion = func(messages []kafka.Message, err error) { 237 if err != nil { 238 select { 239 case <-ctx.Done(): 240 return 241 case aw.errorsChan <- err: 242 default: 243 log.Warn("async writer report error failed, since the err channel is full", 244 zap.String("namespace", aw.changefeedID.Namespace), 245 zap.String("changefeed", aw.changefeedID.ID), 246 zap.Error(err)) 247 } 248 return 249 } 250 251 for _, msg := range messages { 252 callback := msg.WriterData.(func()) 253 if callback != nil { 254 callback() 255 } 256 } 257 } 258 259 return aw, nil 260 } 261 262 // MetricsCollector returns the kafka metrics collector 263 func (f *factory) MetricsCollector( 264 role util.Role, 265 adminClient pkafka.ClusterAdminClient, 266 ) pkafka.MetricsCollector { 267 return NewMetricsCollector(f.changefeedID, role, f.writer) 268 } 269 270 type syncWriter struct { 271 changefeedID model.ChangeFeedID 272 w Writer 273 } 274 275 func (s *syncWriter) SendMessage( 276 ctx context.Context, 277 topic string, partitionNum int32, 278 key []byte, value []byte, 279 ) error { 280 return s.w.WriteMessages(ctx, kafka.Message{ 281 Topic: topic, 282 Partition: int(partitionNum), 283 Key: key, 284 Value: value, 285 }) 286 } 287 288 // SendMessages produces a given set of messages, and returns only when all 289 // messages in the set have either succeeded or failed. Note that messages 290 // can succeed and fail individually; if some succeed and some fail, 291 // SendMessages will return an error. 292 func (s *syncWriter) SendMessages( 293 ctx context.Context, 294 topic string, partitionNum int32, 295 key []byte, value []byte, 296 ) error { 297 msgs := make([]kafka.Message, int(partitionNum)) 298 for i := 0; i < int(partitionNum); i++ { 299 msgs[i] = kafka.Message{ 300 Topic: topic, 301 Key: key, 302 Value: value, 303 Partition: i, 304 } 305 } 306 return s.w.WriteMessages(ctx, msgs...) 307 } 308 309 // Close shuts down the producer; you must call this function before a producer 310 // object passes out of scope, as it may otherwise leak memory. 311 // You must call this before calling Close on the underlying client. 312 func (s *syncWriter) Close() { 313 log.Info("kafka sync producer start closing", 314 zap.String("namespace", s.changefeedID.Namespace), 315 zap.String("changefeed", s.changefeedID.ID)) 316 start := time.Now() 317 if err := s.w.Close(); err != nil { 318 log.Warn("Close kafka sync producer failed", 319 zap.String("namespace", s.changefeedID.Namespace), 320 zap.String("changefeed", s.changefeedID.ID), 321 zap.Duration("duration", time.Since(start)), 322 zap.Error(err)) 323 } else { 324 log.Info("Close kafka sync producer success", 325 zap.String("namespace", s.changefeedID.Namespace), 326 zap.String("changefeed", s.changefeedID.ID), 327 zap.Duration("duration", time.Since(start))) 328 } 329 } 330 331 type asyncWriter struct { 332 w Writer 333 changefeedID model.ChangeFeedID 334 failpointCh chan error 335 errorsChan chan error 336 } 337 338 // Close shuts down the producer and waits for any buffered messages to be 339 // flushed. You must call this function before a producer object passes out of 340 // scope, as it may otherwise leak memory. You must call this before process 341 // shutting down, or you may lose messages. You must call this before calling 342 // Close on the underlying client. 343 func (a *asyncWriter) Close() { 344 log.Info("kafka async producer start closing", 345 zap.String("namespace", a.changefeedID.Namespace), 346 zap.String("changefeed", a.changefeedID.ID)) 347 go func() { 348 start := time.Now() 349 if err := a.w.Close(); err != nil { 350 log.Warn("Close kafka async producer failed", 351 zap.String("namespace", a.changefeedID.Namespace), 352 zap.String("changefeed", a.changefeedID.ID), 353 zap.Duration("duration", time.Since(start)), 354 zap.Error(err)) 355 } else { 356 log.Info("Close kafka async producer success", 357 zap.String("namespace", a.changefeedID.Namespace), 358 zap.String("changefeed", a.changefeedID.ID), 359 zap.Duration("duration", time.Since(start))) 360 } 361 }() 362 } 363 364 // AsyncSend is the input channel for the user to write messages to that they 365 // wish to send. 366 func (a *asyncWriter) AsyncSend(ctx context.Context, topic string, 367 partition int32, key []byte, value []byte, 368 callback func(), 369 ) error { 370 select { 371 case <-ctx.Done(): 372 return errors.Trace(ctx.Err()) 373 default: 374 } 375 return a.w.WriteMessages(ctx, kafka.Message{ 376 Topic: topic, 377 Partition: int(partition), 378 Key: key, 379 Value: value, 380 WriterData: callback, 381 }) 382 } 383 384 // AsyncRunCallback process the messages that has sent to kafka, 385 // and run tha attached callback. the caller should call this 386 // method in a background goroutine 387 func (a *asyncWriter) AsyncRunCallback(ctx context.Context) error { 388 select { 389 case <-ctx.Done(): 390 return errors.Trace(ctx.Err()) 391 case err := <-a.failpointCh: 392 log.Warn("Receive from failpoint chan in kafka producer", 393 zap.String("namespace", a.changefeedID.Namespace), 394 zap.String("changefeed", a.changefeedID.ID), 395 zap.Error(err)) 396 return errors.Trace(err) 397 case err := <-a.errorsChan: 398 // We should not wrap a nil pointer if the pointer 399 // is of a subtype of `error` because Go would store the type info 400 // and the resulted `error` variable would not be nil, 401 // which will cause the pkg/error library to malfunction. 402 // See: https://go.dev/doc/faq#nil_error 403 if err == nil { 404 return nil 405 } 406 return errors.WrapError(errors.ErrKafkaAsyncSendMessage, err) 407 } 408 }