github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/dm/pkg/checker/conn_checker.go (about) 1 // Copyright 2022 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 checker 15 16 import ( 17 "context" 18 "database/sql" 19 20 "github.com/pingcap/tidb/pkg/parser/mysql" 21 "github.com/pingcap/tidb/pkg/util/dbutil" 22 "github.com/pingcap/tiflow/dm/config" 23 "github.com/pingcap/tiflow/dm/pkg/conn" 24 tcontext "github.com/pingcap/tiflow/dm/pkg/context" 25 "github.com/pingcap/tiflow/dm/pkg/log" 26 "go.uber.org/zap" 27 ) 28 29 type connNumberChecker struct { 30 toCheckDB *conn.BaseDB 31 stCfgs []*config.SubTaskConfig 32 33 getConfigConn func(stCfgs []*config.SubTaskConfig) int 34 workerName string 35 unlimitedConn bool 36 } 37 38 func newConnNumberChecker(toCheckDB *conn.BaseDB, stCfgs []*config.SubTaskConfig, fn func(stCfgs []*config.SubTaskConfig) int, workerName string) connNumberChecker { 39 return connNumberChecker{ 40 toCheckDB: toCheckDB, 41 stCfgs: stCfgs, 42 getConfigConn: fn, 43 workerName: workerName, 44 } 45 } 46 47 func (c *connNumberChecker) check(ctx context.Context, checkerName string, neededPriv map[mysql.PrivilegeType]priv) *Result { 48 result := &Result{ 49 Name: checkerName, 50 Desc: "check if connetion concurrency exceeds database's maximum connection limit", 51 State: StateSuccess, 52 } 53 baseConn, err := c.toCheckDB.GetBaseConn(ctx) 54 if err != nil { 55 markCheckError(result, err) 56 return result 57 } 58 defer c.toCheckDB.ForceCloseConnWithoutErr(baseConn) 59 if err != nil { 60 markCheckError(result, err) 61 return result 62 } 63 var rows, processRows *sql.Rows 64 rows, err = baseConn.QuerySQL(tcontext.NewContext(ctx, log.L()), "SHOW GLOBAL VARIABLES LIKE 'max_connections'") 65 if err != nil { 66 markCheckError(result, err) 67 return result 68 } 69 defer func() { 70 _ = rows.Close() 71 _ = rows.Err() 72 }() 73 var ( 74 maxConn int 75 variable string 76 ) 77 for rows.Next() { 78 err = rows.Scan(&variable, &maxConn) 79 if err != nil { 80 markCheckError(result, err) 81 return result 82 } 83 } 84 rows.Close() 85 if maxConn == 0 { 86 // state = success 87 c.unlimitedConn = true 88 return result 89 } 90 // check super privilege for SHOW PROCESSLIST 91 usedConn := 0 92 grants, err := dbutil.ShowGrants(ctx, c.toCheckDB.DB, "", "") 93 if err != nil { 94 markCheckError(result, err) 95 return result 96 } 97 err2 := verifyPrivilegesWithResult(result, grants, neededPriv) 98 if err2 != nil { 99 // no enough privilege to check the user's connection number 100 result.State = StateWarning 101 result.Errors = append(result.Errors, NewWarn(err2.ShortErr)) 102 result.Instruction = err2.Instruction 103 } else { 104 processRows, err = baseConn.QuerySQL(tcontext.NewContext(ctx, log.L()), "SHOW PROCESSLIST") 105 if err != nil { 106 markCheckError(result, err) 107 return result 108 } 109 defer func() { 110 _ = processRows.Close() 111 _ = processRows.Err() 112 }() 113 for processRows.Next() { 114 usedConn++ 115 } 116 usedConn -= 1 // exclude the connection used for show processlist 117 } 118 log.L().Debug("connection checker", zap.Int("maxConnections", maxConn), zap.Int("usedConnections", usedConn)) 119 neededConn := c.getConfigConn(c.stCfgs) 120 if neededConn > maxConn { 121 // nonzero max_connections and needed connections exceed max_connections 122 // FYI: https://github.com/pingcap/tidb/pull/35453 123 // currently, TiDB's max_connections is set to 0 representing unlimited connections, 124 // while for MySQL, 0 is not a legal value (never retrieve from it). 125 result.Errors = append( 126 result.Errors, 127 NewError( 128 "checked database's max_connections: %d is less than the number %s needs: %d", 129 maxConn, 130 c.workerName, 131 neededConn, 132 ), 133 ) 134 result.Instruction = "You need to set a larger max_connection, or adjust the configuration of DM such as reducing the worker count of sycner and reducing the pool size of the dumper and loader." 135 result.State = StateFailure 136 } else if maxConn-usedConn < neededConn { 137 // if we don't have enough privilege to check the user's connection number, 138 // usedConn is 0 139 result.State = StateWarning 140 result.Instruction = "You need to set a larger max_connection, or adjust the configuration of DM such as reducing the worker count of sycner and reducing the pool size of the dumper and loader." 141 result.Errors = append( 142 result.Errors, 143 NewError( 144 "database's max_connections: %d, used_connections: %d, available_connections: %d is less than %s needs: %d", 145 maxConn, 146 usedConn, 147 maxConn-usedConn, 148 c.workerName, 149 neededConn, 150 ), 151 ) 152 } 153 return result 154 } 155 156 type LoaderConnNumberChecker struct { 157 connNumberChecker 158 } 159 160 func NewLoaderConnNumberChecker(targetDB *conn.BaseDB, stCfgs []*config.SubTaskConfig) RealChecker { 161 return &LoaderConnNumberChecker{ 162 connNumberChecker: newConnNumberChecker(targetDB, stCfgs, func(stCfgs []*config.SubTaskConfig) int { 163 loaderConn := 0 164 for _, stCfg := range stCfgs { 165 // loader's worker and checkpoint (always keeps one db connection) 166 loaderConn += stCfg.LoaderConfig.PoolSize + 1 167 } 168 return loaderConn 169 }, "loader"), 170 } 171 } 172 173 func (l *LoaderConnNumberChecker) Name() string { 174 return "loader_conn_number_checker" 175 } 176 177 func (l *LoaderConnNumberChecker) Check(ctx context.Context) *Result { 178 result := l.check(ctx, l.Name(), map[mysql.PrivilegeType]priv{ 179 mysql.SuperPriv: {needGlobal: true}, 180 }) 181 if !l.unlimitedConn && result.State == StateFailure { 182 // if we're using lightning, this error should be omitted 183 // because lightning doesn't need to keep connections while restoring. 184 result.Errors = append( 185 result.Errors, 186 NewWarn("task precheck cannot accurately check the number of connection needed for Lightning."), 187 ) 188 result.State = StateWarning 189 result.Instruction = "You need to set a larger connection for TiDB." 190 } 191 return result 192 } 193 194 func NewDumperConnNumberChecker(sourceDB *conn.BaseDB, dumperThreads int) RealChecker { 195 return &DumperConnNumberChecker{ 196 connNumberChecker: newConnNumberChecker(sourceDB, nil, func(_ []*config.SubTaskConfig) int { 197 // one for generating SQL, another for consistency control 198 return dumperThreads + 2 199 }, "dumper"), 200 } 201 } 202 203 type DumperConnNumberChecker struct { 204 connNumberChecker 205 } 206 207 // Mariadb (process priv): https://mariadb.com/kb/en/show-processlist/ 208 // MySQL(process priv): https://dev.mysql.com/doc/refman/5.7/en/privileges-provided.html 209 // Aurora (process priv): https://aws.amazon.com/cn/premiumsupport/knowledge-center/rds-mysql-running-queries/ 210 func (d *DumperConnNumberChecker) Check(ctx context.Context) *Result { 211 return d.check(ctx, d.Name(), map[mysql.PrivilegeType]priv{ 212 mysql.ProcessPriv: {needGlobal: true}, 213 }) 214 } 215 216 func (d *DumperConnNumberChecker) Name() string { 217 return "dumper_conn_number_checker" 218 }