github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/producer/kafka/kafka_test.go (about)

     1  // Copyright 2020 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  	"sync"
    19  	"testing"
    20  	"time"
    21  
    22  	"github.com/Shopify/sarama"
    23  	"github.com/pingcap/check"
    24  	"github.com/pingcap/errors"
    25  	"github.com/pingcap/ticdc/cdc/sink/codec"
    26  	cerror "github.com/pingcap/ticdc/pkg/errors"
    27  	"github.com/pingcap/ticdc/pkg/security"
    28  	"github.com/pingcap/ticdc/pkg/util"
    29  	"github.com/pingcap/ticdc/pkg/util/testleak"
    30  )
    31  
    32  type kafkaSuite struct{}
    33  
    34  var _ = check.Suite(&kafkaSuite{})
    35  
    36  func Test(t *testing.T) { check.TestingT(t) }
    37  
    38  func (s *kafkaSuite) TestClientID(c *check.C) {
    39  	defer testleak.AfterTest(c)()
    40  	testCases := []struct {
    41  		role         string
    42  		addr         string
    43  		changefeedID string
    44  		configuredID string
    45  		hasError     bool
    46  		expected     string
    47  	}{
    48  		{"owner", "domain:1234", "123-121-121-121", "", false, "TiCDC_sarama_producer_owner_domain_1234_123-121-121-121"},
    49  		{"owner", "127.0.0.1:1234", "123-121-121-121", "", false, "TiCDC_sarama_producer_owner_127.0.0.1_1234_123-121-121-121"},
    50  		{"owner", "127.0.0.1:1234?:,\"", "123-121-121-121", "", false, "TiCDC_sarama_producer_owner_127.0.0.1_1234_____123-121-121-121"},
    51  		{"owner", "中文", "123-121-121-121", "", true, ""},
    52  		{"owner", "127.0.0.1:1234", "123-121-121-121", "cdc-changefeed-1", false, "cdc-changefeed-1"},
    53  	}
    54  	for _, tc := range testCases {
    55  		id, err := kafkaClientID(tc.role, tc.addr, tc.changefeedID, tc.configuredID)
    56  		if tc.hasError {
    57  			c.Assert(err, check.NotNil)
    58  		} else {
    59  			c.Assert(err, check.IsNil)
    60  			c.Assert(id, check.Equals, tc.expected)
    61  		}
    62  	}
    63  }
    64  
    65  func (s *kafkaSuite) TestSaramaProducer(c *check.C) {
    66  	defer testleak.AfterTest(c)()
    67  	ctx, cancel := context.WithCancel(context.Background())
    68  
    69  	topic := "unit_test_1"
    70  	leader := sarama.NewMockBroker(c, 2)
    71  	defer leader.Close()
    72  	metadataResponse := new(sarama.MetadataResponse)
    73  	metadataResponse.AddBroker(leader.Addr(), leader.BrokerID())
    74  	metadataResponse.AddTopicPartition(topic, 0, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError)
    75  	metadataResponse.AddTopicPartition(topic, 1, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError)
    76  	leader.Returns(metadataResponse)
    77  	leader.Returns(metadataResponse)
    78  
    79  	prodSuccess := new(sarama.ProduceResponse)
    80  	prodSuccess.AddTopicPartition(topic, 0, sarama.ErrNoError)
    81  	prodSuccess.AddTopicPartition(topic, 1, sarama.ErrNoError)
    82  	// 200 async messages and 2 sync message, Kafka flush could be in batch,
    83  	// we can set flush.maxmessages to 1 to control message count exactly.
    84  	for i := 0; i < 202; i++ {
    85  		leader.Returns(prodSuccess)
    86  	}
    87  
    88  	errCh := make(chan error, 1)
    89  	config := NewKafkaConfig()
    90  	// Because the sarama mock broker is not compatible with version larger than 1.0.0
    91  	// We use a smaller version in the following producer tests.
    92  	// Ref: https://github.com/Shopify/sarama/blob/89707055369768913defac030c15cf08e9e57925/async_producer_test.go#L1445-L1447
    93  	config.Version = "0.9.0.0"
    94  	config.PartitionNum = int32(2)
    95  	config.TopicPreProcess = false
    96  
    97  	newSaramaConfigImplBak := newSaramaConfigImpl
    98  	newSaramaConfigImpl = func(ctx context.Context, config Config) (*sarama.Config, error) {
    99  		cfg, err := newSaramaConfigImplBak(ctx, config)
   100  		c.Assert(err, check.IsNil)
   101  		cfg.Producer.Flush.MaxMessages = 1
   102  		return cfg, err
   103  	}
   104  	defer func() {
   105  		newSaramaConfigImpl = newSaramaConfigImplBak
   106  	}()
   107  
   108  	producer, err := NewKafkaSaramaProducer(ctx, leader.Addr(), topic, config, errCh)
   109  	c.Assert(err, check.IsNil)
   110  	c.Assert(producer.GetPartitionNum(), check.Equals, int32(2))
   111  	for i := 0; i < 100; i++ {
   112  		err = producer.SendMessage(ctx, &codec.MQMessage{
   113  			Key:   []byte("test-key-1"),
   114  			Value: []byte("test-value"),
   115  		}, int32(0))
   116  		c.Assert(err, check.IsNil)
   117  		err = producer.SendMessage(ctx, &codec.MQMessage{
   118  			Key:   []byte("test-key-1"),
   119  			Value: []byte("test-value"),
   120  		}, int32(1))
   121  		c.Assert(err, check.IsNil)
   122  	}
   123  
   124  	// In TiCDC logic, resolved ts event will always notify the flush loop. Here we
   125  	// trigger the flushedNotifier periodically to prevent the flush loop block.
   126  	var wg sync.WaitGroup
   127  	wg.Add(1)
   128  	go func() {
   129  		defer wg.Done()
   130  		for {
   131  			select {
   132  			case <-ctx.Done():
   133  				return
   134  			case <-time.After(time.Millisecond * 100):
   135  				producer.flushedNotifier.Notify()
   136  			}
   137  		}
   138  	}()
   139  
   140  	err = producer.Flush(ctx)
   141  	c.Assert(err, check.IsNil)
   142  	expected := []struct {
   143  		flushed uint64
   144  		sent    uint64
   145  	}{
   146  		{100, 100},
   147  		{100, 100},
   148  	}
   149  	c.Assert(producer.partitionOffset, check.DeepEquals, expected)
   150  	select {
   151  	case err := <-errCh:
   152  		c.Fatalf("unexpected err: %s", err)
   153  	default:
   154  	}
   155  	// check no events to flush
   156  	err = producer.Flush(ctx)
   157  	c.Assert(err, check.IsNil)
   158  
   159  	err = producer.SyncBroadcastMessage(ctx, &codec.MQMessage{
   160  		Key:   []byte("test-broadcast"),
   161  		Value: nil,
   162  	})
   163  	c.Assert(err, check.IsNil)
   164  
   165  	err = producer.Close()
   166  	c.Assert(err, check.IsNil)
   167  	// check reentrant close
   168  	err = producer.Close()
   169  	c.Assert(err, check.IsNil)
   170  	cancel()
   171  	wg.Wait()
   172  
   173  	// check send messages when context is canceled or producer closed
   174  	err = producer.SendMessage(ctx, &codec.MQMessage{
   175  		Key:   []byte("cancel"),
   176  		Value: nil,
   177  	}, int32(0))
   178  	if err != nil {
   179  		c.Assert(err, check.Equals, context.Canceled)
   180  	}
   181  	err = producer.SyncBroadcastMessage(ctx, &codec.MQMessage{
   182  		Key:   []byte("cancel"),
   183  		Value: nil,
   184  	})
   185  	if err != nil {
   186  		c.Assert(err, check.Equals, context.Canceled)
   187  	}
   188  }
   189  
   190  func (s *kafkaSuite) TestTopicPreProcess(c *check.C) {
   191  	defer testleak.AfterTest(c)
   192  	topic := "unit_test_2"
   193  	ctx, cancel := context.WithCancel(context.Background())
   194  	defer cancel()
   195  
   196  	broker := sarama.NewMockBroker(c, 1)
   197  	defer broker.Close()
   198  	metaResponse := sarama.NewMockMetadataResponse(c).
   199  		SetBroker(broker.Addr(), broker.BrokerID()).
   200  		SetLeader(topic, 0, broker.BrokerID()).
   201  		SetLeader(topic, 1, broker.BrokerID()).
   202  		SetController(broker.BrokerID())
   203  	broker.SetHandlerByMap(map[string]sarama.MockResponse{
   204  		"MetadataRequest":        metaResponse,
   205  		"DescribeConfigsRequest": sarama.NewMockDescribeConfigsResponse(c),
   206  	})
   207  
   208  	config := NewKafkaConfig()
   209  	config.PartitionNum = int32(0)
   210  	cfg, err := newSaramaConfigImpl(ctx, config)
   211  	c.Assert(err, check.IsNil)
   212  	num, err := kafkaTopicPreProcess(topic, broker.Addr(), config, cfg)
   213  	c.Assert(err, check.IsNil)
   214  	c.Assert(num, check.Equals, int32(2))
   215  
   216  	cfg.Metadata.Retry.Max = 1
   217  	_, err = kafkaTopicPreProcess(topic, "", config, cfg)
   218  	c.Assert(errors.Cause(err), check.Equals, sarama.ErrOutOfBrokers)
   219  
   220  	config.PartitionNum = int32(4)
   221  	_, err = kafkaTopicPreProcess(topic, broker.Addr(), config, cfg)
   222  	c.Assert(cerror.ErrKafkaInvalidPartitionNum.Equal(err), check.IsTrue)
   223  }
   224  
   225  func (s *kafkaSuite) TestTopicPreProcessCreate(c *check.C) {
   226  	defer testleak.AfterTest(c)()
   227  	topic := "unit_test_3"
   228  	ctx, cancel := context.WithCancel(context.Background())
   229  	defer cancel()
   230  
   231  	broker := sarama.NewMockBroker(c, 1)
   232  	broker.SetHandlerByMap(map[string]sarama.MockResponse{
   233  		"MetadataRequest": sarama.NewMockMetadataResponse(c).
   234  			SetBroker(broker.Addr(), broker.BrokerID()).
   235  			SetController(broker.BrokerID()),
   236  		"DescribeConfigsRequest": sarama.NewMockDescribeConfigsResponse(c),
   237  		"CreateTopicsRequest":    sarama.NewMockCreateTopicsResponse(c),
   238  	})
   239  	defer broker.Close()
   240  
   241  	config := NewKafkaConfig()
   242  	config.PartitionNum = int32(0)
   243  	cfg, err := newSaramaConfigImpl(ctx, config)
   244  	c.Assert(err, check.IsNil)
   245  	num, err := kafkaTopicPreProcess(topic, broker.Addr(), config, cfg)
   246  	c.Assert(err, check.IsNil)
   247  	c.Assert(num, check.Equals, int32(4))
   248  }
   249  
   250  func (s *kafkaSuite) TestNewSaramaConfig(c *check.C) {
   251  	defer testleak.AfterTest(c)()
   252  	ctx := context.Background()
   253  	config := NewKafkaConfig()
   254  	config.Version = "invalid"
   255  	_, err := newSaramaConfigImpl(ctx, config)
   256  	c.Assert(errors.Cause(err), check.ErrorMatches, "invalid version.*")
   257  
   258  	ctx = util.SetOwnerInCtx(ctx)
   259  	config.Version = "2.6.0"
   260  	config.ClientID = "^invalid$"
   261  	_, err = newSaramaConfigImpl(ctx, config)
   262  	c.Assert(cerror.ErrKafkaInvalidClientID.Equal(err), check.IsTrue)
   263  
   264  	config.ClientID = "test-kafka-client"
   265  	compressionCases := []struct {
   266  		algorithm string
   267  		expected  sarama.CompressionCodec
   268  	}{
   269  		{"none", sarama.CompressionNone},
   270  		{"gzip", sarama.CompressionGZIP},
   271  		{"snappy", sarama.CompressionSnappy},
   272  		{"lz4", sarama.CompressionLZ4},
   273  		{"zstd", sarama.CompressionZSTD},
   274  		{"others", sarama.CompressionNone},
   275  	}
   276  	for _, cc := range compressionCases {
   277  		config.Compression = cc.algorithm
   278  		cfg, err := newSaramaConfigImpl(ctx, config)
   279  		c.Assert(err, check.IsNil)
   280  		c.Assert(cfg.Producer.Compression, check.Equals, cc.expected)
   281  	}
   282  
   283  	config.Credential = &security.Credential{
   284  		CAPath: "/invalid/ca/path",
   285  	}
   286  	_, err = newSaramaConfigImpl(ctx, config)
   287  	c.Assert(errors.Cause(err), check.ErrorMatches, ".*no such file or directory")
   288  
   289  	saslConfig := NewKafkaConfig()
   290  	saslConfig.Version = "2.6.0"
   291  	saslConfig.ClientID = "test-sasl-scram"
   292  	saslConfig.SaslScram = &security.SaslScram{
   293  		SaslUser:      "user",
   294  		SaslPassword:  "password",
   295  		SaslMechanism: sarama.SASLTypeSCRAMSHA256,
   296  	}
   297  
   298  	cfg, err := newSaramaConfigImpl(ctx, saslConfig)
   299  	c.Assert(err, check.IsNil)
   300  	c.Assert(cfg, check.NotNil)
   301  	c.Assert(cfg.Net.SASL.User, check.Equals, "user")
   302  	c.Assert(cfg.Net.SASL.Password, check.Equals, "password")
   303  	c.Assert(cfg.Net.SASL.Mechanism, check.Equals, sarama.SASLMechanism("SCRAM-SHA-256"))
   304  }
   305  
   306  func (s *kafkaSuite) TestCreateProducerFailed(c *check.C) {
   307  	defer testleak.AfterTest(c)()
   308  	ctx := context.Background()
   309  	errCh := make(chan error, 1)
   310  	config := NewKafkaConfig()
   311  	config.Version = "invalid"
   312  	_, err := NewKafkaSaramaProducer(ctx, "127.0.0.1:1111", "topic", config, errCh)
   313  	c.Assert(errors.Cause(err), check.ErrorMatches, "invalid version.*")
   314  
   315  	config.Version = "0.8.2.0"
   316  	config.PartitionNum = int32(-1)
   317  	_, err = NewKafkaSaramaProducer(ctx, "127.0.0.1:1111", "topic", config, errCh)
   318  	c.Assert(cerror.ErrKafkaInvalidPartitionNum.Equal(err), check.IsTrue)
   319  }