github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/syncer/dbconn/db.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 dbconn 15 16 import ( 17 "context" 18 "database/sql" 19 "net" 20 "strings" 21 "time" 22 23 "github.com/go-sql-driver/mysql" 24 "github.com/pingcap/failpoint" 25 "github.com/pingcap/tidb/pkg/util/dbutil" 26 "github.com/pingcap/tiflow/dm/config" 27 "github.com/pingcap/tiflow/dm/pkg/conn" 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/pingcap/tiflow/dm/syncer/metrics" 34 "github.com/prometheus/client_golang/prometheus" 35 "go.uber.org/atomic" 36 "go.uber.org/zap" 37 ) 38 39 var retryTimeout = 3 * time.Second 40 41 // CloseBaseDB closes baseDB to release all connection generated by this baseDB and this baseDB. 42 func CloseBaseDB(logCtx *tcontext.Context, baseDB *conn.BaseDB) { 43 if baseDB != nil { 44 err := baseDB.Close() 45 if err != nil { 46 logCtx.L().Error("fail to close baseDB", log.ShortError(err)) 47 } 48 } 49 } 50 51 // DBConn represents a live DB connection 52 // it's not thread-safe. 53 type DBConn struct { 54 cfg *config.SubTaskConfig 55 baseConn *conn.BaseConn 56 57 // generate new BaseConn and close old one 58 ResetBaseConnFn func(*tcontext.Context, *conn.BaseConn) (*conn.BaseConn, error) 59 } 60 61 func NewDBConn(cfg *config.SubTaskConfig, baseConn *conn.BaseConn) *DBConn { 62 return &DBConn{ 63 cfg: cfg, 64 baseConn: baseConn, 65 } 66 } 67 68 // Scope return connection scope. 69 func (conn *DBConn) Scope() terror.ErrScope { 70 if conn == nil || conn.baseConn == nil { 71 return terror.ScopeNotSet 72 } 73 return conn.baseConn.Scope 74 } 75 76 // ResetConn reset one worker connection from specify *BaseDB. 77 func (conn *DBConn) ResetConn(tctx *tcontext.Context) error { 78 baseConn, err := conn.ResetBaseConnFn(tctx, conn.baseConn) 79 if err != nil { 80 return err 81 } 82 conn.baseConn = baseConn 83 return nil 84 } 85 86 // QuerySQL does one query. 87 func (conn *DBConn) QuerySQL( 88 tctx *tcontext.Context, 89 metricProxies *metrics.Proxies, 90 query string, 91 args ...interface{}, 92 ) (*sql.Rows, error) { 93 if conn == nil || conn.baseConn == nil { 94 return nil, terror.ErrDBUnExpect.Generate("database base connection not valid") 95 } 96 // nolint:dupl 97 params := retry.Params{ 98 RetryCount: 10, 99 FirstRetryDuration: retryTimeout, 100 BackoffStrategy: retry.Stable, 101 IsRetryableFn: conn.retryableFn(tctx, query, args), 102 } 103 104 ret, _, err := conn.baseConn.ApplyRetryStrategy( 105 tctx, 106 params, 107 func(ctx *tcontext.Context) (interface{}, error) { 108 startTime := time.Now() 109 ret, err := conn.baseConn.QuerySQL(ctx, query, args...) 110 if err == nil { 111 if ret.Err() != nil { 112 return err, ret.Err() 113 } 114 cost := time.Since(startTime) 115 // duration seconds 116 ds := cost.Seconds() 117 if metricProxies != nil { 118 metricProxies.Metrics.QueryHistogram.Observe(ds) 119 } 120 if ds > 1 { 121 ctx.L().Warn("query statement too slow", 122 zap.Duration("cost time", cost), 123 zap.String("query", utils.TruncateString(query, -1)), 124 zap.String("argument", utils.TruncateInterface(args, -1))) 125 } 126 } 127 return ret, err 128 }, 129 ) 130 if err != nil { 131 tctx.L().ErrorFilterContextCanceled("query statement failed after retry", 132 zap.String("query", utils.TruncateString(query, -1)), 133 zap.String("argument", utils.TruncateInterface(args, -1)), 134 log.ShortError(err)) 135 return nil, err 136 } 137 return ret.(*sql.Rows), nil 138 } 139 140 // ExecuteSQLWithIgnore do some SQL executions and can ignore some error by `ignoreError`. 141 func (conn *DBConn) ExecuteSQLWithIgnore( 142 tctx *tcontext.Context, 143 metricProxies *metrics.Proxies, 144 ignoreError func(error) bool, 145 queries []string, 146 args ...[]interface{}, 147 ) (int, error) { 148 failpoint.Inject("ExecuteSQLWithIgnoreFailed", func(val failpoint.Value) { 149 queryPattern := val.(string) 150 if len(queries) == 1 && strings.Contains(queries[0], queryPattern) { 151 tctx.L().Warn("executeSQLWithIgnore failed", zap.String("failpoint", "ExecuteSQLWithIgnoreFailed")) 152 failpoint.Return(0, terror.ErrDBUnExpect.Generate("invalid connection")) 153 } 154 }) 155 156 if len(queries) == 0 { 157 return 0, nil 158 } 159 160 if conn == nil || conn.baseConn == nil { 161 return 0, terror.ErrDBUnExpect.Generate("database base connection not valid") 162 } 163 164 // nolint:dupl 165 params := retry.Params{ 166 RetryCount: 100, 167 FirstRetryDuration: retryTimeout, 168 BackoffStrategy: retry.Stable, 169 IsRetryableFn: conn.retryableFn(tctx, queries, args), 170 } 171 172 ret, _, err := conn.baseConn.ApplyRetryStrategy( 173 tctx, 174 params, 175 func(ctx *tcontext.Context) (interface{}, error) { 176 startTime := time.Now() 177 var histProxy *prometheus.HistogramVec 178 if metricProxies != nil { 179 histProxy = metricProxies.StmtHistogram 180 } 181 ret, err := conn.baseConn.ExecuteSQLWithIgnoreError(ctx, histProxy, conn.cfg.Name, ignoreError, queries, args...) 182 if err == nil { 183 cost := time.Since(startTime) 184 // duration seconds 185 ds := cost.Seconds() 186 if metricProxies != nil { 187 metricProxies.Metrics.TxnHistogram.Observe(ds) 188 } 189 // calculate idealJobCount metric: connection count * 1 / (one sql cost time) 190 qps := float64(conn.cfg.WorkerCount) / (cost.Seconds() / float64(len(queries))) 191 if metricProxies != nil { 192 metricProxies.Metrics.IdealQPS.Set(qps) 193 } 194 if ds > 1 { 195 ctx.L().Warn("execute transaction too slow", 196 zap.Duration("cost time", cost), 197 zap.String("query", utils.TruncateInterface(queries, -1)), 198 zap.String("argument", utils.TruncateInterface(args, -1))) 199 } 200 } 201 return ret, err 202 }) 203 if err != nil { 204 tctx.L().ErrorFilterContextCanceled("execute statements failed after retry", 205 zap.String("queries", utils.TruncateInterface(queries, -1)), 206 zap.String("arguments", utils.TruncateInterface(args, -1)), 207 log.ShortError(err)) 208 return ret.(int), err 209 } 210 return ret.(int), nil 211 } 212 213 // ExecuteSQL does some SQL executions. 214 func (conn *DBConn) ExecuteSQL( 215 tctx *tcontext.Context, 216 metricProxies *metrics.Proxies, 217 queries []string, 218 args ...[]interface{}, 219 ) (int, error) { 220 return conn.ExecuteSQLWithIgnore(tctx, metricProxies, nil, queries, args...) 221 } 222 223 // ExecuteSQLAutoSplit wraps BaseConn.ExecuteSQLAutoSplit. 224 // TODO: refine DBConn and BaseConn. 225 func (conn *DBConn) ExecuteSQLAutoSplit( 226 tctx *tcontext.Context, 227 metricProxies *metrics.Proxies, 228 queries []string, 229 args ...[]interface{}, 230 ) error { 231 if conn == nil { 232 // only happens in test 233 return nil 234 } 235 var m *prometheus.HistogramVec 236 if metricProxies != nil { 237 m = metricProxies.StmtHistogram 238 } 239 return conn.baseConn.ExecuteSQLsAutoSplit(tctx, m, conn.cfg.Name, queries, args...) 240 } 241 242 func (conn *DBConn) retryableFn(tctx *tcontext.Context, queries, args any) func(int, error) bool { 243 return func(retryTime int, err error) bool { 244 if retry.IsConnectionError(err) { 245 err = conn.ResetConn(tctx) 246 if err != nil { 247 tctx.L().Error("reset connection failed", zap.Int("retry", retryTime), 248 zap.String("queries", utils.TruncateInterface(queries, -1)), 249 zap.String("arguments", utils.TruncateInterface(args, -1)), 250 log.ShortError(err)) 251 return false 252 } 253 tctx.L().Warn("execute sql failed by connection error", zap.Int("retry", retryTime), 254 zap.Error(err)) 255 return true 256 } 257 if dbutil.IsRetryableError(err) { 258 tctx.L().Warn("execute statements", zap.Int("retry", retryTime), 259 zap.String("queries", utils.TruncateInterface(queries, -1)), 260 zap.String("arguments", utils.TruncateInterface(args, -1)), 261 log.ShortError(err)) 262 return true 263 } 264 return false 265 } 266 } 267 268 // CreateConns returns a opened DB from dbCfg and number of `count` connections of that DB. 269 func CreateConns(tctx *tcontext.Context, cfg *config.SubTaskConfig, dbCfg conn.ScopedDBConfig, count int, ioCounter *atomic.Uint64, uuid string) (*conn.BaseDB, []*DBConn, error) { 270 if ioCounter != nil { 271 tctx.L().Debug("create connection with io total bytes") 272 mysql.RegisterDialContext(uuid, func(ctx context.Context, addr string) (net.Conn, error) { 273 d := &net.Dialer{} 274 conn, err := d.DialContext(ctx, "tcp", addr) 275 if err != nil { 276 return nil, err 277 } 278 tcpConn := conn.(*net.TCPConn) 279 // try https://github.com/go-sql-driver/mysql/blob/bcc459a906419e2890a50fc2c99ea6dd927a88f2/connector.go#L56-L64 280 err = tcpConn.SetKeepAlive(true) 281 if err != nil { 282 tctx.L().Warn("set TCP keep alive failed", zap.Error(err)) 283 } 284 return NewTCPConnWithIOCounter(tcpConn, ioCounter), nil 285 }) 286 dbCfg.Net = uuid 287 } 288 289 conns := make([]*DBConn, 0, count) 290 baseDB, err := conn.DefaultDBProvider.Apply(dbCfg) 291 if err != nil { 292 return nil, nil, err 293 } 294 for i := 0; i < count; i++ { 295 baseConn, err := baseDB.GetBaseConn(tctx.Context()) 296 if err != nil { 297 CloseBaseDB(tctx, baseDB) 298 return nil, nil, terror.WithScope(err, terror.ScopeDownstream) 299 } 300 resetBaseConnFn := func(tctx *tcontext.Context, baseConn *conn.BaseConn) (*conn.BaseConn, error) { 301 err := baseDB.ForceCloseConn(baseConn) 302 if err != nil { 303 tctx.L().Warn("failed to close BaseConn in reset") 304 } 305 return baseDB.GetBaseConn(tctx.Context()) 306 } 307 conns = append(conns, &DBConn{baseConn: baseConn, cfg: cfg, ResetBaseConnFn: resetBaseConnFn}) 308 } 309 return baseDB, conns, nil 310 }