github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/util.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 syncer 15 16 import ( 17 "crypto/tls" 18 "database/sql" 19 "fmt" 20 "strconv" 21 "time" 22 23 "github.com/go-mysql-org/go-mysql/replication" 24 "github.com/pingcap/tidb/br/pkg/version" 25 "github.com/pingcap/tidb/dumpling/export" 26 dlog "github.com/pingcap/tidb/dumpling/log" 27 "github.com/pingcap/tidb/pkg/parser/ast" 28 "github.com/pingcap/tidb/pkg/util" 29 "github.com/pingcap/tidb/pkg/util/filter" 30 "github.com/pingcap/tiflow/dm/config" 31 "github.com/pingcap/tiflow/dm/pkg/binlog/common" 32 "github.com/pingcap/tiflow/dm/pkg/conn" 33 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 34 "github.com/pingcap/tiflow/dm/pkg/terror" 35 "github.com/pingcap/tiflow/dm/pkg/utils" 36 "github.com/pingcap/tiflow/dm/syncer/dbconn" 37 "go.uber.org/zap" 38 ) 39 40 const ( 41 // the time layout for TiDB SHOW DDL statements. 42 timeLayout = "2006-01-02 15:04:05" 43 // everytime retrieve 10 new rows from TiDB history jobs. 44 linesOfRows = 10 45 // max capacity of the block/allow list. 46 maxCapacity = 100000 47 ) 48 49 // getTableByDML gets table from INSERT/UPDATE/DELETE statement. 50 func getTableByDML(dml ast.DMLNode) (*filter.Table, error) { 51 switch stmt := dml.(type) { 52 case *ast.InsertStmt: 53 if stmt.Table == nil || stmt.Table.TableRefs == nil || stmt.Table.TableRefs.Left == nil { 54 return nil, terror.ErrSyncUnitInvalidTableName.Generate(fmt.Sprintf("INSERT statement %s not valid", stmt.Text())) 55 } 56 table, err := tableNameResultSet(stmt.Table.TableRefs.Left) 57 return table, terror.Annotatef(err, "INSERT statement %s", stmt.Text()) 58 case *ast.UpdateStmt: 59 if stmt.TableRefs == nil || stmt.TableRefs.TableRefs == nil || stmt.TableRefs.TableRefs.Left == nil { 60 return nil, terror.ErrSyncUnitInvalidTableName.Generate(fmt.Sprintf("UPDATE statement %s not valid", stmt.Text())) 61 } 62 table, err := tableNameResultSet(stmt.TableRefs.TableRefs.Left) 63 return table, terror.Annotatef(err, "UPDATE statement %s", stmt.Text()) 64 case *ast.DeleteStmt: 65 if stmt.TableRefs == nil || stmt.TableRefs.TableRefs == nil || stmt.TableRefs.TableRefs.Left == nil { 66 return nil, terror.ErrSyncUnitInvalidTableName.Generate(fmt.Sprintf("DELETE statement %s not valid", stmt.Text())) 67 } 68 table, err := tableNameResultSet(stmt.TableRefs.TableRefs.Left) 69 return table, terror.Annotatef(err, "DELETE statement %s", stmt.Text()) 70 } 71 return nil, terror.ErrSyncUnitNotSupportedDML.Generate(dml) 72 } 73 74 func tableNameResultSet(rs ast.ResultSetNode) (*filter.Table, error) { 75 ts, ok := rs.(*ast.TableSource) 76 if !ok { 77 return nil, terror.ErrSyncUnitTableNameQuery.Generate(fmt.Sprintf("ResultSetNode %s", rs.Text())) 78 } 79 tn, ok := ts.Source.(*ast.TableName) 80 if !ok { 81 return nil, terror.ErrSyncUnitTableNameQuery.Generate(fmt.Sprintf("TableSource %s", ts.Text())) 82 } 83 return &filter.Table{Schema: tn.Schema.O, Name: tn.Name.O}, nil 84 } 85 86 // record source tbls record the tables that need to flush checkpoints. 87 func recordSourceTbls(sourceTbls map[string]map[string]struct{}, stmt ast.StmtNode, table *filter.Table) { 88 schema, name := table.Schema, table.Name 89 switch stmt.(type) { 90 // these ddls' relative table checkpoints will be deleted during track ddl, 91 // so we shouldn't flush these checkpoints 92 case *ast.DropDatabaseStmt: 93 delete(sourceTbls, schema) 94 case *ast.DropTableStmt: 95 if _, ok := sourceTbls[schema]; ok { 96 delete(sourceTbls[schema], name) 97 } 98 // these ddls won't update schema tracker, no need to update them 99 case *ast.LockTablesStmt, *ast.UnlockTablesStmt, *ast.CleanupTableLockStmt, *ast.TruncateTableStmt: 100 break 101 // flush other tables schema tracker info into checkpoint 102 default: 103 if _, ok := sourceTbls[schema]; !ok { 104 sourceTbls[schema] = make(map[string]struct{}) 105 } 106 sourceTbls[schema][name] = struct{}{} 107 } 108 } 109 110 func printServerVersion(tctx *tcontext.Context, db *conn.BaseDB, scope string) { 111 logger := dlog.NewAppLogger(tctx.Logger.With(zap.String("scope", scope))) 112 versionInfo, err := export.SelectVersion(db.DB) 113 if err != nil { 114 logger.Warn("fail to get version info", zap.Error(err)) 115 return 116 } 117 version.ParseServerInfo(versionInfo) 118 } 119 120 func str2TimezoneOrFromDB(tctx *tcontext.Context, tzStr string, dbCfg conn.ScopedDBConfig) (*time.Location, string, error) { 121 var err error 122 if len(tzStr) == 0 { 123 baseDB, err2 := conn.DefaultDBProvider.Apply(dbCfg) 124 if err2 != nil { 125 return nil, "", err2 126 } 127 defer baseDB.Close() 128 tzStr, err = config.FetchTimeZoneSetting(tctx.Ctx, baseDB.DB) 129 if err != nil { 130 return nil, "", err 131 } 132 } 133 loc, err := utils.ParseTimeZone(tzStr) 134 if err != nil { 135 return nil, "", err 136 } 137 tctx.L().Info("use timezone", zap.String("location", loc.String()), 138 zap.String("host", dbCfg.Host), zap.Int("port", dbCfg.Port)) 139 return loc, tzStr, nil 140 } 141 142 func subtaskCfg2BinlogSyncerCfg(cfg *config.SubTaskConfig, timezone *time.Location, baList *filter.Filter) (replication.BinlogSyncerConfig, error) { 143 var tlsConfig *tls.Config 144 var err error 145 if cfg.From.Security != nil { 146 if loadErr := cfg.From.Security.LoadTLSContent(); loadErr != nil { 147 return replication.BinlogSyncerConfig{}, terror.ErrCtlLoadTLSCfg.Delegate(loadErr) 148 } 149 tlsConfig, err = util.NewTLSConfig( 150 util.WithCAContent(cfg.From.Security.SSLCABytes), 151 util.WithCertAndKeyContent(cfg.From.Security.SSLCertBytes, cfg.From.Security.SSLKeyBytes), 152 util.WithVerifyCommonName(cfg.From.Security.CertAllowedCN), 153 util.WithMinTLSVersion(tls.VersionTLS10), 154 ) 155 if err != nil { 156 return replication.BinlogSyncerConfig{}, terror.ErrConnInvalidTLSConfig.Delegate(err) 157 } 158 } 159 160 var rowsEventDecodeFunc func(*replication.RowsEvent, []byte) error 161 if baList != nil { 162 // we don't track delete table events, so simply reset the cache if it's full 163 // TODO: use LRU or CLOCK cache if needed. 164 // NOTE: use Table as Key rather than TableID 165 // because TableID may change when upstream switches master, and also RenameTable will not change TableID. 166 allowListCache := make(map[filter.Table]struct{}, maxCapacity) 167 blockListCache := make(map[filter.Table]struct{}, maxCapacity) 168 169 rowsEventDecodeFunc = func(re *replication.RowsEvent, data []byte) error { 170 pos, err := re.DecodeHeader(data) 171 if err != nil { 172 return err 173 } 174 tb := filter.Table{ 175 Schema: string(re.Table.Schema), 176 Name: string(re.Table.Table), 177 } 178 if _, ok := blockListCache[tb]; ok { 179 return nil 180 } else if _, ok := allowListCache[tb]; ok { 181 return re.DecodeData(pos, data) 182 } 183 184 if skipByTable(baList, &tb) { 185 if len(blockListCache) >= maxCapacity { 186 blockListCache = make(map[filter.Table]struct{}, maxCapacity) 187 } 188 blockListCache[tb] = struct{}{} 189 return nil 190 } 191 192 if len(allowListCache) >= maxCapacity { 193 allowListCache = make(map[filter.Table]struct{}, maxCapacity) 194 } 195 allowListCache[tb] = struct{}{} 196 return re.DecodeData(pos, data) 197 } 198 } 199 200 h := cfg.WorkerName 201 // https://github.com/mysql/mysql-server/blob/1bfe02bdad6604d54913c62614bde57a055c8332/include/my_hostname.h#L33-L42 202 if len(h) > 60 { 203 h = h[:60] 204 } 205 syncCfg := replication.BinlogSyncerConfig{ 206 ServerID: cfg.ServerID, 207 Flavor: cfg.Flavor, 208 Host: cfg.From.Host, 209 Port: uint16(cfg.From.Port), 210 User: cfg.From.User, 211 Password: cfg.From.Password, 212 TimestampStringLocation: timezone, 213 TLSConfig: tlsConfig, 214 RowsEventDecodeFunc: rowsEventDecodeFunc, 215 Localhost: h, 216 } 217 // when retry count > 1, go-mysql will retry sync from the previous GTID set in GTID mode, 218 // which may get duplicate binlog event after retry success. so just set retry count = 1, and task 219 // will exit when meet error, and then auto resume by DM itself. 220 common.SetDefaultReplicationCfg(&syncCfg, 1) 221 return syncCfg, nil 222 } 223 224 func safeToRedirect(e *replication.BinlogEvent) bool { 225 if e != nil { 226 switch e.Event.(type) { 227 case *replication.GTIDEvent, *replication.MariadbGTIDEvent: 228 return true 229 } 230 } 231 return false 232 } 233 234 // getDDLStatusFromTiDB retrieves the synchronizing status of DDL from TiDB 235 // hence here db should be TiDB database 236 // createTime should be based on the timezone of downstream, and its unit is second. 237 func getDDLStatusFromTiDB(tctx *tcontext.Context, db *dbconn.DBConn, ddl string, createTime int64) (string, error) { 238 rowNum := linesOfRows 239 rowOffset := 0 240 queryMap := make(map[int]string) 241 242 for { 243 // every attempt try 10 history jobs 244 showJobs := fmt.Sprintf("ADMIN SHOW DDL JOBS %d", rowNum) 245 //nolint:rowserrcheck 246 jobsRows, err := db.QuerySQL(tctx, nil, showJobs) 247 if err != nil { 248 return "", err 249 } 250 251 var jobsResults [][]string 252 jobsResults, err = export.GetSpecifiedColumnValuesAndClose(jobsRows, "JOB_ID", "CREATE_TIME", "STATE") 253 if err != nil { 254 return "", err 255 } 256 257 for i := rowNum - linesOfRows; i < rowNum && i < len(jobsResults); i++ { 258 ddlCreateTimeStr := jobsResults[i][1] 259 var ddlCreateTimeParse time.Time 260 ddlCreateTimeParse, err = time.Parse(timeLayout, ddlCreateTimeStr) 261 if err != nil { 262 return "", err 263 } 264 ddlCreateTime := ddlCreateTimeParse.Unix() 265 266 // ddlCreateTime and createTime are both based on timezone of downstream 267 if ddlCreateTime >= createTime { 268 var jobID int 269 jobID, err = strconv.Atoi(jobsResults[i][0]) 270 if err != nil { 271 return "", err 272 } 273 274 for { 275 ddlQuery, ok := queryMap[jobID] 276 if !ok { 277 // jobID does not exist, expand queryMap for deeper search 278 showJobsLimitNext := fmt.Sprintf("ADMIN SHOW DDL JOB QUERIES LIMIT 10 OFFSET %d", rowOffset) 279 var rowsLimitNext *sql.Rows 280 //nolint:rowserrcheck 281 rowsLimitNext, err = db.QuerySQL(tctx, nil, showJobsLimitNext) 282 if err != nil { 283 return "", err 284 } 285 286 var resultsLimitNext [][]string 287 resultsLimitNext, err = export.GetSpecifiedColumnValuesAndClose(rowsLimitNext, "JOB_ID", "QUERY") 288 if err != nil { 289 return "", err 290 } 291 if len(resultsLimitNext) == 0 { 292 // JOB QUERIES has been used up 293 // requested DDL cannot be found 294 return "", nil 295 } 296 297 // if new DDLs are written to TiDB after the last query 'ADMIN SHOW DDL JOB QUERIES LIMIT 10 OFFSET' 298 // we may get duplicate rows here, but it does not affect the checking 299 for k := range resultsLimitNext { 300 var jobIDForLimit int 301 jobIDForLimit, err = strconv.Atoi(resultsLimitNext[k][0]) 302 if err != nil { 303 return "", err 304 } 305 queryMap[jobIDForLimit] = resultsLimitNext[k][1] 306 } 307 rowOffset += linesOfRows 308 } else { 309 if ddl == ddlQuery { 310 return jobsResults[i][2], nil 311 } 312 break 313 } 314 } 315 } else { 316 // ddlCreateTime is monotonous in jobsResults 317 // requested DDL cannot be found 318 return "", nil 319 } 320 } 321 if len(jobsResults) == rowNum { 322 rowNum += linesOfRows 323 } else { 324 // jobsResults has been checked thoroughly 325 return "", nil 326 } 327 } 328 }