Skip to content

Commit

Permalink
resource_control: Separate the mark for watch and rule and enhance te…
Browse files Browse the repository at this point in the history
…sting (#52197)

ref #43691
  • Loading branch information
CabinfeverB authored Apr 22, 2024
1 parent cfcc770 commit 62f3aea
Show file tree
Hide file tree
Showing 4 changed files with 63 additions and 21 deletions.
4 changes: 4 additions & 0 deletions pkg/ddl/tests/resourcegroup/resource_group_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -327,6 +327,10 @@ func TestResourceGroupRunaway(t *testing.T) {
"rg2 select /*+ resource_group(rg2) */ * from t watch"), maxWaitDuration, tryInterval)
tk.MustQuery("select SQL_NO_CACHE resource_group_name, watch_text from mysql.tidb_runaway_watch").
Check(testkit.Rows("rg2 select /*+ resource_group(rg2) */ * from t"))
// wait for the runaway watch to be cleaned up
tk.EventuallyMustQueryAndCheck("select SQL_NO_CACHE resource_group_name, watch_text from mysql.tidb_runaway_watch", nil, testkit.Rows(), maxWaitDuration, tryInterval)
err = tk.QueryToErr("select /*+ resource_group(rg2) */ * from t")
require.ErrorContains(t, err, "Query execution was interrupted, identified as runaway query")

tk.EventuallyMustQueryAndCheck("select SQL_NO_CACHE resource_group_name, original_sql, time from mysql.tidb_runaway_queries", nil,
nil, maxWaitDuration, tryInterval)
Expand Down
42 changes: 28 additions & 14 deletions pkg/domain/resourcegroup/runaway.go
Original file line number Diff line number Diff line change
Expand Up @@ -468,7 +468,9 @@ type RunawayChecker struct {
deadline time.Time
setting *rmpb.RunawaySettings

marked atomic.Bool
markedByRule atomic.Bool
markedByWatch bool
watchAction rmpb.RunawayAction
}

func newRunawayChecker(manager *RunawayManager, resourceGroupName string, setting *rmpb.RunawaySettings, originalSQL, sqlDigest, planDigest string) *RunawayChecker {
Expand All @@ -479,7 +481,8 @@ func newRunawayChecker(manager *RunawayManager, resourceGroupName string, settin
sqlDigest: sqlDigest,
planDigest: planDigest,
setting: setting,
marked: atomic.Bool{},
markedByRule: atomic.Bool{},
markedByWatch: false,
}
if setting != nil {
c.deadline = time.Now().Add(time.Duration(setting.Rule.ExecElapsedTimeMs) * time.Millisecond)
Expand All @@ -498,15 +501,16 @@ func (r *RunawayChecker) BeforeExecutor() error {
if action == rmpb.RunawayAction_NoneAction && r.setting != nil {
action = r.setting.Action
}
if r.marked.CompareAndSwap(false, true) {
now := time.Now()
r.markRunaway(RunawayMatchTypeWatch, action, &now)
}
r.markedByWatch = true
now := time.Now()
r.watchAction = action
r.markRunaway(RunawayMatchTypeWatch, action, &now)
// If no match action, it will do nothing.
switch action {
case rmpb.RunawayAction_Kill:
return exeerrors.ErrResourceGroupQueryRunawayQuarantine
case rmpb.RunawayAction_CoolDown:
// This action should be done in BeforeCopRequest.
return nil
case rmpb.RunawayAction_DryRun:
return nil
Expand All @@ -519,12 +523,18 @@ func (r *RunawayChecker) BeforeExecutor() error {

// BeforeCopRequest checks runaway and modifies the request if necessary before sending coprocessor request.
func (r *RunawayChecker) BeforeCopRequest(req *tikvrpc.Request) error {
if r.setting == nil {
if r.setting == nil && !r.markedByWatch {
return nil
}
marked := r.marked.Load()
marked := r.markedByRule.Load()
if !marked {
// note: now we don't check whether query is in watch list again.
if r.markedByWatch {
if r.watchAction == rmpb.RunawayAction_CoolDown {
req.ResourceControlContext.OverridePriority = 1 // set priority to lowest
}
}

now := time.Now()
until := r.deadline.Sub(now)
if until > 0 {
Expand All @@ -537,9 +547,11 @@ func (r *RunawayChecker) BeforeCopRequest(req *tikvrpc.Request) error {
return nil
}
// execution time exceeds the threshold, mark the query as runaway
if r.marked.CompareAndSwap(false, true) {
if r.markedByRule.CompareAndSwap(false, true) {
r.markRunaway(RunawayMatchTypeIdentify, r.setting.Action, &now)
r.markQuarantine(&now)
if !r.markedByWatch {
r.markQuarantine(&now)
}
}
}
switch r.setting.Action {
Expand All @@ -561,16 +573,18 @@ func (r *RunawayChecker) CheckCopRespError(err error) error {
return err
}
if strings.HasPrefix(err.Error(), "Coprocessor task terminated due to exceeding the deadline") {
if !r.marked.Load() {
if !r.markedByRule.Load() {
now := time.Now()
if r.deadline.Before(now) && r.marked.CompareAndSwap(false, true) {
if r.deadline.Before(now) && r.markedByRule.CompareAndSwap(false, true) {
r.markRunaway(RunawayMatchTypeIdentify, r.setting.Action, &now)
r.markQuarantine(&now)
if !r.markedByWatch {
r.markQuarantine(&now)
}
return exeerrors.ErrResourceGroupQueryRunawayInterrupted
}
}
// Due to concurrency, check again.
if r.marked.Load() {
if r.markedByRule.Load() {
return exeerrors.ErrResourceGroupQueryRunawayInterrupted
}
}
Expand Down
1 change: 1 addition & 0 deletions pkg/executor/internal/querywatch/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ go_test(
"//pkg/meta/autoid",
"//pkg/testkit",
"//pkg/testkit/testsetup",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//tikv",
"@org_uber_go_goleak//:goleak",
Expand Down
37 changes: 30 additions & 7 deletions pkg/executor/internal/querywatch/query_watch_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,11 @@ package querywatch_test

import (
"context"
"fmt"
"testing"
"time"

"github.com/pingcap/failpoint"
mysql "github.com/pingcap/tidb/pkg/errno"
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/testkit"
Expand Down Expand Up @@ -54,11 +56,12 @@ func TestQueryWatch(t *testing.T) {
testkit.Rows("default select * from test.t1 1 1", "default select * from test.t2 2 1"), maxWaitDuration, tryInterval)

tk.MustExec("create resource group rg1 RU_PER_SEC=1000 QUERY_LIMIT=(EXEC_ELAPSED='50ms' ACTION=KILL)")
tk.MustExec("create resource group rg2 RU_PER_SEC=1000 QUERY_LIMIT=(EXEC_ELAPSED='50ms' ACTION=KILL)")

tk.MustQuery("query watch add resource group rg1 sql text exact to 'select * from test.t1'").Check(testkit.Rows("3"))
tk.MustQuery("query watch add resource group rg1 sql text similar to 'select * from test.t2'").Check(testkit.Rows("4"))
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnPrivilege)
tk.MustQueryWithContext(ctx, "query watch add resource group rg1 sql text plan to 'select * from test.t3'").Check(testkit.Rows("5"))
tk.MustQueryWithContext(ctx, "query watch add resource group rg1 action DRYRUN sql text plan to 'select * from test.t3'").Check(testkit.Rows("5"))

tk.MustQuery("query watch add action KILL SQL DIGEST '4ea0618129ffc6a7effbc0eff4bbcb41a7f5d4c53a6fa0b2e9be81c7010915b0'").Check(testkit.Rows("6"))
tk.MustQuery("query watch add action KILL PLAN DIGEST 'd08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57'").Check(testkit.Rows("7"))
Expand All @@ -68,48 +71,67 @@ func TestQueryWatch(t *testing.T) {
"default select * from test.t2 2 1",
"rg1 select * from test.t1 3 1",
"rg1 02576c15e1f35a8aa3eb7e3b1f977c9f9f9921a22421b3e9f42bad5ab632b4f6 3 2",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 3 3",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 1 3",
"default 4ea0618129ffc6a7effbc0eff4bbcb41a7f5d4c53a6fa0b2e9be81c7010915b0 3 2",
"default d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 3 3",
), maxWaitDuration, tryInterval)
tk.MustQuery("query watch add action COOLDOWN sql text similar to 'select * from test.t1'").Check(testkit.Rows("8"))
tk.MustQueryWithContext(ctx, "query watch add resource group rg2 action KILL sql text plan to 'select * from test.t3'").Check(testkit.Rows("9"))

tk.EventuallyMustQueryAndCheck("select SQL_NO_CACHE resource_group_name, watch_text, action, watch from mysql.tidb_runaway_watch order by id", nil,
testkit.Rows("default select * from test.t1 1 1",
"default select * from test.t2 2 1",
"rg1 select * from test.t1 3 1",
"rg1 02576c15e1f35a8aa3eb7e3b1f977c9f9f9921a22421b3e9f42bad5ab632b4f6 3 2",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 3 3",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 1 3",
"default d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 3 3",
"default 4ea0618129ffc6a7effbc0eff4bbcb41a7f5d4c53a6fa0b2e9be81c7010915b0 2 2",
"rg2 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 3 3",
), maxWaitDuration, tryInterval)

tk.EventuallyMustQueryAndCheck("select SQL_NO_CACHE resource_group_name, watch_text, action, watch from information_schema.runaway_watches order by id", nil,
testkit.Rows("default select * from test.t1 DryRun Exact",
"default select * from test.t2 CoolDown Exact",
"rg1 select * from test.t1 Kill Exact",
"rg1 02576c15e1f35a8aa3eb7e3b1f977c9f9f9921a22421b3e9f42bad5ab632b4f6 Kill Similar",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 DryRun Plan",
"default d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
"default 4ea0618129ffc6a7effbc0eff4bbcb41a7f5d4c53a6fa0b2e9be81c7010915b0 CoolDown Similar",
"rg2 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
), maxWaitDuration, tryInterval)

// avoid the default resource group to be recorded.
tk.MustExec("alter resource group default QUERY_LIMIT=(EXEC_ELAPSED='1000ms' ACTION=DRYRUN)")

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/store/copr/sleepCoprRequest", fmt.Sprintf("return(%d)", 60)))
err = tk.QueryToErr("select /*+ resource_group(rg1) */ * from t3")
require.ErrorContains(t, err, "[executor:8253]Query execution was interrupted, identified as runaway query")
tk.EventuallyMustQueryAndCheck("select SQL_NO_CACHE resource_group_name, original_sql, match_type from mysql.tidb_runaway_queries", nil,
testkit.Rows(
"rg1 select /*+ resource_group(rg1) */ * from t3 watch",
"rg1 select /*+ resource_group(rg1) */ * from t3 identify",
), maxWaitDuration, tryInterval)
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/copr/sleepCoprRequest"))

tk.MustExec("alter resource group default QUERY_LIMIT=(EXEC_ELAPSED='50ms' ACTION=DRYRUN)")
tk.MustGetErrCode("select * from t3", mysql.ErrResourceGroupQueryRunawayQuarantine)
tk.MustQuery("select * from t2").Check(testkit.Rows("1"))
tk.MustQuery("select /*+ resource_group(rg1) */ * from t1").Check(testkit.Rows("1"))
tk.MustExec("SET RESOURCE GROUP rg1")
// hit and schema will affect sql digest
tk.MustGetErrCode("select * from test.t2", mysql.ErrResourceGroupQueryRunawayQuarantine)
tk.MustGetErrCode("select /*+ resource_group(rg1) */ * from t3", mysql.ErrResourceGroupQueryRunawayQuarantine)
tk.MustGetErrCode("select /*+ resource_group(rg2) */ * from t3", mysql.ErrResourceGroupQueryRunawayQuarantine)

tk.MustExec("alter resource group rg1 RU_PER_SEC=1000 QUERY_LIMIT=()")
tk.EventuallyMustQueryAndCheck("select SQL_NO_CACHE resource_group_name, watch_text, action, watch from information_schema.runaway_watches order by id", nil,
testkit.Rows("default select * from test.t1 DryRun Exact",
"default select * from test.t2 CoolDown Exact",
"rg1 select * from test.t1 Kill Exact",
"rg1 02576c15e1f35a8aa3eb7e3b1f977c9f9f9921a22421b3e9f42bad5ab632b4f6 Kill Similar",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 DryRun Plan",
"default d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
"default 4ea0618129ffc6a7effbc0eff4bbcb41a7f5d4c53a6fa0b2e9be81c7010915b0 CoolDown Similar",
"rg2 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
), maxWaitDuration, tryInterval)

tk.MustExec("alter resource group rg1 RU_PER_SEC=1000 QUERY_LIMIT=(EXEC_ELAPSED='50ms' ACTION=KILL)")
Expand All @@ -118,9 +140,10 @@ func TestQueryWatch(t *testing.T) {
"default select * from test.t2 CoolDown Exact",
"rg1 select * from test.t1 Kill Exact",
"rg1 02576c15e1f35a8aa3eb7e3b1f977c9f9f9921a22421b3e9f42bad5ab632b4f6 Kill Similar",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
"rg1 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 DryRun Plan",
"default d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
"default 4ea0618129ffc6a7effbc0eff4bbcb41a7f5d4c53a6fa0b2e9be81c7010915b0 CoolDown Similar",
"rg2 d08bc323a934c39dc41948b0a073725be3398479b6fa4f6dd1db2a9b115f7f57 Kill Plan",
), maxWaitDuration, tryInterval)

// test remove
Expand Down

0 comments on commit 62f3aea

Please sign in to comment.