github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/codec/open/open_protocol_message_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 open 15 16 import ( 17 "testing" 18 19 "github.com/pingcap/tidb/pkg/parser/mysql" 20 "github.com/pingcap/tidb/pkg/types" 21 "github.com/pingcap/tiflow/cdc/entry" 22 "github.com/pingcap/tiflow/cdc/model" 23 "github.com/pingcap/tiflow/pkg/config" 24 cerror "github.com/pingcap/tiflow/pkg/errors" 25 "github.com/pingcap/tiflow/pkg/sink/codec/common" 26 "github.com/pingcap/tiflow/pkg/sink/codec/internal" 27 "github.com/stretchr/testify/require" 28 ) 29 30 func TestFormatCol(t *testing.T) { 31 t.Parallel() 32 row := &messageRow{Update: map[string]internal.Column{"test": { 33 Type: mysql.TypeString, 34 Value: "测", 35 }}} 36 rowEncode, err := row.encode() 37 require.NoError(t, err) 38 row2 := new(messageRow) 39 err = row2.decode(rowEncode) 40 require.NoError(t, err) 41 require.Equal(t, row, row2) 42 // 43 row = &messageRow{Update: map[string]internal.Column{"test": { 44 Type: mysql.TypeBlob, 45 Value: []byte("测"), 46 }}} 47 rowEncode, err = row.encode() 48 require.NoError(t, err) 49 row2 = new(messageRow) 50 err = row2.decode(rowEncode) 51 require.NoError(t, err) 52 require.Equal(t, row, row2) 53 } 54 55 func TestNonBinaryStringCol(t *testing.T) { 56 t.Parallel() 57 col := &model.Column{ 58 Name: "test", 59 Type: mysql.TypeString, 60 Value: "value", 61 } 62 mqCol := internal.Column{} 63 mqCol.FromRowChangeColumn(col) 64 row := &messageRow{Update: map[string]internal.Column{"test": mqCol}} 65 rowEncode, err := row.encode() 66 require.NoError(t, err) 67 row2 := new(messageRow) 68 err = row2.decode(rowEncode) 69 require.NoError(t, err) 70 require.Equal(t, row, row2) 71 mqCol2 := row2.Update["test"] 72 col2 := mqCol2.ToRowChangeColumn("test") 73 col2.Value = string(col2.Value.([]byte)) 74 require.Equal(t, col, col2) 75 } 76 77 func TestVarBinaryCol(t *testing.T) { 78 t.Parallel() 79 col := &model.Column{ 80 Name: "test", 81 Type: mysql.TypeString, 82 Flag: model.BinaryFlag, 83 Value: []byte{0x89, 0x50, 0x4E, 0x47, 0x0D, 0x0A, 0x1A, 0x0A}, 84 } 85 mqCol := internal.Column{} 86 mqCol.FromRowChangeColumn(col) 87 row := &messageRow{Update: map[string]internal.Column{"test": mqCol}} 88 rowEncode, err := row.encode() 89 require.NoError(t, err) 90 row2 := new(messageRow) 91 err = row2.decode(rowEncode) 92 require.NoError(t, err) 93 require.Equal(t, row, row2) 94 mqCol2 := row2.Update["test"] 95 col2 := mqCol2.ToRowChangeColumn("test") 96 require.Equal(t, col, col2) 97 } 98 99 func TestOnlyOutputUpdatedColumn(t *testing.T) { 100 helper := entry.NewSchemaTestHelper(t) 101 defer helper.Close() 102 103 _ = helper.DDL2Event(`create table test.t (a int primary key, b int, c int)`) 104 event := helper.DML2Event(`insert into test.t values (1, 1, 1)`, "test", "t") 105 event.PreColumns = make([]*model.ColumnData, len(event.Columns)) 106 for idx, col := range event.Columns { 107 event.PreColumns[idx] = &model.ColumnData{ 108 ColumnID: col.ColumnID, 109 Value: col.Value, 110 ApproximateBytes: 0, 111 } 112 } 113 codecConfig := common.NewConfig(config.ProtocolOpen) 114 codecConfig.OnlyOutputUpdatedColumns = true 115 116 // column not updated, so ignore it. 117 _, row, err := rowChangeToMsg(event, codecConfig, false) 118 require.NoError(t, err) 119 require.Len(t, row.PreColumns, 0) 120 121 event.TableInfo.Columns[1].FieldType = *types.NewFieldType(mysql.TypeFloat) 122 event.Columns[1].Value = float32(event.Columns[1].Value.(int64)) 123 event.PreColumns[1].Value = float64(event.PreColumns[1].Value.(int64)) 124 _, row, err = rowChangeToMsg(event, codecConfig, false) 125 require.NoError(t, err) 126 _, ok := row.PreColumns["b"] 127 require.True(t, ok) 128 129 event.TableInfo.Columns[1].FieldType = *types.NewFieldType(mysql.TypeLong) 130 event.Columns[1].Value = int64(event.Columns[1].Value.(float32)) 131 event.PreColumns[1].Value = uint64(event.PreColumns[1].Value.(float64)) 132 _, row, err = rowChangeToMsg(event, codecConfig, false) 133 require.NoError(t, err) 134 _, ok = row.PreColumns["b"] 135 require.True(t, ok) 136 } 137 138 func TestRowChanged2MsgOnlyHandleKeyColumns(t *testing.T) { 139 replicaConfig := config.GetDefaultReplicaConfig() 140 replicaConfig.ForceReplicate = true 141 142 helper := entry.NewSchemaTestHelperWithReplicaConfig(t, replicaConfig) 143 defer helper.Close() 144 145 _ = helper.DDL2Event(`create table test.t(id int primary key, a int)`) 146 insertEvent := helper.DML2Event(`insert into test.t values (1, 1)`, "test", "t") 147 148 config := common.NewConfig(config.ProtocolOpen) 149 config.DeleteOnlyHandleKeyColumns = true 150 151 _, value, err := rowChangeToMsg(insertEvent, config, false) 152 require.NoError(t, err) 153 require.Contains(t, value.Update, "id") 154 require.Contains(t, value.Update, "a") 155 156 config.DeleteOnlyHandleKeyColumns = false 157 key, value, err := rowChangeToMsg(insertEvent, config, true) 158 require.NoError(t, err) 159 require.True(t, key.OnlyHandleKey) 160 require.Contains(t, value.Update, "id") 161 require.NotContains(t, value.Update, "a") 162 163 _ = helper.DDL2Event(`create table test.t1(id varchar(10), a varchar(10))`) 164 insertEventNoHandleKey := helper.DML2Event(`insert into test.t1 values ("1", "1")`, "test", "t1") 165 _, _, err = rowChangeToMsg(insertEventNoHandleKey, config, true) 166 require.Error(t, err, cerror.ErrOpenProtocolCodecInvalidData) 167 168 updateEvent := *insertEvent 169 updateEvent.PreColumns = updateEvent.Columns 170 171 config.DeleteOnlyHandleKeyColumns = true 172 _, value, err = rowChangeToMsg(&updateEvent, config, false) 173 require.NoError(t, err) 174 require.Contains(t, value.PreColumns, "a") 175 176 config.DeleteOnlyHandleKeyColumns = false 177 key, value, err = rowChangeToMsg(&updateEvent, config, true) 178 require.NoError(t, err) 179 require.True(t, key.OnlyHandleKey) 180 require.NotContains(t, value.PreColumns, "a") 181 require.NotContains(t, value.Update, "a") 182 183 updateEventNoHandleKey := *insertEventNoHandleKey 184 updateEventNoHandleKey.PreColumns = updateEventNoHandleKey.Columns 185 _, _, err = rowChangeToMsg(&updateEventNoHandleKey, config, true) 186 require.Error(t, err, cerror.ErrOpenProtocolCodecInvalidData) 187 188 deleteEvent := *insertEvent 189 deleteEvent.PreColumns = deleteEvent.Columns 190 deleteEvent.Columns = nil 191 config.DeleteOnlyHandleKeyColumns = true 192 _, value, err = rowChangeToMsg(&deleteEvent, config, false) 193 require.NoError(t, err) 194 require.Contains(t, value.Delete, "id") 195 require.NotContains(t, value.Delete, "a") 196 197 config.DeleteOnlyHandleKeyColumns = false 198 _, value, err = rowChangeToMsg(&deleteEvent, config, false) 199 require.NoError(t, err) 200 require.Contains(t, value.Delete, "id") 201 require.Contains(t, value.Delete, "a") 202 203 config.DeleteOnlyHandleKeyColumns = false 204 key, value, err = rowChangeToMsg(&deleteEvent, config, true) 205 require.NoError(t, err) 206 require.True(t, key.OnlyHandleKey) 207 require.NotContains(t, value.Delete, "a") 208 209 deleteEventNoHandleKey := *insertEventNoHandleKey 210 deleteEventNoHandleKey.PreColumns = deleteEvent.Columns 211 deleteEventNoHandleKey.Columns = nil 212 _, _, err = rowChangeToMsg(&deleteEventNoHandleKey, config, true) 213 require.Error(t, err, cerror.ErrOpenProtocolCodecInvalidData) 214 }