github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/conn/baseconn.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 conn 15 16 import ( 17 "database/sql" 18 "database/sql/driver" 19 "fmt" 20 "strings" 21 "time" 22 23 gmysql "github.com/go-mysql-org/go-mysql/mysql" 24 "github.com/go-sql-driver/mysql" 25 "github.com/pingcap/errors" 26 "github.com/pingcap/failpoint" 27 "github.com/pingcap/tidb/pkg/errno" 28 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 29 "github.com/pingcap/tiflow/dm/pkg/log" 30 "github.com/pingcap/tiflow/dm/pkg/retry" 31 "github.com/pingcap/tiflow/dm/pkg/terror" 32 "github.com/pingcap/tiflow/dm/pkg/utils" 33 "github.com/prometheus/client_golang/prometheus" 34 "go.uber.org/zap" 35 ) 36 37 // BaseConn is the basic connection we use in dm 38 // BaseDB -> BaseConn correspond to sql.DB -> sql.Conn 39 // In our scenario, there are two main reasons why we need BaseConn 40 // 1. we often need one fixed DB connection to execute sql 41 // 2. we need own retry policy during execute failed 42 // 43 // So we split a fixed sql.Conn out of sql.DB, and wraps it to BaseConn 44 // And Similar with sql.Conn, all BaseConn generated from one BaseDB shares this BaseDB to reset 45 // 46 // Basic usage: 47 // For Syncer and Loader Unit, they both have different amount of connections due to config 48 // Currently we have some types of connections exist 49 // 50 // Syncer: 51 // Worker Connection: 52 // DML connection: 53 // execute some DML on Downstream DB, one unit has `syncer.WorkerCount` worker connections 54 // DDL Connection: 55 // execute some DDL on Downstream DB, one unit has one connection 56 // CheckPoint Connection: 57 // interact with CheckPoint DB, one unit has one connection 58 // OnlineDDL connection: 59 // interact with Online DDL DB, one unit has one connection 60 // ShardGroupKeeper connection: 61 // interact with ShardGroupKeeper DB, one unit has one connection 62 // 63 // Loader: 64 // Worker Connection: 65 // execute some DML to Downstream DB, one unit has `loader.PoolSize` worker connections 66 // CheckPoint Connection: 67 // interact with CheckPoint DB, one unit has one connection 68 // Restore Connection: 69 // only use to create schema and table in restoreData, 70 // it ignore already exists error and it should be removed after use, one unit has one connection 71 // 72 // each connection should have ability to retry on some common errors (e.g. tmysql.ErrTiKVServerTimeout) or maybe some specify errors in the future 73 // and each connection also should have ability to reset itself during some specify connection error (e.g. driver.ErrBadConn). 74 type BaseConn struct { 75 DBConn *sql.Conn 76 Scope terror.ErrScope 77 RetryStrategy retry.Strategy 78 } 79 80 // NewBaseConn builds BaseConn to connect real DB. 81 func NewBaseConn(conn *sql.Conn, scope terror.ErrScope, strategy retry.Strategy) *BaseConn { 82 if strategy == nil { 83 strategy = &retry.FiniteRetryStrategy{} 84 } 85 return &BaseConn{ 86 DBConn: conn, 87 Scope: scope, 88 RetryStrategy: strategy, 89 } 90 } 91 92 // NewBaseConnForTest builds BaseConn to connect real DB for test. 93 func NewBaseConnForTest(conn *sql.Conn, strategy retry.Strategy) *BaseConn { 94 if strategy == nil { 95 strategy = &retry.FiniteRetryStrategy{} 96 } 97 return &BaseConn{ 98 DBConn: conn, 99 Scope: terror.ScopeNotSet, 100 RetryStrategy: strategy, 101 } 102 } 103 104 // SetRetryStrategy set retry strategy for baseConn. 105 func (conn *BaseConn) SetRetryStrategy(strategy retry.Strategy) error { 106 if conn == nil { 107 return terror.ErrDBUnExpect.Generate("database connection not valid") 108 } 109 conn.RetryStrategy = strategy 110 return nil 111 } 112 113 // QuerySQL runs a query statement. 114 func (conn *BaseConn) QuerySQL(tctx *tcontext.Context, query string, args ...interface{}) (*sql.Rows, error) { 115 if conn == nil || conn.DBConn == nil { 116 return nil, terror.ErrDBUnExpect.Generate("database connection not valid") 117 } 118 tctx.L().Debug("query statement", 119 zap.String("query", utils.TruncateString(query, -1)), 120 zap.String("argument", utils.TruncateInterface(args, -1))) 121 122 rows, err := conn.DBConn.QueryContext(tctx.Context(), query, args...) 123 if err != nil { 124 tctx.L().ErrorFilterContextCanceled("query statement failed", 125 zap.String("query", utils.TruncateString(query, -1)), 126 zap.String("argument", utils.TruncateInterface(args, -1)), 127 log.ShortError(err)) 128 return nil, terror.ErrDBQueryFailed.Delegate(err, utils.TruncateString(query, -1)) 129 } 130 return rows, nil 131 } 132 133 // ExecuteSQLWithIgnoreError executes sql on real DB, and will ignore some error and continue execute the next query. 134 // return 135 // 1. failed: (the index of sqls executed error, error) 136 // 2. succeed: (rows affected, nil). 137 func (conn *BaseConn) ExecuteSQLWithIgnoreError(tctx *tcontext.Context, hVec *prometheus.HistogramVec, task string, ignoreErr func(error) bool, queries []string, args ...[]interface{}) (int, error) { 138 var affect int64 139 // inject an error to trigger retry, this should be placed before the real execution of the SQL statement. 140 failpoint.Inject("retryableError", func(val failpoint.Value) { 141 if mark, ok := val.(string); ok { 142 enabled := false 143 for _, query := range queries { 144 if strings.Contains(query, mark) { 145 enabled = true // only enable if the `mark` matched. 146 } 147 } 148 if enabled { 149 tctx.L().Info("", zap.String("failpoint", "retryableError"), zap.String("mark", mark)) 150 failpoint.Return(0, &mysql.MySQLError{ 151 Number: gmysql.ER_LOCK_DEADLOCK, 152 Message: fmt.Sprintf("failpoint inject retryable error for %s", mark), 153 }) 154 } 155 } 156 }) 157 158 if len(queries) == 0 { 159 return 0, nil 160 } 161 if conn == nil || conn.DBConn == nil { 162 return 0, terror.ErrDBUnExpect.Generate("database connection not valid") 163 } 164 165 startTime := time.Now() 166 txn, err := conn.DBConn.BeginTx(tctx.Context(), nil) 167 if err != nil { 168 return 0, terror.ErrDBExecuteFailed.Delegate(err, "begin") 169 } 170 if hVec != nil { 171 hVec.WithLabelValues("begin", task).Observe(time.Since(startTime).Seconds()) 172 } 173 174 l := len(queries) 175 176 for i, query := range queries { 177 var arg []interface{} 178 if len(args) > i { 179 arg = args[i] 180 } 181 182 // avoid use TruncateInterface for all log level which will slow the speed of DML 183 if tctx.L().Core().Enabled(zap.DebugLevel) { 184 tctx.L().Debug("execute statement", 185 zap.String("query", utils.TruncateString(query, -1)), 186 zap.String("argument", utils.TruncateInterface(arg, -1))) 187 } 188 189 startTime = time.Now() 190 result, err2 := txn.ExecContext(tctx.Context(), query, arg...) 191 if err2 == nil { 192 rows, _ := result.RowsAffected() 193 affect += rows 194 if hVec != nil { 195 hVec.WithLabelValues("stmt", task).Observe(time.Since(startTime).Seconds()) 196 } 197 } else { 198 if ignoreErr != nil && ignoreErr(err2) { 199 tctx.L().Warn("execute statement failed and will ignore this error", 200 zap.String("query", utils.TruncateString(query, -1)), 201 zap.String("argument", utils.TruncateInterface(arg, -1)), 202 log.ShortError(err2)) 203 continue 204 } 205 206 tctx.L().ErrorFilterContextCanceled("execute statement failed", 207 zap.String("query", utils.TruncateString(query, -1)), 208 zap.String("argument", utils.TruncateInterface(arg, -1)), log.ShortError(err2)) 209 210 startTime = time.Now() 211 rerr := txn.Rollback() 212 if rerr != nil { 213 tctx.L().Error("rollback failed", 214 zap.String("query", utils.TruncateString(query, -1)), 215 zap.String("argument", utils.TruncateInterface(arg, -1)), 216 log.ShortError(rerr)) 217 } else if hVec != nil { 218 hVec.WithLabelValues("rollback", task).Observe(time.Since(startTime).Seconds()) 219 } 220 // we should return the exec err, instead of the rollback rerr. 221 return i, terror.ErrDBExecuteFailed.Delegate(err2, utils.TruncateString(query, -1)) 222 } 223 } 224 startTime = time.Now() 225 err = txn.Commit() 226 if err != nil { 227 return l - 1, terror.ErrDBExecuteFailed.Delegate(err, "commit") // mark failed on the last one 228 } 229 if hVec != nil { 230 hVec.WithLabelValues("commit", task).Observe(time.Since(startTime).Seconds()) 231 } 232 return int(affect), nil 233 } 234 235 // ExecuteSQL executes sql on real DB, 236 // return 237 // 1. failed: (the index of sqls executed error, error) 238 // 2. succeed: (rows affected, nil). 239 func (conn *BaseConn) ExecuteSQL(tctx *tcontext.Context, hVec *prometheus.HistogramVec, task string, queries []string, args ...[]interface{}) (int, error) { 240 return conn.ExecuteSQLWithIgnoreError(tctx, hVec, task, nil, queries, args...) 241 } 242 243 // ExecuteSQLsAutoSplit executes sqls and when meet "transaction too large" error, 244 // it will try to split the sqls into two parts and execute them again. 245 // The `queries` and `args` should be the same length. 246 func (conn *BaseConn) ExecuteSQLsAutoSplit( 247 tctx *tcontext.Context, 248 hVec *prometheus.HistogramVec, 249 task string, 250 queries []string, 251 args ...[]interface{}, 252 ) error { 253 _, err := conn.ExecuteSQL(tctx, hVec, task, queries, args...) 254 mysqlErr, ok := errors.Cause(err).(*mysql.MySQLError) 255 if !ok { 256 return err 257 } 258 259 if mysqlErr.Number != errno.ErrTxnTooLarge || len(queries) == 1 { 260 return err 261 } 262 263 mid := len(queries) / 2 264 err = conn.ExecuteSQLsAutoSplit(tctx, hVec, task, queries[:mid], args[:mid]...) 265 if err != nil { 266 return err 267 } 268 return conn.ExecuteSQLsAutoSplit(tctx, hVec, task, queries[mid:], args[mid:]...) 269 } 270 271 // ApplyRetryStrategy apply specify strategy for BaseConn. 272 func (conn *BaseConn) ApplyRetryStrategy(tctx *tcontext.Context, params retry.Params, 273 operateFn func(*tcontext.Context) (interface{}, error), 274 ) (interface{}, int, error) { 275 return conn.RetryStrategy.Apply(tctx, params, operateFn) 276 } 277 278 // close returns the connection to the connection pool, has the same meaning of sql.Conn.Close. 279 func (conn *BaseConn) close() error { 280 if conn == nil || conn.DBConn == nil { 281 return nil 282 } 283 return conn.DBConn.Close() 284 } 285 286 // forceClose will close the underlying connection completely, 287 // should not be used by functions other than BaseDB.ForceCloseConn. 288 func (conn *BaseConn) forceClose() error { 289 if conn == nil || conn.DBConn == nil { 290 return nil 291 } 292 293 err := conn.DBConn.Raw(func(dc interface{}) error { 294 // return an `ErrBadConn` to ensure close the connection, but do not put it back to the pool. 295 // if we choose to use `Close`, it will always put the connection back to the pool. 296 return driver.ErrBadConn 297 }) 298 if err != driver.ErrBadConn { 299 return terror.ErrDBUnExpect.Delegate(err, "close") 300 } 301 return nil 302 }