github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/ddlsink/mq/mq_ddl_sink_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 mq
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"net/url"
    20  	"testing"
    21  
    22  	mm "github.com/pingcap/tidb/pkg/parser/model"
    23  	"github.com/pingcap/tiflow/cdc/model"
    24  	"github.com/pingcap/tiflow/cdc/sink/ddlsink/mq/ddlproducer"
    25  	"github.com/pingcap/tiflow/pkg/config"
    26  	"github.com/pingcap/tiflow/pkg/sink/kafka"
    27  	"github.com/stretchr/testify/require"
    28  )
    29  
    30  func TestNewKafkaDDLSinkFailed(t *testing.T) {
    31  	t.Parallel()
    32  
    33  	changefeedID := model.DefaultChangeFeedID("test")
    34  	ctx, cancel := context.WithCancel(context.Background())
    35  	defer cancel()
    36  
    37  	uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" +
    38  		"&max-message-bytes=1048576&partition-num=1" +
    39  		"&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip&protocol=avro"
    40  	uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName)
    41  
    42  	sinkURI, err := url.Parse(uri)
    43  	require.NoError(t, err)
    44  	replicaConfig := config.GetDefaultReplicaConfig()
    45  	require.NoError(t, replicaConfig.ValidateAndAdjust(sinkURI))
    46  
    47  	ctx = context.WithValue(ctx, "testing.T", t)
    48  	s, err := NewKafkaDDLSink(ctx, changefeedID, sinkURI, replicaConfig,
    49  		kafka.NewMockFactory, ddlproducer.NewMockDDLProducer)
    50  	require.ErrorContains(t, err, "Avro protocol requires parameter \"schema-registry\"",
    51  		"should report error when protocol is avro but schema-registry is not set")
    52  	require.Nil(t, s)
    53  }
    54  
    55  func TestWriteDDLEventToAllPartitions(t *testing.T) {
    56  	t.Parallel()
    57  
    58  	changefeedID := model.DefaultChangeFeedID("test")
    59  	ctx, cancel := context.WithCancel(context.Background())
    60  	defer cancel()
    61  
    62  	// partition-number is 2, so only send DDL events to 2 partitions.
    63  	uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" +
    64  		"&max-message-bytes=1048576&partition-num=2" +
    65  		"&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip&protocol=open-protocol"
    66  	uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName)
    67  
    68  	sinkURI, err := url.Parse(uri)
    69  	require.NoError(t, err)
    70  	replicaConfig := config.GetDefaultReplicaConfig()
    71  	require.NoError(t, replicaConfig.ValidateAndAdjust(sinkURI))
    72  
    73  	ctx = context.WithValue(ctx, "testing.T", t)
    74  	s, err := NewKafkaDDLSink(ctx, changefeedID, sinkURI, replicaConfig,
    75  		kafka.NewMockFactory,
    76  		ddlproducer.NewMockDDLProducer)
    77  	require.NoError(t, err)
    78  	require.NotNil(t, s)
    79  
    80  	ddl := &model.DDLEvent{
    81  		CommitTs: 417318403368288260,
    82  		TableInfo: &model.TableInfo{
    83  			TableName: model.TableName{
    84  				Schema: "cdc", Table: "person",
    85  			},
    86  		},
    87  		Query: "create table person(id int, name varchar(32), primary key(id))",
    88  		Type:  mm.ActionCreateTable,
    89  	}
    90  	err = s.WriteDDLEvent(ctx, ddl)
    91  	require.NoError(t, err)
    92  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetAllEvents(),
    93  		2, "All partitions should be broadcast")
    94  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 0), 1)
    95  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 1), 1)
    96  }
    97  
    98  func TestWriteDDLEventToZeroPartition(t *testing.T) {
    99  	t.Parallel()
   100  
   101  	ctx, cancel := context.WithCancel(context.Background())
   102  	defer cancel()
   103  
   104  	uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" +
   105  		"&max-message-bytes=1048576&partition-num=1" +
   106  		"&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip&protocol=canal-json"
   107  	uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName)
   108  
   109  	sinkURI, err := url.Parse(uri)
   110  	require.NoError(t, err)
   111  	replicaConfig := config.GetDefaultReplicaConfig()
   112  	require.NoError(t, replicaConfig.ValidateAndAdjust(sinkURI))
   113  
   114  	ctx = context.WithValue(ctx, "testing.T", t)
   115  	s, err := NewKafkaDDLSink(ctx, model.DefaultChangeFeedID("test"),
   116  		sinkURI, replicaConfig,
   117  		kafka.NewMockFactory,
   118  		ddlproducer.NewMockDDLProducer)
   119  	require.NoError(t, err)
   120  	require.NotNil(t, s)
   121  
   122  	ddl := &model.DDLEvent{
   123  		CommitTs: 417318403368288260,
   124  		TableInfo: &model.TableInfo{
   125  			TableName: model.TableName{
   126  				Schema: "cdc", Table: "person",
   127  			},
   128  		},
   129  		Query: "create table person(id int, name varchar(32), primary key(id))",
   130  		Type:  mm.ActionCreateTable,
   131  	}
   132  	err = s.WriteDDLEvent(ctx, ddl)
   133  	require.NoError(t, err)
   134  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetAllEvents(),
   135  		1, "Only zero partition")
   136  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 0), 1)
   137  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 1), 0)
   138  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 2), 0)
   139  }
   140  
   141  func TestWriteCheckpointTsToDefaultTopic(t *testing.T) {
   142  	t.Parallel()
   143  
   144  	ctx, cancel := context.WithCancel(context.Background())
   145  	defer cancel()
   146  
   147  	// partition-num is set to 2, so send checkpoint to 2 partitions.
   148  	uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" +
   149  		"&max-message-bytes=1048576&partition-num=2" +
   150  		"&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip" +
   151  		"&protocol=canal-json&enable-tidb-extension=true"
   152  	uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName)
   153  
   154  	sinkURI, err := url.Parse(uri)
   155  	require.Nil(t, err)
   156  	replicaConfig := config.GetDefaultReplicaConfig()
   157  	require.Nil(t, replicaConfig.ValidateAndAdjust(sinkURI))
   158  
   159  	ctx = context.WithValue(ctx, "testing.T", t)
   160  	s, err := NewKafkaDDLSink(ctx, model.DefaultChangeFeedID("test"),
   161  		sinkURI, replicaConfig,
   162  		kafka.NewMockFactory,
   163  		ddlproducer.NewMockDDLProducer)
   164  	require.Nil(t, err)
   165  	require.NotNil(t, s)
   166  
   167  	checkpointTs := uint64(417318403368288260)
   168  	var tables []*model.TableInfo
   169  	err = s.WriteCheckpointTs(ctx, checkpointTs, tables)
   170  	require.Nil(t, err)
   171  
   172  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetAllEvents(),
   173  		2, "All partitions should be broadcast")
   174  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 0), 1)
   175  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 1), 1)
   176  }
   177  
   178  func TestWriteCheckpointTsToTableTopics(t *testing.T) {
   179  	t.Parallel()
   180  
   181  	ctx, cancel := context.WithCancel(context.Background())
   182  	defer cancel()
   183  
   184  	// Notice: auto create topic is true. Auto created topic will have 1 partition.
   185  	uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" +
   186  		"&max-message-bytes=1048576&partition-num=1" +
   187  		"&kafka-client-id=unit-test&auto-create-topic=true&compression=gzip" +
   188  		"&protocol=canal-json&enable-tidb-extension=true"
   189  	uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName)
   190  
   191  	sinkURI, err := url.Parse(uri)
   192  	require.NoError(t, err)
   193  	replicaConfig := config.GetDefaultReplicaConfig()
   194  	require.NoError(t, replicaConfig.ValidateAndAdjust(sinkURI))
   195  	replicaConfig.Sink.DispatchRules = []*config.DispatchRule{
   196  		{
   197  			Matcher:   []string{"*.*"},
   198  			TopicRule: "{schema}_{table}",
   199  		},
   200  	}
   201  
   202  	ctx = context.WithValue(ctx, "testing.T", t)
   203  	s, err := NewKafkaDDLSink(ctx, model.DefaultChangeFeedID("test"),
   204  		sinkURI, replicaConfig,
   205  		kafka.NewMockFactory,
   206  		ddlproducer.NewMockDDLProducer)
   207  	require.NoError(t, err)
   208  	require.NotNil(t, s)
   209  
   210  	checkpointTs := uint64(417318403368288260)
   211  	tables := []*model.TableInfo{
   212  		{
   213  			TableName: model.TableName{
   214  				Schema: "cdc",
   215  				Table:  "person",
   216  			},
   217  		},
   218  		{
   219  			TableName: model.TableName{
   220  				Schema: "cdc",
   221  				Table:  "person1",
   222  			},
   223  		},
   224  		{
   225  			TableName: model.TableName{
   226  				Schema: "cdc",
   227  				Table:  "person2",
   228  			},
   229  		},
   230  	}
   231  
   232  	err = s.WriteCheckpointTs(ctx, checkpointTs, tables)
   233  	require.NoError(t, err)
   234  
   235  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetAllEvents(),
   236  		4, "All topics and partitions should be broadcast")
   237  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("mock_topic", 0), 1)
   238  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("cdc_person", 0), 1)
   239  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("cdc_person1", 0), 1)
   240  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetEvents("cdc_person2", 0), 1)
   241  }
   242  
   243  func TestWriteCheckpointTsWhenCanalJsonTiDBExtensionIsDisable(t *testing.T) {
   244  	t.Parallel()
   245  
   246  	ctx, cancel := context.WithCancel(context.Background())
   247  	defer cancel()
   248  
   249  	// Notice: tidb extension is disabled.
   250  	uriTemplate := "kafka://%s/%s?kafka-version=0.9.0.0&max-batch-size=1" +
   251  		"&max-message-bytes=1048576&partition-num=1" +
   252  		"&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip" +
   253  		"&protocol=canal-json&enable-tidb-extension=false"
   254  	uri := fmt.Sprintf(uriTemplate, "127.0.0.1:9092", kafka.DefaultMockTopicName)
   255  
   256  	sinkURI, err := url.Parse(uri)
   257  	require.NoError(t, err)
   258  	replicaConfig := config.GetDefaultReplicaConfig()
   259  	replicaConfig.Sink.KafkaConfig = &config.KafkaConfig{
   260  		LargeMessageHandle: config.NewDefaultLargeMessageHandleConfig(),
   261  	}
   262  	require.NoError(t, replicaConfig.ValidateAndAdjust(sinkURI))
   263  
   264  	ctx = context.WithValue(ctx, "testing.T", t)
   265  	s, err := NewKafkaDDLSink(ctx, model.DefaultChangeFeedID("test"),
   266  		sinkURI, replicaConfig,
   267  		kafka.NewMockFactory,
   268  		ddlproducer.NewMockDDLProducer)
   269  	require.NoError(t, err)
   270  	require.NotNil(t, s)
   271  
   272  	checkpointTs := uint64(417318403368288260)
   273  	var tables []*model.TableInfo
   274  	err = s.WriteCheckpointTs(ctx, checkpointTs, tables)
   275  	require.NoError(t, err)
   276  
   277  	require.Len(t, s.producer.(*ddlproducer.MockDDLProducer).GetAllEvents(),
   278  		0, "No topic and partition should be broadcast")
   279  }
   280  
   281  func TestGetDLLDispatchRuleByProtocol(t *testing.T) {
   282  	t.Parallel()
   283  
   284  	require.Equal(t, PartitionZero, getDDLDispatchRule(config.ProtocolCanal))
   285  	require.Equal(t, PartitionZero, getDDLDispatchRule(config.ProtocolCanalJSON))
   286  
   287  	require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolOpen))
   288  	require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolDefault))
   289  	require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolAvro))
   290  	require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolMaxwell))
   291  	require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolCraft))
   292  	require.Equal(t, PartitionAll, getDDLDispatchRule(config.ProtocolSimple))
   293  }