Skip to content

Commit

Permalink
Add support check for MAX_EXECUTION_TIME.
Browse files Browse the repository at this point in the history
- This feature is request by [Feature 7008](pingcap#7008).
- Add support for system variable "session.max_execution_time" and "global.max_execution_time".
- Support hint in  Select Statement linke: "SELECT /*+ MAX_EXECUTION_TIME(1000) */ * FROM t1".
- Add some unit test for the above(all tests passed).
  • Loading branch information
db-storage committed May 29, 2019
1 parent aabc00f commit 1643cc1
Show file tree
Hide file tree
Showing 6 changed files with 55 additions and 17 deletions.
2 changes: 1 addition & 1 deletion server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -1302,9 +1302,9 @@ func (cc *clientConn) writeChunks(ctx context.Context, rs ResultSet, binary bool
failpoint.Inject("SleepInwriteChunks", func(val failpoint.Value) {
time.Sleep(time.Duration(val.(int)) * time.Millisecond)
})
//MaxExecDuration() return 0 if there is no limit
if rs.MaxExecDuration().Nanoseconds() > 0 &&
time.Now().After(rs.StartExecTime().Add(rs.MaxExecDuration())) {
logutil.Logger(ctx).Warn("1907", zap.Int64("timeout:", rs.MaxExecDuration().Nanoseconds()))
return errors.New("Query execution was interrupted, max_execution_time exceeded")
}

Expand Down
10 changes: 10 additions & 0 deletions server/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,16 @@ func (ts ConnTestSuite) TestConnExecutionTimeout(c *C) {
_, err = se.Execute(context.Background(), "select * FROM tidb;")
c.Assert(err, IsNil)

_, err = se.Execute(context.Background(), "select * FROM tidb;")
c.Assert(err, IsNil)

_, err = se.Execute(context.Background(), "set @@max_execution_time = 100;")
c.Assert(err, IsNil)

//session's max_execution_time has been set before
err = cc.handleQuery(context.Background(), "select * FROM tidb;")
c.Assert(err.Error(), Equals, errors.New("Query execution was interrupted, max_execution_time exceeded").Error())

_, err = se.Execute(context.Background(), "set @@max_execution_time = 100;")
c.Assert(err, IsNil)

Expand Down
2 changes: 0 additions & 2 deletions server/driver.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,8 +141,6 @@ type ResultSet interface {
StoreFetchedRows(rows []chunk.Row)
GetFetchedRows() []chunk.Row
Close() error
// SetMaxExecDuration(time.Duration)
MaxExecDuration() time.Duration
//SetStartExecTime(time.Time)
StartExecTime() time.Time
}
7 changes: 3 additions & 4 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -964,11 +964,10 @@ func (s *session) Execute(ctx context.Context, sql string) (recordSets []sqlexec
}

const (
// TiDBMaxExecutionTime is currently used for selection Statement only
TiDBMaxExecutionTime = "max_execution_time"
// MaxExecutionTime is currently used for selection Statement only
MaxExecutionTime = "max_execution_time"
)

//exeutionHints contains only maxExecutionTime now, but there maybe more in the future
type exeutionHints struct {
maxExecutionTime uint64
}
Expand All @@ -977,7 +976,7 @@ func (s *session) getExecutionHints(hints []*ast.TableOptimizerHint) (execHints
execHints = &exeutionHints{maxExecutionTime: 0}
for _, hint := range hints {
switch hint.HintName.L {
case TiDBMaxExecutionTime:
case MaxExecutionTime:
execHints.maxExecutionTime = hint.MaxExecutionTime
default:
}
Expand Down
26 changes: 26 additions & 0 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2698,3 +2698,29 @@ func (s *testSessionSuite) TestGrantViewRelated(c *C) {
tkUser.MustQuery("select current_user();").Check(testkit.Rows("u_version29@%"))
tkUser.MustExec("create view v_version29_c as select * from v_version29;")
}

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;")
}
25 changes: 15 additions & 10 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ type TransactionContext struct {
TableDeltaMap map[int64]TableDelta
IsPessimistic bool

// For metrics.
// CreateTime For metrics.
CreateTime time.Time
StatementCount int
}
Expand Down Expand Up @@ -205,25 +205,24 @@ 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.
KVVars *kv.Variables

// 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
}
Expand Down Expand Up @@ -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

Expand All @@ -380,6 +379,8 @@ type SessionVars struct {
MaxExecutionTime uint64
// LowResolutionTSO is used for reading data with low resolution TSO which is updated once every two seconds.
LowResolutionTSO bool
// PessimisticLock indicates whether new transaction should be pessimistic .
PessimisticLock bool
}

// ConnectionInfo present connection used by audit.
Expand Down Expand Up @@ -690,7 +691,11 @@ func (s *SessionVars) SetSystemVar(name string, val string) error {
s.SetStatusFlag(mysql.ServerStatusInTrans, false)
}
case MaxExecutionTime:
s.MaxExecutionTime = uint64(tidbOptInt64(val, 0))
timeoutMS := tidbOptInt64(val, 0)
if timeoutMS < 0 {
timeoutMS = 0
}
s.MaxExecutionTime = uint64(timeoutMS)
case TiDBSkipUTF8Check:
s.SkipUTF8Check = TiDBOptOn(val)
case TiDBOptAggPushDown:
Expand Down Expand Up @@ -890,7 +895,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.
Expand Down

0 comments on commit 1643cc1

Please sign in to comment.