github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/entry/schema_storage.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 "context" 18 "sort" 19 "sync" 20 "sync/atomic" 21 "time" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/log" 25 tidbkv "github.com/pingcap/tidb/pkg/kv" 26 timodel "github.com/pingcap/tidb/pkg/parser/model" 27 "github.com/pingcap/tiflow/cdc/entry/schema" 28 "github.com/pingcap/tiflow/cdc/kv" 29 "github.com/pingcap/tiflow/cdc/model" 30 cerror "github.com/pingcap/tiflow/pkg/errors" 31 "github.com/pingcap/tiflow/pkg/filter" 32 "github.com/pingcap/tiflow/pkg/retry" 33 "github.com/pingcap/tiflow/pkg/util" 34 "go.uber.org/zap" 35 "go.uber.org/zap/zapcore" 36 ) 37 38 // SchemaStorage stores the schema information with multi-version 39 type SchemaStorage interface { 40 // GetSnapshot returns the nearest snapshot which currentTs is less than or 41 // equal to the ts. 42 // It may block caller when ts is larger than the resolvedTs of SchemaStorage. 43 GetSnapshot(ctx context.Context, ts uint64) (*schema.Snapshot, error) 44 // GetLastSnapshot returns the last snapshot 45 GetLastSnapshot() *schema.Snapshot 46 // HandleDDLJob creates a new snapshot in storage and handles the ddl job 47 HandleDDLJob(job *timodel.Job) error 48 49 // AllPhysicalTables returns the table IDs of all tables and partition tables. 50 AllPhysicalTables(ctx context.Context, ts model.Ts) ([]model.TableID, error) 51 52 // AllTables returns table info of all tables that are being replicated. 53 AllTables(ctx context.Context, ts model.Ts) ([]*model.TableInfo, error) 54 55 // BuildDDLEvents by parsing the DDL job 56 BuildDDLEvents(ctx context.Context, job *timodel.Job) (ddlEvents []*model.DDLEvent, err error) 57 58 // IsIneligibleTable returns whether the table is ineligible. 59 // Ineligible means that the table does not have a primary key or unique key. 60 IsIneligibleTable(ctx context.Context, tableID model.TableID, ts model.Ts) (bool, error) 61 62 // AdvanceResolvedTs advances the resolved ts 63 AdvanceResolvedTs(ts uint64) 64 // ResolvedTs returns the resolved ts of the schema storage 65 ResolvedTs() uint64 66 // DoGC removes snaps that are no longer needed at the specified TS. 67 // It returns the TS from which the oldest maintained snapshot is valid. 68 DoGC(ts uint64) (lastSchemaTs uint64) 69 } 70 71 type schemaStorage struct { 72 snaps []*schema.Snapshot 73 snapsMu sync.RWMutex 74 75 gcTs uint64 76 resolvedTs uint64 77 schemaVersion int64 78 79 filter filter.Filter 80 81 forceReplicate bool 82 83 id model.ChangeFeedID 84 role util.Role 85 } 86 87 // NewSchemaStorage creates a new schema storage 88 func NewSchemaStorage( 89 storage tidbkv.Storage, startTs uint64, 90 forceReplicate bool, id model.ChangeFeedID, 91 role util.Role, filter filter.Filter, 92 ) (SchemaStorage, error) { 93 var ( 94 snap *schema.Snapshot 95 version int64 96 err error 97 ) 98 // storage may be nil in some unit test cases. 99 if storage == nil { 100 snap = schema.NewEmptySnapshot(forceReplicate) 101 } else { 102 meta := kv.GetSnapshotMeta(storage, startTs) 103 snap, err = schema.NewSnapshotFromMeta(id, meta, startTs, forceReplicate, filter) 104 if err != nil { 105 return nil, errors.Trace(err) 106 } 107 version, err = schema.GetSchemaVersion(meta) 108 if err != nil { 109 return nil, errors.Trace(err) 110 } 111 } 112 return &schemaStorage{ 113 snaps: []*schema.Snapshot{snap}, 114 resolvedTs: startTs, 115 forceReplicate: forceReplicate, 116 filter: filter, 117 id: id, 118 schemaVersion: version, 119 role: role, 120 }, nil 121 } 122 123 // getSnapshot returns the snapshot which currentTs is less than(but most close to) 124 // or equal to the ts. 125 func (s *schemaStorage) getSnapshot(ts uint64) (*schema.Snapshot, error) { 126 gcTs := atomic.LoadUint64(&s.gcTs) 127 if ts < gcTs { 128 // Unexpected error, caller should fail immediately. 129 return nil, cerror.ErrSchemaStorageGCed.GenWithStackByArgs(ts, gcTs) 130 } 131 resolvedTs := atomic.LoadUint64(&s.resolvedTs) 132 if ts > resolvedTs { 133 // Caller should retry. 134 return nil, cerror.ErrSchemaStorageUnresolved.GenWithStackByArgs(ts, resolvedTs) 135 } 136 s.snapsMu.RLock() 137 defer s.snapsMu.RUnlock() 138 // Here we search for the first snapshot whose currentTs is larger than ts. 139 // So the result index -1 is the snapshot we want. 140 i := sort.Search(len(s.snaps), func(i int) bool { 141 return s.snaps[i].CurrentTs() > ts 142 }) 143 // i == 0 has two meanings: 144 // 1. The schema storage is empty. 145 // 2. The ts is smaller than the first snapshot. 146 // In both cases, we should return an error. 147 if i == 0 { 148 // Unexpected error, caller should fail immediately. 149 return nil, cerror.ErrSchemaSnapshotNotFound.GenWithStackByArgs(ts) 150 } 151 return s.snaps[i-1], nil 152 } 153 154 // GetSnapshot returns the snapshot which of ts is specified 155 func (s *schemaStorage) GetSnapshot(ctx context.Context, ts uint64) (*schema.Snapshot, error) { 156 var snap *schema.Snapshot 157 158 // The infinite retry here is a temporary solution to the `ErrSchemaStorageUnresolved` caused by 159 // DDL puller lagging too much. 160 startTime := time.Now() 161 logTime := startTime 162 err := retry.Do(ctx, func() error { 163 var err error 164 snap, err = s.getSnapshot(ts) 165 now := time.Now() 166 if now.Sub(logTime) >= 30*time.Second && isRetryable(err) { 167 log.Warn("GetSnapshot is taking too long, DDL puller stuck?", 168 zap.Error(err), 169 zap.Uint64("ts", ts), 170 zap.Duration("duration", now.Sub(startTime)), 171 zap.String("namespace", s.id.Namespace), 172 zap.String("changefeed", s.id.ID), 173 zap.String("role", s.role.String())) 174 logTime = now 175 } 176 return err 177 }, retry.WithBackoffBaseDelay(10), retry.WithIsRetryableErr(isRetryable)) 178 179 return snap, err 180 } 181 182 func isRetryable(err error) bool { 183 return cerror.IsRetryableError(err) && cerror.ErrSchemaStorageUnresolved.Equal(err) 184 } 185 186 // GetLastSnapshot returns the last snapshot 187 func (s *schemaStorage) GetLastSnapshot() *schema.Snapshot { 188 s.snapsMu.RLock() 189 defer s.snapsMu.RUnlock() 190 return s.snaps[len(s.snaps)-1] 191 } 192 193 // HandleDDLJob creates a new snapshot in storage and handles the ddl job 194 func (s *schemaStorage) HandleDDLJob(job *timodel.Job) error { 195 if s.skipJob(job) { 196 s.schemaVersion = job.BinlogInfo.SchemaVersion 197 s.AdvanceResolvedTs(job.BinlogInfo.FinishedTS) 198 return nil 199 } 200 s.snapsMu.Lock() 201 defer s.snapsMu.Unlock() 202 var snap *schema.Snapshot 203 if len(s.snaps) > 0 { 204 lastSnap := s.snaps[len(s.snaps)-1] 205 // We use schemaVersion to check if an already-executed DDL job is processed for a second time. 206 // Unexecuted DDL jobs should have largest schemaVersions. 207 if job.BinlogInfo.FinishedTS <= lastSnap.CurrentTs() || job.BinlogInfo.SchemaVersion <= s.schemaVersion { 208 log.Info("schemaStorage: ignore foregone DDL", 209 zap.String("namespace", s.id.Namespace), 210 zap.String("changefeed", s.id.ID), 211 zap.String("DDL", job.Query), 212 zap.Int64("jobID", job.ID), 213 zap.Uint64("finishTs", job.BinlogInfo.FinishedTS), 214 zap.Int64("schemaVersion", s.schemaVersion), 215 zap.Int64("jobSchemaVersion", job.BinlogInfo.SchemaVersion), 216 zap.String("role", s.role.String())) 217 return nil 218 } 219 snap = lastSnap.Copy() 220 } else { 221 snap = schema.NewEmptySnapshot(s.forceReplicate) 222 } 223 if err := snap.HandleDDL(job); err != nil { 224 log.Error("schemaStorage: update snapshot by the DDL job failed", 225 zap.String("namespace", s.id.Namespace), 226 zap.String("changefeed", s.id.ID), 227 zap.String("schema", job.SchemaName), 228 zap.String("table", job.TableName), 229 zap.String("query", job.Query), 230 zap.Uint64("finishedTs", job.BinlogInfo.FinishedTS), 231 zap.String("role", s.role.String()), 232 zap.Error(err)) 233 return errors.Trace(err) 234 } 235 s.snaps = append(s.snaps, snap) 236 s.schemaVersion = job.BinlogInfo.SchemaVersion 237 s.AdvanceResolvedTs(job.BinlogInfo.FinishedTS) 238 log.Info("schemaStorage: update snapshot by the DDL job", 239 zap.String("namespace", s.id.Namespace), 240 zap.String("changefeed", s.id.ID), 241 zap.String("schema", job.SchemaName), 242 zap.String("table", job.TableName), 243 zap.String("query", job.Query), 244 zap.Uint64("finishedTs", job.BinlogInfo.FinishedTS), 245 zap.Uint64("schemaVersion", uint64(s.schemaVersion)), 246 zap.String("role", s.role.String())) 247 return nil 248 } 249 250 // AllPhysicalTables returns the table IDs of all tables and partition tables. 251 func (s *schemaStorage) AllPhysicalTables(ctx context.Context, ts model.Ts) ([]model.TableID, error) { 252 // NOTE: it's better to pre-allocate the vector. However, in the current implementation 253 // we can't know how many valid tables in the snapshot. 254 res := make([]model.TableID, 0) 255 snap, err := s.GetSnapshot(ctx, ts) 256 if err != nil { 257 return nil, err 258 } 259 260 snap.IterTables(true, func(tblInfo *model.TableInfo) { 261 if s.shouldIgnoreTable(tblInfo) { 262 return 263 } 264 if pi := tblInfo.GetPartitionInfo(); pi != nil { 265 for _, partition := range pi.Definitions { 266 res = append(res, partition.ID) 267 } 268 } else { 269 res = append(res, tblInfo.ID) 270 } 271 }) 272 log.Debug("get new schema snapshot", 273 zap.Uint64("ts", ts), 274 zap.Uint64("snapTs", snap.CurrentTs()), 275 zap.Any("tables", res), 276 zap.String("snapshot", snap.DumpToString())) 277 278 return res, nil 279 } 280 281 // AllTables returns table info of all tables that are being replicated. 282 func (s *schemaStorage) AllTables(ctx context.Context, ts model.Ts) ([]*model.TableInfo, error) { 283 tables := make([]*model.TableInfo, 0) 284 snap, err := s.GetSnapshot(ctx, ts) 285 if err != nil { 286 return nil, err 287 } 288 snap.IterTables(true, func(tblInfo *model.TableInfo) { 289 if !s.shouldIgnoreTable(tblInfo) { 290 tables = append(tables, tblInfo) 291 } 292 }) 293 return tables, nil 294 } 295 296 func (s *schemaStorage) shouldIgnoreTable(t *model.TableInfo) bool { 297 schemaName := t.TableName.Schema 298 tableName := t.TableName.Table 299 if s.filter.ShouldIgnoreTable(schemaName, tableName) { 300 return true 301 } 302 if t.IsEligible(s.forceReplicate) { 303 return false 304 } 305 306 // Sequence is not supported yet, and always ineligible. 307 // Skip Warn to avoid confusion. 308 // See https://github.com/pingcap/tiflow/issues/4559 309 if !t.IsSequence() { 310 log.Warn("skip ineligible table", 311 zap.String("namespace", s.id.Namespace), 312 zap.String("changefeed", s.id.ID), 313 zap.Int64("tableID", t.ID), 314 zap.Stringer("tableName", t.TableName), 315 ) 316 } 317 return true 318 } 319 320 // IsIneligibleTable returns whether the table is ineligible. 321 // It uses the snapshot of the given ts to check the table. 322 // Ineligible means that the table does not have a primary key 323 // or not null unique key. 324 func (s *schemaStorage) IsIneligibleTable( 325 ctx context.Context, tableID model.TableID, ts model.Ts, 326 ) (bool, error) { 327 snap, err := s.GetSnapshot(ctx, ts) 328 if err != nil { 329 return false, err 330 } 331 return snap.IsIneligibleTableID(tableID), nil 332 } 333 334 // AdvanceResolvedTs advances the resolved. Not thread safe. 335 // NOTE: SHOULD NOT call it concurrently 336 func (s *schemaStorage) AdvanceResolvedTs(ts uint64) { 337 if ts > s.ResolvedTs() { 338 atomic.StoreUint64(&s.resolvedTs, ts) 339 } 340 } 341 342 // ResolvedTs returns the resolved ts of the schema storage 343 func (s *schemaStorage) ResolvedTs() uint64 { 344 return atomic.LoadUint64(&s.resolvedTs) 345 } 346 347 // DoGC removes snaps which of ts less than this specified ts 348 func (s *schemaStorage) DoGC(ts uint64) (lastSchemaTs uint64) { 349 s.snapsMu.Lock() 350 defer s.snapsMu.Unlock() 351 var startIdx int 352 for i, snap := range s.snaps { 353 if snap.CurrentTs() > ts { 354 break 355 } 356 startIdx = i 357 } 358 if startIdx == 0 { 359 return s.snaps[0].CurrentTs() 360 } 361 if log.GetLevel() == zapcore.DebugLevel { 362 log.Debug("Do GC in schema storage") 363 for i := 0; i < startIdx; i++ { 364 s.snaps[i].PrintStatus(log.Debug) 365 } 366 } 367 368 // NOTE: Drop must be called to remove stale versions. 369 s.snaps[startIdx-1].Drop() 370 371 // copy the part of the slice that is needed instead of re-slicing it 372 // to maximize efficiency of Go runtime GC. 373 newSnaps := make([]*schema.Snapshot, len(s.snaps)-startIdx) 374 copy(newSnaps, s.snaps[startIdx:]) 375 s.snaps = newSnaps 376 377 lastSchemaTs = s.snaps[0].CurrentTs() 378 atomic.StoreUint64(&s.gcTs, lastSchemaTs) 379 return 380 } 381 382 // SkipJob skip the job should not be executed 383 // TiDB write DDL Binlog for every DDL Job, 384 // we must ignore jobs that are cancelled or rollback 385 // For older version TiDB, it writes DDL Binlog in the txn 386 // that the state of job is changed to *synced* 387 // Now, it writes DDL Binlog in the txn that the state of 388 // job is changed to *done* (before change to *synced*) 389 // At state *done*, it will be always and only changed to *synced*. 390 func (s *schemaStorage) skipJob(job *timodel.Job) bool { 391 log.Debug("handle DDL new commit", 392 zap.String("DDL", job.Query), zap.Stringer("job", job), 393 zap.String("namespace", s.id.Namespace), 394 zap.String("changefeed", s.id.ID), 395 zap.String("role", s.role.String())) 396 return !job.IsDone() 397 } 398 399 // BuildDDLEvents by parsing the DDL job 400 func (s *schemaStorage) BuildDDLEvents( 401 ctx context.Context, job *timodel.Job, 402 ) (ddlEvents []*model.DDLEvent, err error) { 403 switch job.Type { 404 case timodel.ActionRenameTables: 405 // The result contains more than one DDLEvent for a rename tables job. 406 ddlEvents, err = s.buildRenameEvents(ctx, job) 407 if err != nil { 408 return nil, errors.Trace(err) 409 } 410 default: 411 // parse preTableInfo 412 preSnap, err := s.GetSnapshot(ctx, job.BinlogInfo.FinishedTS-1) 413 if err != nil { 414 return nil, errors.Trace(err) 415 } 416 preTableInfo, err := preSnap.PreTableInfo(job) 417 if err != nil { 418 return nil, errors.Trace(err) 419 } 420 421 // parse tableInfo 422 var tableInfo *model.TableInfo 423 err = preSnap.FillSchemaName(job) 424 if err != nil { 425 log.Error("build DDL event fail", zap.Any("job", job), zap.Error(err)) 426 return nil, errors.Trace(err) 427 } 428 // TODO: find a better way to refactor this. For example, drop table job should not 429 // have table info. 430 if job.BinlogInfo != nil && job.BinlogInfo.TableInfo != nil { 431 tableInfo = model.WrapTableInfo(job.SchemaID, job.SchemaName, job.BinlogInfo.FinishedTS, job.BinlogInfo.TableInfo) 432 433 // TODO: remove this after job is fixed by TiDB. 434 // ref: https://github.com/pingcap/tidb/issues/43819 435 if job.Type == timodel.ActionExchangeTablePartition { 436 oldTableInfo, ok := preSnap.PhysicalTableByID(job.BinlogInfo.TableInfo.ID) 437 if !ok { 438 return nil, cerror.ErrSchemaStorageTableMiss.GenWithStackByArgs(job.TableID) 439 } 440 tableInfo.SchemaID = oldTableInfo.SchemaID 441 tableInfo.TableName = oldTableInfo.TableName 442 } 443 } else { 444 // Just retrieve the schema name for a DDL job that does not contain TableInfo. 445 // Currently supported by cdc are: ActionCreateSchema, ActionDropSchema, 446 // and ActionModifySchemaCharsetAndCollate. 447 tableInfo = &model.TableInfo{ 448 TableName: model.TableName{Schema: job.SchemaName}, 449 Version: job.BinlogInfo.FinishedTS, 450 } 451 } 452 event := new(model.DDLEvent) 453 event.FromJob(job, preTableInfo, tableInfo) 454 ddlEvents = append(ddlEvents, event) 455 } 456 return ddlEvents, nil 457 } 458 459 // TODO: find a better way to refactor this function. 460 // buildRenameEvents gets a list of DDLEvent from a rename tables DDL job. 461 func (s *schemaStorage) buildRenameEvents( 462 ctx context.Context, job *timodel.Job, 463 ) ([]*model.DDLEvent, error) { 464 var ( 465 oldSchemaIDs, newSchemaIDs, oldTableIDs []int64 466 newTableNames, oldSchemaNames []*timodel.CIStr 467 ddlEvents []*model.DDLEvent 468 ) 469 err := job.DecodeArgs(&oldSchemaIDs, &newSchemaIDs, 470 &newTableNames, &oldTableIDs, &oldSchemaNames) 471 if err != nil { 472 return nil, errors.Trace(err) 473 } 474 475 multiTableInfos := job.BinlogInfo.MultipleTableInfos 476 if len(multiTableInfos) != len(oldSchemaIDs) || 477 len(multiTableInfos) != len(newSchemaIDs) || 478 len(multiTableInfos) != len(newTableNames) || 479 len(multiTableInfos) != len(oldTableIDs) || 480 len(multiTableInfos) != len(oldSchemaNames) { 481 return nil, cerror.ErrInvalidDDLJob.GenWithStackByArgs(job.ID) 482 } 483 484 preSnap, err := s.GetSnapshot(ctx, job.BinlogInfo.FinishedTS-1) 485 if err != nil { 486 return nil, errors.Trace(err) 487 } 488 489 for i, tableInfo := range multiTableInfos { 490 newSchema, ok := preSnap.SchemaByID(newSchemaIDs[i]) 491 if !ok { 492 return nil, cerror.ErrSnapshotSchemaNotFound.GenWithStackByArgs( 493 newSchemaIDs[i]) 494 } 495 newSchemaName := newSchema.Name.O 496 oldSchemaName := oldSchemaNames[i].O 497 event := new(model.DDLEvent) 498 preTableInfo, ok := preSnap.PhysicalTableByID(tableInfo.ID) 499 if !ok { 500 return nil, cerror.ErrSchemaStorageTableMiss.GenWithStackByArgs( 501 job.TableID) 502 } 503 504 tableInfo := model.WrapTableInfo(newSchemaIDs[i], newSchemaName, 505 job.BinlogInfo.FinishedTS, tableInfo) 506 event.FromJobWithArgs(job, preTableInfo, tableInfo, oldSchemaName, newSchemaName) 507 ddlEvents = append(ddlEvents, event) 508 } 509 return ddlEvents, nil 510 } 511 512 // MockSchemaStorage is for tests. 513 type MockSchemaStorage struct { 514 Resolved uint64 515 } 516 517 // AllPhysicalTables implements SchemaStorage. 518 func (s *MockSchemaStorage) AllPhysicalTables(ctx context.Context, ts model.Ts) ([]model.TableID, error) { 519 return nil, nil 520 } 521 522 // IsIneligibleTable implements SchemaStorage. 523 func (s *MockSchemaStorage) IsIneligibleTable(ctx context.Context, tableID model.TableID, ts model.Ts) (bool, error) { 524 return true, nil 525 } 526 527 // AllTables implements SchemaStorage. 528 func (s *MockSchemaStorage) AllTables(ctx context.Context, ts model.Ts) ([]*model.TableInfo, error) { 529 return nil, nil 530 } 531 532 // GetSnapshot implements SchemaStorage. 533 func (s *MockSchemaStorage) GetSnapshot(ctx context.Context, ts uint64) (*schema.Snapshot, error) { 534 return nil, nil 535 } 536 537 // GetLastSnapshot implements SchemaStorage. 538 func (s *MockSchemaStorage) GetLastSnapshot() *schema.Snapshot { 539 return nil 540 } 541 542 // HandleDDLJob implements SchemaStorage. 543 func (s *MockSchemaStorage) HandleDDLJob(job *timodel.Job) error { 544 return nil 545 } 546 547 // BuildDDLEvents implements SchemaStorage. 548 func (s *MockSchemaStorage) BuildDDLEvents( 549 _ context.Context, _ *timodel.Job, 550 ) (ddlEvents []*model.DDLEvent, err error) { 551 return nil, nil 552 } 553 554 // AdvanceResolvedTs implements SchemaStorage. 555 func (s *MockSchemaStorage) AdvanceResolvedTs(ts uint64) { 556 atomic.StoreUint64(&s.Resolved, ts) 557 } 558 559 // ResolvedTs implements SchemaStorage. 560 func (s *MockSchemaStorage) ResolvedTs() uint64 { 561 return atomic.LoadUint64(&s.Resolved) 562 } 563 564 // DoGC implements SchemaStorage. 565 func (s *MockSchemaStorage) DoGC(ts uint64) uint64 { 566 return atomic.LoadUint64(&s.Resolved) 567 }