From 05309d7230f04b350ff78d1cce2c16c9dc868203 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 2 Jul 2019 11:55:57 +0800 Subject: [PATCH 1/5] *: Add support for MAX_EXECUTION_TIME (#10541) --- executor/adapter.go | 19 +++++++- server/conn.go | 22 +++++++-- server/conn_stmt.go | 2 +- server/conn_test.go | 70 +++++++++++++++++++++++++++ server/driver.go | 2 +- server/driver_tidb.go | 4 +- server/server.go | 26 +++++----- session/session.go | 33 ++++++++----- session/session_test.go | 26 ++++++++++ sessionctx/variable/session.go | 26 ++++++---- sessionctx/variable/sysvar.go | 3 +- sessionctx/variable/tidb_vars.go | 11 +++-- sessionctx/variable/varsutil.go | 6 ++- tidb-server/main.go | 2 +- util/expensivequery/expensivequery.go | 26 +++++----- util/processinfo.go | 3 ++ 16 files changed, 220 insertions(+), 61 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 6f6af7de5545a..e7fbc4927e95c 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -50,7 +50,7 @@ import ( // processinfoSetter is the interface use to set current running process info. type processinfoSetter interface { - SetProcessInfo(string, time.Time, byte) + SetProcessInfo(string, time.Time, byte, uint64) } // recordSet wraps an executor, implements sqlexec.RecordSet interface @@ -245,8 +245,9 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { sql = ss.SecureText() } } + maxExecutionTime := getMaxExecutionTime(sctx, a.StmtNode) // Update processinfo, ShowProcess() will use it. - pi.SetProcessInfo(sql, time.Now(), cmd) + pi.SetProcessInfo(sql, time.Now(), cmd, maxExecutionTime) a.Ctx.GetSessionVars().StmtCtx.StmtType = GetStmtLabel(a.StmtNode) } @@ -285,6 +286,20 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) { }, nil } +// getMaxExecutionTime get the max execution timeout value. +func getMaxExecutionTime(sctx sessionctx.Context, stmtNode ast.StmtNode) uint64 { + ret := sctx.GetSessionVars().MaxExecutionTime + if sel, ok := stmtNode.(*ast.SelectStmt); ok { + for _, hint := range sel.TableHints { + if hint.HintName.L == variable.MaxExecutionTime { + ret = hint.MaxExecutionTime + break + } + } + } + return ret +} + type chunkRowRecordSet struct { rows []chunk.Row idx int diff --git a/server/conn.go b/server/conn.go index 8f2b99598a30c..a23db09e377a1 100644 --- a/server/conn.go +++ b/server/conn.go @@ -50,6 +50,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -262,6 +263,11 @@ func (cc *clientConn) readPacket() ([]byte, error) { } func (cc *clientConn) writePacket(data []byte) error { + failpoint.Inject("FakeClientConn", func() { + if cc.pkt == nil { + failpoint.Return(nil) + } + }) return cc.pkt.writePacket(data) } @@ -845,7 +851,11 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { cc.lastCmd = string(hack.String(data)) token := cc.server.getToken() defer func() { - cc.ctx.SetProcessInfo("", t, mysql.ComSleep) + // if handleChangeUser failed, cc.ctx may be nil + if cc.ctx != nil { + cc.ctx.SetProcessInfo("", t, mysql.ComSleep, 0) + } + cc.server.releaseToken(token) span.Finish() }() @@ -860,9 +870,9 @@ func (cc *clientConn) dispatch(ctx context.Context, data []byte) error { switch cmd { case mysql.ComPing, mysql.ComStmtClose, mysql.ComStmtSendLongData, mysql.ComStmtReset, mysql.ComSetOption, mysql.ComChangeUser: - cc.ctx.SetProcessInfo("", t, cmd) + cc.ctx.SetProcessInfo("", t, cmd, 0) case mysql.ComInitDB: - cc.ctx.SetProcessInfo("use "+dataStr, t, cmd) + cc.ctx.SetProcessInfo("use "+dataStr, t, cmd, 0) } switch cmd { @@ -925,6 +935,11 @@ func (cc *clientConn) useDB(ctx context.Context, db string) (err error) { } func (cc *clientConn) flush() error { + failpoint.Inject("FakeClientConn", func() { + if cc.pkt == nil { + failpoint.Return(nil) + } + }) return cc.pkt.flush() } @@ -1255,6 +1270,7 @@ func (cc *clientConn) writeResultset(ctx context.Context, rs ResultSet, binary b if err != nil { return err } + return cc.flush() } diff --git a/server/conn_stmt.go b/server/conn_stmt.go index d55ba0eea3a0f..96c8220db4de2 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -223,7 +223,7 @@ func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err err if prepared, ok := cc.ctx.GetStatement(int(stmtID)).(*TiDBStatement); ok { sql = prepared.sql } - cc.ctx.SetProcessInfo(sql, time.Now(), mysql.ComStmtExecute) + cc.ctx.SetProcessInfo(sql, time.Now(), mysql.ComStmtExecute, 0) rs := stmt.GetResultSet() if rs == nil { return mysql.NewErr(mysql.ErrUnknownStmtHandler, diff --git a/server/conn_test.go b/server/conn_test.go index d139fbd3aff32..132a91553d4a8 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -18,8 +18,11 @@ import ( "bytes" "context" "encoding/binary" + "fmt" + "io" . "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -207,3 +210,70 @@ func mapBelong(m1, m2 map[string]string) bool { } return true } + +func (ts ConnTestSuite) TestConnExecutionTimeout(c *C) { + //There is no underlying netCon, use failpoint to avoid panic + c.Assert(failpoint.Enable("github.com/pingcap/tidb/server/FakeClientConn", "return(1)"), IsNil) + + c.Parallel() + var err error + ts.store, err = mockstore.NewMockTikvStore() + c.Assert(err, IsNil) + ts.dom, err = session.BootstrapSession(ts.store) + c.Assert(err, IsNil) + se, err := session.CreateSession4Test(ts.store) + c.Assert(err, IsNil) + + connID := 1 + se.SetConnectionID(uint64(connID)) + tc := &TiDBContext{ + session: se, + stmts: make(map[int]*TiDBStatement), + } + cc := &clientConn{ + connectionID: uint32(connID), + server: &Server{ + capability: defaultCapability, + }, + ctx: tc, + alloc: arena.NewAllocator(32 * 1024), + } + srv := &Server{ + clients: map[uint32]*clientConn{ + uint32(connID): cc, + }, + } + handle := ts.dom.ExpensiveQueryHandle().SetSessionManager(srv) + go handle.Run() + defer handle.Close() + + _, err = se.Execute(context.Background(), "use test;") + c.Assert(err, IsNil) + _, err = se.Execute(context.Background(), "CREATE TABLE testTable2 (id bigint PRIMARY KEY, age int)") + c.Assert(err, IsNil) + for i := 0; i < 10; i++ { + str := fmt.Sprintf("insert into testTable2 values(%d, %d)", i, i%80) + _, err = se.Execute(context.Background(), str) + c.Assert(err, IsNil) + } + + _, err = se.Execute(context.Background(), "select SLEEP(1);") + c.Assert(err, IsNil) + + _, err = se.Execute(context.Background(), "set @@max_execution_time = 500;") + c.Assert(err, IsNil) + + err = cc.handleQuery(context.Background(), "select * FROM testTable2 WHERE SLEEP(1);") + c.Assert(err, NotNil) + + _, err = se.Execute(context.Background(), "set @@max_execution_time = 0;") + c.Assert(err, IsNil) + + err = cc.handleQuery(context.Background(), "select * FROM testTable2 WHERE SLEEP(1);") + c.Assert(err, IsNil) + + err = cc.handleQuery(context.Background(), "select /*+ MAX_EXECUTION_TIME(100)*/ * FROM testTable2 WHERE SLEEP(1);") + c.Assert(err, NotNil) + + c.Assert(failpoint.Disable("github.com/pingcap/tidb/server/FakeClientConn"), IsNil) +} diff --git a/server/driver.go b/server/driver.go index b88137391d69e..14468b0d64398 100644 --- a/server/driver.go +++ b/server/driver.go @@ -51,7 +51,7 @@ type QueryCtx interface { // SetValue saves a value associated with this context for key. SetValue(key fmt.Stringer, value interface{}) - SetProcessInfo(sql string, t time.Time, command byte) + SetProcessInfo(sql string, t time.Time, command byte, maxExecutionTime uint64) // CommitTxn commits the transaction operations. CommitTxn(ctx context.Context) error diff --git a/server/driver_tidb.go b/server/driver_tidb.go index 0d5180f1318fe..5d34dbbc3d706 100644 --- a/server/driver_tidb.go +++ b/server/driver_tidb.go @@ -212,8 +212,8 @@ func (tc *TiDBContext) CommitTxn(ctx context.Context) error { } // SetProcessInfo implements QueryCtx SetProcessInfo method. -func (tc *TiDBContext) SetProcessInfo(sql string, t time.Time, command byte) { - tc.session.SetProcessInfo(sql, t, command) +func (tc *TiDBContext) SetProcessInfo(sql string, t time.Time, command byte, maxExecutionTime uint64) { + tc.session.SetProcessInfo(sql, t, command, maxExecutionTime) } // RollbackTxn implements QueryCtx RollbackTxn method. diff --git a/server/server.go b/server/server.go index c258f074d0ed0..bcd0e83bf4baa 100644 --- a/server/server.go +++ b/server/server.go @@ -84,12 +84,13 @@ func init() { } var ( - errUnknownFieldType = terror.ClassServer.New(codeUnknownFieldType, "unknown field type") - errInvalidPayloadLen = terror.ClassServer.New(codeInvalidPayloadLen, "invalid payload length") - errInvalidSequence = terror.ClassServer.New(codeInvalidSequence, "invalid sequence") - errInvalidType = terror.ClassServer.New(codeInvalidType, "invalid type") - errNotAllowedCommand = terror.ClassServer.New(codeNotAllowedCommand, "the used command is not allowed with this TiDB version") - errAccessDenied = terror.ClassServer.New(codeAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDenied]) + errUnknownFieldType = terror.ClassServer.New(codeUnknownFieldType, "unknown field type") + errInvalidPayloadLen = terror.ClassServer.New(codeInvalidPayloadLen, "invalid payload length") + errInvalidSequence = terror.ClassServer.New(codeInvalidSequence, "invalid sequence") + errInvalidType = terror.ClassServer.New(codeInvalidType, "invalid type") + errNotAllowedCommand = terror.ClassServer.New(codeNotAllowedCommand, "the used command is not allowed with this TiDB version") + errAccessDenied = terror.ClassServer.New(codeAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDenied]) + errMaxExecTimeExceeded = terror.ClassServer.New(codeMaxExecTimeExceeded, mysql.MySQLErrName[mysql.ErrMaxExecTimeExceeded]) ) // DefaultCapability is the capability of the server when it is created using the default configuration. @@ -107,7 +108,7 @@ type Server struct { driver IDriver listener net.Listener socket net.Listener - rwlock *sync.RWMutex + rwlock sync.RWMutex concurrentLimiter *TokenLimiter clients map[uint32]*clientConn capability uint32 @@ -199,7 +200,6 @@ func NewServer(cfg *config.Config, driver IDriver) (*Server, error) { cfg: cfg, driver: driver, concurrentLimiter: NewTokenLimiter(cfg.TokenLimit), - rwlock: &sync.RWMutex{}, clients: make(map[uint32]*clientConn), stopListenerCh: make(chan struct{}, 1), } @@ -618,14 +618,16 @@ const ( codeInvalidSequence = 3 codeInvalidType = 4 - codeNotAllowedCommand = 1148 - codeAccessDenied = mysql.ErrAccessDenied + codeNotAllowedCommand = 1148 + codeAccessDenied = mysql.ErrAccessDenied + codeMaxExecTimeExceeded = mysql.ErrMaxExecTimeExceeded ) func init() { serverMySQLErrCodes := map[terror.ErrCode]uint16{ - codeNotAllowedCommand: mysql.ErrNotAllowedCommand, - codeAccessDenied: mysql.ErrAccessDenied, + codeNotAllowedCommand: mysql.ErrNotAllowedCommand, + codeAccessDenied: mysql.ErrAccessDenied, + codeMaxExecTimeExceeded: mysql.ErrMaxExecTimeExceeded, } terror.ErrClassToMySQLCodes[terror.ClassServer] = serverMySQLErrCodes } diff --git a/session/session.go b/session/session.go index 38c95182a3046..a3985f0e3ee09 100644 --- a/session/session.go +++ b/session/session.go @@ -116,7 +116,7 @@ type Session interface { SetClientCapability(uint32) // Set client capability flags. SetConnectionID(uint64) SetCommandValue(byte) - SetProcessInfo(string, time.Time, byte) + SetProcessInfo(string, time.Time, byte, uint64) SetTLSState(*tls.ConnectionState) SetCollation(coID int) error SetSessionManager(util.SessionManager) @@ -780,6 +780,10 @@ func createSessionFunc(store kv.Storage) pools.Factory { if err != nil { return nil, err } + err = variable.SetSessionSystemVar(se.sessionVars, variable.MaxExecutionTime, types.NewUintDatum(0)) + if err != nil { + return nil, errors.Trace(err) + } se.sessionVars.CommonGlobalLoaded = true se.sessionVars.InRestrictedSQL = true return se, nil @@ -796,6 +800,10 @@ func createSessionWithDomainFunc(store kv.Storage) func(*domain.Domain) (pools.R if err != nil { return nil, err } + err = variable.SetSessionSystemVar(se.sessionVars, variable.MaxExecutionTime, types.NewUintDatum(0)) + if err != nil { + return nil, errors.Trace(err) + } se.sessionVars.CommonGlobalLoaded = true se.sessionVars.InRestrictedSQL = true return se, nil @@ -907,18 +915,19 @@ func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) return s.parser.Parse(sql, charset, collation) } -func (s *session) SetProcessInfo(sql string, t time.Time, command byte) { +func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecutionTime uint64) { pi := util.ProcessInfo{ - ID: s.sessionVars.ConnectionID, - DB: s.sessionVars.CurrentDB, - Command: command, - Plan: s.currentPlan, - Time: t, - State: s.Status(), - Info: sql, - CurTxnStartTS: s.sessionVars.TxnCtx.StartTS, - StmtCtx: s.sessionVars.StmtCtx, - StatsInfo: plannercore.GetStatsInfo, + ID: s.sessionVars.ConnectionID, + DB: s.sessionVars.CurrentDB, + Command: command, + Plan: s.currentPlan, + Time: t, + State: s.Status(), + Info: sql, + CurTxnStartTS: s.sessionVars.TxnCtx.StartTS, + StmtCtx: s.sessionVars.StmtCtx, + StatsInfo: plannercore.GetStatsInfo, + MaxExecutionTime: maxExecutionTime, } if s.sessionVars.User != nil { pi.User = s.sessionVars.User.Username diff --git a/session/session_test.go b/session/session_test.go index 19e8d88ecc91a..bc8897daa5b0f 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2640,6 +2640,32 @@ func (s *testSessionSuite) TestTxnGoString(c *C) { c.Assert(fmt.Sprintf("%#v", txn), Equals, "Txn{state=invalid}") } +func (s *testSessionSuite) TestMaxExeucteTime(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + + tk.MustExec("create table MaxExecTime( id int,name varchar(128),age int);") + tk.MustExec("begin") + tk.MustExec("insert into MaxExecTime (id,name,age) values (1,'john',18),(2,'lary',19),(3,'lily',18);") + + tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) + tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("0")) + tk.MustQuery("select /*+ MAX_EXECUTION_TIME(1000) */ * FROM MaxExecTime;") + + tk.MustExec("set @@global.MAX_EXECUTION_TIME = 300;") + tk.MustQuery("select * FROM MaxExecTime;") + + tk.MustExec("set @@MAX_EXECUTION_TIME = 150;") + tk.MustQuery("select * FROM MaxExecTime;") + + tk.MustQuery("select @@global.MAX_EXECUTION_TIME;").Check(testkit.Rows("300")) + tk.MustQuery("select @@MAX_EXECUTION_TIME;").Check(testkit.Rows("150")) + + tk.MustExec("set @@global.MAX_EXECUTION_TIME = 0;") + tk.MustExec("set @@MAX_EXECUTION_TIME = 0;") + tk.MustExec("commit") + tk.MustExec("drop table if exists MaxExecTime;") +} + func (s *testSessionSuite) TestGrantViewRelated(c *C) { tkRoot := testkit.NewTestKitWithInit(c, s.store) tkUser := testkit.NewTestKitWithInit(c, s.store) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 83d060a814f93..c4c228f3b864a 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -111,7 +111,7 @@ type TransactionContext struct { TableDeltaMap map[int64]TableDelta IsPessimistic bool - // For metrics. + // CreateTime For metrics. CreateTime time.Time StatementCount int } @@ -205,15 +205,14 @@ type SessionVars struct { PreparedStmtNameToID map[string]uint32 // preparedStmtID is id of prepared statement. preparedStmtID uint32 - // params for prepared statements + // PreparedParams params for prepared statements PreparedParams []types.Datum // ActiveRoles stores active roles for current user ActiveRoles []*auth.RoleIdentity - // retry information RetryInfo *RetryInfo - // Should be reset on transaction finished. + // TxnCtx Should be reset on transaction finished. TxnCtx *TransactionContext // KVVars is the variables for KV storage. @@ -221,9 +220,9 @@ type SessionVars struct { // TxnIsolationLevelOneShot is used to implements "set transaction isolation level ..." TxnIsolationLevelOneShot struct { - // state 0 means default - // state 1 means it's set in current transaction. - // state 2 means it should be used in current transaction. + // State 0 means default + // State 1 means it's set in current transaction. + // State 2 means it should be used in current transaction. State int Value string } @@ -364,7 +363,7 @@ type SessionVars struct { // CommandValue indicates which command current session is doing. CommandValue uint32 - // TIDBOptJoinOrderAlgoThreshold defines the minimal number of join nodes + // TiDBOptJoinReorderThreshold defines the minimal number of join nodes // to use the greedy join reorder algorithm. TiDBOptJoinReorderThreshold int @@ -380,6 +379,11 @@ type SessionVars struct { // LowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds. LowResolutionTSO bool + // MaxExecutionTime is the timeout for select statement, in milliseconds. + // If the value is 0, timeouts are not enabled. + // See https://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_max_execution_time + MaxExecutionTime uint64 + // Killed is a flag to indicate that this query is killed. Killed uint32 } @@ -691,6 +695,9 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { if isAutocommit { s.SetStatusFlag(mysql.ServerStatusInTrans, false) } + case MaxExecutionTime: + timeoutMS := tidbOptPositiveInt32(val, 0) + s.MaxExecutionTime = uint64(timeoutMS) case TiDBSkipUTF8Check: s.SkipUTF8Check = TiDBOptOn(val) case TiDBOptAggPushDown: @@ -845,6 +852,7 @@ const ( TxnIsolation = "tx_isolation" TransactionIsolation = "transaction_isolation" TxnIsolationOneShot = "tx_isolation_one_shot" + MaxExecutionTime = "max_execution_time" ) // these variables are useless for TiDB, but still need to validate their values for some compatible issues. @@ -891,7 +899,7 @@ type Concurrency struct { // HashAggPartialConcurrency is the number of concurrent hash aggregation partial worker. HashAggPartialConcurrency int - // HashAggPartialConcurrency is the number of concurrent hash aggregation final worker. + // HashAggFinalConcurrency is the number of concurrent hash aggregation final worker. HashAggFinalConcurrency int // IndexSerialScanConcurrency is the number of concurrent index serial scan worker. diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index fdbc116d2c3ef..de5d21406e25e 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -174,7 +174,8 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal, "innodb_max_undo_log_size", ""}, {ScopeGlobal | ScopeSession, "range_alloc_block_size", "4096"}, {ScopeGlobal, ConnectTimeout, "10"}, - {ScopeGlobal | ScopeSession, "collation_server", mysql.DefaultCollationName}, + {ScopeGlobal | ScopeSession, MaxExecutionTime, "0"}, + {ScopeGlobal | ScopeSession, CollationServer, mysql.DefaultCollationName}, {ScopeNone, "have_rtree_keys", "YES"}, {ScopeGlobal, "innodb_old_blocks_pct", "37"}, {ScopeGlobal, "innodb_file_format", "Antelope"}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 5df9769b845ef..e13d60778402e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -348,9 +348,10 @@ var ( MaxDDLReorgBatchSize int32 = 10240 MinDDLReorgBatchSize int32 = 32 // DDLSlowOprThreshold is the threshold for ddl slow operations, uint is millisecond. - DDLSlowOprThreshold uint32 = DefTiDBDDLSlowOprThreshold - ForcePriority = int32(DefTiDBForcePriority) - ServerHostname, _ = os.Hostname() - MaxOfMaxAllowedPacket uint64 = 1073741824 - ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold + DDLSlowOprThreshold uint32 = DefTiDBDDLSlowOprThreshold + ForcePriority = int32(DefTiDBForcePriority) + ServerHostname, _ = os.Hostname() + MaxOfMaxAllowedPacket uint64 = 1073741824 + ExpensiveQueryTimeThreshold uint64 = DefTiDBExpensiveQueryTimeThreshold + MinExpensiveQueryTimeThreshold uint64 = 10 //10s ) diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 316766287b3a1..46d0757e70896 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -424,6 +424,8 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return value, nil } return value, ErrWrongValueForVar.GenWithStackByArgs(name, value) + case MaxExecutionTime: + return checkUInt64SystemVar(name, value, 0, math.MaxUint64, vars) case TiDBEnableTablePartition: switch { case strings.EqualFold(value, "ON") || value == "1": @@ -438,6 +440,8 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return checkUInt64SystemVar(name, value, uint64(MinDDLReorgBatchSize), uint64(MaxDDLReorgBatchSize), vars) case TiDBDDLErrorCountLimit: return checkUInt64SystemVar(name, value, uint64(0), math.MaxInt64, vars) + case TiDBExpensiveQueryTimeThreshold: + return checkUInt64SystemVar(name, value, MinExpensiveQueryTimeThreshold, math.MaxInt64, vars) case TiDBIndexLookupConcurrency, TiDBIndexLookupJoinConcurrency, TiDBIndexJoinBatchSize, TiDBIndexLookupSize, TiDBHashJoinConcurrency, @@ -446,7 +450,7 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, TiDBDistSQLScanConcurrency, TiDBIndexSerialScanConcurrency, TiDBDDLReorgWorkerCount, TiDBBackoffLockFast, TiDBBackOffWeight, - TiDBDMLBatchSize, TiDBOptimizerSelectivityLevel, TiDBExpensiveQueryTimeThreshold: + TiDBDMLBatchSize, TiDBOptimizerSelectivityLevel: v, err := strconv.Atoi(value) if err != nil { return value, ErrWrongTypeForVar.GenWithStackByArgs(name) diff --git a/tidb-server/main.go b/tidb-server/main.go index 8356c5fc4392f..8d267cbe6d37b 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" - "github.com/pingcap/pd/client" + pd "github.com/pingcap/pd/client" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" diff --git a/util/expensivequery/expensivequery.go b/util/expensivequery/expensivequery.go index 2689eac6092a8..92120e0eeadb5 100644 --- a/util/expensivequery/expensivequery.go +++ b/util/expensivequery/expensivequery.go @@ -18,6 +18,7 @@ import ( "fmt" "strconv" "strings" + "sync" "sync/atomic" "time" @@ -31,6 +32,7 @@ import ( // Handle is the handler for expensive query. type Handle struct { + mu sync.RWMutex exitCh chan struct{} sm util.SessionManager } @@ -50,36 +52,38 @@ func (eqh *Handle) SetSessionManager(sm util.SessionManager) *Handle { // Run starts a expensive query checker goroutine at the start time of the server. func (eqh *Handle) Run() { threshold := atomic.LoadUint64(&variable.ExpensiveQueryTimeThreshold) - curInterval := time.Second * time.Duration(threshold) - ticker := time.NewTicker(curInterval / 2) + // use 100ms as tickInterval temply, may use given interval or use defined variable later + tickInterval := time.Millisecond * time.Duration(100) + ticker := time.NewTicker(tickInterval) for { select { case <-ticker.C: - if log.GetLevel() > zapcore.WarnLevel { - continue - } processInfo := eqh.sm.ShowProcessList() for _, info := range processInfo { if len(info.Info) == 0 || info.ExceedExpensiveTimeThresh { continue } - if costTime := time.Since(info.Time); costTime >= curInterval { + costTime := time.Since(info.Time) + if costTime >= time.Second*time.Duration(threshold) && log.GetLevel() <= zapcore.WarnLevel { logExpensiveQuery(costTime, info) info.ExceedExpensiveTimeThresh = true + + } else if info.MaxExecutionTime > 0 && costTime > time.Duration(info.MaxExecutionTime)*time.Millisecond { + eqh.sm.Kill(info.ID, true) } } threshold = atomic.LoadUint64(&variable.ExpensiveQueryTimeThreshold) - if newInterval := time.Second * time.Duration(threshold); curInterval != newInterval { - curInterval = newInterval - ticker.Stop() - ticker = time.NewTicker(curInterval / 2) - } case <-eqh.exitCh: return } } } +// Close closes the handle and release the background goroutine. +func (eqh *Handle) Close() { + close(eqh.exitCh) +} + // LogOnQueryExceedMemQuota prints a log when memory usage of connID is out of memory quota. func (eqh *Handle) LogOnQueryExceedMemQuota(connID uint64) { if log.GetLevel() > zapcore.WarnLevel { diff --git a/util/processinfo.go b/util/processinfo.go index 956196cdfdb42..deaf3cab08cde 100644 --- a/util/processinfo.go +++ b/util/processinfo.go @@ -36,6 +36,9 @@ type ProcessInfo struct { StmtCtx *stmtctx.StatementContext StatsInfo func(interface{}) map[string]uint64 ExceedExpensiveTimeThresh bool + // MaxExecutionTime is the timeout for select statement, in milliseconds. + // If the query takes too long, kill it. + MaxExecutionTime uint64 } // ToRow returns []interface{} for the row data of "show processlist" and "select * from infoschema.processlist". From c02c9ec9a1f3c0d654289cd69437f05b9e5b7d54 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 2 Jul 2019 11:55:30 +0800 Subject: [PATCH 2/5] session,parser: make MAX_EXECUTION_TIME sql hint and global variable work --- go.mod | 4 ++-- go.sum | 8 ++++---- session/session_test.go | 7 +++++++ 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/go.mod b/go.mod index f277f82c30489..4154cb830aeb7 100644 --- a/go.mod +++ b/go.mod @@ -43,8 +43,8 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190619024611-a4759dfe3753 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190613045206-37cc370a20a4 - github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 + github.com/pingcap/parser v0.0.0-20190627064259-e9c42442aa72 + github.com/pingcap/pd v0.0.0-20190617100349-293d4b5189bf github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 github.com/prometheus/client_golang v0.9.0 diff --git a/go.sum b/go.sum index 2658354e7ee2a..df37cf75d5326 100644 --- a/go.sum +++ b/go.sum @@ -166,10 +166,10 @@ github.com/pingcap/kvproto v0.0.0-20190619024611-a4759dfe3753/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190613045206-37cc370a20a4 h1:r5BvCTM1R9U9EjJntFREb67GMsgn8IK9vLTQ/HzRZBc= -github.com/pingcap/parser v0.0.0-20190613045206-37cc370a20a4/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= -github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= -github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= +github.com/pingcap/parser v0.0.0-20190627064259-e9c42442aa72 h1:yPYZB/rPuTost36QcoOLzvkDxVjqfJ9w0xTpohwlSlA= +github.com/pingcap/parser v0.0.0-20190627064259-e9c42442aa72/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/pd v0.0.0-20190617100349-293d4b5189bf h1:vmlN6DpZI5LtHd8r9YRAsyCeTU2pxRq+WlWn5CZ+ax4= +github.com/pingcap/pd v0.0.0-20190617100349-293d4b5189bf/go.mod h1:3DlDlFT7EF64A1bmb/tulZb6wbPSagm5G4p1AlhaEDs= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 h1:rRMLMjIMFulCX9sGKZ1hoov/iROMsKyC8Snc02nSukw= diff --git a/session/session_test.go b/session/session_test.go index bc8897daa5b0f..90619107515fd 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -451,6 +451,13 @@ func (s *testSessionSuite) TestGlobalVarAccessor(c *C) { c.Assert(err, IsNil) c.Assert(v, Equals, varValue2) + // For issue 10955, make sure the new session load `max_execution_time` into sessionVars. + s.dom.GetGlobalVarsCache().Disable() + tk1.MustExec("set @@global.max_execution_time = 100") + tk2 := testkit.NewTestKitWithInit(c, s.store) + c.Assert(tk2.Se.GetSessionVars().MaxExecutionTime, Equals, uint64(100)) + tk1.MustExec("set @@global.max_execution_time = 0") + result := tk.MustQuery("show global variables where variable_name='sql_select_limit';") result.Check(testkit.Rows("sql_select_limit 18446744073709551615")) result = tk.MustQuery("show session variables where variable_name='sql_select_limit';") From 7f411bc213dc7fd3f6267d2806ea92c2b2d9c84b Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 2 Jul 2019 15:00:10 +0800 Subject: [PATCH 3/5] fix CI --- go.mod | 6 +++--- go.sum | 15 ++++++++------- server/conn_test.go | 2 +- session/session.go | 1 + sessionctx/variable/sysvar.go | 2 +- 5 files changed, 14 insertions(+), 12 deletions(-) diff --git a/go.mod b/go.mod index 4154cb830aeb7..9238365eab576 100644 --- a/go.mod +++ b/go.mod @@ -39,12 +39,12 @@ require ( github.com/opentracing/opentracing-go v1.0.2 github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8 github.com/pingcap/errors v0.11.4 - github.com/pingcap/failpoint v0.0.0-20190422094118-d8535965f59b + github.com/pingcap/failpoint v0.0.0-20190512135322-30cc7431d99c github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190619024611-a4759dfe3753 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190627064259-e9c42442aa72 - github.com/pingcap/pd v0.0.0-20190617100349-293d4b5189bf + github.com/pingcap/parser v0.0.0-20190702054329-5922bb20961d + github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 github.com/prometheus/client_golang v0.9.0 diff --git a/go.sum b/go.sum index df37cf75d5326..1fc8072b842d4 100644 --- a/go.sum +++ b/go.sum @@ -15,7 +15,6 @@ github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx2 github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20171208011716-f6d7a1f6fbf3/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/client9/misspell v0.3.4 h1:ta993UF76GwbvJcIo3Y68y/M3WxlpEHPWIGDkJYwzJI= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd h1:qMd81Ts1T2OTKmB4acZcyKaMtRnY5Y44NuXGX2GFJ1w= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= @@ -154,8 +153,8 @@ github.com/pingcap/errors v0.10.1/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTw github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/failpoint v0.0.0-20190422094118-d8535965f59b h1:gFQVlQbthX4C2WKV/zqGBF3bZFr7oceKK9jGOVNkfws= -github.com/pingcap/failpoint v0.0.0-20190422094118-d8535965f59b/go.mod h1:fdAkVXuIXHAPZ7a280nj9bRORfK9NuSsOguvBH0+W6c= +github.com/pingcap/failpoint v0.0.0-20190512135322-30cc7431d99c h1:hvQd3aOLKLF7xvRV6DzvPkKY4QXzfVbjU1BhW0d9yL8= +github.com/pingcap/failpoint v0.0.0-20190512135322-30cc7431d99c/go.mod h1:DNS3Qg7bEDhU6EXNHF+XSv/PGznQaMJ5FWvctpm6pQI= github.com/pingcap/gofail v0.0.0-20181217135706-6a951c1e42c3 h1:04yuCf5NMvLU8rB2m4Qs3rynH7EYpMno3lHkewIOdMo= github.com/pingcap/gofail v0.0.0-20181217135706-6a951c1e42c3/go.mod h1:DazNTg0PTldtpsQiT9I5tVJwV1onHMKBBgXzmJUlMns= github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e h1:P73/4dPCL96rGrobssy1nVy2VaVpNCuLpCbr+FEaTA8= @@ -166,10 +165,10 @@ github.com/pingcap/kvproto v0.0.0-20190619024611-a4759dfe3753/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190627064259-e9c42442aa72 h1:yPYZB/rPuTost36QcoOLzvkDxVjqfJ9w0xTpohwlSlA= -github.com/pingcap/parser v0.0.0-20190627064259-e9c42442aa72/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= -github.com/pingcap/pd v0.0.0-20190617100349-293d4b5189bf h1:vmlN6DpZI5LtHd8r9YRAsyCeTU2pxRq+WlWn5CZ+ax4= -github.com/pingcap/pd v0.0.0-20190617100349-293d4b5189bf/go.mod h1:3DlDlFT7EF64A1bmb/tulZb6wbPSagm5G4p1AlhaEDs= +github.com/pingcap/parser v0.0.0-20190702054329-5922bb20961d h1:/+VW8Dhnq2X6rg02KQCUMVAl5aEE8DslNbJZZiN0phM= +github.com/pingcap/parser v0.0.0-20190702054329-5922bb20961d/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= +github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 h1:rRMLMjIMFulCX9sGKZ1hoov/iROMsKyC8Snc02nSukw= @@ -194,6 +193,8 @@ github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d h1:GoAlyOgbOEIFd github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/remyoudompheng/bigfft v0.0.0-20170806203942-52369c62f446 h1:/NRJ5vAYoqz+7sG51ubIDHXeWO8DlTSrToPu6q11ziA= github.com/remyoudompheng/bigfft v0.0.0-20170806203942-52369c62f446/go.mod h1:uYEyJGbgTkfkS4+E/PavXkNJcbFIpEtjt2B0KDQ5+9M= +github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= +github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.18.10+incompatible h1:cy84jW6EVRPa5g9HAHrlbxMSIjBhDSX0OFYyMYminYs= github.com/shirou/gopsutil v2.18.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shurcooL/httpfs v0.0.0-20171119174359-809beceb2371 h1:SWV2fHctRpRrp49VXJ6UZja7gU9QLHwRpIPBN89SKEo= diff --git a/server/conn_test.go b/server/conn_test.go index 132a91553d4a8..e1c6fe812cebe 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -19,7 +19,6 @@ import ( "context" "encoding/binary" "fmt" - "io" . "github.com/pingcap/check" "github.com/pingcap/failpoint" @@ -28,6 +27,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/util/arena" ) type ConnTestSuite struct { diff --git a/session/session.go b/session/session.go index a3985f0e3ee09..92c6ca0351424 100644 --- a/session/session.go +++ b/session/session.go @@ -1632,6 +1632,7 @@ var builtinGlobalVariable = []string{ variable.WaitTimeout, variable.InteractiveTimeout, variable.MaxPreparedStmtCount, + variable.MaxExecutionTime, /* TiDB specific global variables: */ variable.TiDBSkipUTF8Check, variable.TiDBIndexJoinBatchSize, diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index de5d21406e25e..92f34448b36b7 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -175,7 +175,7 @@ var defaultSysVars = []*SysVar{ {ScopeGlobal | ScopeSession, "range_alloc_block_size", "4096"}, {ScopeGlobal, ConnectTimeout, "10"}, {ScopeGlobal | ScopeSession, MaxExecutionTime, "0"}, - {ScopeGlobal | ScopeSession, CollationServer, mysql.DefaultCollationName}, + {ScopeGlobal | ScopeSession, "collation_server", mysql.DefaultCollationName}, {ScopeNone, "have_rtree_keys", "YES"}, {ScopeGlobal, "innodb_old_blocks_pct", "37"}, {ScopeGlobal, "innodb_file_format", "Antelope"}, From 027a328116676f974ef57e0c4646cea68cf1b16f Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 3 Jul 2019 14:58:09 +0800 Subject: [PATCH 4/5] fix CI --- server/conn_test.go | 14 +++++++++----- session/session.go | 2 +- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/server/conn_test.go b/server/conn_test.go index e1c6fe812cebe..12bf3949416fb 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -19,6 +19,7 @@ import ( "context" "encoding/binary" "fmt" + "time" . "github.com/pingcap/check" "github.com/pingcap/failpoint" @@ -263,17 +264,20 @@ func (ts ConnTestSuite) TestConnExecutionTimeout(c *C) { _, err = se.Execute(context.Background(), "set @@max_execution_time = 500;") c.Assert(err, IsNil) - err = cc.handleQuery(context.Background(), "select * FROM testTable2 WHERE SLEEP(1);") - c.Assert(err, NotNil) + now := time.Now() + err = cc.handleQuery(context.Background(), "select * FROM testTable2 WHERE SLEEP(3);") + c.Assert(time.Since(now) < 3*time.Second, IsTrue) _, err = se.Execute(context.Background(), "set @@max_execution_time = 0;") c.Assert(err, IsNil) + now = time.Now() err = cc.handleQuery(context.Background(), "select * FROM testTable2 WHERE SLEEP(1);") - c.Assert(err, IsNil) + c.Assert(time.Since(now) > 500*time.Millisecond, IsTrue) - err = cc.handleQuery(context.Background(), "select /*+ MAX_EXECUTION_TIME(100)*/ * FROM testTable2 WHERE SLEEP(1);") - c.Assert(err, NotNil) + now = time.Now() + err = cc.handleQuery(context.Background(), "select /*+ MAX_EXECUTION_TIME(100)*/ * FROM testTable2 WHERE SLEEP(3);") + c.Assert(time.Since(now) < 3*time.Second, IsTrue) c.Assert(failpoint.Disable("github.com/pingcap/tidb/server/FakeClientConn"), IsNil) } diff --git a/session/session.go b/session/session.go index 6bb7d3df1a89d..593db28078e88 100644 --- a/session/session.go +++ b/session/session.go @@ -915,7 +915,7 @@ func (s *session) ParseSQL(ctx context.Context, sql, charset, collation string) return s.parser.Parse(sql, charset, collation) } -func (s *session) SetProcessInfo(sql string, t time.Time, command byte) { +func (s *session) SetProcessInfo(sql string, t time.Time, command byte, maxExecutionTime uint64) { var db interface{} if len(s.sessionVars.CurrentDB) > 0 { db = s.sessionVars.CurrentDB From 90512044b98ed921755ddc55b1be2802bf781844 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Wed, 3 Jul 2019 15:13:31 +0800 Subject: [PATCH 5/5] make golint happy --- server/conn_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/conn_test.go b/server/conn_test.go index 12bf3949416fb..572e7140ae32b 100644 --- a/server/conn_test.go +++ b/server/conn_test.go @@ -266,6 +266,7 @@ func (ts ConnTestSuite) TestConnExecutionTimeout(c *C) { now := time.Now() err = cc.handleQuery(context.Background(), "select * FROM testTable2 WHERE SLEEP(3);") + c.Assert(err, IsNil) c.Assert(time.Since(now) < 3*time.Second, IsTrue) _, err = se.Execute(context.Background(), "set @@max_execution_time = 0;") @@ -273,10 +274,12 @@ func (ts ConnTestSuite) TestConnExecutionTimeout(c *C) { now = time.Now() err = cc.handleQuery(context.Background(), "select * FROM testTable2 WHERE SLEEP(1);") + c.Assert(err, IsNil) c.Assert(time.Since(now) > 500*time.Millisecond, IsTrue) now = time.Now() err = cc.handleQuery(context.Background(), "select /*+ MAX_EXECUTION_TIME(100)*/ * FROM testTable2 WHERE SLEEP(3);") + c.Assert(err, IsNil) c.Assert(time.Since(now) < 3*time.Second, IsTrue) c.Assert(failpoint.Disable("github.com/pingcap/tidb/server/FakeClientConn"), IsNil)