github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/dmlproducer/kafka_dml_producer_test.go (about)

     1  // Copyright 2022 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 dmlproducer
    15  
    16  import (
    17  	"context"
    18  	"errors"
    19  	"sync"
    20  	"testing"
    21  	"time"
    22  
    23  	"github.com/IBM/sarama"
    24  	"github.com/pingcap/tiflow/cdc/model"
    25  	cerror "github.com/pingcap/tiflow/pkg/errors"
    26  	"github.com/pingcap/tiflow/pkg/sink/codec/common"
    27  	"github.com/pingcap/tiflow/pkg/sink/kafka"
    28  	"github.com/pingcap/tiflow/pkg/util"
    29  	"github.com/stretchr/testify/require"
    30  	"go.uber.org/atomic"
    31  )
    32  
    33  func getOptions() *kafka.Options {
    34  	options := kafka.NewOptions()
    35  	options.Version = "0.9.0.0"
    36  	options.ClientID = "test-client"
    37  	options.PartitionNum = int32(2)
    38  	options.AutoCreate = false
    39  	options.BrokerEndpoints = []string{"127.0.0.1:9092"}
    40  
    41  	return options
    42  }
    43  
    44  func TestProducerAck(t *testing.T) {
    45  	options := getOptions()
    46  	options.MaxMessages = 1
    47  
    48  	errCh := make(chan error, 1)
    49  	ctx, cancel := context.WithCancel(context.Background())
    50  	config, err := kafka.NewSaramaConfig(ctx, options)
    51  	require.Nil(t, err)
    52  	require.Equal(t, 1, config.Producer.Flush.MaxMessages)
    53  
    54  	ctx = context.WithValue(ctx, "testing.T", t)
    55  	changefeed := model.DefaultChangeFeedID("changefeed-test")
    56  	factory, err := kafka.NewMockFactory(options, changefeed)
    57  	require.NoError(t, err)
    58  
    59  	adminClient, err := factory.AdminClient(ctx)
    60  	require.NoError(t, err)
    61  	metricsCollector := factory.MetricsCollector(util.RoleTester, adminClient)
    62  
    63  	failpointCh := make(chan error, 1)
    64  	asyncProducer, err := factory.AsyncProducer(ctx, failpointCh)
    65  	require.NoError(t, err)
    66  
    67  	producer := NewKafkaDMLProducer(ctx, changefeed,
    68  		asyncProducer, metricsCollector, errCh, failpointCh)
    69  	require.NotNil(t, producer)
    70  
    71  	messageCount := 20
    72  	for i := 0; i < messageCount; i++ {
    73  		asyncProducer.(*kafka.MockSaramaAsyncProducer).AsyncProducer.ExpectInputAndSucceed()
    74  	}
    75  
    76  	count := atomic.NewInt64(0)
    77  	for i := 0; i < 10; i++ {
    78  		err = producer.AsyncSendMessage(ctx, kafka.DefaultMockTopicName, int32(0), &common.Message{
    79  			Key:   []byte("test-key-1"),
    80  			Value: []byte("test-value"),
    81  			Callback: func() {
    82  				count.Add(1)
    83  			},
    84  		})
    85  		require.NoError(t, err)
    86  		err = producer.AsyncSendMessage(ctx, kafka.DefaultMockTopicName, int32(1), &common.Message{
    87  			Key:   []byte("test-key-1"),
    88  			Value: []byte("test-value"),
    89  			Callback: func() {
    90  				count.Add(1)
    91  			},
    92  		})
    93  		require.NoError(t, err)
    94  	}
    95  	// Test all messages are sent and callback is called.
    96  	require.Eventuallyf(t, func() bool {
    97  		return count.Load() == 20
    98  	}, time.Second*5, time.Millisecond*10, "All msgs should be acked")
    99  
   100  	// No error should be returned.
   101  	select {
   102  	case err := <-errCh:
   103  		t.Fatalf("unexpected err: %s", err)
   104  	default:
   105  	}
   106  
   107  	producer.Close()
   108  	cancel()
   109  	// check send messages when context is producer closed
   110  	err = producer.AsyncSendMessage(ctx, kafka.DefaultMockTopicName, int32(0), &common.Message{
   111  		Key:   []byte("cancel"),
   112  		Value: nil,
   113  	})
   114  	require.ErrorIs(t, err, cerror.ErrKafkaProducerClosed)
   115  }
   116  
   117  func TestProducerSendMsgFailed(t *testing.T) {
   118  	options := getOptions()
   119  	errCh := make(chan error, 1)
   120  	ctx, cancel := context.WithTimeout(context.Background(), time.Second*10)
   121  	defer cancel()
   122  	_, err := kafka.NewSaramaConfig(ctx, options)
   123  	require.Nil(t, err)
   124  	options.MaxMessages = 1
   125  	options.MaxMessageBytes = 1
   126  
   127  	ctx = context.WithValue(ctx, "testing.T", t)
   128  	changefeed := model.DefaultChangeFeedID("changefeed-test")
   129  	factory, err := kafka.NewMockFactory(options, changefeed)
   130  	require.NoError(t, err)
   131  
   132  	adminClient, err := factory.AdminClient(ctx)
   133  	require.NoError(t, err)
   134  	metricsCollector := factory.MetricsCollector(util.RoleTester, adminClient)
   135  
   136  	failpointCh := make(chan error, 1)
   137  	asyncProducer, err := factory.AsyncProducer(ctx, failpointCh)
   138  	require.NoError(t, err)
   139  
   140  	producer := NewKafkaDMLProducer(ctx, changefeed,
   141  		asyncProducer, metricsCollector, errCh, failpointCh)
   142  	require.NoError(t, err)
   143  	require.NotNil(t, producer)
   144  
   145  	defer func() {
   146  		producer.Close()
   147  
   148  		// Close reentry.
   149  		producer.Close()
   150  	}()
   151  
   152  	var wg sync.WaitGroup
   153  
   154  	wg.Add(1)
   155  	go func(t *testing.T) {
   156  		defer wg.Done()
   157  
   158  		asyncProducer.(*kafka.MockSaramaAsyncProducer).AsyncProducer.ExpectInputAndFail(sarama.ErrMessageTooLarge)
   159  		err = producer.AsyncSendMessage(ctx, kafka.DefaultMockTopicName, int32(0), &common.Message{
   160  			Key:   []byte("test-key-1"),
   161  			Value: []byte("test-value"),
   162  		})
   163  
   164  		if err != nil {
   165  			require.Condition(t, func() bool {
   166  				return errors.Is(err, cerror.ErrKafkaProducerClosed) ||
   167  					errors.Is(err, context.DeadlineExceeded)
   168  			}, "should return error")
   169  		}
   170  	}(t)
   171  
   172  	wg.Add(1)
   173  	go func() {
   174  		defer wg.Done()
   175  		select {
   176  		case <-ctx.Done():
   177  			t.Errorf("TestProducerSendMessageFailed timed out")
   178  		case err := <-errCh:
   179  			require.ErrorIs(t, err, sarama.ErrMessageTooLarge)
   180  		}
   181  	}()
   182  
   183  	wg.Wait()
   184  }
   185  
   186  func TestProducerDoubleClose(t *testing.T) {
   187  	options := getOptions()
   188  
   189  	errCh := make(chan error, 1)
   190  	ctx, cancel := context.WithCancel(context.Background())
   191  	defer cancel()
   192  
   193  	ctx = context.WithValue(ctx, "testing.T", t)
   194  	changefeed := model.DefaultChangeFeedID("changefeed-test")
   195  	factory, err := kafka.NewMockFactory(options, changefeed)
   196  	require.NoError(t, err)
   197  
   198  	adminClient, err := factory.AdminClient(ctx)
   199  	require.NoError(t, err)
   200  	metricsCollector := factory.MetricsCollector(util.RoleTester, adminClient)
   201  
   202  	failpointCh := make(chan error, 1)
   203  	asyncProducer, err := factory.AsyncProducer(ctx, failpointCh)
   204  	require.NoError(t, err)
   205  
   206  	producer := NewKafkaDMLProducer(ctx, changefeed,
   207  		asyncProducer, metricsCollector, errCh, failpointCh)
   208  	require.NoError(t, err)
   209  	require.NotNil(t, producer)
   210  
   211  	producer.Close()
   212  	producer.Close()
   213  }