github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/kafka/sarama.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  	"crypto/tls"
    19  	"math/rand"
    20  	"strings"
    21  	"time"
    22  
    23  	"github.com/IBM/sarama"
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/log"
    26  	cerror "github.com/pingcap/tiflow/pkg/errors"
    27  	"github.com/pingcap/tiflow/pkg/security"
    28  	"go.uber.org/zap"
    29  )
    30  
    31  var (
    32  	defaultKafkaVersion = sarama.V2_0_0_0
    33  	maxKafkaVersion     = sarama.V2_8_0_0
    34  )
    35  
    36  // NewSaramaConfig return the default config and set the according version and metrics
    37  func NewSaramaConfig(ctx context.Context, o *Options) (*sarama.Config, error) {
    38  	config := sarama.NewConfig()
    39  	config.ClientID = o.ClientID
    40  	var err error
    41  	// Admin client would refresh metadata periodically,
    42  	// if metadata cannot be refreshed easily, this would indicate the network condition between the
    43  	// capture server and kafka broker is not good.
    44  	// Set the timeout to 2 minutes to ensure that the underlying client does not retry for too long.
    45  	// If retrying to obtain the metadata fails, simply return the error and let sinkManager rebuild the sink.
    46  	config.Metadata.Retry.Max = 10
    47  	config.Metadata.Retry.Backoff = 200 * time.Millisecond
    48  	config.Metadata.Timeout = 2 * time.Minute
    49  
    50  	config.Admin.Retry.Max = 10
    51  	config.Admin.Retry.Backoff = 200 * time.Millisecond
    52  	// This timeout control the request timeout for each admin request.
    53  	// set it as the read timeout.
    54  	config.Admin.Timeout = 10 * time.Second
    55  
    56  	// Producer.Retry take effect when the producer try to send message to kafka
    57  	// brokers. If kafka cluster is healthy, just the default value should be enough.
    58  	// For kafka cluster with a bad network condition, producer should not try to
    59  	// waster too much time on sending a message, get response no matter success
    60  	// or fail as soon as possible is preferred.
    61  	config.Producer.Retry.Max = 3
    62  	config.Producer.Retry.Backoff = 100 * time.Millisecond
    63  
    64  	// make sure sarama producer flush messages as soon as possible.
    65  	config.Producer.Flush.Bytes = 0
    66  	config.Producer.Flush.Messages = 0
    67  	config.Producer.Flush.Frequency = time.Duration(0)
    68  	config.Producer.Flush.MaxMessages = o.MaxMessages
    69  
    70  	config.Net.DialTimeout = o.DialTimeout
    71  	config.Net.WriteTimeout = o.WriteTimeout
    72  	config.Net.ReadTimeout = o.ReadTimeout
    73  
    74  	config.Producer.Partitioner = sarama.NewManualPartitioner
    75  	config.Producer.MaxMessageBytes = o.MaxMessageBytes
    76  	config.Producer.Return.Successes = true
    77  	config.Producer.Return.Errors = true
    78  	config.Producer.RequiredAcks = sarama.RequiredAcks(o.RequiredAcks)
    79  	compression := strings.ToLower(strings.TrimSpace(o.Compression))
    80  	switch compression {
    81  	case "none":
    82  		config.Producer.Compression = sarama.CompressionNone
    83  	case "gzip":
    84  		config.Producer.Compression = sarama.CompressionGZIP
    85  	case "snappy":
    86  		config.Producer.Compression = sarama.CompressionSnappy
    87  	case "lz4":
    88  		config.Producer.Compression = sarama.CompressionLZ4
    89  	case "zstd":
    90  		config.Producer.Compression = sarama.CompressionZSTD
    91  	default:
    92  		log.Warn("Unsupported compression algorithm", zap.String("compression", o.Compression))
    93  		config.Producer.Compression = sarama.CompressionNone
    94  	}
    95  	if config.Producer.Compression != sarama.CompressionNone {
    96  		log.Info("Kafka producer uses " + compression + " compression algorithm")
    97  	}
    98  
    99  	if o.EnableTLS {
   100  		// for SSL encryption with a trust CA certificate, we must populate the
   101  		// following two params of config.Net.TLS
   102  		config.Net.TLS.Enable = true
   103  		config.Net.TLS.Config = &tls.Config{
   104  			MinVersion: tls.VersionTLS12,
   105  			NextProtos: []string{"h2", "http/1.1"},
   106  		}
   107  
   108  		// for SSL encryption with self-signed CA certificate, we reassign the
   109  		// config.Net.TLS.Config using the relevant credential files.
   110  		if o.Credential != nil && o.Credential.IsTLSEnabled() {
   111  			config.Net.TLS.Config, err = o.Credential.ToTLSConfig()
   112  			if err != nil {
   113  				return nil, errors.Trace(err)
   114  			}
   115  		}
   116  
   117  		config.Net.TLS.Config.InsecureSkipVerify = o.InsecureSkipVerify
   118  	}
   119  
   120  	err = completeSaramaSASLConfig(ctx, config, o)
   121  	if err != nil {
   122  		return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, err)
   123  	}
   124  
   125  	kafkaVersion, err := getKafkaVersion(config, o)
   126  	if err != nil {
   127  		log.Warn("Can't get Kafka version by broker. ticdc will use default version",
   128  			zap.String("defaultVersion", kafkaVersion.String()))
   129  	}
   130  	config.Version = kafkaVersion
   131  
   132  	if o.IsAssignedVersion {
   133  		version, err := sarama.ParseKafkaVersion(o.Version)
   134  		if err != nil {
   135  			return nil, cerror.WrapError(cerror.ErrKafkaInvalidVersion, err)
   136  		}
   137  		config.Version = version
   138  		if !version.IsAtLeast(maxKafkaVersion) && version.String() != kafkaVersion.String() {
   139  			log.Warn("The Kafka version you assigned may not be correct. "+
   140  				"Please assign a version equal to or less than the specified version",
   141  				zap.String("assignedVersion", version.String()),
   142  				zap.String("desiredVersion", kafkaVersion.String()))
   143  		}
   144  	}
   145  	return config, nil
   146  }
   147  
   148  func completeSaramaSASLConfig(ctx context.Context, config *sarama.Config, o *Options) error {
   149  	if o.SASL != nil && o.SASL.SASLMechanism != "" {
   150  		config.Net.SASL.Enable = true
   151  		config.Net.SASL.Mechanism = sarama.SASLMechanism(o.SASL.SASLMechanism)
   152  		switch o.SASL.SASLMechanism {
   153  		case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512, SASLTypePlaintext:
   154  			config.Net.SASL.User = o.SASL.SASLUser
   155  			config.Net.SASL.Password = o.SASL.SASLPassword
   156  			if strings.EqualFold(string(o.SASL.SASLMechanism), SASLTypeSCRAMSHA256) {
   157  				config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient {
   158  					return &security.XDGSCRAMClient{HashGeneratorFcn: security.SHA256}
   159  				}
   160  			} else if strings.EqualFold(string(o.SASL.SASLMechanism), SASLTypeSCRAMSHA512) {
   161  				config.Net.SASL.SCRAMClientGeneratorFunc = func() sarama.SCRAMClient {
   162  					return &security.XDGSCRAMClient{HashGeneratorFcn: security.SHA512}
   163  				}
   164  			}
   165  		case SASLTypeGSSAPI:
   166  			config.Net.SASL.GSSAPI.AuthType = int(o.SASL.GSSAPI.AuthType)
   167  			config.Net.SASL.GSSAPI.Username = o.SASL.GSSAPI.Username
   168  			config.Net.SASL.GSSAPI.ServiceName = o.SASL.GSSAPI.ServiceName
   169  			config.Net.SASL.GSSAPI.KerberosConfigPath = o.SASL.GSSAPI.KerberosConfigPath
   170  			config.Net.SASL.GSSAPI.Realm = o.SASL.GSSAPI.Realm
   171  			config.Net.SASL.GSSAPI.DisablePAFXFAST = o.SASL.GSSAPI.DisablePAFXFAST
   172  			switch o.SASL.GSSAPI.AuthType {
   173  			case security.UserAuth:
   174  				config.Net.SASL.GSSAPI.Password = o.SASL.GSSAPI.Password
   175  			case security.KeyTabAuth:
   176  				config.Net.SASL.GSSAPI.KeyTabPath = o.SASL.GSSAPI.KeyTabPath
   177  			}
   178  
   179  		case SASLTypeOAuth:
   180  			p, err := newTokenProvider(ctx, o)
   181  			if err != nil {
   182  				return errors.Trace(err)
   183  			}
   184  			config.Net.SASL.TokenProvider = p
   185  		}
   186  	}
   187  
   188  	return nil
   189  }
   190  
   191  func getKafkaVersion(config *sarama.Config, o *Options) (sarama.KafkaVersion, error) {
   192  	var err error
   193  	version := defaultKafkaVersion
   194  	addrs := o.BrokerEndpoints
   195  	if len(addrs) > 1 {
   196  		// Shuffle the list of addresses to randomize the order in which
   197  		// connections are attempted. This prevents routing all connections
   198  		// to the first broker (which will usually succeed).
   199  		rand.Shuffle(len(addrs), func(i, j int) {
   200  			addrs[i], addrs[j] = addrs[j], addrs[i]
   201  		})
   202  	}
   203  	for i := range addrs {
   204  		version, err := getKafkaVersionFromBroker(config, o.RequestVersion, addrs[i])
   205  		if err == nil {
   206  			return version, err
   207  		}
   208  	}
   209  	return version, err
   210  }
   211  
   212  func getKafkaVersionFromBroker(config *sarama.Config, requestVersion int16, addr string) (sarama.KafkaVersion, error) {
   213  	KafkaVersion := defaultKafkaVersion
   214  	broker := sarama.NewBroker(addr)
   215  	err := broker.Open(config)
   216  	defer func() {
   217  		broker.Close()
   218  	}()
   219  	if err != nil {
   220  		log.Warn("Kafka fail to open broker", zap.String("addr", addr), zap.Error(err))
   221  		return KafkaVersion, err
   222  	}
   223  	apiResponse, err := broker.ApiVersions(&sarama.ApiVersionsRequest{Version: requestVersion})
   224  	if err != nil {
   225  		log.Warn("Kafka fail to get ApiVersions", zap.String("addr", addr), zap.Error(err))
   226  		return KafkaVersion, err
   227  	}
   228  	// ApiKey method
   229  	// 0      Produce
   230  	// 3      Metadata (default)
   231  	version := apiResponse.ApiKeys[3].MaxVersion
   232  	if version >= 10 {
   233  		KafkaVersion = sarama.V2_8_0_0
   234  	} else if version >= 9 {
   235  		KafkaVersion = sarama.V2_4_0_0
   236  	} else if version >= 8 {
   237  		KafkaVersion = sarama.V2_3_0_0
   238  	} else if version >= 7 {
   239  		KafkaVersion = sarama.V2_1_0_0
   240  	} else if version >= 6 {
   241  		KafkaVersion = sarama.V2_0_0_0
   242  	} else if version >= 5 {
   243  		KafkaVersion = sarama.V1_0_0_0
   244  	} else if version >= 3 {
   245  		KafkaVersion = sarama.V0_11_0_0
   246  	} else if version >= 2 {
   247  		KafkaVersion = sarama.V0_10_1_0
   248  	} else if version >= 1 {
   249  		KafkaVersion = sarama.V0_10_0_0
   250  	} else if version >= 0 {
   251  		KafkaVersion = sarama.V0_8_2_0
   252  	}
   253  	return KafkaVersion, nil
   254  }