github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/binlog/status.go (about) 1 // Copyright 2021 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 binlog 15 16 import ( 17 "path" 18 "strconv" 19 "time" 20 21 gmysql "github.com/go-mysql-org/go-mysql/mysql" 22 "github.com/pingcap/tidb/dumpling/export" 23 "github.com/pingcap/tiflow/dm/pkg/conn" 24 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 25 "github.com/pingcap/tiflow/dm/pkg/terror" 26 "github.com/pingcap/tiflow/dm/pkg/utils" 27 ) 28 29 // in MySQL, we can set `max_binlog_size` to control the max size of a binlog file. 30 // but this is not absolute: 31 // > A transaction is written in one chunk to the binary log, so it is never split between several binary logs. 32 // > Therefore, if you have big transactions, you might see binary log files larger than max_binlog_size. 33 // ref: https://dev.mysql.com/doc/refman/5.7/en/replication-options-binary-log.html#sysvar_max_binlog_size 34 // The max value of `max_binlog_size` is 1073741824 (1GB) 35 // but the actual file size still can be larger, and it may exceed the range of an uint32 36 // so, if we use go-mysql.Position(with uint32 Pos) to store the binlog size, it may become out of range. 37 // ps, use go-mysql.Position to store a position of binlog event (position of the next event) is enough. 38 type binlogSize struct { 39 name string 40 size int64 41 } 42 43 // FileSizes is a list of binlog filename and size. 44 type FileSizes []binlogSize 45 46 // GetBinaryLogs returns binlog filename and size of upstream. 47 func GetBinaryLogs(ctx *tcontext.Context, db *conn.BaseDB) (FileSizes, error) { 48 query := "SHOW BINARY LOGS" 49 rows, err := db.QueryContext(ctx, query) 50 if err != nil { 51 return nil, terror.DBErrorAdapt(err, db.Scope, terror.ErrDBDriverError) 52 } 53 defer func() { 54 _ = rows.Close() 55 _ = rows.Err() 56 }() 57 files := make([]binlogSize, 0, 10) 58 var rowsResult [][]string 59 rowsResult, err = export.GetSpecifiedColumnValuesAndClose(rows, "Log_name", "File_size") 60 if err != nil { 61 return nil, terror.DBErrorAdapt(err, db.Scope, terror.ErrDBDriverError) 62 } 63 for _, rowResult := range rowsResult { 64 pos, err := strconv.ParseInt(rowResult[1], 10, 64) 65 if err != nil { 66 return nil, terror.DBErrorAdapt(err, db.Scope, terror.ErrDBDriverError) 67 } 68 files = append(files, binlogSize{name: rowResult[0], size: pos}) 69 } 70 return files, nil 71 } 72 73 // After returns the total size of binlog after `fromFile` in FileSizes. 74 func (b FileSizes) After(fromFile gmysql.Position) int64 { 75 var total int64 76 for _, file := range b { 77 switch gmysql.CompareBinlogFileName(file.name, fromFile.Name) { 78 case -1: 79 continue 80 case 1: 81 total += file.size 82 case 0: 83 if file.size > int64(fromFile.Pos) { 84 total += file.size - int64(fromFile.Pos) 85 } 86 } 87 } 88 89 return total 90 } 91 92 func GetLocalBinaryLogs(dir string) (FileSizes, error) { 93 fileNames, err := ReadSortedBinlogFromDir(dir) 94 if err != nil { 95 return nil, err 96 } 97 files := make([]binlogSize, 0, len(fileNames)) 98 for _, fileName := range fileNames { 99 size, err := utils.GetFileSize(path.Join(dir, fileName)) 100 if err != nil { 101 return nil, err 102 } 103 files = append(files, binlogSize{name: fileName, size: size}) 104 } 105 return files, nil 106 } 107 108 // SourceStatus collects all information of upstream. 109 type SourceStatus struct { 110 Location Location 111 Binlogs FileSizes 112 UpdateTime time.Time 113 } 114 115 func GetSourceStatus(tctx *tcontext.Context, db *conn.BaseDB, flavor string) (*SourceStatus, error) { 116 ret := &SourceStatus{} 117 ctx, cancel := tctx.WithTimeout(conn.DefaultDBTimeout) 118 defer cancel() 119 pos, gtidSet, err := conn.GetPosAndGs(ctx, db, flavor) 120 if err != nil { 121 return nil, err 122 } 123 ret.Location = NewLocation(pos, gtidSet) 124 ctx2, cancel2 := tctx.WithTimeout(conn.DefaultDBTimeout) 125 defer cancel2() 126 binlogs, err := GetBinaryLogs(ctx2, db) 127 if err != nil { 128 return nil, err 129 } 130 ret.Binlogs = binlogs 131 132 ret.UpdateTime = time.Now() 133 return ret, nil 134 }