diff --git a/executor/adapter.go b/executor/adapter.go index 42ce555c6b3c3..6edacf1c56443 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" +<<<<<<< HEAD:executor/adapter.go "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl/placement" @@ -69,6 +70,50 @@ import ( "github.com/pingcap/tidb/util/topsql" topsqlstate "github.com/pingcap/tidb/util/topsql/state" "github.com/pingcap/tidb/util/tracing" +======= + "github.com/pingcap/tidb/pkg/bindinfo" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/ddl/placement" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/executor/internal/exec" + executor_metrics "github.com/pingcap/tidb/pkg/executor/metrics" + "github.com/pingcap/tidb/pkg/expression" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/keyspace" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/metrics" + "github.com/pingcap/tidb/pkg/parser" + "github.com/pingcap/tidb/pkg/parser/ast" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/planner" + plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/plugin" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessionctx/sessionstates" + "github.com/pingcap/tidb/pkg/sessionctx/stmtctx" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/sessiontxn/staleread" + "github.com/pingcap/tidb/pkg/types" + util2 "github.com/pingcap/tidb/pkg/util" + "github.com/pingcap/tidb/pkg/util/breakpoint" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" + "github.com/pingcap/tidb/pkg/util/execdetails" + "github.com/pingcap/tidb/pkg/util/hint" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/pingcap/tidb/pkg/util/plancodec" + "github.com/pingcap/tidb/pkg/util/replayer" + "github.com/pingcap/tidb/pkg/util/sqlexec" + "github.com/pingcap/tidb/pkg/util/stmtsummary" + stmtsummaryv2 "github.com/pingcap/tidb/pkg/util/stmtsummary/v2" + "github.com/pingcap/tidb/pkg/util/stringutil" + "github.com/pingcap/tidb/pkg/util/topsql" + topsqlstate "github.com/pingcap/tidb/pkg/util/topsql/state" + "github.com/pingcap/tidb/pkg/util/tracing" +>>>>>>> 33480e8c8d8 (*: add last ru consumption for tidb_last_query_info (#49769)):pkg/executor/adapter.go "github.com/prometheus/client_golang/prometheus" tikverr "github.com/tikv/client-go/v2/error" "github.com/tikv/client-go/v2/oracle" @@ -1361,7 +1406,7 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo } } sessVars.PrevStmt = FormatSQL(a.GetTextToLog(false)) - + a.recordLastQueryInfo(err) a.observePhaseDurations(sessVars.InRestrictedSQL, execDetail.CommitDetail) executeDuration := time.Since(sessVars.StartTime) - sessVars.DurationCompile if sessVars.InRestrictedSQL { @@ -1405,6 +1450,38 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo } } +func (a *ExecStmt) recordLastQueryInfo(err error) { + sessVars := a.Ctx.GetSessionVars() + // Record diagnostic information for DML statements + recordLastQuery := false + switch typ := a.StmtNode.(type) { + case *ast.ShowStmt: + recordLastQuery = typ.Tp != ast.ShowSessionStates + case *ast.ExecuteStmt, ast.DMLNode: + recordLastQuery = true + } + if recordLastQuery { + var lastRUConsumption float64 + if ruDetailRaw := a.GoCtx.Value(util.RUDetailsCtxKey); ruDetailRaw != nil { + ruDetail := ruDetailRaw.(*util.RUDetails) + lastRUConsumption = ruDetail.RRU() + ruDetail.WRU() + } + failpoint.Inject("mockRUConsumption", func(_ failpoint.Value) { + lastRUConsumption = float64(len(sessVars.StmtCtx.OriginalSQL)) + }) + // Keep the previous queryInfo for `show session_states` because the statement needs to encode it. + sessVars.LastQueryInfo = sessionstates.QueryInfo{ + TxnScope: sessVars.CheckAndGetTxnScope(), + StartTS: sessVars.TxnCtx.StartTS, + ForUpdateTS: sessVars.TxnCtx.GetForUpdateTS(), + LastRUConsumption: lastRUConsumption, + } + if err != nil { + sessVars.LastQueryInfo.ErrMsg = err.Error() + } + } +} + func (a *ExecStmt) checkPlanReplayerCapture(txnTS uint64) { if kv.GetInternalSourceType(a.GoCtx) == kv.InternalTxnStats { return diff --git a/pkg/session/test/variable/BUILD.bazel b/pkg/session/test/variable/BUILD.bazel new file mode 100644 index 0000000000000..380a6dff7e4b8 --- /dev/null +++ b/pkg/session/test/variable/BUILD.bazel @@ -0,0 +1,28 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_test") + +go_test( + name = "variable_test", + timeout = "short", + srcs = [ + "main_test.go", + "variable_test.go", + ], + flaky = True, + shard_count = 10, + deps = [ + "//pkg/config", + "//pkg/kv", + "//pkg/session", + "//pkg/sessionctx/variable", + "//pkg/store/copr", + "//pkg/testkit", + "//pkg/testkit/testmain", + "//pkg/testkit/testsetup", + "//pkg/util/dbterror/exeerrors", + "//pkg/util/memory", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_stretchr_testify//require", + "@com_github_tikv_client_go_v2//tikv", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/pkg/session/test/variable/variable_test.go b/pkg/session/test/variable/variable_test.go new file mode 100644 index 0000000000000..84354537e445e --- /dev/null +++ b/pkg/session/test/variable/variable_test.go @@ -0,0 +1,386 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package variable + +import ( + "context" + "fmt" + "strings" + "testing" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/session" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/store/copr" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util/dbterror/exeerrors" + "github.com/pingcap/tidb/pkg/util/memory" + "github.com/stretchr/testify/require" +) + +func TestForbidSettingBothTSVariable(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + // For mock tikv, safe point is not initialized, we manually insert it for snapshot to use. + safePointName := "tikv_gc_safe_point" + safePointValue := "20060102-15:04:05 -0700" + safePointComment := "All versions after safe point can be accessed. (DO NOT EDIT)" + updateSafePoint := fmt.Sprintf(`INSERT INTO mysql.tidb VALUES ('%[1]s', '%[2]s', '%[3]s') + ON DUPLICATE KEY + UPDATE variable_value = '%[2]s', comment = '%[3]s'`, safePointName, safePointValue, safePointComment) + tk.MustExec(updateSafePoint) + + // Set tidb_snapshot and assert tidb_read_staleness + tk.MustExec("set @@tidb_snapshot = '2007-01-01 15:04:05.999999'") + tk.MustGetErrMsg("set @@tidb_read_staleness='-5'", "tidb_snapshot should be clear before setting tidb_read_staleness") + tk.MustExec("set @@tidb_snapshot = ''") + tk.MustExec("set @@tidb_read_staleness='-5'") + + // Set tidb_read_staleness and assert tidb_snapshot + tk.MustExec("set @@tidb_read_staleness='-5'") + tk.MustGetErrMsg("set @@tidb_snapshot = '2007-01-01 15:04:05.999999'", "tidb_read_staleness should be clear before setting tidb_snapshot") + tk.MustExec("set @@tidb_read_staleness = ''") + tk.MustExec("set @@tidb_snapshot = '2007-01-01 15:04:05.999999'") +} + +func TestCoprocessorOOMAction(t *testing.T) { + // Assert Coprocessor OOMAction + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("set @@tidb_enable_rate_limit_action=true") + tk.MustExec("create database testoom") + tk.MustExec("use testoom") + tk.MustExec(`set @@tidb_wait_split_region_finish=1`) + // create table for non keep-order case + tk.MustExec("drop table if exists t5") + tk.MustExec("create table t5(id int)") + tk.MustQuery(`split table t5 between (0) and (10000) regions 10`).Check(testkit.Rows("9 1")) + // create table for keep-order case + tk.MustExec("drop table if exists t6") + tk.MustExec("create table t6(id int, index(id))") + tk.MustQuery(`split table t6 between (0) and (10000) regions 10`).Check(testkit.Rows("10 1")) + tk.MustQuery("split table t6 INDEX id between (0) and (10000) regions 10;").Check(testkit.Rows("10 1")) + count := 10 + for i := 0; i < count; i++ { + tk.MustExec(fmt.Sprintf("insert into t5 (id) values (%v)", i)) + tk.MustExec(fmt.Sprintf("insert into t6 (id) values (%v)", i)) + } + + testcases := []struct { + name string + sql string + }{ + { + name: "keep Order", + sql: "select id from t6 order by id", + }, + { + name: "non keep Order", + sql: "select id from t5", + }, + } + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockConsumeAndAssert", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockConsumeAndAssert")) + }() + + enableOOM := func(tk *testkit.TestKit, name, sql string) { + t.Logf("enable OOM, testcase: %v", name) + // larger than 4 copResponse, smaller than 5 copResponse + quota := 5*copr.MockResponseSizeForTest - 100 + defer tk.MustExec("SET GLOBAL tidb_mem_oom_action = DEFAULT") + tk.MustExec("SET GLOBAL tidb_mem_oom_action='CANCEL'") + tk.MustExec("use testoom") + tk.MustExec("set @@tidb_enable_rate_limit_action=1") + tk.MustExec("set @@tidb_distsql_scan_concurrency = 10") + tk.MustExec(fmt.Sprintf("set @@tidb_mem_quota_query=%v;", quota)) + var expect []string + for i := 0; i < count; i++ { + expect = append(expect, fmt.Sprintf("%v", i)) + } + tk.MustQuery(sql).Sort().Check(testkit.Rows(expect...)) + // assert oom action worked by max consumed > memory quota + require.Greater(t, tk.Session().GetSessionVars().StmtCtx.MemTracker.MaxConsumed(), int64(quota)) + } + + disableOOM := func(tk *testkit.TestKit, name, sql string) { + t.Logf("disable OOM, testcase: %v", name) + quota := 5*copr.MockResponseSizeForTest - 100 + tk.MustExec("use testoom") + tk.MustExec("set @@tidb_enable_rate_limit_action=0") + tk.MustExec("set @@tidb_distsql_scan_concurrency = 10") + tk.MustExec(fmt.Sprintf("set @@tidb_mem_quota_query=%v;", quota)) + err := tk.QueryToErr(sql) + require.Error(t, err) + require.True(t, exeerrors.ErrMemoryExceedForQuery.Equal(err)) + } + + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockWaitMax", `return(true)`)) + // assert oom action and switch + for _, testcase := range testcases { + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + tk.SetSession(se) + enableOOM(tk, testcase.name, testcase.sql) + tk.MustExec("set @@tidb_enable_rate_limit_action = 0") + disableOOM(tk, testcase.name, testcase.sql) + tk.MustExec("set @@tidb_enable_rate_limit_action = 1") + enableOOM(tk, testcase.name, testcase.sql) + se.Close() + } + globaltk := testkit.NewTestKit(t, store) + globaltk.MustExec("use testoom") + globaltk.MustExec("set global tidb_enable_rate_limit_action= 0") + for _, testcase := range testcases { + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + tk.SetSession(se) + disableOOM(tk, testcase.name, testcase.sql) + se.Close() + } + globaltk.MustExec("set global tidb_enable_rate_limit_action= 1") + for _, testcase := range testcases { + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + tk.SetSession(se) + enableOOM(tk, testcase.name, testcase.sql) + se.Close() + } + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/copr/testRateLimitActionMockWaitMax")) + + // assert oom fallback + for _, testcase := range testcases { + t.Log(testcase.name) + se, err := session.CreateSession4Test(store) + require.NoError(t, err) + tk.SetSession(se) + tk.MustExec("use testoom") + tk.MustExec("set tidb_distsql_scan_concurrency = 1") + tk.MustExec("set @@tidb_mem_quota_query=1;") + err = tk.QueryToErr(testcase.sql) + require.Error(t, err) + require.True(t, exeerrors.ErrMemoryExceedForQuery.Equal(err)) + se.Close() + } +} + +func TestStatementCountLimit(t *testing.T) { + store := testkit.CreateMockStore(t) + setTxnTk := testkit.NewTestKit(t, store) + setTxnTk.MustExec("set global tidb_txn_mode=''") + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table stmt_count_limit (id int)") + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.Performance.StmtCountLimit = 3 + }) + tk.MustExec("set tidb_disable_txn_auto_retry = 0") + tk.MustExec("begin") + tk.MustExec("insert into stmt_count_limit values (1)") + tk.MustExec("insert into stmt_count_limit values (2)") + _, err := tk.Exec("insert into stmt_count_limit values (3)") + require.Error(t, err) + + // begin is counted into history but this one is not. + tk.MustExec("SET SESSION autocommit = false") + tk.MustExec("insert into stmt_count_limit values (1)") + tk.MustExec("insert into stmt_count_limit values (2)") + tk.MustExec("insert into stmt_count_limit values (3)") + _, err = tk.Exec("insert into stmt_count_limit values (4)") + require.Error(t, err) +} + +func TestCorrectScopeError(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeNone, Name: "sv_none", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal, Name: "sv_global", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeSession, Name: "sv_session", Value: "acdc"}) + variable.RegisterSysVar(&variable.SysVar{Scope: variable.ScopeGlobal | variable.ScopeSession, Name: "sv_both", Value: "acdc"}) + + // check set behavior + + // none + _, err := tk.Exec("SET sv_none='acdc'") + require.Equal(t, "[variable:1238]Variable 'sv_none' is a read only variable", err.Error()) + _, err = tk.Exec("SET GLOBAL sv_none='acdc'") + require.Equal(t, "[variable:1238]Variable 'sv_none' is a read only variable", err.Error()) + + // global + tk.MustExec("SET GLOBAL sv_global='acdc'") + _, err = tk.Exec("SET sv_global='acdc'") + require.Equal(t, "[variable:1229]Variable 'sv_global' is a GLOBAL variable and should be set with SET GLOBAL", err.Error()) + + // session + _, err = tk.Exec("SET GLOBAL sv_session='acdc'") + require.Equal(t, "[variable:1228]Variable 'sv_session' is a SESSION variable and can't be used with SET GLOBAL", err.Error()) + tk.MustExec("SET sv_session='acdc'") + + // both + tk.MustExec("SET GLOBAL sv_both='acdc'") + tk.MustExec("SET sv_both='acdc'") + + // unregister + variable.UnregisterSysVar("sv_none") + variable.UnregisterSysVar("sv_global") + variable.UnregisterSysVar("sv_session") + variable.UnregisterSysVar("sv_both") +} + +func TestReadDMLBatchSize(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("set global tidb_dml_batch_size=1000") + se, err := session.CreateSession(store) + require.NoError(t, err) + + // `select 1` to load the global variables. + _, _ = se.Execute(context.TODO(), "select 1") + require.Equal(t, 1000, se.GetSessionVars().DMLBatchSize) +} + +func TestSetEnableRateLimitAction(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("set @@tidb_enable_rate_limit_action=true") + // assert default value + result := tk.MustQuery("select @@tidb_enable_rate_limit_action;") + result.Check(testkit.Rows("1")) + tk.MustExec("use test") + tk.MustExec("create table tmp123(id int)") + rs, err := tk.Exec("select * from tmp123;") + require.NoError(t, err) + haveRateLimitAction := false + action := tk.Session().GetSessionVars().MemTracker.GetFallbackForTest(false) + for ; action != nil; action = action.GetFallback() { + if action.GetPriority() == memory.DefRateLimitPriority { + haveRateLimitAction = true + break + } + } + require.True(t, haveRateLimitAction) + err = rs.Close() + require.NoError(t, err) + + // assert set sys variable + tk.MustExec("set global tidb_enable_rate_limit_action= '0';") + tk.Session().Close() + + tk.RefreshSession() + result = tk.MustQuery("select @@tidb_enable_rate_limit_action;") + result.Check(testkit.Rows("0")) + + haveRateLimitAction = false + action = tk.Session().GetSessionVars().MemTracker.GetFallbackForTest(false) + for ; action != nil; action = action.GetFallback() { + if action.GetPriority() == memory.DefRateLimitPriority { + haveRateLimitAction = true + break + } + } + require.False(t, haveRateLimitAction) +} + +func TestMaxExecutionTime(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + tk.MustExec("use test") + 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(1000) MAX_EXECUTION_TIME(500) */ * FROM MaxExecTime;") + require.Len(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings(), 1) + require.EqualError(t, tk.Session().GetSessionVars().StmtCtx.GetWarnings()[0].Err, "MAX_EXECUTION_TIME() is defined more than once, only the last definition takes effect: MAX_EXECUTION_TIME(500)") + require.True(t, tk.Session().GetSessionVars().StmtCtx.HasMaxExecutionTime) + require.Equal(t, uint64(500), tk.Session().GetSessionVars().StmtCtx.MaxExecutionTime) + + 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 TestReplicaRead(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + require.Equal(t, kv.ReplicaReadLeader, tk.Session().GetSessionVars().GetReplicaRead()) + tk.MustExec("set @@tidb_replica_read = 'follower';") + require.Equal(t, kv.ReplicaReadFollower, tk.Session().GetSessionVars().GetReplicaRead()) + tk.MustExec("set @@tidb_replica_read = 'leader';") + require.Equal(t, kv.ReplicaReadLeader, tk.Session().GetSessionVars().GetReplicaRead()) +} + +func TestIsolationRead(t *testing.T) { + store := testkit.CreateMockStore(t) + + tk := testkit.NewTestKit(t, store) + require.Len(t, tk.Session().GetSessionVars().GetIsolationReadEngines(), 3) + tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash';") + engines := tk.Session().GetSessionVars().GetIsolationReadEngines() + require.Len(t, engines, 1) + _, hasTiFlash := engines[kv.TiFlash] + _, hasTiKV := engines[kv.TiKV] + require.True(t, hasTiFlash) + require.False(t, hasTiKV) +} + +func TestLastQueryInfo(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/mockRUConsumption", `return()`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mockRUConsumption")) + }() + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, index idx(a))") + tk.MustExec(`prepare stmt1 from 'select * from t'`) + tk.MustExec("execute stmt1") + checkMatch := func(actual []string, expected []interface{}) bool { + return strings.Contains(actual[0], expected[0].(string)) + } + tk.MustQuery("select @@tidb_last_query_info;").CheckWithFunc(testkit.Rows(`"last_ru_consumption":15`), checkMatch) + tk.MustExec("select a from t where a = 1") + tk.MustQuery("select @@tidb_last_query_info;").CheckWithFunc(testkit.Rows(`"last_ru_consumption":27`), checkMatch) + tk.MustQuery("select @@tidb_last_query_info;").CheckWithFunc(testkit.Rows(`"last_ru_consumption":30`), checkMatch) +} diff --git a/session/session.go b/session/session.go index 36d7d78608195..84cb2c6ac21f0 100644 --- a/session/session.go +++ b/session/session.go @@ -2387,23 +2387,6 @@ func runStmt(ctx context.Context, se *session, s sqlexec.Statement) (rs sqlexec. sessVars := se.sessionVars - // Record diagnostic information for DML statements - if stmt, ok := s.(*executor.ExecStmt).StmtNode.(ast.DMLNode); ok { - // Keep the previous queryInfo for `show session_states` because the statement needs to encode it. - if showStmt, ok := stmt.(*ast.ShowStmt); !ok || showStmt.Tp != ast.ShowSessionStates { - defer func() { - sessVars.LastQueryInfo = sessionstates.QueryInfo{ - TxnScope: sessVars.CheckAndGetTxnScope(), - StartTS: sessVars.TxnCtx.StartTS, - ForUpdateTS: sessVars.TxnCtx.GetForUpdateTS(), - } - if err != nil { - sessVars.LastQueryInfo.ErrMsg = err.Error() - } - }() - } - } - // Save origTxnCtx here to avoid it reset in the transaction retry. origTxnCtx := sessVars.TxnCtx err = se.checkTxnAborted(s) diff --git a/sessionctx/sessionstates/session_states.go b/sessionctx/sessionstates/session_states.go index b66f662a20368..32cc2f25dee31 100644 --- a/sessionctx/sessionstates/session_states.go +++ b/sessionctx/sessionstates/session_states.go @@ -48,10 +48,11 @@ type PreparedStmtInfo struct { // QueryInfo represents the information of last executed query. It's used to expose information for test purpose. type QueryInfo struct { - TxnScope string `json:"txn_scope"` - StartTS uint64 `json:"start_ts"` - ForUpdateTS uint64 `json:"for_update_ts"` - ErrMsg string `json:"error,omitempty"` + TxnScope string `json:"txn_scope"` + StartTS uint64 `json:"start_ts"` + ForUpdateTS uint64 `json:"for_update_ts"` + LastRUConsumption float64 `json:"last_ru_consumption"` + ErrMsg string `json:"error,omitempty"` } // LastDDLInfo represents the information of last DDL. It's used to expose information for test purpose.