github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/entry/mounter.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 entry 15 16 import ( 17 "bytes" 18 "context" 19 "encoding/json" 20 "fmt" 21 "math" 22 "math/rand" 23 "time" 24 25 "github.com/pingcap/errors" 26 "github.com/pingcap/log" 27 timodel "github.com/pingcap/parser/model" 28 "github.com/pingcap/parser/mysql" 29 "github.com/pingcap/ticdc/cdc/model" 30 cerror "github.com/pingcap/ticdc/pkg/errors" 31 "github.com/pingcap/ticdc/pkg/util" 32 "github.com/pingcap/tidb/kv" 33 "github.com/pingcap/tidb/table" 34 "github.com/pingcap/tidb/tablecodec" 35 "github.com/pingcap/tidb/types" 36 "go.uber.org/zap" 37 "golang.org/x/sync/errgroup" 38 ) 39 40 const ( 41 defaultOutputChanSize = 128000 42 ) 43 44 type baseKVEntry struct { 45 StartTs uint64 46 // Commit or resolved TS 47 CRTs uint64 48 49 PhysicalTableID int64 50 RecordID kv.Handle 51 Delete bool 52 } 53 54 type rowKVEntry struct { 55 baseKVEntry 56 Row map[int64]types.Datum 57 PreRow map[int64]types.Datum 58 59 // In some cases, row data may exist but not contain any Datum, 60 // use this RowExist/PreRowExist variable to distinguish between row data that does not exist 61 // or row data that does not contain any Datum. 62 RowExist bool 63 PreRowExist bool 64 } 65 66 // Mounter is used to parse SQL events from KV events 67 type Mounter interface { 68 Run(ctx context.Context) error 69 Input() chan<- *model.PolymorphicEvent 70 } 71 72 type mounterImpl struct { 73 schemaStorage SchemaStorage 74 rawRowChangedChs []chan *model.PolymorphicEvent 75 tz *time.Location 76 workerNum int 77 enableOldValue bool 78 } 79 80 // NewMounter creates a mounter 81 func NewMounter(schemaStorage SchemaStorage, workerNum int, enableOldValue bool) Mounter { 82 if workerNum <= 0 { 83 workerNum = defaultMounterWorkerNum 84 } 85 chs := make([]chan *model.PolymorphicEvent, workerNum) 86 for i := 0; i < workerNum; i++ { 87 chs[i] = make(chan *model.PolymorphicEvent, defaultOutputChanSize) 88 } 89 return &mounterImpl{ 90 schemaStorage: schemaStorage, 91 rawRowChangedChs: chs, 92 workerNum: workerNum, 93 enableOldValue: enableOldValue, 94 } 95 } 96 97 const defaultMounterWorkerNum = 32 98 99 func (m *mounterImpl) Run(ctx context.Context) error { 100 m.tz = util.TimezoneFromCtx(ctx) 101 errg, ctx := errgroup.WithContext(ctx) 102 errg.Go(func() error { 103 m.collectMetrics(ctx) 104 return nil 105 }) 106 for i := 0; i < m.workerNum; i++ { 107 index := i 108 errg.Go(func() error { 109 return m.codecWorker(ctx, index) 110 }) 111 } 112 return errg.Wait() 113 } 114 115 func (m *mounterImpl) codecWorker(ctx context.Context, index int) error { 116 captureAddr := util.CaptureAddrFromCtx(ctx) 117 changefeedID := util.ChangefeedIDFromCtx(ctx) 118 metricMountDuration := mountDuration.WithLabelValues(captureAddr, changefeedID) 119 120 for { 121 var pEvent *model.PolymorphicEvent 122 select { 123 case <-ctx.Done(): 124 return errors.Trace(ctx.Err()) 125 case pEvent = <-m.rawRowChangedChs[index]: 126 } 127 if pEvent.RawKV.OpType == model.OpTypeResolved { 128 pEvent.PrepareFinished() 129 continue 130 } 131 startTime := time.Now() 132 rowEvent, err := m.unmarshalAndMountRowChanged(ctx, pEvent.RawKV) 133 if err != nil { 134 return errors.Trace(err) 135 } 136 pEvent.Row = rowEvent 137 pEvent.RawKV.Value = nil 138 pEvent.RawKV.OldValue = nil 139 pEvent.PrepareFinished() 140 metricMountDuration.Observe(time.Since(startTime).Seconds()) 141 } 142 } 143 144 func (m *mounterImpl) Input() chan<- *model.PolymorphicEvent { 145 return m.rawRowChangedChs[rand.Intn(m.workerNum)] 146 } 147 148 func (m *mounterImpl) collectMetrics(ctx context.Context) { 149 captureAddr := util.CaptureAddrFromCtx(ctx) 150 changefeedID := util.ChangefeedIDFromCtx(ctx) 151 metricMounterInputChanSize := mounterInputChanSizeGauge.WithLabelValues(captureAddr, changefeedID) 152 153 for { 154 select { 155 case <-ctx.Done(): 156 return 157 case <-time.After(time.Second * 15): 158 chSize := 0 159 for _, ch := range m.rawRowChangedChs { 160 chSize += len(ch) 161 } 162 metricMounterInputChanSize.Set(float64(chSize)) 163 } 164 } 165 } 166 167 func (m *mounterImpl) unmarshalAndMountRowChanged(ctx context.Context, raw *model.RawKVEntry) (*model.RowChangedEvent, error) { 168 if !bytes.HasPrefix(raw.Key, tablePrefix) { 169 return nil, nil 170 } 171 key, physicalTableID, err := decodeTableID(raw.Key) 172 if err != nil { 173 return nil, err 174 } 175 if len(raw.OldValue) == 0 && len(raw.Value) == 0 { 176 log.Warn("empty value and old value", zap.Any("row", raw)) 177 } 178 baseInfo := baseKVEntry{ 179 StartTs: raw.StartTs, 180 CRTs: raw.CRTs, 181 PhysicalTableID: physicalTableID, 182 Delete: raw.OpType == model.OpTypeDelete, 183 } 184 // when async commit is enabled, the commitTs of DMLs may be equals with DDL finishedTs 185 // a DML whose commitTs is equal to a DDL finishedTs using the schema info before the DDL 186 snap, err := m.schemaStorage.GetSnapshot(ctx, raw.CRTs-1) 187 if err != nil { 188 return nil, errors.Trace(err) 189 } 190 row, err := func() (*model.RowChangedEvent, error) { 191 if snap.IsIneligibleTableID(physicalTableID) { 192 log.Debug("skip the DML of ineligible table", zap.Uint64("ts", raw.CRTs), zap.Int64("tableID", physicalTableID)) 193 return nil, nil 194 } 195 tableInfo, exist := snap.PhysicalTableByID(physicalTableID) 196 if !exist { 197 if snap.IsTruncateTableID(physicalTableID) { 198 log.Debug("skip the DML of truncated table", zap.Uint64("ts", raw.CRTs), zap.Int64("tableID", physicalTableID)) 199 return nil, nil 200 } 201 return nil, cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(physicalTableID) 202 } 203 if bytes.HasPrefix(key, recordPrefix) { 204 rowKV, err := m.unmarshalRowKVEntry(tableInfo, raw.Key, raw.Value, raw.OldValue, baseInfo) 205 if err != nil { 206 return nil, errors.Trace(err) 207 } 208 if rowKV == nil { 209 return nil, nil 210 } 211 return m.mountRowKVEntry(tableInfo, rowKV, raw.ApproximateSize()) 212 } 213 return nil, nil 214 }() 215 if err != nil { 216 log.Error("failed to mount and unmarshals entry, start to print debug info", zap.Error(err)) 217 snap.PrintStatus(log.Error) 218 } 219 return row, err 220 } 221 222 func (m *mounterImpl) unmarshalRowKVEntry(tableInfo *model.TableInfo, rawKey []byte, rawValue []byte, rawOldValue []byte, base baseKVEntry) (*rowKVEntry, error) { 223 recordID, err := tablecodec.DecodeRowKey(rawKey) 224 if err != nil { 225 return nil, errors.Trace(err) 226 } 227 decodeRow := func(rawColValue []byte) (map[int64]types.Datum, bool, error) { 228 if len(rawColValue) == 0 { 229 return nil, false, nil 230 } 231 row, err := decodeRow(rawColValue, recordID, tableInfo, m.tz) 232 if err != nil { 233 return nil, false, errors.Trace(err) 234 } 235 return row, true, nil 236 } 237 238 row, rowExist, err := decodeRow(rawValue) 239 if err != nil { 240 return nil, errors.Trace(err) 241 } 242 preRow, preRowExist, err := decodeRow(rawOldValue) 243 if err != nil { 244 return nil, errors.Trace(err) 245 } 246 247 if base.Delete && !m.enableOldValue && (tableInfo.PKIsHandle || tableInfo.IsCommonHandle) { 248 handleColIDs, fieldTps, _ := tableInfo.GetRowColInfos() 249 preRow, err = tablecodec.DecodeHandleToDatumMap(recordID, handleColIDs, fieldTps, m.tz, nil) 250 if err != nil { 251 return nil, errors.Trace(err) 252 } 253 preRowExist = true 254 } 255 256 base.RecordID = recordID 257 return &rowKVEntry{ 258 baseKVEntry: base, 259 Row: row, 260 PreRow: preRow, 261 RowExist: rowExist, 262 PreRowExist: preRowExist, 263 }, nil 264 } 265 266 const ( 267 ddlJobListKey = "DDLJobList" 268 ddlAddIndexJobListKey = "DDLJobAddIdxList" 269 ) 270 271 // UnmarshalDDL unmarshals the ddl job from RawKVEntry 272 func UnmarshalDDL(raw *model.RawKVEntry) (*timodel.Job, error) { 273 if raw.OpType != model.OpTypePut || !bytes.HasPrefix(raw.Key, metaPrefix) { 274 return nil, nil 275 } 276 meta, err := decodeMetaKey(raw.Key) 277 if err != nil { 278 return nil, errors.Trace(err) 279 } 280 if meta.getType() != ListData { 281 return nil, nil 282 } 283 k := meta.(metaListData) 284 if k.key != ddlJobListKey && k.key != ddlAddIndexJobListKey { 285 return nil, nil 286 } 287 job := &timodel.Job{} 288 err = json.Unmarshal(raw.Value, job) 289 if err != nil { 290 return nil, errors.Trace(err) 291 } 292 if !job.IsDone() && !job.IsSynced() { 293 return nil, nil 294 } 295 // FinishedTS is only set when the job is synced, 296 // but we can use the entry's ts here 297 job.StartTS = raw.StartTs 298 job.BinlogInfo.FinishedTS = raw.CRTs 299 return job, nil 300 } 301 302 func datum2Column(tableInfo *model.TableInfo, datums map[int64]types.Datum, fillWithDefaultValue bool) ([]*model.Column, error) { 303 cols := make([]*model.Column, len(tableInfo.RowColumnsOffset)) 304 for _, colInfo := range tableInfo.Columns { 305 if !model.IsColCDCVisible(colInfo) { 306 continue 307 } 308 colName := colInfo.Name.O 309 colDatums, exist := datums[colInfo.ID] 310 var colValue interface{} 311 if exist { 312 var err error 313 var warn string 314 colValue, warn, err = formatColVal(colDatums, colInfo.Tp) 315 if err != nil { 316 return nil, errors.Trace(err) 317 } 318 if warn != "" { 319 log.Warn(warn, zap.String("table", tableInfo.TableName.String()), zap.String("column", colInfo.Name.String())) 320 } 321 } else if fillWithDefaultValue { 322 colValue = getDefaultOrZeroValue(colInfo) 323 } else { 324 continue 325 } 326 cols[tableInfo.RowColumnsOffset[colInfo.ID]] = &model.Column{ 327 Name: colName, 328 Type: colInfo.Tp, 329 Value: colValue, 330 Flag: tableInfo.ColumnsFlag[colInfo.ID], 331 } 332 } 333 return cols, nil 334 } 335 336 func (m *mounterImpl) mountRowKVEntry(tableInfo *model.TableInfo, row *rowKVEntry, dataSize int64) (*model.RowChangedEvent, error) { 337 var err error 338 // Decode previous columns. 339 var preCols []*model.Column 340 // Since we now always use old value internally, 341 // we need to control the output(sink will use the PreColumns field to determine whether to output old value). 342 // Normally old value is output when only enableOldValue is on, 343 // but for the Delete event, when the old value feature is off, 344 // the HandleKey column needs to be included as well. So we need to do the following filtering. 345 if row.PreRowExist { 346 // FIXME(leoppro): using pre table info to mounter pre column datum 347 // the pre column and current column in one event may using different table info 348 preCols, err = datum2Column(tableInfo, row.PreRow, m.enableOldValue) 349 if err != nil { 350 return nil, errors.Trace(err) 351 } 352 353 // NOTICE: When the old Value feature is off, 354 // the Delete event only needs to keep the handle key column. 355 if row.Delete && !m.enableOldValue { 356 for i := range preCols { 357 col := preCols[i] 358 if col != nil && !col.Flag.IsHandleKey() { 359 preCols[i] = nil 360 } 361 } 362 } 363 } 364 365 var cols []*model.Column 366 if row.RowExist { 367 cols, err = datum2Column(tableInfo, row.Row, true) 368 if err != nil { 369 return nil, errors.Trace(err) 370 } 371 } 372 373 schemaName := tableInfo.TableName.Schema 374 tableName := tableInfo.TableName.Table 375 var intRowID int64 376 if row.RecordID.IsInt() { 377 intRowID = row.RecordID.IntValue() 378 } 379 380 var tableInfoVersion uint64 381 // Align with the old format if old value disabled. 382 if row.Delete && !m.enableOldValue { 383 tableInfoVersion = 0 384 } else { 385 tableInfoVersion = tableInfo.TableInfoVersion 386 } 387 388 return &model.RowChangedEvent{ 389 StartTs: row.StartTs, 390 CommitTs: row.CRTs, 391 RowID: intRowID, 392 TableInfoVersion: tableInfoVersion, 393 Table: &model.TableName{ 394 Schema: schemaName, 395 Table: tableName, 396 TableID: row.PhysicalTableID, 397 IsPartition: tableInfo.GetPartitionInfo() != nil, 398 }, 399 Columns: cols, 400 PreColumns: preCols, 401 IndexColumns: tableInfo.IndexColumnsOffset, 402 ApproximateSize: dataSize, 403 }, nil 404 } 405 406 var emptyBytes = make([]byte, 0) 407 408 func formatColVal(datum types.Datum, tp byte) (value interface{}, warn string, err error) { 409 if datum.IsNull() { 410 return nil, "", nil 411 } 412 switch tp { 413 case mysql.TypeDate, mysql.TypeDatetime, mysql.TypeNewDate, mysql.TypeTimestamp: 414 return datum.GetMysqlTime().String(), "", nil 415 case mysql.TypeDuration: 416 return datum.GetMysqlDuration().String(), "", nil 417 case mysql.TypeJSON: 418 return datum.GetMysqlJSON().String(), "", nil 419 case mysql.TypeNewDecimal: 420 v := datum.GetMysqlDecimal() 421 if v == nil { 422 return nil, "", nil 423 } 424 return v.String(), "", nil 425 case mysql.TypeEnum: 426 return datum.GetMysqlEnum().Value, "", nil 427 case mysql.TypeSet: 428 return datum.GetMysqlSet().Value, "", nil 429 case mysql.TypeBit: 430 // Encode bits as integers to avoid pingcap/tidb#10988 (which also affects MySQL itself) 431 v, err := datum.GetBinaryLiteral().ToInt(nil) 432 return v, "", err 433 case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar, 434 mysql.TypeTinyBlob, mysql.TypeMediumBlob, mysql.TypeLongBlob, mysql.TypeBlob: 435 b := datum.GetBytes() 436 if b == nil { 437 b = emptyBytes 438 } 439 return b, "", nil 440 case mysql.TypeFloat, mysql.TypeDouble: 441 v := datum.GetFloat64() 442 if math.IsNaN(v) || math.IsInf(v, 1) || math.IsInf(v, -1) { 443 warn = fmt.Sprintf("the value is invalid in column: %f", v) 444 v = 0 445 } 446 return v, warn, nil 447 default: 448 return datum.GetValue(), "", nil 449 } 450 } 451 452 func getDefaultOrZeroValue(col *timodel.ColumnInfo) interface{} { 453 // see https://github.com/pingcap/tidb/issues/9304 454 // must use null if TiDB not write the column value when default value is null 455 // and the value is null 456 if !mysql.HasNotNullFlag(col.Flag) { 457 d := types.NewDatum(nil) 458 return d.GetValue() 459 } 460 461 if col.GetDefaultValue() != nil { 462 d := types.NewDatum(col.GetDefaultValue()) 463 return d.GetValue() 464 } 465 switch col.Tp { 466 case mysql.TypeEnum: 467 // For enum type, if no default value and not null is set, 468 // the default value is the first element of the enum list 469 d := types.NewDatum(col.FieldType.Elems[0]) 470 return d.GetValue() 471 case mysql.TypeString, mysql.TypeVarString, mysql.TypeVarchar: 472 return emptyBytes 473 } 474 475 d := table.GetZeroValue(col) 476 return d.GetValue() 477 } 478 479 // DecodeTableID decodes the raw key to a table ID 480 func DecodeTableID(key []byte) (model.TableID, error) { 481 _, physicalTableID, err := decodeTableID(key) 482 if err != nil { 483 return 0, errors.Trace(err) 484 } 485 return physicalTableID, nil 486 }