github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/mq_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 sink
    15  
    16  import (
    17  	"context"
    18  	"fmt"
    19  	"net/url"
    20  
    21  	"github.com/pingcap/failpoint"
    22  	"github.com/pingcap/ticdc/cdc/sink/codec"
    23  
    24  	"github.com/Shopify/sarama"
    25  	"github.com/pingcap/check"
    26  	"github.com/pingcap/errors"
    27  	"github.com/pingcap/ticdc/cdc/model"
    28  	"github.com/pingcap/ticdc/pkg/config"
    29  	cerror "github.com/pingcap/ticdc/pkg/errors"
    30  	"github.com/pingcap/ticdc/pkg/filter"
    31  	"github.com/pingcap/ticdc/pkg/util/testleak"
    32  )
    33  
    34  type mqSinkSuite struct{}
    35  
    36  var _ = check.Suite(&mqSinkSuite{})
    37  
    38  func (s mqSinkSuite) TestKafkaSink(c *check.C) {
    39  	defer testleak.AfterTest(c)()
    40  	ctx, cancel := context.WithCancel(context.Background())
    41  
    42  	topic := "kafka-test"
    43  	leader := sarama.NewMockBroker(c, 1)
    44  	defer leader.Close()
    45  	metadataResponse := new(sarama.MetadataResponse)
    46  	metadataResponse.AddBroker(leader.Addr(), leader.BrokerID())
    47  	metadataResponse.AddTopicPartition(topic, 0, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError)
    48  	leader.Returns(metadataResponse)
    49  	leader.Returns(metadataResponse)
    50  
    51  	prodSuccess := new(sarama.ProduceResponse)
    52  	prodSuccess.AddTopicPartition(topic, 0, sarama.ErrNoError)
    53  
    54  	uriTemplate := "kafka://%s/kafka-test?kafka-version=0.9.0.0&max-batch-size=1" +
    55  		"&max-message-bytes=4194304&partition-num=1" +
    56  		"&kafka-client-id=unit-test&auto-create-topic=false&compression=gzip"
    57  	uri := fmt.Sprintf(uriTemplate, leader.Addr())
    58  	sinkURI, err := url.Parse(uri)
    59  	c.Assert(err, check.IsNil)
    60  	replicaConfig := config.GetDefaultReplicaConfig()
    61  	fr, err := filter.NewFilter(replicaConfig)
    62  	c.Assert(err, check.IsNil)
    63  	opts := map[string]string{}
    64  	errCh := make(chan error, 1)
    65  	sink, err := newKafkaSaramaSink(ctx, sinkURI, fr, replicaConfig, opts, errCh)
    66  	c.Assert(err, check.IsNil)
    67  
    68  	encoder := sink.newEncoder()
    69  	c.Assert(encoder, check.FitsTypeOf, &codec.JSONEventBatchEncoder{})
    70  	c.Assert(encoder.(*codec.JSONEventBatchEncoder).GetMaxBatchSize(), check.Equals, 1)
    71  	c.Assert(encoder.(*codec.JSONEventBatchEncoder).GetMaxKafkaMessageSize(), check.Equals, 4194304)
    72  
    73  	// mock kafka broker processes 1 row changed event
    74  	leader.Returns(prodSuccess)
    75  	row := &model.RowChangedEvent{
    76  		Table: &model.TableName{
    77  			Schema: "test",
    78  			Table:  "t1",
    79  		},
    80  		StartTs:  100,
    81  		CommitTs: 120,
    82  		Columns:  []*model.Column{{Name: "col1", Type: 1, Value: "aa"}},
    83  	}
    84  	err = sink.EmitRowChangedEvents(ctx, row)
    85  	c.Assert(err, check.IsNil)
    86  	checkpointTs, err := sink.FlushRowChangedEvents(ctx, uint64(120))
    87  	c.Assert(err, check.IsNil)
    88  	c.Assert(checkpointTs, check.Equals, uint64(120))
    89  	// flush older resolved ts
    90  	checkpointTs, err = sink.FlushRowChangedEvents(ctx, uint64(110))
    91  	c.Assert(err, check.IsNil)
    92  	c.Assert(checkpointTs, check.Equals, uint64(120))
    93  
    94  	// mock kafka broker processes 1 checkpoint ts event
    95  	leader.Returns(prodSuccess)
    96  	err = sink.EmitCheckpointTs(ctx, uint64(120))
    97  	c.Assert(err, check.IsNil)
    98  
    99  	// mock kafka broker processes 1 ddl event
   100  	leader.Returns(prodSuccess)
   101  	ddl := &model.DDLEvent{
   102  		StartTs:  130,
   103  		CommitTs: 140,
   104  		TableInfo: &model.SimpleTableInfo{
   105  			Schema: "a", Table: "b",
   106  		},
   107  		Query: "create table a",
   108  		Type:  1,
   109  	}
   110  	err = sink.EmitDDLEvent(ctx, ddl)
   111  	c.Assert(err, check.IsNil)
   112  
   113  	cancel()
   114  	err = sink.EmitRowChangedEvents(ctx, row)
   115  	if err != nil {
   116  		c.Assert(errors.Cause(err), check.Equals, context.Canceled)
   117  	}
   118  	err = sink.EmitDDLEvent(ctx, ddl)
   119  	if err != nil {
   120  		c.Assert(errors.Cause(err), check.Equals, context.Canceled)
   121  	}
   122  	err = sink.EmitCheckpointTs(ctx, uint64(140))
   123  	if err != nil {
   124  		c.Assert(errors.Cause(err), check.Equals, context.Canceled)
   125  	}
   126  
   127  	err = sink.Close(ctx)
   128  	if err != nil {
   129  		c.Assert(errors.Cause(err), check.Equals, context.Canceled)
   130  	}
   131  }
   132  
   133  func (s mqSinkSuite) TestKafkaSinkFilter(c *check.C) {
   134  	defer testleak.AfterTest(c)()
   135  	ctx, cancel := context.WithCancel(context.Background())
   136  	defer cancel()
   137  
   138  	topic := "kafka-test"
   139  	leader := sarama.NewMockBroker(c, 1)
   140  	defer leader.Close()
   141  	metadataResponse := new(sarama.MetadataResponse)
   142  	metadataResponse.AddBroker(leader.Addr(), leader.BrokerID())
   143  	metadataResponse.AddTopicPartition(topic, 0, leader.BrokerID(), nil, nil, nil, sarama.ErrNoError)
   144  	leader.Returns(metadataResponse)
   145  	leader.Returns(metadataResponse)
   146  
   147  	prodSuccess := new(sarama.ProduceResponse)
   148  	prodSuccess.AddTopicPartition(topic, 0, sarama.ErrNoError)
   149  
   150  	uriTemplate := "kafka://%s/kafka-test?kafka-version=0.9.0.0&auto-create-topic=false"
   151  	uri := fmt.Sprintf(uriTemplate, leader.Addr())
   152  	sinkURI, err := url.Parse(uri)
   153  	c.Assert(err, check.IsNil)
   154  	replicaConfig := config.GetDefaultReplicaConfig()
   155  	replicaConfig.Filter = &config.FilterConfig{
   156  		Rules: []string{"test.*"},
   157  	}
   158  	fr, err := filter.NewFilter(replicaConfig)
   159  	c.Assert(err, check.IsNil)
   160  	opts := map[string]string{}
   161  	errCh := make(chan error, 1)
   162  	sink, err := newKafkaSaramaSink(ctx, sinkURI, fr, replicaConfig, opts, errCh)
   163  	c.Assert(err, check.IsNil)
   164  
   165  	row := &model.RowChangedEvent{
   166  		Table: &model.TableName{
   167  			Schema: "order",
   168  			Table:  "t1",
   169  		},
   170  		StartTs:  100,
   171  		CommitTs: 120,
   172  	}
   173  	err = sink.EmitRowChangedEvents(ctx, row)
   174  	c.Assert(err, check.IsNil)
   175  	c.Assert(sink.statistics.TotalRowsCount(), check.Equals, uint64(0))
   176  
   177  	ddl := &model.DDLEvent{
   178  		StartTs:  130,
   179  		CommitTs: 140,
   180  		TableInfo: &model.SimpleTableInfo{
   181  			Schema: "lineitem", Table: "t2",
   182  		},
   183  		Query: "create table lineitem.t2",
   184  		Type:  1,
   185  	}
   186  	err = sink.EmitDDLEvent(ctx, ddl)
   187  	c.Assert(cerror.ErrDDLEventIgnored.Equal(err), check.IsTrue)
   188  
   189  	err = sink.Close(ctx)
   190  	if err != nil {
   191  		c.Assert(errors.Cause(err), check.Equals, context.Canceled)
   192  	}
   193  }
   194  
   195  func (s mqSinkSuite) TestPulsarSinkEncoderConfig(c *check.C) {
   196  	defer testleak.AfterTest(c)()
   197  	ctx, cancel := context.WithCancel(context.Background())
   198  	defer cancel()
   199  
   200  	err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sink/producer/pulsar/MockPulsar", "return(true)")
   201  	c.Assert(err, check.IsNil)
   202  
   203  	uri := "pulsar://127.0.0.1:1234/kafka-test?" +
   204  		"max-message-bytes=4194304&max-batch-size=1"
   205  
   206  	sinkURI, err := url.Parse(uri)
   207  	c.Assert(err, check.IsNil)
   208  	replicaConfig := config.GetDefaultReplicaConfig()
   209  	fr, err := filter.NewFilter(replicaConfig)
   210  	c.Assert(err, check.IsNil)
   211  	opts := map[string]string{}
   212  	errCh := make(chan error, 1)
   213  	sink, err := newPulsarSink(ctx, sinkURI, fr, replicaConfig, opts, errCh)
   214  	c.Assert(err, check.IsNil)
   215  
   216  	encoder := sink.newEncoder()
   217  	c.Assert(encoder, check.FitsTypeOf, &codec.JSONEventBatchEncoder{})
   218  	c.Assert(encoder.(*codec.JSONEventBatchEncoder).GetMaxBatchSize(), check.Equals, 1)
   219  	c.Assert(encoder.(*codec.JSONEventBatchEncoder).GetMaxKafkaMessageSize(), check.Equals, 4194304)
   220  }