github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/dispatcher/partition/index_value_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 partition
    15  
    16  import (
    17  	"testing"
    18  
    19  	timodel "github.com/pingcap/tidb/pkg/parser/model"
    20  	"github.com/pingcap/tidb/pkg/parser/mysql"
    21  	"github.com/pingcap/tidb/pkg/types"
    22  	"github.com/pingcap/tiflow/cdc/model"
    23  	"github.com/pingcap/tiflow/pkg/errors"
    24  	"github.com/stretchr/testify/require"
    25  )
    26  
    27  func TestIndexValueDispatcher(t *testing.T) {
    28  	t.Parallel()
    29  
    30  	tableInfoWithSinglePK := model.BuildTableInfo("test", "t1", []*model.Column{
    31  		{
    32  			Name: "a",
    33  			Flag: model.HandleKeyFlag | model.PrimaryKeyFlag,
    34  		}, {
    35  			Name: "b",
    36  		},
    37  	}, [][]int{{0}})
    38  
    39  	tableInfoWithCompositePK := model.BuildTableInfo("test", "t2", []*model.Column{
    40  		{
    41  			Name: "a",
    42  			Flag: model.HandleKeyFlag | model.PrimaryKeyFlag,
    43  		}, {
    44  			Name: "b",
    45  			Flag: model.HandleKeyFlag | model.PrimaryKeyFlag,
    46  		},
    47  	}, [][]int{{0, 1}})
    48  	testCases := []struct {
    49  		row             *model.RowChangedEvent
    50  		expectPartition int32
    51  	}{
    52  		{row: &model.RowChangedEvent{
    53  			TableInfo: tableInfoWithSinglePK,
    54  			Columns: model.Columns2ColumnDatas([]*model.Column{
    55  				{
    56  					Name:  "a",
    57  					Value: 11,
    58  				}, {
    59  					Name:  "b",
    60  					Value: 22,
    61  				},
    62  			}, tableInfoWithSinglePK),
    63  		}, expectPartition: 2},
    64  		{row: &model.RowChangedEvent{
    65  			TableInfo: tableInfoWithSinglePK,
    66  			Columns: model.Columns2ColumnDatas([]*model.Column{
    67  				{
    68  					Name:  "a",
    69  					Value: 22,
    70  				}, {
    71  					Name:  "b",
    72  					Value: 22,
    73  				},
    74  			}, tableInfoWithSinglePK),
    75  		}, expectPartition: 11},
    76  		{row: &model.RowChangedEvent{
    77  			TableInfo: tableInfoWithSinglePK,
    78  			Columns: model.Columns2ColumnDatas([]*model.Column{
    79  				{
    80  					Name:  "a",
    81  					Value: 11,
    82  				}, {
    83  					Name:  "b",
    84  					Value: 33,
    85  				},
    86  			}, tableInfoWithSinglePK),
    87  		}, expectPartition: 2},
    88  		{row: &model.RowChangedEvent{
    89  			TableInfo: tableInfoWithCompositePK,
    90  			Columns: model.Columns2ColumnDatas([]*model.Column{
    91  				{
    92  					Name:  "a",
    93  					Value: 11,
    94  				}, {
    95  					Name:  "b",
    96  					Value: 22,
    97  				},
    98  			}, tableInfoWithCompositePK),
    99  		}, expectPartition: 5},
   100  		{row: &model.RowChangedEvent{
   101  			TableInfo: tableInfoWithCompositePK,
   102  			Columns: model.Columns2ColumnDatas([]*model.Column{
   103  				{
   104  					Name:  "b",
   105  					Value: 22,
   106  				}, {
   107  					Name:  "a",
   108  					Value: 11,
   109  				},
   110  			}, tableInfoWithCompositePK),
   111  		}, expectPartition: 5},
   112  		{row: &model.RowChangedEvent{
   113  			TableInfo: tableInfoWithCompositePK,
   114  			Columns: model.Columns2ColumnDatas([]*model.Column{
   115  				{
   116  					Name:  "a",
   117  					Value: 11,
   118  				}, {
   119  					Name:  "b",
   120  					Value: 0,
   121  				},
   122  			}, tableInfoWithCompositePK),
   123  		}, expectPartition: 14},
   124  		{row: &model.RowChangedEvent{
   125  			TableInfo: tableInfoWithCompositePK,
   126  			Columns: model.Columns2ColumnDatas([]*model.Column{
   127  				{
   128  					Name:  "a",
   129  					Value: 11,
   130  				}, {
   131  					Name:  "b",
   132  					Value: 33,
   133  				},
   134  			}, tableInfoWithCompositePK),
   135  		}, expectPartition: 2},
   136  	}
   137  	p := NewIndexValueDispatcher("")
   138  	for _, tc := range testCases {
   139  		index, _, err := p.DispatchRowChangedEvent(tc.row, 16)
   140  		require.Equal(t, tc.expectPartition, index)
   141  		require.NoError(t, err)
   142  	}
   143  }
   144  
   145  func TestIndexValueDispatcherWithIndexName(t *testing.T) {
   146  	t.Parallel()
   147  
   148  	tidbTableInfo := &timodel.TableInfo{
   149  		ID:   100,
   150  		Name: timodel.NewCIStr("t1"),
   151  		Columns: []*timodel.ColumnInfo{
   152  			{ID: 1, Name: timodel.NewCIStr("a"), FieldType: *types.NewFieldType(mysql.TypeLong)},
   153  		},
   154  		Indices: []*timodel.IndexInfo{
   155  			{
   156  				Name: timodel.CIStr{
   157  					O: "index1",
   158  				},
   159  				Columns: []*timodel.IndexColumn{
   160  					{
   161  						Name: timodel.CIStr{
   162  							O: "a",
   163  						},
   164  					},
   165  				},
   166  			},
   167  		},
   168  	}
   169  	tableInfo := model.WrapTableInfo(100, "test", 33, tidbTableInfo)
   170  
   171  	event := &model.RowChangedEvent{
   172  		TableInfo: tableInfo,
   173  		Columns: []*model.ColumnData{
   174  			{ColumnID: 1, Value: 11},
   175  		},
   176  	}
   177  
   178  	p := NewIndexValueDispatcher("index2")
   179  	_, _, err := p.DispatchRowChangedEvent(event, 16)
   180  	require.ErrorIs(t, err, errors.ErrDispatcherFailed)
   181  
   182  	p = NewIndexValueDispatcher("index1")
   183  	index, _, err := p.DispatchRowChangedEvent(event, 16)
   184  	require.NoError(t, err)
   185  	require.Equal(t, int32(2), index)
   186  }