github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/sink/codec/maxwell.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 codec 15 16 import ( 17 "bytes" 18 "encoding/binary" 19 "encoding/json" 20 21 "github.com/pingcap/errors" 22 model2 "github.com/pingcap/parser/model" 23 "github.com/pingcap/parser/mysql" 24 "github.com/pingcap/ticdc/cdc/model" 25 cerror "github.com/pingcap/ticdc/pkg/errors" 26 "github.com/tikv/pd/pkg/tsoutil" 27 ) 28 29 // MaxwellEventBatchEncoder is a maxwell format encoder implementation 30 type MaxwellEventBatchEncoder struct { 31 keyBuf *bytes.Buffer 32 valueBuf *bytes.Buffer 33 batchSize int 34 } 35 36 type maxwellMessage struct { 37 Database string `json:"database"` 38 Table string `json:"table"` 39 Type string `json:"type"` 40 Ts int64 `json:"ts"` 41 Xid int `json:"xid,omitempty"` 42 Xoffset int `json:"xoffset,omitempty"` 43 Position string `json:"position,omitempty"` 44 Gtid string `json:"gtid,omitempty"` 45 Data map[string]interface{} `json:"data,omitempty"` 46 Old map[string]interface{} `json:"old,omitempty"` 47 } 48 49 // Encode encodes the message to bytes 50 func (m *maxwellMessage) Encode() ([]byte, error) { 51 data, err := json.Marshal(m) 52 return data, cerror.WrapError(cerror.ErrMaxwellEncodeFailed, err) 53 } 54 55 // Encode encodes the message to bytes 56 func (m *DdlMaxwellMessage) Encode() ([]byte, error) { 57 data, err := json.Marshal(m) 58 return data, cerror.WrapError(cerror.ErrMaxwellEncodeFailed, err) 59 } 60 61 // EncodeCheckpointEvent implements the EventBatchEncoder interface 62 func (d *MaxwellEventBatchEncoder) EncodeCheckpointEvent(ts uint64) (*MQMessage, error) { 63 // For maxwell now, there is no such a corresponding type to ResolvedEvent so far. 64 // Therefore the event is ignored. 65 return nil, nil 66 } 67 68 // AppendResolvedEvent implements the EventBatchEncoder interface 69 func (d *MaxwellEventBatchEncoder) AppendResolvedEvent(ts uint64) (EncoderResult, error) { 70 return EncoderNoOperation, nil 71 } 72 73 func rowEventToMaxwellMessage(e *model.RowChangedEvent) (*mqMessageKey, *maxwellMessage) { 74 var partition *int64 75 if e.Table.IsPartition { 76 partition = &e.Table.TableID 77 } 78 key := &mqMessageKey{ 79 Ts: e.CommitTs, 80 Schema: e.Table.Schema, 81 Table: e.Table.Table, 82 Partition: partition, 83 Type: model.MqMessageTypeRow, 84 } 85 value := &maxwellMessage{ 86 Ts: 0, 87 Database: e.Table.Schema, 88 Table: e.Table.Table, 89 Data: make(map[string]interface{}), 90 Old: make(map[string]interface{}), 91 } 92 93 physicalTime, _ := tsoutil.ParseTS(e.CommitTs) 94 value.Ts = physicalTime.Unix() 95 if e.IsDelete() { 96 value.Type = "delete" 97 for _, v := range e.PreColumns { 98 switch v.Type { 99 case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: 100 if v.Value == nil { 101 value.Old[v.Name] = nil 102 } else if v.Flag.IsBinary() { 103 value.Old[v.Name] = v.Value 104 } else { 105 value.Old[v.Name] = string(v.Value.([]byte)) 106 } 107 default: 108 value.Old[v.Name] = v.Value 109 } 110 } 111 } else { 112 for _, v := range e.Columns { 113 switch v.Type { 114 case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: 115 if v.Value == nil { 116 value.Data[v.Name] = nil 117 } else if v.Flag.IsBinary() { 118 value.Data[v.Name] = v.Value 119 } else { 120 value.Data[v.Name] = string(v.Value.([]byte)) 121 } 122 default: 123 value.Data[v.Name] = v.Value 124 } 125 } 126 if e.PreColumns == nil { 127 value.Type = "insert" 128 } else { 129 value.Type = "update" 130 for _, v := range e.PreColumns { 131 switch v.Type { 132 case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: 133 if v.Value == nil { 134 if value.Data[v.Name] != nil { 135 value.Old[v.Name] = nil 136 } 137 } else if v.Flag.IsBinary() { 138 if value.Data[v.Name] != v.Value { 139 value.Old[v.Name] = v.Value 140 } 141 } else { 142 if value.Data[v.Name] != string(v.Value.([]byte)) { 143 value.Old[v.Name] = string(v.Value.([]byte)) 144 } 145 } 146 default: 147 if value.Data[v.Name] != v.Value { 148 value.Old[v.Name] = v.Value 149 } 150 } 151 } 152 153 } 154 } 155 return key, value 156 } 157 158 // AppendRowChangedEvent implements the EventBatchEncoder interface 159 func (d *MaxwellEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) (EncoderResult, error) { 160 _, valueMsg := rowEventToMaxwellMessage(e) 161 value, err := valueMsg.Encode() 162 if err != nil { 163 return EncoderNoOperation, errors.Trace(err) 164 } 165 d.valueBuf.Write(value) 166 d.batchSize++ 167 return EncoderNeedAsyncWrite, nil 168 } 169 170 // SetParams is no-op for Maxwell for now 171 func (d *MaxwellEventBatchEncoder) SetParams(params map[string]string) error { 172 return nil 173 } 174 175 // Column represents a column in maxwell 176 type Column struct { 177 Type string `json:"type"` 178 Name string `json:"name"` 179 // Do not mark the unique key temporarily 180 Signed bool `json:"signed,omitempty"` 181 ColumnLength int `json:"column-length,omitempty"` 182 Charset string `json:"charset,omitempty"` 183 } 184 185 // TableStruct represents a table structure includes some table info 186 type TableStruct struct { 187 Database string `json:"database"` 188 Charset string `json:"charset,omitempty"` 189 Table string `json:"table"` 190 Columns []*Column `json:"columns"` 191 // Do not output whether it is a primary key temporarily 192 PrimaryKey []string `json:"primary-key"` 193 } 194 195 // DdlMaxwellMessage represents a DDL maxwell message 196 // Old for table old schema 197 // Def for table after ddl schema 198 type DdlMaxwellMessage struct { 199 Type string `json:"type"` 200 Database string `json:"database"` 201 Table string `json:"table"` 202 Old TableStruct `json:"old,omitempty"` 203 Def TableStruct `json:"def,omitempty"` 204 Ts uint64 `json:"ts"` 205 SQL string `json:"sql"` 206 Position string `json:"position,omitempty"` 207 } 208 209 func ddlEventtoMaxwellMessage(e *model.DDLEvent) (*mqMessageKey, *DdlMaxwellMessage) { 210 key := &mqMessageKey{ 211 Ts: e.CommitTs, 212 Schema: e.TableInfo.Schema, 213 Table: e.TableInfo.Table, 214 Type: model.MqMessageTypeDDL, 215 } 216 value := &DdlMaxwellMessage{ 217 Ts: e.CommitTs, 218 Database: e.TableInfo.Schema, 219 Type: "table-create", 220 Table: e.TableInfo.Table, 221 Old: TableStruct{}, 222 Def: TableStruct{}, 223 SQL: e.Query, 224 } 225 226 value.Type = ddlToMaxwellType(e.Type) 227 228 if e.PreTableInfo != nil { 229 value.Old.Database = e.PreTableInfo.Schema 230 value.Old.Table = e.PreTableInfo.Table 231 for _, v := range e.PreTableInfo.ColumnInfo { 232 maxwellcolumntype, _ := columnToMaxwellType(v.Type) 233 value.Old.Columns = append(value.Old.Columns, &Column{ 234 Name: v.Name, 235 Type: maxwellcolumntype, 236 }) 237 } 238 } 239 240 value.Def.Database = e.TableInfo.Schema 241 value.Def.Table = e.TableInfo.Table 242 for _, v := range e.TableInfo.ColumnInfo { 243 maxwellcolumntype, err := columnToMaxwellType(v.Type) 244 if err != nil { 245 value.Old.Columns = append(value.Old.Columns, &Column{ 246 Name: v.Name, 247 Type: err.Error(), 248 }) 249 } 250 value.Def.Columns = append(value.Def.Columns, &Column{ 251 Name: v.Name, 252 Type: maxwellcolumntype, 253 }) 254 } 255 return key, value 256 } 257 258 // EncodeDDLEvent implements the EventBatchEncoder interface 259 // DDL message unresolved tso 260 func (d *MaxwellEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, error) { 261 keyMsg, valueMsg := ddlEventtoMaxwellMessage(e) 262 key, err := keyMsg.Encode() 263 if err != nil { 264 return nil, errors.Trace(err) 265 } 266 value, err := valueMsg.Encode() 267 if err != nil { 268 return nil, errors.Trace(err) 269 } 270 271 return newDDLMQMessage(ProtocolMaxwell, key, value, e), nil 272 } 273 274 // Build implements the EventBatchEncoder interface 275 func (d *MaxwellEventBatchEncoder) Build() []*MQMessage { 276 if d.batchSize == 0 { 277 return nil 278 } 279 280 ret := NewMQMessage(ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) 281 d.Reset() 282 return []*MQMessage{ret} 283 } 284 285 // MixedBuild implements the EventBatchEncoder interface 286 func (d *MaxwellEventBatchEncoder) MixedBuild(withVersion bool) []byte { 287 return nil 288 } 289 290 // Reset implements the EventBatchEncoder interface 291 func (d *MaxwellEventBatchEncoder) Reset() { 292 d.keyBuf.Reset() 293 d.valueBuf.Reset() 294 d.batchSize = 0 295 var versionByte [8]byte 296 binary.BigEndian.PutUint64(versionByte[:], BatchVersion1) 297 d.keyBuf.Write(versionByte[:]) 298 } 299 300 // Size implements the EventBatchEncoder interface 301 func (d *MaxwellEventBatchEncoder) Size() int { 302 return d.keyBuf.Len() + d.valueBuf.Len() 303 } 304 305 // NewMaxwellEventBatchEncoder creates a new MaxwellEventBatchEncoder. 306 func NewMaxwellEventBatchEncoder() EventBatchEncoder { 307 batch := &MaxwellEventBatchEncoder{ 308 keyBuf: &bytes.Buffer{}, 309 valueBuf: &bytes.Buffer{}, 310 } 311 batch.Reset() 312 return batch 313 } 314 315 // ddl typecode from parser/model/ddl.go 316 func ddlToMaxwellType(ddlType model2.ActionType) string { 317 if ddlType >= model2.ActionAddColumn && ddlType <= model2.ActionDropTablePartition { 318 return "table-alter" 319 } 320 switch ddlType { 321 case model2.ActionCreateTable: 322 return "table-create" 323 case model2.ActionDropTable: 324 return "table-drop" 325 case 22, 23, 27, 28, 29, 33, 37, 38, 41, 42: 326 return "table-alter" 327 case model2.ActionCreateSchema: 328 return "database-create" 329 case model2.ActionDropSchema: 330 return "database-drop" 331 case model2.ActionModifySchemaCharsetAndCollate: 332 return "database-alter" 333 default: 334 return ddlType.String() 335 } 336 } 337 338 // Convert column type code to maxwell column type 339 func columnToMaxwellType(columnType byte) (string, error) { 340 switch columnType { 341 case mysql.TypeTiny, mysql.TypeShort, mysql.TypeLong, mysql.TypeInt24: 342 return "int", nil 343 case mysql.TypeLonglong: 344 return "bigint", nil 345 case mysql.TypeTinyBlob, mysql.TypeBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeString, mysql.TypeVarchar: 346 return "string", nil 347 case mysql.TypeDate: 348 return "date", nil 349 case mysql.TypeTimestamp, mysql.TypeDatetime: 350 return "datetime", nil 351 case mysql.TypeDuration: 352 return "time", nil 353 case mysql.TypeYear: 354 return "year", nil 355 case mysql.TypeEnum: 356 return "enum", nil 357 case mysql.TypeSet: 358 return "set", nil 359 case mysql.TypeBit: 360 return "bit", nil 361 case mysql.TypeJSON: 362 return "json", nil 363 case mysql.TypeFloat, mysql.TypeDouble: 364 return "float", nil 365 case mysql.TypeNewDecimal: 366 return "decimal", nil 367 default: 368 return "", cerror.ErrMaxwellInvalidData.GenWithStack("unsupported column type - %v", columnType) 369 } 370 }