github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/schema/tracker.go (about) 1 // Copyright 2019 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 schema 15 16 import ( 17 "bytes" 18 "context" 19 "fmt" 20 "strings" 21 "sync" 22 23 "github.com/pingcap/errors" 24 "github.com/pingcap/tidb/pkg/ddl" 25 "github.com/pingcap/tidb/pkg/ddl/schematracker" 26 "github.com/pingcap/tidb/pkg/executor" 27 "github.com/pingcap/tidb/pkg/infoschema" 28 "github.com/pingcap/tidb/pkg/kv" 29 "github.com/pingcap/tidb/pkg/meta/autoid" 30 "github.com/pingcap/tidb/pkg/parser" 31 "github.com/pingcap/tidb/pkg/parser/ast" 32 "github.com/pingcap/tidb/pkg/parser/model" 33 "github.com/pingcap/tidb/pkg/parser/mysql" 34 "github.com/pingcap/tidb/pkg/sessionctx" 35 "github.com/pingcap/tidb/pkg/sessionctx/variable" 36 "github.com/pingcap/tidb/pkg/util/chunk" 37 "github.com/pingcap/tidb/pkg/util/filter" 38 "github.com/pingcap/tidb/pkg/util/mock" 39 "github.com/pingcap/tidb/pkg/util/sqlexec" 40 "github.com/pingcap/tiflow/dm/pkg/conn" 41 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 42 fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" 43 "github.com/pingcap/tiflow/dm/pkg/log" 44 dmterror "github.com/pingcap/tiflow/dm/pkg/terror" 45 "github.com/pingcap/tiflow/dm/pkg/utils" 46 "github.com/pingcap/tiflow/dm/syncer/dbconn" 47 "github.com/pingcap/tiflow/pkg/sqlmodel" 48 "go.uber.org/atomic" 49 "go.uber.org/zap" 50 ) 51 52 // Tracker is used to track schema locally. 53 type Tracker struct { 54 // The Tracker uses tidb DDL library to track table structure changes. 55 // where there was basically no parallel operation at the beginning. 56 // However, since the validator is introduced and heavily dependent on the Tracker, we need to make sure 57 // the synchronization between the reading from the validator and the modification from the syncer (e.g. 58 // when the checkpoint is being rolled back, we have to make sure the validator can still vision the original tables) 59 // From this point, we add an extra layer of the synchronization for the following operations: 60 // 1. GetTableInfo: the validator reads table infos. 61 // 2. Init: when the syncer restarts, it may re-initialize the Tracker while the validator may read the Tracker at the same time. 62 // 3. Close: Being similar as above, the validator can read the Tracker while the syncer is closing the Tracker. 63 sync.RWMutex 64 lowerCaseTableNames int 65 se sessionctx.Context 66 upstreamTracker schematracker.SchemaTracker 67 downstreamTracker *downstreamTracker 68 logger log.Logger 69 closed atomic.Bool 70 } 71 72 // downstreamTracker tracks downstream schema. 73 type downstreamTracker struct { 74 sync.RWMutex 75 se sessionctx.Context 76 downstreamConn *dbconn.DBConn // downstream connection 77 stmtParser *parser.Parser // statement parser 78 tableInfos map[string]*DownstreamTableInfo // downstream table infos 79 } 80 81 // DownstreamTableInfo contains tableinfo and index cache. 82 type DownstreamTableInfo struct { 83 TableInfo *model.TableInfo // tableInfo which comes from parse create statement syntaxtree 84 WhereHandle *sqlmodel.WhereHandle 85 } 86 87 type executorContext struct { 88 sessionctx.Context 89 } 90 91 var _ sqlexec.RestrictedSQLExecutor = executorContext{} 92 93 func (se executorContext) GetRestrictedSQLExecutor() sqlexec.RestrictedSQLExecutor { 94 return se 95 } 96 97 func (se executorContext) ParseWithParams(context.Context, string, ...interface{}) (ast.StmtNode, error) { 98 return nil, nil 99 } 100 101 func (se executorContext) ExecRestrictedStmt(context.Context, ast.StmtNode, ...sqlexec.OptionFuncAlias) ([]chunk.Row, []*ast.ResultField, error) { 102 return nil, nil, nil 103 } 104 105 func (se executorContext) ExecRestrictedSQL(context.Context, []sqlexec.OptionFuncAlias, string, ...interface{}) ([]chunk.Row, []*ast.ResultField, error) { 106 return nil, nil, nil 107 } 108 109 // NewTracker simply returns an empty Tracker, 110 // which should be followed by an initialization before used. 111 func NewTracker() *Tracker { 112 return &Tracker{} 113 } 114 115 // Init initializes the Tracker. `sessionCfg` will be set as tracker's session variables if specified, or retrieve 116 // some variable from downstream using `downstreamConn`. 117 // NOTE **sessionCfg is a reference to caller**. 118 func (tr *Tracker) Init( 119 ctx context.Context, 120 task string, 121 lowerCaseTableNames int, 122 downstreamConn *dbconn.DBConn, 123 logger log.Logger, 124 ) error { 125 if tr == nil { 126 return nil 127 } 128 var err error 129 130 rollbackHolder := fr.NewRollbackHolder("schema-tracker") 131 defer func() { 132 if err != nil { 133 rollbackHolder.RollbackReverseOrder() 134 } 135 }() 136 137 logger = logger.WithFields(zap.String("component", "schema-tracker"), zap.String("task", task)) 138 139 upTracker := schematracker.NewSchemaTracker(lowerCaseTableNames) 140 dsSession := mock.NewContext() 141 dsSession.SetValue(ddl.SuppressErrorTooLongKeyKey, true) 142 downTracker := &downstreamTracker{ 143 downstreamConn: downstreamConn, 144 se: dsSession, 145 tableInfos: make(map[string]*DownstreamTableInfo), 146 } 147 // TODO: need to use upstream timezone to correctly check literal is in [1970, 2038] 148 sctx := mock.NewContext() 149 se := executorContext{Context: sctx} 150 tr.Lock() 151 defer tr.Unlock() 152 tr.lowerCaseTableNames = lowerCaseTableNames 153 tr.se = se 154 tr.upstreamTracker = upTracker 155 tr.downstreamTracker = downTracker 156 tr.logger = logger 157 tr.closed.Store(false) 158 return nil 159 } 160 161 // NewTestTracker creates an empty Tracker and initializes it subsequently. 162 // It is useful for test. 163 func NewTestTracker( 164 ctx context.Context, 165 task string, 166 downstreamConn *dbconn.DBConn, 167 logger log.Logger, 168 ) (*Tracker, error) { 169 tr := NewTracker() 170 err := tr.Init(ctx, task, int(conn.LCTableNamesSensitive), downstreamConn, logger) 171 if err != nil { 172 return nil, err 173 } 174 return tr, nil 175 } 176 177 // Exec runs an SQL (DDL) statement. 178 func (tr *Tracker) Exec(ctx context.Context, db string, stmt ast.StmtNode) (errRet error) { 179 defer func() { 180 if r := recover(); r != nil { 181 errRet = fmt.Errorf("tracker panicked: %v", r) 182 } 183 }() 184 visitor := currentDBSetter{ 185 currentDB: db, 186 } 187 stmt.Accept(&visitor) 188 189 switch v := stmt.(type) { 190 case *ast.CreateDatabaseStmt: 191 return tr.upstreamTracker.CreateSchema(tr.se, v) 192 case *ast.AlterDatabaseStmt: 193 return tr.upstreamTracker.AlterSchema(tr.se, v) 194 case *ast.DropDatabaseStmt: 195 return tr.upstreamTracker.DropSchema(tr.se, v) 196 case *ast.CreateTableStmt: 197 return tr.upstreamTracker.CreateTable(tr.se, v) 198 case *ast.AlterTableStmt: 199 return tr.upstreamTracker.AlterTable(ctx, tr.se, v) 200 case *ast.RenameTableStmt: 201 return tr.upstreamTracker.RenameTable(tr.se, v) 202 case *ast.DropTableStmt: 203 return tr.upstreamTracker.DropTable(tr.se, v) 204 case *ast.CreateIndexStmt: 205 return tr.upstreamTracker.CreateIndex(tr.se, v) 206 case *ast.DropIndexStmt: 207 return tr.upstreamTracker.DropIndex(tr.se, v) 208 case *ast.TruncateTableStmt: 209 ident := ast.Ident{Schema: v.Table.Schema, Name: v.Table.Name} 210 return tr.upstreamTracker.TruncateTable(tr.se, ident) 211 default: 212 tr.logger.DPanic("unexpected statement type", zap.String("type", fmt.Sprintf("%T", v))) 213 } 214 return nil 215 } 216 217 // GetTableInfo returns the schema associated with the table. 218 func (tr *Tracker) GetTableInfo(table *filter.Table) (*model.TableInfo, error) { 219 tr.RLock() 220 defer tr.RUnlock() 221 if tr.closed.Load() { 222 return nil, dmterror.ErrSchemaTrackerIsClosed.New("fail to get table info") 223 } 224 return tr.upstreamTracker.TableByName(model.NewCIStr(table.Schema), model.NewCIStr(table.Name)) 225 } 226 227 // GetCreateTable returns the `CREATE TABLE` statement of the table. 228 func (tr *Tracker) GetCreateTable(ctx context.Context, table *filter.Table) (string, error) { 229 tableInfo, err := tr.upstreamTracker.TableByName(model.NewCIStr(table.Schema), model.NewCIStr(table.Name)) 230 if err != nil { 231 return "", err 232 } 233 result := bytes.NewBuffer(make([]byte, 0, 512)) 234 err = executor.ConstructResultOfShowCreateTable(tr.se, tableInfo, autoid.Allocators{}, result) 235 if err != nil { 236 return "", err 237 } 238 return conn.CreateTableSQLToOneRow(result.String()), nil 239 } 240 241 // AllSchemas returns all schemas visible to the tracker (excluding system tables). 242 func (tr *Tracker) AllSchemas() []string { 243 return tr.upstreamTracker.AllSchemaNames() 244 } 245 246 // ListSchemaTables lists all tables in the schema. 247 func (tr *Tracker) ListSchemaTables(schema string) ([]string, error) { 248 ret, err := tr.upstreamTracker.AllTableNamesOfSchema(model.NewCIStr(schema)) 249 if err != nil { 250 return nil, dmterror.ErrSchemaTrackerUnSchemaNotExist.Generate(schema) 251 } 252 return ret, nil 253 } 254 255 // GetSingleColumnIndices returns indices of input column if input column only has single-column indices 256 // returns nil if input column has no indices, or has multi-column indices. 257 // TODO: move out of this package! 258 func (tr *Tracker) GetSingleColumnIndices(db, tbl, col string) ([]*model.IndexInfo, error) { 259 col = strings.ToLower(col) 260 t, err := tr.upstreamTracker.TableByName(model.NewCIStr(db), model.NewCIStr(tbl)) 261 if err != nil { 262 return nil, err 263 } 264 265 var idxInfos []*model.IndexInfo 266 for _, idx := range t.Indices { 267 for _, col2 := range idx.Columns { 268 // found an index covers input column 269 if col2.Name.L == col { 270 if len(idx.Columns) == 1 { 271 idxInfos = append(idxInfos, idx) 272 } else { 273 // temporary use errors.New, won't propagate further 274 return nil, errors.New("found multi-column index") 275 } 276 } 277 } 278 } 279 return idxInfos, nil 280 } 281 282 // IsTableNotExists checks if err means the database or table does not exist. 283 func IsTableNotExists(err error) bool { 284 return infoschema.ErrTableNotExists.Equal(err) || infoschema.ErrDatabaseNotExists.Equal(err) 285 } 286 287 // Reset drops all tables inserted into this tracker. 288 func (tr *Tracker) Reset() { 289 // TODO: lock? 290 tr.upstreamTracker = schematracker.NewSchemaTracker(tr.lowerCaseTableNames) 291 } 292 293 // Close closes a tracker. 294 func (tr *Tracker) Close() { 295 if tr == nil { 296 return 297 } 298 // prevent SchemaTracker being closed when 299 // other components are getting/setting table info 300 tr.Lock() 301 defer tr.Unlock() 302 tr.closed.Store(true) 303 } 304 305 // DropTable drops a table from this tracker. 306 func (tr *Tracker) DropTable(table *filter.Table) error { 307 return tr.upstreamTracker.DeleteTable(model.NewCIStr(table.Schema), model.NewCIStr(table.Name)) 308 } 309 310 // CreateSchemaIfNotExists creates a SCHEMA of the given name if it did not exist. 311 func (tr *Tracker) CreateSchemaIfNotExists(db string) error { 312 dbName := model.NewCIStr(db) 313 if tr.upstreamTracker.SchemaByName(dbName) != nil { 314 return nil 315 } 316 stmt := &ast.CreateDatabaseStmt{ 317 Name: dbName, 318 IfNotExists: true, 319 } 320 return tr.upstreamTracker.CreateSchema(tr.se, stmt) 321 } 322 323 // cloneTableInfo creates a clone of the TableInfo. 324 func cloneTableInfo(ti *model.TableInfo) *model.TableInfo { 325 ret := ti.Clone() 326 ret.Lock = nil 327 // FIXME pingcap/parser's Clone() doesn't clone Partition yet 328 if ret.Partition != nil { 329 pi := *ret.Partition 330 pi.Definitions = append([]model.PartitionDefinition(nil), ret.Partition.Definitions...) 331 ret.Partition = &pi 332 } 333 return ret 334 } 335 336 // CreateTableIfNotExists creates a TABLE of the given name if it did not exist. 337 func (tr *Tracker) CreateTableIfNotExists(table *filter.Table, ti *model.TableInfo) error { 338 schemaName := model.NewCIStr(table.Schema) 339 tableName := model.NewCIStr(table.Name) 340 ti = cloneTableInfo(ti) 341 ti.Name = tableName 342 return tr.upstreamTracker.CreateTableWithInfo(tr.se, schemaName, ti, ddl.OnExistIgnore) 343 } 344 345 // SplitBatchCreateTableAndHandle will split the batch if it exceeds the kv entry size limit. 346 func (tr *Tracker) SplitBatchCreateTableAndHandle(schema model.CIStr, info []*model.TableInfo, l int, r int) error { 347 var err error 348 if err = tr.upstreamTracker.BatchCreateTableWithInfo(tr.se, schema, info[l:r], ddl.OnExistIgnore); kv.ErrEntryTooLarge.Equal(err) { 349 if r-l == 1 { 350 return err 351 } 352 err = tr.SplitBatchCreateTableAndHandle(schema, info, l, (l+r)/2) 353 if err != nil { 354 return err 355 } 356 err = tr.SplitBatchCreateTableAndHandle(schema, info, (l+r)/2, r) 357 if err != nil { 358 return err 359 } 360 return nil 361 } 362 return err 363 } 364 365 // BatchCreateTableIfNotExist will batch creating tables per schema. If the schema does not exist, it will create it. 366 // The argument is { database name -> { table name -> TableInfo } }. 367 func (tr *Tracker) BatchCreateTableIfNotExist(tablesToCreate map[string]map[string]*model.TableInfo) error { 368 for schema, tableNameInfo := range tablesToCreate { 369 if err := tr.CreateSchemaIfNotExists(schema); err != nil { 370 return err 371 } 372 373 var cloneTis []*model.TableInfo 374 for table, ti := range tableNameInfo { 375 cloneTi := cloneTableInfo(ti) // clone TableInfo w.r.t the warning of the CreateTable function 376 cloneTi.Name = model.NewCIStr(table) // TableInfo has no `TableName` 377 cloneTis = append(cloneTis, cloneTi) 378 } 379 schemaName := model.NewCIStr(schema) 380 if err := tr.SplitBatchCreateTableAndHandle(schemaName, cloneTis, 0, len(cloneTis)); err != nil { 381 return err 382 } 383 } 384 return nil 385 } 386 387 // GetDownStreamTableInfo gets downstream table info. 388 // note. this function will init downstreamTrack's table info. 389 func (tr *Tracker) GetDownStreamTableInfo(tctx *tcontext.Context, tableID string, originTI *model.TableInfo) (*DownstreamTableInfo, error) { 390 return tr.downstreamTracker.getOrInit(tctx, tableID, originTI) 391 } 392 393 // RemoveDownstreamSchema just remove schema or table in downstreamTrack. 394 func (tr *Tracker) RemoveDownstreamSchema(tctx *tcontext.Context, targetTables []*filter.Table) { 395 if len(targetTables) == 0 { 396 return 397 } 398 399 for _, targetTable := range targetTables { 400 tr.downstreamTracker.remove(tctx, targetTable) 401 } 402 } 403 404 func (dt *downstreamTracker) getOrInit(tctx *tcontext.Context, tableID string, originTI *model.TableInfo) (*DownstreamTableInfo, error) { 405 dt.RLock() 406 dti, ok := dt.tableInfos[tableID] 407 dt.RUnlock() 408 if ok { 409 return dti, nil 410 } 411 412 // cache miss, get from downstream 413 dt.Lock() 414 defer dt.Unlock() 415 dti, ok = dt.tableInfos[tableID] 416 if !ok { 417 tctx.Logger.Info("Downstream schema tracker init. ", zap.String("tableID", tableID)) 418 downstreamTI, err := dt.getTableInfoByCreateStmt(tctx, tableID) 419 if err != nil { 420 tctx.Logger.Error("Init dowstream schema info error. ", zap.String("tableID", tableID), zap.Error(err)) 421 return nil, err 422 } 423 424 dti = &DownstreamTableInfo{ 425 TableInfo: downstreamTI, 426 WhereHandle: sqlmodel.GetWhereHandle(originTI, downstreamTI), 427 } 428 dt.tableInfos[tableID] = dti 429 } 430 return dti, nil 431 } 432 433 func (dt *downstreamTracker) remove(tctx *tcontext.Context, targetTable *filter.Table) { 434 dt.Lock() 435 defer dt.Unlock() 436 437 tableID := utils.GenTableID(targetTable) 438 if _, ok := dt.tableInfos[tableID]; !ok { 439 // handle just have schema 440 if targetTable.Schema != "" && targetTable.Name == "" { 441 for k := range dt.tableInfos { 442 if strings.HasPrefix(k, tableID+".") { 443 delete(dt.tableInfos, k) 444 tctx.Logger.Info("Remove downstream schema tracker", zap.String("tableID", k)) 445 } 446 } 447 } 448 } else { 449 delete(dt.tableInfos, tableID) 450 tctx.Logger.Info("Remove downstream schema tracker", zap.String("tableID", tableID)) 451 } 452 } 453 454 // getTableInfoByCreateStmt get downstream tableInfo by "SHOW CREATE TABLE" stmt. 455 func (dt *downstreamTracker) getTableInfoByCreateStmt(tctx *tcontext.Context, tableID string) (*model.TableInfo, error) { 456 if dt.stmtParser == nil { 457 err := dt.initDownStreamSQLModeAndParser(tctx) 458 if err != nil { 459 return nil, err 460 } 461 } 462 createStr, err := dbconn.GetTableCreateSQL(tctx, dt.downstreamConn, tableID) 463 if err != nil { 464 return nil, dmterror.ErrSchemaTrackerCannotFetchDownstreamCreateTableStmt.Delegate(err, tableID) 465 } 466 467 tctx.Logger.Info("Show create table info", zap.String("tableID", tableID), zap.String("create string", createStr)) 468 // parse create table stmt. 469 stmtNode, err := dt.stmtParser.ParseOneStmt(createStr, "", "") 470 if err != nil { 471 return nil, dmterror.ErrSchemaTrackerInvalidCreateTableStmt.Delegate(err, createStr) 472 } 473 474 // suppress ErrTooLongKey 475 dt.se.SetValue(ddl.SuppressErrorTooLongKeyKey, true) 476 // support drop PK 477 enableClusteredIndexBackup := dt.se.GetSessionVars().EnableClusteredIndex 478 dt.se.GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOff 479 defer func() { 480 dt.se.ClearValue(ddl.SuppressErrorTooLongKeyKey) 481 dt.se.GetSessionVars().EnableClusteredIndex = enableClusteredIndexBackup 482 }() 483 484 ti, err := ddl.BuildTableInfoWithStmt(dt.se, stmtNode.(*ast.CreateTableStmt), mysql.DefaultCharset, "", nil) 485 if err != nil { 486 return nil, dmterror.ErrSchemaTrackerCannotMockDownstreamTable.Delegate(err, createStr) 487 } 488 ti.State = model.StatePublic 489 return ti, nil 490 } 491 492 // initDownStreamTrackerParser init downstream tracker parser by default sql_mode. 493 func (dt *downstreamTracker) initDownStreamSQLModeAndParser(tctx *tcontext.Context) error { 494 setSQLMode := fmt.Sprintf("SET SESSION SQL_MODE = '%s'", mysql.DefaultSQLMode) 495 _, err := dt.downstreamConn.ExecuteSQL(tctx, nil, []string{setSQLMode}) 496 if err != nil { 497 return dmterror.ErrSchemaTrackerCannotSetDownstreamSQLMode.Delegate(err, mysql.DefaultSQLMode) 498 } 499 stmtParser, err := conn.GetParserFromSQLModeStr(mysql.DefaultSQLMode) 500 if err != nil { 501 return dmterror.ErrSchemaTrackerCannotInitDownstreamParser.Delegate(err, mysql.DefaultSQLMode) 502 } 503 dt.stmtParser = stmtParser 504 return nil 505 }