github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/kafka/options.go (about) 1 // Copyright 2021 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 "encoding/base64" 19 "fmt" 20 "net/http" 21 "net/url" 22 "regexp" 23 "strconv" 24 "strings" 25 "time" 26 27 "github.com/gin-gonic/gin/binding" 28 "github.com/imdario/mergo" 29 "github.com/pingcap/errors" 30 "github.com/pingcap/log" 31 "github.com/pingcap/tiflow/cdc/model" 32 "github.com/pingcap/tiflow/pkg/config" 33 cerror "github.com/pingcap/tiflow/pkg/errors" 34 "github.com/pingcap/tiflow/pkg/security" 35 "go.uber.org/zap" 36 ) 37 38 const ( 39 // defaultPartitionNum specifies the default number of partitions when we create the topic. 40 defaultPartitionNum = 3 41 42 // the `max-message-bytes` is set equal to topic's `max.message.bytes`, and is used to check 43 // whether the message is larger than the max size limit. It's found some message pass the message 44 // size limit check at the client side and failed at the broker side since message enlarged during 45 // the network transmission. so we set the `max-message-bytes` to a smaller value to avoid this problem. 46 // maxMessageBytesOverhead is used to reduce the `max-message-bytes`. 47 maxMessageBytesOverhead = 128 48 ) 49 50 const ( 51 // BrokerMessageMaxBytesConfigName specifies the largest record batch size allowed by 52 // Kafka brokers. 53 // See: https://kafka.apache.org/documentation/#brokerconfigs_message.max.bytes 54 BrokerMessageMaxBytesConfigName = "message.max.bytes" 55 // TopicMaxMessageBytesConfigName specifies the largest record batch size allowed by 56 // Kafka topics. 57 // See: https://kafka.apache.org/documentation/#topicconfigs_max.message.bytes 58 TopicMaxMessageBytesConfigName = "max.message.bytes" 59 // MinInsyncReplicasConfigName the minimum number of replicas that must acknowledge a write 60 // for the write to be considered successful. 61 // Only works if the producer's acks is "all" (or "-1"). 62 // See: https://kafka.apache.org/documentation/#brokerconfigs_min.insync.replicas and 63 // https://kafka.apache.org/documentation/#topicconfigs_min.insync.replicas 64 MinInsyncReplicasConfigName = "min.insync.replicas" 65 ) 66 67 const ( 68 // SASLTypePlaintext represents the plain mechanism 69 SASLTypePlaintext = "PLAIN" 70 // SASLTypeSCRAMSHA256 represents the SCRAM-SHA-256 mechanism. 71 SASLTypeSCRAMSHA256 = "SCRAM-SHA-256" 72 // SASLTypeSCRAMSHA512 represents the SCRAM-SHA-512 mechanism. 73 SASLTypeSCRAMSHA512 = "SCRAM-SHA-512" 74 // SASLTypeGSSAPI represents the gssapi mechanism. 75 SASLTypeGSSAPI = "GSSAPI" 76 // SASLTypeOAuth represents the SASL/OAUTHBEARER mechanism (Kafka 2.0.0+) 77 SASLTypeOAuth = "OAUTHBEARER" 78 ) 79 80 // RequiredAcks is used in Produce Requests to tell the broker how many replica acknowledgements 81 // it must see before responding. Any of the constants defined here are valid. On broker versions 82 // prior to 0.8.2.0 any other positive int16 is also valid (the broker will wait for that many 83 // acknowledgements) but in 0.8.2.0 and later this will raise an exception (it has been replaced 84 // by setting the `min.isr` value in the brokers configuration). 85 type RequiredAcks int16 86 87 const ( 88 // NoResponse doesn't send any response, the TCP ACK is all you get. 89 NoResponse RequiredAcks = 0 90 // WaitForLocal waits for only the local commit to succeed before responding. 91 WaitForLocal RequiredAcks = 1 92 // WaitForAll waits for all in-sync replicas to commit before responding. 93 // The minimum number of in-sync replicas is configured on the broker via 94 // the `min.insync.replicas` configuration key. 95 WaitForAll RequiredAcks = -1 96 // Unknown should never have been use in real config. 97 Unknown RequiredAcks = 2 98 ) 99 100 func requireAcksFromString(acks int) (RequiredAcks, error) { 101 switch acks { 102 case int(WaitForAll): 103 return WaitForAll, nil 104 case int(WaitForLocal): 105 return WaitForLocal, nil 106 case int(NoResponse): 107 return NoResponse, nil 108 default: 109 return Unknown, cerror.ErrKafkaInvalidRequiredAcks.GenWithStackByArgs(acks) 110 } 111 } 112 113 type urlConfig struct { 114 PartitionNum *int32 `form:"partition-num"` 115 ReplicationFactor *int16 `form:"replication-factor"` 116 KafkaVersion *string `form:"kafka-version"` 117 MaxMessageBytes *int `form:"max-message-bytes"` 118 Compression *string `form:"compression"` 119 KafkaClientID *string `form:"kafka-client-id"` 120 AutoCreateTopic *bool `form:"auto-create-topic"` 121 DialTimeout *string `form:"dial-timeout"` 122 WriteTimeout *string `form:"write-timeout"` 123 ReadTimeout *string `form:"read-timeout"` 124 RequiredAcks *int `form:"required-acks"` 125 SASLUser *string `form:"sasl-user"` 126 SASLPassword *string `form:"sasl-password"` 127 SASLMechanism *string `form:"sasl-mechanism"` 128 SASLGssAPIAuthType *string `form:"sasl-gssapi-auth-type"` 129 SASLGssAPIKeytabPath *string `form:"sasl-gssapi-keytab-path"` 130 SASLGssAPIKerberosConfigPath *string `form:"sasl-gssapi-kerberos-config-path"` 131 SASLGssAPIServiceName *string `form:"sasl-gssapi-service-name"` 132 SASLGssAPIUser *string `form:"sasl-gssapi-user"` 133 SASLGssAPIPassword *string `form:"sasl-gssapi-password"` 134 SASLGssAPIRealm *string `form:"sasl-gssapi-realm"` 135 SASLGssAPIDisablePafxfast *bool `form:"sasl-gssapi-disable-pafxfast"` 136 EnableTLS *bool `form:"enable-tls"` 137 CA *string `form:"ca"` 138 Cert *string `form:"cert"` 139 Key *string `form:"key"` 140 InsecureSkipVerify *bool `form:"insecure-skip-verify"` 141 } 142 143 // Options stores user specified configurations 144 type Options struct { 145 BrokerEndpoints []string 146 147 // control whether to create topic 148 AutoCreate bool 149 PartitionNum int32 150 // User should make sure that `replication-factor` not greater than the number of kafka brokers. 151 ReplicationFactor int16 152 Version string 153 IsAssignedVersion bool 154 RequestVersion int16 155 MaxMessageBytes int 156 Compression string 157 ClientID string 158 RequiredAcks RequiredAcks 159 // Only for test. User can not set this value. 160 // The current prod default value is 0. 161 MaxMessages int 162 163 // Credential is used to connect to kafka cluster. 164 EnableTLS bool 165 Credential *security.Credential 166 InsecureSkipVerify bool 167 SASL *security.SASL 168 169 // Timeout for network configurations, default to `10s` 170 DialTimeout time.Duration 171 WriteTimeout time.Duration 172 ReadTimeout time.Duration 173 } 174 175 // NewOptions returns a default Kafka configuration 176 func NewOptions() *Options { 177 return &Options{ 178 Version: "2.4.0", 179 // MaxMessageBytes will be used to initialize producer 180 MaxMessageBytes: config.DefaultMaxMessageBytes, 181 ReplicationFactor: 1, 182 Compression: "none", 183 RequiredAcks: WaitForAll, 184 Credential: &security.Credential{}, 185 InsecureSkipVerify: false, 186 SASL: &security.SASL{}, 187 AutoCreate: true, 188 DialTimeout: 10 * time.Second, 189 WriteTimeout: 10 * time.Second, 190 ReadTimeout: 10 * time.Second, 191 } 192 } 193 194 // SetPartitionNum set the partition-num by the topic's partition count. 195 func (o *Options) SetPartitionNum(realPartitionCount int32) error { 196 // user does not specify the `partition-num` in the sink-uri 197 if o.PartitionNum == 0 { 198 o.PartitionNum = realPartitionCount 199 log.Info("partitionNum is not set, set by topic's partition-num", 200 zap.Int32("partitionNum", realPartitionCount)) 201 return nil 202 } 203 204 if o.PartitionNum < realPartitionCount { 205 log.Warn("number of partition specified in sink-uri is less than that of the actual topic. "+ 206 "Some partitions will not have messages dispatched to", 207 zap.Int32("sinkUriPartitions", o.PartitionNum), 208 zap.Int32("topicPartitions", realPartitionCount)) 209 return nil 210 } 211 212 // Make sure that the user-specified `partition-num` is not greater than 213 // the real partition count, since messages would be dispatched to different 214 // partitions, this could prevent potential correctness problems. 215 if o.PartitionNum > realPartitionCount { 216 return cerror.ErrKafkaInvalidPartitionNum.GenWithStack( 217 "the number of partition (%d) specified in sink-uri is more than that of actual topic (%d)", 218 o.PartitionNum, realPartitionCount) 219 } 220 return nil 221 } 222 223 // Apply the sinkURI to update Options 224 func (o *Options) Apply(changefeedID model.ChangeFeedID, 225 sinkURI *url.URL, replicaConfig *config.ReplicaConfig, 226 ) error { 227 o.BrokerEndpoints = strings.Split(sinkURI.Host, ",") 228 229 var err error 230 req := &http.Request{URL: sinkURI} 231 urlParameter := &urlConfig{} 232 if err := binding.Query.Bind(req, urlParameter); err != nil { 233 return cerror.WrapError(cerror.ErrMySQLInvalidConfig, err) 234 } 235 if urlParameter, err = mergeConfig(replicaConfig, urlParameter); err != nil { 236 return err 237 } 238 if urlParameter.PartitionNum != nil { 239 o.PartitionNum = *urlParameter.PartitionNum 240 if o.PartitionNum <= 0 { 241 return cerror.ErrKafkaInvalidPartitionNum.GenWithStackByArgs(o.PartitionNum) 242 } 243 } 244 245 if urlParameter.ReplicationFactor != nil { 246 o.ReplicationFactor = *urlParameter.ReplicationFactor 247 } 248 249 if urlParameter.KafkaVersion != nil { 250 o.Version = *urlParameter.KafkaVersion 251 o.IsAssignedVersion = true 252 } 253 254 if urlParameter.MaxMessageBytes != nil { 255 o.MaxMessageBytes = *urlParameter.MaxMessageBytes 256 } 257 258 if urlParameter.Compression != nil { 259 o.Compression = *urlParameter.Compression 260 } 261 262 var kafkaClientID string 263 if urlParameter.KafkaClientID != nil { 264 kafkaClientID = *urlParameter.KafkaClientID 265 } 266 clientID, err := NewKafkaClientID( 267 config.GetGlobalServerConfig().AdvertiseAddr, 268 changefeedID, 269 kafkaClientID) 270 if err != nil { 271 return err 272 } 273 o.ClientID = clientID 274 275 if urlParameter.AutoCreateTopic != nil { 276 o.AutoCreate = *urlParameter.AutoCreateTopic 277 } 278 279 if urlParameter.DialTimeout != nil && *urlParameter.DialTimeout != "" { 280 a, err := time.ParseDuration(*urlParameter.DialTimeout) 281 if err != nil { 282 return err 283 } 284 o.DialTimeout = a 285 } 286 287 if urlParameter.WriteTimeout != nil && *urlParameter.WriteTimeout != "" { 288 a, err := time.ParseDuration(*urlParameter.WriteTimeout) 289 if err != nil { 290 return err 291 } 292 o.WriteTimeout = a 293 } 294 295 if urlParameter.ReadTimeout != nil && *urlParameter.ReadTimeout != "" { 296 a, err := time.ParseDuration(*urlParameter.ReadTimeout) 297 if err != nil { 298 return err 299 } 300 o.ReadTimeout = a 301 } 302 303 if urlParameter.RequiredAcks != nil { 304 r, err := requireAcksFromString(*urlParameter.RequiredAcks) 305 if err != nil { 306 return err 307 } 308 o.RequiredAcks = r 309 } 310 311 err = o.applySASL(urlParameter, replicaConfig) 312 if err != nil { 313 return err 314 } 315 316 err = o.applyTLS(urlParameter) 317 if err != nil { 318 return err 319 } 320 321 return nil 322 } 323 324 func mergeConfig( 325 replicaConfig *config.ReplicaConfig, 326 urlParameters *urlConfig, 327 ) (*urlConfig, error) { 328 dest := &urlConfig{} 329 if replicaConfig.Sink != nil && replicaConfig.Sink.KafkaConfig != nil { 330 fileConifg := replicaConfig.Sink.KafkaConfig 331 dest.PartitionNum = fileConifg.PartitionNum 332 dest.ReplicationFactor = fileConifg.ReplicationFactor 333 dest.KafkaVersion = fileConifg.KafkaVersion 334 dest.MaxMessageBytes = fileConifg.MaxMessageBytes 335 dest.Compression = fileConifg.Compression 336 dest.KafkaClientID = fileConifg.KafkaClientID 337 dest.AutoCreateTopic = fileConifg.AutoCreateTopic 338 dest.DialTimeout = fileConifg.DialTimeout 339 dest.WriteTimeout = fileConifg.WriteTimeout 340 dest.ReadTimeout = fileConifg.ReadTimeout 341 dest.RequiredAcks = fileConifg.RequiredAcks 342 dest.SASLUser = fileConifg.SASLUser 343 dest.SASLPassword = fileConifg.SASLPassword 344 dest.SASLMechanism = fileConifg.SASLMechanism 345 dest.SASLGssAPIDisablePafxfast = fileConifg.SASLGssAPIDisablePafxfast 346 dest.SASLGssAPIAuthType = fileConifg.SASLGssAPIAuthType 347 dest.SASLGssAPIKeytabPath = fileConifg.SASLGssAPIKeytabPath 348 dest.SASLGssAPIServiceName = fileConifg.SASLGssAPIServiceName 349 dest.SASLGssAPIKerberosConfigPath = fileConifg.SASLGssAPIKerberosConfigPath 350 dest.SASLGssAPIRealm = fileConifg.SASLGssAPIRealm 351 dest.SASLGssAPIUser = fileConifg.SASLGssAPIUser 352 dest.SASLGssAPIPassword = fileConifg.SASLGssAPIPassword 353 dest.EnableTLS = fileConifg.EnableTLS 354 dest.CA = fileConifg.CA 355 dest.Cert = fileConifg.Cert 356 dest.Key = fileConifg.Key 357 dest.InsecureSkipVerify = fileConifg.InsecureSkipVerify 358 } 359 if err := mergo.Merge(dest, urlParameters, mergo.WithOverride); err != nil { 360 return nil, err 361 } 362 return dest, nil 363 } 364 365 func (o *Options) applyTLS(params *urlConfig) error { 366 if params.CA != nil && *params.CA != "" { 367 o.Credential.CAPath = *params.CA 368 } 369 370 if params.Cert != nil && *params.Cert != "" { 371 o.Credential.CertPath = *params.Cert 372 } 373 374 if params.Key != nil && *params.Key != "" { 375 o.Credential.KeyPath = *params.Key 376 } 377 378 if o.Credential != nil && !o.Credential.IsEmpty() && 379 !o.Credential.IsTLSEnabled() { 380 return cerror.WrapError(cerror.ErrKafkaInvalidConfig, 381 errors.New("ca, cert and key files should all be supplied")) 382 } 383 384 // if enable-tls is not set, but credential files are set, 385 // then tls should be enabled, and the self-signed CA certificate is used. 386 // if enable-tls is set to true, and credential files are not set, 387 // then tls should be enabled, and the trusted CA certificate on OS is used. 388 // if enable-tls is set to false, and credential files are set, 389 // then an error is returned. 390 if params.EnableTLS != nil { 391 enableTLS := *params.EnableTLS 392 393 if o.Credential != nil && o.Credential.IsTLSEnabled() && !enableTLS { 394 return cerror.WrapError(cerror.ErrKafkaInvalidConfig, 395 errors.New("credential files are supplied, but 'enable-tls' is set to false")) 396 } 397 o.EnableTLS = enableTLS 398 } else { 399 if o.Credential != nil && o.Credential.IsTLSEnabled() { 400 o.EnableTLS = true 401 } 402 } 403 404 // Only set InsecureSkipVerify when enable the TLS. 405 if o.EnableTLS && params.InsecureSkipVerify != nil { 406 o.InsecureSkipVerify = *params.InsecureSkipVerify 407 } 408 409 return nil 410 } 411 412 func (o *Options) applySASL(urlParameter *urlConfig, replicaConfig *config.ReplicaConfig) error { 413 if urlParameter.SASLUser != nil && *urlParameter.SASLUser != "" { 414 o.SASL.SASLUser = *urlParameter.SASLUser 415 } 416 417 if urlParameter.SASLPassword != nil && *urlParameter.SASLPassword != "" { 418 o.SASL.SASLPassword = *urlParameter.SASLPassword 419 } 420 421 if urlParameter.SASLMechanism != nil && *urlParameter.SASLMechanism != "" { 422 mechanism, err := security.SASLMechanismFromString(*urlParameter.SASLMechanism) 423 if err != nil { 424 return cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 425 } 426 o.SASL.SASLMechanism = mechanism 427 } 428 429 if urlParameter.SASLGssAPIAuthType != nil && *urlParameter.SASLGssAPIAuthType != "" { 430 authType, err := security.AuthTypeFromString(*urlParameter.SASLGssAPIAuthType) 431 if err != nil { 432 return cerror.WrapError(cerror.ErrKafkaInvalidConfig, err) 433 } 434 o.SASL.GSSAPI.AuthType = authType 435 } 436 437 if urlParameter.SASLGssAPIKeytabPath != nil && *urlParameter.SASLGssAPIKeytabPath != "" { 438 o.SASL.GSSAPI.KeyTabPath = *urlParameter.SASLGssAPIKeytabPath 439 } 440 441 if urlParameter.SASLGssAPIKerberosConfigPath != nil && 442 *urlParameter.SASLGssAPIKerberosConfigPath != "" { 443 o.SASL.GSSAPI.KerberosConfigPath = *urlParameter.SASLGssAPIKerberosConfigPath 444 } 445 446 if urlParameter.SASLGssAPIServiceName != nil && *urlParameter.SASLGssAPIServiceName != "" { 447 o.SASL.GSSAPI.ServiceName = *urlParameter.SASLGssAPIServiceName 448 } 449 450 if urlParameter.SASLGssAPIUser != nil && *urlParameter.SASLGssAPIUser != "" { 451 o.SASL.GSSAPI.Username = *urlParameter.SASLGssAPIUser 452 } 453 454 if urlParameter.SASLGssAPIPassword != nil && *urlParameter.SASLGssAPIPassword != "" { 455 o.SASL.GSSAPI.Password = *urlParameter.SASLGssAPIPassword 456 } 457 458 if urlParameter.SASLGssAPIRealm != nil && *urlParameter.SASLGssAPIRealm != "" { 459 o.SASL.GSSAPI.Realm = *urlParameter.SASLGssAPIRealm 460 } 461 462 if urlParameter.SASLGssAPIDisablePafxfast != nil { 463 o.SASL.GSSAPI.DisablePAFXFAST = *urlParameter.SASLGssAPIDisablePafxfast 464 } 465 466 if replicaConfig.Sink != nil && replicaConfig.Sink.KafkaConfig != nil { 467 if replicaConfig.Sink.KafkaConfig.SASLOAuthClientID != nil { 468 clientID := *replicaConfig.Sink.KafkaConfig.SASLOAuthClientID 469 if clientID == "" { 470 return cerror.ErrKafkaInvalidConfig.GenWithStack("OAuth2 client ID cannot be empty") 471 } 472 o.SASL.OAuth2.ClientID = clientID 473 } 474 475 if replicaConfig.Sink.KafkaConfig.SASLOAuthClientSecret != nil { 476 clientSecret := *replicaConfig.Sink.KafkaConfig.SASLOAuthClientSecret 477 if clientSecret == "" { 478 return cerror.ErrKafkaInvalidConfig.GenWithStack( 479 "OAuth2 client secret cannot be empty") 480 } 481 482 // BASE64 decode the client secret 483 decodedClientSecret, err := base64.StdEncoding.DecodeString(clientSecret) 484 if err != nil { 485 log.Error("OAuth2 client secret is not base64 encoded", zap.Error(err)) 486 return cerror.ErrKafkaInvalidConfig.GenWithStack( 487 "OAuth2 client secret is not base64 encoded") 488 } 489 o.SASL.OAuth2.ClientSecret = string(decodedClientSecret) 490 } 491 492 if replicaConfig.Sink.KafkaConfig.SASLOAuthTokenURL != nil { 493 tokenURL := *replicaConfig.Sink.KafkaConfig.SASLOAuthTokenURL 494 if tokenURL == "" { 495 return cerror.ErrKafkaInvalidConfig.GenWithStack( 496 "OAuth2 token URL cannot be empty") 497 } 498 o.SASL.OAuth2.TokenURL = tokenURL 499 } 500 501 if o.SASL.OAuth2.IsEnable() { 502 if o.SASL.SASLMechanism != security.OAuthMechanism { 503 return cerror.ErrKafkaInvalidConfig.GenWithStack( 504 "OAuth2 is only supported with SASL mechanism type OAUTHBEARER, but got %s", 505 o.SASL.SASLMechanism) 506 } 507 508 if err := o.SASL.OAuth2.Validate(); err != nil { 509 return cerror.ErrKafkaInvalidConfig.Wrap(err) 510 } 511 o.SASL.OAuth2.SetDefault() 512 } 513 514 if replicaConfig.Sink.KafkaConfig.SASLOAuthScopes != nil { 515 o.SASL.OAuth2.Scopes = replicaConfig.Sink.KafkaConfig.SASLOAuthScopes 516 } 517 518 if replicaConfig.Sink.KafkaConfig.SASLOAuthGrantType != nil { 519 o.SASL.OAuth2.GrantType = *replicaConfig.Sink.KafkaConfig.SASLOAuthGrantType 520 } 521 522 if replicaConfig.Sink.KafkaConfig.SASLOAuthAudience != nil { 523 o.SASL.OAuth2.Audience = *replicaConfig.Sink.KafkaConfig.SASLOAuthAudience 524 } 525 } 526 527 return nil 528 } 529 530 // AutoCreateTopicConfig is used to create topic configuration. 531 type AutoCreateTopicConfig struct { 532 AutoCreate bool 533 PartitionNum int32 534 ReplicationFactor int16 535 } 536 537 // DeriveTopicConfig derive a `topicConfig` from the `Options` 538 func (o *Options) DeriveTopicConfig() *AutoCreateTopicConfig { 539 return &AutoCreateTopicConfig{ 540 AutoCreate: o.AutoCreate, 541 PartitionNum: o.PartitionNum, 542 ReplicationFactor: o.ReplicationFactor, 543 } 544 } 545 546 var ( 547 validClientID = regexp.MustCompile(`\A[A-Za-z0-9._-]+\z`) 548 commonInvalidChar = regexp.MustCompile(`[\?:,"]`) 549 ) 550 551 // NewKafkaClientID generates kafka client id 552 func NewKafkaClientID(captureAddr string, 553 changefeedID model.ChangeFeedID, 554 configuredClientID string, 555 ) (clientID string, err error) { 556 if configuredClientID != "" { 557 clientID = configuredClientID 558 } else { 559 clientID = fmt.Sprintf("TiCDC_producer_%s_%s_%s", 560 captureAddr, changefeedID.Namespace, changefeedID.ID) 561 clientID = commonInvalidChar.ReplaceAllString(clientID, "_") 562 } 563 if !validClientID.MatchString(clientID) { 564 return "", cerror.ErrKafkaInvalidClientID.GenWithStackByArgs(clientID) 565 } 566 return 567 } 568 569 // AdjustOptions adjust the `Options` and `sarama.Config` by condition. 570 func AdjustOptions( 571 ctx context.Context, 572 admin ClusterAdminClient, 573 options *Options, 574 topic string, 575 ) error { 576 topics, err := admin.GetTopicsMeta(ctx, []string{topic}, true) 577 if err != nil { 578 return errors.Trace(err) 579 } 580 581 // Only check replicationFactor >= minInsyncReplicas when producer's required acks is -1. 582 // If we don't check it, the producer probably can not send message to the topic. 583 // Because it will wait for the ack from all replicas. But we do not have enough replicas. 584 if options.RequiredAcks == WaitForAll { 585 err = validateMinInsyncReplicas(ctx, admin, topics, topic, int(options.ReplicationFactor)) 586 if err != nil { 587 return errors.Trace(err) 588 } 589 } 590 591 info, exists := topics[topic] 592 // once we have found the topic, no matter `auto-create-topic`, 593 // make sure user input parameters are valid. 594 if exists { 595 // make sure that producer's `MaxMessageBytes` smaller than topic's `max.message.bytes` 596 topicMaxMessageBytesStr, err := getTopicConfig( 597 ctx, admin, info.Name, 598 TopicMaxMessageBytesConfigName, 599 BrokerMessageMaxBytesConfigName, 600 ) 601 if err != nil { 602 return errors.Trace(err) 603 } 604 topicMaxMessageBytes, err := strconv.Atoi(topicMaxMessageBytesStr) 605 if err != nil { 606 return errors.Trace(err) 607 } 608 609 maxMessageBytes := topicMaxMessageBytes - maxMessageBytesOverhead 610 if topicMaxMessageBytes <= options.MaxMessageBytes { 611 log.Warn("topic's `max.message.bytes` less than the `max-message-bytes`,"+ 612 "use topic's `max.message.bytes` to initialize the Kafka producer", 613 zap.Int("max.message.bytes", topicMaxMessageBytes), 614 zap.Int("max-message-bytes", options.MaxMessageBytes), 615 zap.Int("real-max-message-bytes", maxMessageBytes)) 616 options.MaxMessageBytes = maxMessageBytes 617 } else { 618 if maxMessageBytes < options.MaxMessageBytes { 619 options.MaxMessageBytes = maxMessageBytes 620 } 621 } 622 623 // no need to create the topic, 624 // but we would have to log user if they found enter wrong topic name later 625 if options.AutoCreate { 626 log.Warn("topic already exist, TiCDC will not create the topic", 627 zap.String("topic", topic), zap.Any("detail", info)) 628 } 629 630 if err := options.SetPartitionNum(info.NumPartitions); err != nil { 631 return errors.Trace(err) 632 } 633 634 return nil 635 } 636 637 brokerMessageMaxBytesStr, err := admin.GetBrokerConfig( 638 ctx, 639 BrokerMessageMaxBytesConfigName, 640 ) 641 if err != nil { 642 log.Warn("TiCDC cannot find `message.max.bytes` from broker's configuration") 643 return errors.Trace(err) 644 } 645 brokerMessageMaxBytes, err := strconv.Atoi(brokerMessageMaxBytesStr) 646 if err != nil { 647 return errors.Trace(err) 648 } 649 650 // when create the topic, `max.message.bytes` is decided by the broker, 651 // it would use broker's `message.max.bytes` to set topic's `max.message.bytes`. 652 // TiCDC need to make sure that the producer's `MaxMessageBytes` won't larger than 653 // broker's `message.max.bytes`. 654 maxMessageBytes := brokerMessageMaxBytes - maxMessageBytesOverhead 655 if brokerMessageMaxBytes <= options.MaxMessageBytes { 656 log.Warn("broker's `message.max.bytes` less than the `max-message-bytes`,"+ 657 "use broker's `message.max.bytes` to initialize the Kafka producer", 658 zap.Int("message.max.bytes", brokerMessageMaxBytes), 659 zap.Int("max-message-bytes", options.MaxMessageBytes), 660 zap.Int("real-max-message-bytes", maxMessageBytes)) 661 options.MaxMessageBytes = maxMessageBytes 662 } else { 663 if maxMessageBytes < options.MaxMessageBytes { 664 options.MaxMessageBytes = maxMessageBytes 665 } 666 } 667 668 // topic not exists yet, and user does not specify the `partition-num` in the sink uri. 669 if options.PartitionNum == 0 { 670 options.PartitionNum = defaultPartitionNum 671 log.Warn("partition-num is not set, use the default partition count", 672 zap.String("topic", topic), zap.Int32("partitions", options.PartitionNum)) 673 } 674 return nil 675 } 676 677 func validateMinInsyncReplicas( 678 ctx context.Context, 679 admin ClusterAdminClient, 680 topics map[string]TopicDetail, 681 topic string, 682 replicationFactor int, 683 ) error { 684 minInsyncReplicasConfigGetter := func() (string, bool, error) { 685 info, exists := topics[topic] 686 if exists { 687 minInsyncReplicasStr, err := getTopicConfig( 688 ctx, admin, info.Name, 689 MinInsyncReplicasConfigName, 690 MinInsyncReplicasConfigName) 691 if err != nil { 692 return "", true, err 693 } 694 return minInsyncReplicasStr, true, nil 695 } 696 697 minInsyncReplicasStr, err := admin.GetBrokerConfig(ctx, 698 MinInsyncReplicasConfigName) 699 if err != nil { 700 return "", false, err 701 } 702 703 return minInsyncReplicasStr, false, nil 704 } 705 706 minInsyncReplicasStr, exists, err := minInsyncReplicasConfigGetter() 707 if err != nil { 708 // 'min.insync.replica' is invisible to us in Confluent Cloud Kafka. 709 if cerror.ErrKafkaConfigNotFound.Equal(err) { 710 log.Warn("TiCDC cannot find `min.insync.replicas` from broker's configuration, " + 711 "please make sure that the replication factor is greater than or equal " + 712 "to the minimum number of in-sync replicas" + 713 "if you want to use `required-acks` = -1." + 714 "Otherwise, TiCDC will not be able to send messages to the topic.") 715 return nil 716 } 717 return err 718 } 719 minInsyncReplicas, err := strconv.Atoi(minInsyncReplicasStr) 720 if err != nil { 721 return err 722 } 723 724 configFrom := "topic" 725 if !exists { 726 configFrom = "broker" 727 } 728 729 if replicationFactor < minInsyncReplicas { 730 msg := fmt.Sprintf("`replication-factor` cannot be smaller than the `%s` of %s", 731 MinInsyncReplicasConfigName, configFrom) 732 log.Error(msg, zap.Int("replication-factor", replicationFactor), 733 zap.Int("min.insync.replicas", minInsyncReplicas)) 734 return cerror.ErrKafkaInvalidConfig.GenWithStack( 735 "TiCDC Kafka producer's `request.required.acks` defaults to -1, "+ 736 "TiCDC cannot deliver messages when the `replication-factor` %d "+ 737 "is smaller than the `min.insync.replicas` %d of %s", 738 replicationFactor, minInsyncReplicas, configFrom, 739 ) 740 } 741 742 return nil 743 } 744 745 // getTopicConfig gets topic config by name. 746 // If the topic does not have this configuration, 747 // we will try to get it from the broker's configuration. 748 // NOTICE: The configuration names of topic and broker may be different for the same configuration. 749 func getTopicConfig( 750 ctx context.Context, 751 admin ClusterAdminClient, 752 topicName string, 753 topicConfigName string, 754 brokerConfigName string, 755 ) (string, error) { 756 if c, err := admin.GetTopicConfig(ctx, topicName, topicConfigName); err == nil { 757 return c, nil 758 } 759 760 log.Info("TiCDC cannot find the configuration from topic, try to get it from broker", 761 zap.String("topic", topicName), zap.String("config", topicConfigName)) 762 return admin.GetBrokerConfig(ctx, brokerConfigName) 763 }