Skip to content

Commit

Permalink
*: format the error message when query/instance exceeds memory quota (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-chi-bot authored Aug 10, 2023
1 parent d22f9bc commit 8d62c87
Show file tree
Hide file tree
Showing 18 changed files with 68 additions and 43 deletions.
1 change: 1 addition & 0 deletions ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@ go_library(
"//util/hack",
"//util/logutil",
"//util/mathutil",
"//util/memory",
"//util/mock",
"//util/ranger",
"//util/resourcegrouptag",
Expand Down
3 changes: 2 additions & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ import (
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/set"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -2847,7 +2848,7 @@ func checkPartitionByHash(ctx sessionctx.Context, tbInfo *model.TableInfo) error
// checkPartitionByRange checks validity of a "BY RANGE" partition.
func checkPartitionByRange(ctx sessionctx.Context, tbInfo *model.TableInfo) error {
failpoint.Inject("CheckPartitionByRangeErr", func() {
panic("Out Of Memory Quota!")
panic(memory.PanicMemoryExceedWarnMsg)
})
pi := tbInfo.Partition

Expand Down
2 changes: 1 addition & 1 deletion executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -449,7 +449,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) {
}
return
}
if str, ok := r.(string); !ok || !strings.Contains(str, memory.PanicMemoryExceed) {
if str, ok := r.(string); !ok || !strings.Contains(str, memory.PanicMemoryExceedWarnMsg) {
panic(r)
}
err = errors.Errorf("%v", r)
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ func getAnalyzePanicErr(r interface{}) error {
if msg == globalPanicAnalyzeMemoryExceed {
return errAnalyzeOOM
}
if strings.Contains(msg, memory.PanicMemoryExceed) {
if strings.Contains(msg, memory.PanicMemoryExceedWarnMsg) {
return errors.Errorf(msg, errAnalyzeOOM)
}
}
Expand Down
1 change: 1 addition & 0 deletions executor/analyzetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ go_test(
"//types",
"//util",
"//util/codec",
"//util/memory",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
Expand Down
7 changes: 4 additions & 3 deletions executor/analyzetest/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/memory"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/testutils"
"github.com/tikv/client-go/v2/tikv"
Expand Down Expand Up @@ -3189,7 +3190,7 @@ func TestGlobalMemoryControlForAnalyze(t *testing.T) {
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/memory/ReadMemStats", `return(536870912)`))
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/mockAnalyzeMergeWorkerSlowConsume", `return(100)`))
_, err := tk0.Exec(sql)
require.True(t, strings.Contains(err.Error(), "Out Of Memory Quota!"))
require.True(t, strings.Contains(err.Error(), memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForInstance))
runtime.GC()
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/memory/ReadMemStats"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeMergeWorkerSlowConsume"))
Expand Down Expand Up @@ -3230,7 +3231,7 @@ func TestGlobalMemoryControlForPrepareAnalyze(t *testing.T) {
_, err0 := tk0.Exec(sqlPrepare)
require.NoError(t, err0)
_, err1 := tk0.Exec(sqlExecute)
require.True(t, strings.Contains(err1.Error(), "Out Of Memory Quota!"))
require.True(t, strings.Contains(err1.Error(), "Your query has been cancelled due to exceeding the allowed memory limit for the tidb-server instance and this query is currently using the most memory."))
runtime.GC()
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/memory/ReadMemStats"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/mockAnalyzeMergeWorkerSlowConsume"))
Expand Down Expand Up @@ -3312,7 +3313,7 @@ func TestGlobalMemoryControlForAutoAnalyze(t *testing.T) {
h.HandleAutoAnalyze(dom.InfoSchema())
rs := tk.MustQuery("select fail_reason from mysql.analyze_jobs where table_name=? and state=? limit 1", "t", "failed")
failReason := rs.Rows()[0][0].(string)
require.True(t, strings.Contains(failReason, "Out Of Memory Quota!"))
require.True(t, strings.Contains(failReason, memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForInstance))

childTrackers = executor.GlobalAnalyzeMemoryTracker.GetChildrenForTest()
require.Len(t, childTrackers, 0)
Expand Down
2 changes: 1 addition & 1 deletion executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (_ *ExecS
if r == nil {
return
}
if str, ok := r.(string); !ok || !strings.Contains(str, memory.PanicMemoryExceed) {
if str, ok := r.(string); !ok || !strings.Contains(str, memory.PanicMemoryExceedWarnMsg) {
panic(r)
}
err = errors.Errorf("%v", r)
Expand Down
30 changes: 15 additions & 15 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -3542,42 +3542,42 @@ func TestOOMPanicAction(t *testing.T) {
tk.MustExec("set @@tidb_mem_quota_query=1;")
err := tk.QueryToErr("select sum(b) from t group by a;")
require.Error(t, err)
require.Regexp(t, "Out Of Memory Quota!.*", err.Error())
require.Regexp(t, memory.PanicMemoryExceedWarnMsg, err.Error())

// Test insert from select oom panic.
tk.MustExec("drop table if exists t,t1")
tk.MustExec("create table t (a bigint);")
tk.MustExec("create table t1 (a bigint);")
tk.MustExec("set @@tidb_mem_quota_query=200;")
tk.MustMatchErrMsg("insert into t1 values (1),(2),(3),(4),(5);", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("replace into t1 values (1),(2),(3),(4),(5);", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("insert into t1 values (1),(2),(3),(4),(5);", memory.PanicMemoryExceedWarnMsg)
tk.MustMatchErrMsg("replace into t1 values (1),(2),(3),(4),(5);", memory.PanicMemoryExceedWarnMsg)
tk.MustExec("set @@tidb_mem_quota_query=10000")
tk.MustExec("insert into t1 values (1),(2),(3),(4),(5);")
tk.MustExec("set @@tidb_mem_quota_query=10;")
tk.MustMatchErrMsg("insert into t select a from t1 order by a desc;", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("replace into t select a from t1 order by a desc;", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("insert into t select a from t1 order by a desc;", memory.PanicMemoryExceedWarnMsg)
tk.MustMatchErrMsg("replace into t select a from t1 order by a desc;", memory.PanicMemoryExceedWarnMsg)

tk.MustExec("set @@tidb_mem_quota_query=10000")
tk.MustExec("insert into t values (1),(2),(3),(4),(5);")
// Set the memory quota to 244 to make this SQL panic during the DeleteExec
// instead of the TableReaderExec.
tk.MustExec("set @@tidb_mem_quota_query=244;")
tk.MustMatchErrMsg("delete from t", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("delete from t", memory.PanicMemoryExceedWarnMsg)

tk.MustExec("set @@tidb_mem_quota_query=10000;")
tk.MustExec("delete from t1")
tk.MustExec("insert into t1 values(1)")
tk.MustExec("insert into t values (1),(2),(3),(4),(5);")
tk.MustExec("set @@tidb_mem_quota_query=244;")
tk.MustMatchErrMsg("delete t, t1 from t join t1 on t.a = t1.a", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("delete t, t1 from t join t1 on t.a = t1.a", memory.PanicMemoryExceedWarnMsg)

tk.MustExec("set @@tidb_mem_quota_query=100000;")
tk.MustExec("truncate table t")
tk.MustExec("insert into t values(1),(2),(3)")
// set the memory to quota to make the SQL panic during UpdateExec instead
// of TableReader.
tk.MustExec("set @@tidb_mem_quota_query=244;")
tk.MustMatchErrMsg("update t set a = 4", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("update t set a = 4", memory.PanicMemoryExceedWarnMsg)
}

func TestPointGetPreparedPlan(t *testing.T) {
Expand Down Expand Up @@ -5972,7 +5972,7 @@ func TestSummaryFailedUpdate(t *testing.T) {
tk.MustExec("SET GLOBAL tidb_mem_oom_action='CANCEL'")
require.NoError(t, tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil))
tk.MustExec("set @@tidb_mem_quota_query=1")
tk.MustMatchErrMsg("update t set t.a = t.a - 1 where t.a in (select a from t where a < 4)", "Out Of Memory Quota!.*")
tk.MustMatchErrMsg("update t set t.a = t.a - 1 where t.a in (select a from t where a < 4)", memory.PanicMemoryExceedWarnMsg)
tk.MustExec("set @@tidb_mem_quota_query=1000000000")
tk.MustQuery("select stmt_type from information_schema.statements_summary where digest_text = 'update `t` set `t` . `a` = `t` . `a` - ? where `t` . `a` in ( select `a` from `t` where `a` < ? )'").Check(testkit.Rows("Update"))
}
Expand Down Expand Up @@ -6137,7 +6137,7 @@ func TestGlobalMemoryControl(t *testing.T) {
func() {
tracker3.Consume(1)
}, func(r interface{}) {
require.True(t, strings.Contains(r.(string), "Out Of Memory Quota!"))
require.True(t, strings.Contains(r.(string), memory.PanicMemoryExceedWarnMsg))
})
tracker2.Consume(300 << 20) // Sum 500MB, Not Panic, Waiting t3 cancel finish.
time.Sleep(500 * time.Millisecond)
Expand All @@ -6156,7 +6156,7 @@ func TestGlobalMemoryControl(t *testing.T) {
func() {
tracker2.Consume(1)
}, func(r interface{}) {
require.True(t, strings.Contains(r.(string), "Out Of Memory Quota!"))
require.True(t, strings.Contains(r.(string), memory.PanicMemoryExceedWarnMsg))
})
require.Equal(t, test[0], 0) // Keep 1GB HeapInUse
}
Expand Down Expand Up @@ -6191,7 +6191,7 @@ func TestGlobalMemoryControl2(t *testing.T) {
wg.Done()
}()
sql := "select * from t t1 join t t2 join t t3 on t1.a=t2.a and t1.a=t3.a order by t1.a;" // Need 500MB
require.True(t, strings.Contains(tk0.QueryToErr(sql).Error(), "Out Of Memory Quota!"))
require.True(t, strings.Contains(tk0.QueryToErr(sql).Error(), memory.PanicMemoryExceedWarnMsg))
require.Equal(t, tk0.Session().GetSessionVars().DiskTracker.MaxConsumed(), int64(0))
wg.Wait()
test[0] = 0
Expand All @@ -6210,7 +6210,7 @@ func TestCompileOutOfMemoryQuota(t *testing.T) {
tk.MustExec("create table t1(a int, c int, index idx(a))")
tk.MustExec("set tidb_mem_quota_query=10")
err := tk.ExecToErr("select t.a, t1.a from t use index(idx), t1 use index(idx) where t.a = t1.a")
require.Contains(t, err.Error(), "Out Of Memory Quota!")
require.Contains(t, err.Error(), memory.PanicMemoryExceedWarnMsg)
}

func TestSignalCheckpointForSort(t *testing.T) {
Expand All @@ -6236,7 +6236,7 @@ func TestSignalCheckpointForSort(t *testing.T) {
tk.Session().GetSessionVars().ConnectionID = 123456

err := tk.QueryToErr("select * from t order by a")
require.Contains(t, err.Error(), "Out Of Memory Quota!")
require.Contains(t, err.Error(), memory.PanicMemoryExceedWarnMsg)
}

func TestSessionRootTrackerDetach(t *testing.T) {
Expand All @@ -6248,7 +6248,7 @@ func TestSessionRootTrackerDetach(t *testing.T) {
tk.MustExec("create table t(a int, b int, index idx(a))")
tk.MustExec("create table t1(a int, c int, index idx(a))")
tk.MustExec("set tidb_mem_quota_query=10")
tk.MustContainErrMsg("select /*+hash_join(t1)*/ t.a, t1.a from t use index(idx), t1 use index(idx) where t.a = t1.a", "Out Of Memory Quota!")
tk.MustContainErrMsg("select /*+hash_join(t1)*/ t.a, t1.a from t use index(idx), t1 use index(idx) where t.a = t1.a", memory.PanicMemoryExceedWarnMsg)
tk.MustExec("set tidb_mem_quota_query=1000")
rs, err := tk.Exec("select /*+hash_join(t1)*/ t.a, t1.a from t use index(idx), t1 use index(idx) where t.a = t1.a")
require.NoError(t, err)
Expand Down
3 changes: 2 additions & 1 deletion executor/index_merge_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/testutil"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/memory"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -796,7 +797,7 @@ func TestIntersectionMemQuota(t *testing.T) {
defer tk.MustExec("set global tidb_mem_oom_action = DEFAULT")
tk.MustExec("set @@tidb_mem_quota_query = 4000")
err := tk.QueryToErr("select /*+ use_index_merge(t1, primary, idx1, idx2) */ c1 from t1 where c1 < 1024 and c2 < 1024")
require.Contains(t, err.Error(), "Out Of Memory Quota!")
require.Contains(t, err.Error(), memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery)
}

func setupPartitionTableHelper(tk *testkit.TestKit) {
Expand Down
8 changes: 4 additions & 4 deletions executor/issuetest/executor_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -301,9 +301,9 @@ func TestIssue28650(t *testing.T) {
tk.MustExec("set @@tidb_mem_quota_query = 1073741824") // 1GB
require.Nil(t, tk.QueryToErr(sql))
tk.MustExec("set @@tidb_mem_quota_query = 33554432") // 32MB, out of memory during executing
require.True(t, strings.Contains(tk.QueryToErr(sql).Error(), "Out Of Memory Quota!"))
require.True(t, strings.Contains(tk.QueryToErr(sql).Error(), memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery))
tk.MustExec("set @@tidb_mem_quota_query = 65536") // 64KB, out of memory during building the plan
require.True(t, strings.Contains(tk.ExecToErr(sql).Error(), "Out Of Memory Quota!"))
require.True(t, strings.Contains(tk.ExecToErr(sql).Error(), memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery))
}
}

Expand Down Expand Up @@ -483,10 +483,10 @@ func TestIndexJoin31494(t *testing.T) {
for i := 0; i < 10; i++ {
err := tk.QueryToErr("select /*+ inl_join(t1) */ * from t1 right join t2 on t1.b=t2.b;")
require.Error(t, err)
require.Regexp(t, "Out Of Memory Quota!.*", err.Error())
require.Regexp(t, memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery, err.Error())
err = tk.QueryToErr("select /*+ inl_hash_join(t1) */ * from t1 right join t2 on t1.b=t2.b;")
require.Error(t, err)
require.Regexp(t, "Out Of Memory Quota!.*", err.Error())
require.Regexp(t, memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery, err.Error())
}
}

Expand Down
11 changes: 6 additions & 5 deletions executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ import (
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/testdata"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/memory"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -2307,14 +2308,14 @@ func TestIssue18070(t *testing.T) {
tk.MustExec("insert into t1 values(1),(2)")
tk.MustExec("insert into t2 values(1),(1),(2),(2)")
tk.MustExec("set @@tidb_mem_quota_query=1000")
tk.MustContainErrMsg("select /*+ inl_hash_join(t1)*/ * from t1 join t2 on t1.a = t2.a;", "Out Of Memory Quota!")
tk.MustContainErrMsg("select /*+ inl_hash_join(t1)*/ * from t1 join t2 on t1.a = t2.a;", memory.PanicMemoryExceedWarnMsg)

fpName := "github.com/pingcap/tidb/executor/mockIndexMergeJoinOOMPanic"
require.NoError(t, failpoint.Enable(fpName, `panic("ERROR 1105 (HY000): Out Of Memory Quota![conn_id=1]")`))
defer func() {
require.NoError(t, failpoint.Disable(fpName))
}()
tk.MustContainErrMsg("select /*+ inl_merge_join(t1)*/ * from t1 join t2 on t1.a = t2.a;", "Out Of Memory Quota!")
tk.MustContainErrMsg("select /*+ inl_merge_join(t1)*/ * from t1 join t2 on t1.a = t2.a;", memory.PanicMemoryExceedWarnMsg)
}

func TestIssue18564(t *testing.T) {
Expand Down Expand Up @@ -2714,9 +2715,9 @@ func TestIssue30211(t *testing.T) {
tk.MustExec("SET GLOBAL tidb_mem_oom_action = 'CANCEL'")
defer tk.MustExec("SET GLOBAL tidb_mem_oom_action='LOG'")
err := tk.QueryToErr("select /*+ inl_join(t1) */ * from t1 join t2 on t1.a = t2.a;").Error()
require.True(t, strings.Contains(err, "Out Of Memory Quota"))
require.True(t, strings.Contains(err, memory.PanicMemoryExceedWarnMsg))
err = tk.QueryToErr("select /*+ inl_hash_join(t1) */ * from t1 join t2 on t1.a = t2.a;").Error()
require.True(t, strings.Contains(err, "Out Of Memory Quota"))
require.True(t, strings.Contains(err, memory.PanicMemoryExceedWarnMsg))
}

func TestIssue31129(t *testing.T) {
Expand Down Expand Up @@ -2907,7 +2908,7 @@ func TestCartesianJoinPanic(t *testing.T) {
}
err := tk.QueryToErr("desc analyze select * from t t1, t t2, t t3, t t4, t t5, t t6;")
require.NotNil(t, err)
require.True(t, strings.Contains(err.Error(), "Out Of Memory Quota!"))
require.True(t, strings.Contains(err.Error(), memory.PanicMemoryExceedWarnMsg))
}

func TestTiDBNAAJ(t *testing.T) {
Expand Down
1 change: 1 addition & 0 deletions executor/tiflashtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_test(
"//testkit",
"//testkit/external",
"//util/israce",
"//util/memory",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_kvproto//pkg/metapb",
Expand Down
3 changes: 2 additions & 1 deletion executor/tiflashtest/tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/testkit/external"
"github.com/pingcap/tidb/util/israce"
"github.com/pingcap/tidb/util/memory"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/testutils"
)
Expand Down Expand Up @@ -1277,5 +1278,5 @@ func TestMPPMemoryTracker(t *testing.T) {
}()
err = tk.QueryToErr("select * from t")
require.NotNil(t, err)
require.True(t, strings.Contains(err.Error(), "Out Of Memory Quota!"))
require.True(t, strings.Contains(err.Error(), memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery))
}
3 changes: 2 additions & 1 deletion infoschema/cluster_tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ import (
"github.com/pingcap/tidb/store/mockstore/mockstorage"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/pdapi"
"github.com/pingcap/tidb/util/resourcegrouptag"
"github.com/pingcap/tidb/util/set"
Expand Down Expand Up @@ -696,7 +697,7 @@ select * from t1;

err = tk.QueryToErr("select * from `information_schema`.`slow_query` where time > '2022-04-14 00:00:00' and time < '2022-04-15 00:00:00'")
require.Error(t, err, quota)
require.Contains(t, err.Error(), "Out Of Memory Quota!", quota)
require.Contains(t, err.Error(), memory.PanicMemoryExceedWarnMsg, quota)
}
memQuotas := []int{128, 512, 1024, 2048, 4096}
for _, quota := range memQuotas {
Expand Down
2 changes: 1 addition & 1 deletion server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -2267,7 +2267,7 @@ func (cc *clientConn) writeResultset(ctx context.Context, rs ResultSet, binary b
if r == nil {
return
}
if str, ok := r.(string); !ok || !strings.HasPrefix(str, memory.PanicMemoryExceed) {
if str, ok := r.(string); !ok || !strings.HasPrefix(str, memory.PanicMemoryExceedWarnMsg) {
panic(r)
}
// TODO(jianzhang.zj: add metrics here)
Expand Down
4 changes: 2 additions & 2 deletions session/session_test/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1787,7 +1787,7 @@ func TestCoprocessorOOMAction(t *testing.T) {
tk.MustExec(fmt.Sprintf("set @@tidb_mem_quota_query=%v;", quota))
err := tk.QueryToErr(sql)
require.Error(t, err)
require.Regexp(t, "Out Of Memory Quota.*", err)
require.Regexp(t, memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery, err)
}

require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/copr/testRateLimitActionMockWaitMax", `return(true)`))
Expand Down Expand Up @@ -1834,7 +1834,7 @@ func TestCoprocessorOOMAction(t *testing.T) {
tk.MustExec("set @@tidb_mem_quota_query=1;")
err = tk.QueryToErr(testcase.sql)
require.Error(t, err)
require.Regexp(t, "Out Of Memory Quota.*", err)
require.Regexp(t, memory.PanicMemoryExceedWarnMsg+memory.WarnMsgSuffixForSingleQuery, err)
se.Close()
}
}
Expand Down
Loading

0 comments on commit 8d62c87

Please sign in to comment.