Skip to content

Commit

Permalink
This is an automated cherry-pick of pingcap#42803
Browse files Browse the repository at this point in the history
Signed-off-by: ti-chi-bot <ti-community-prow-bot@tidb.io>
  • Loading branch information
XuHuaiyu authored and ti-chi-bot committed Apr 7, 2023
1 parent c9d322c commit f66bd48
Show file tree
Hide file tree
Showing 11 changed files with 200 additions and 29 deletions.
17 changes: 2 additions & 15 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -734,12 +734,7 @@ func (a *ExecStmt) handleNoDelay(ctx context.Context, e Executor, isPessimistic
// done in the `defer` function. If the rs is not nil, the detachment will be done in
// `rs.Close` in `handleStmt`
if handled && sc != nil && rs == nil {
if sc.MemTracker != nil {
sc.MemTracker.Detach()
}
if sc.DiskTracker != nil {
sc.DiskTracker.Detach()
}
sc.DetachMemDiskTracker()
}
}()

Expand Down Expand Up @@ -1417,15 +1412,7 @@ func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, err error, hasMoreResults boo
func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) {
a.FinishExecuteStmt(txnStartTS, lastErr, false)
a.logAudit()
// Detach the Memory and disk tracker for the previous stmtCtx from GlobalMemoryUsageTracker and GlobalDiskUsageTracker
if stmtCtx := a.Ctx.GetSessionVars().StmtCtx; stmtCtx != nil {
if stmtCtx.DiskTracker != nil {
stmtCtx.DiskTracker.Detach()
}
if stmtCtx.MemTracker != nil {
stmtCtx.MemTracker.Detach()
}
}
a.Ctx.GetSessionVars().StmtCtx.DetachMemDiskTracker()
}

// LogSlowQuery is used to print the slow query in the log files.
Expand Down
2 changes: 2 additions & 0 deletions executor/issuetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,10 +15,12 @@ go_test(
"//parser/auth",
"//parser/charset",
"//parser/mysql",
"//session",
"//sessionctx/variable",
"//statistics",
"//testkit",
"//util",
"//util/memory",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//tikv",
Expand Down
114 changes: 114 additions & 0 deletions executor/issuetest/executor_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,17 +20,20 @@ import (
"math/rand"
"strings"
"testing"
"time"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/testkit"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/memory"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -1348,3 +1351,114 @@ func TestIssue40158(t *testing.T) {
tk.MustExec("insert into t1 values (1, null);")
tk.MustQuery("select * from t1 where c1 is null and _id < 1;").Check(testkit.Rows())
}
<<<<<<< HEAD
=======

func TestIssue40596(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec(`CREATE TABLE t1 (
c1 double DEFAULT '1.335088259490289',
c2 set('mj','4s7ht','z','3i','b26','9','cg11','uvzcp','c','ns','fl9') NOT NULL DEFAULT 'mj,z,3i,9,cg11,c',
PRIMARY KEY (c2) /*T![clustered_index] CLUSTERED */,
KEY i1 (c1),
KEY i2 (c1),
KEY i3 (c1)
) ENGINE=InnoDB DEFAULT CHARSET=gbk COLLATE=gbk_chinese_ci;`)
tk.MustExec("INSERT INTO t1 VALUES (634.2783557491367,''),(2000.5041449792013,'4s7ht'),(634.2783557491367,'3i'),(634.2783557491367,'9'),(7803.173688589342,'uvzcp'),(634.2783557491367,'ns'),(634.2783557491367,'fl9');")
tk.MustExec(`CREATE TABLE t2 (
c3 decimal(56,16) DEFAULT '931359772706767457132645278260455518957.9866038319986886',
c4 set('3bqx','g','6op3','2g','jf','arkd3','y0b','jdy','1g','ff5z','224b') DEFAULT '3bqx,2g,ff5z,224b',
c5 smallint(6) NOT NULL DEFAULT '-25973',
c6 year(4) DEFAULT '2122',
c7 text DEFAULT NULL,
PRIMARY KEY (c5) /*T![clustered_index] CLUSTERED */,
KEY i4 (c6),
KEY i5 (c5)
) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin COMMENT=''
PARTITION BY HASH (c5) PARTITIONS 4;`)
tk.MustExec("INSERT INTO t2 VALUES (465.0000000000000000,'jdy',-8542,2008,'FgZXe');")
tk.MustExec("set @@sql_mode='';")
tk.MustExec("set tidb_partition_prune_mode=dynamic;")
tk.MustExec("analyze table t1;")
tk.MustExec("analyze table t2;")

// No nil pointer panic
tk.MustQuery("select /*+ inl_join( t1 , t2 ) */ avg( t2.c5 ) as r0 , repeat( t2.c7 , t2.c5 ) as r1 , locate( t2.c7 , t2.c7 ) as r2 , unhex( t1.c1 ) as r3 from t1 right join t2 on t1.c2 = t2.c5 where not( t2.c5 in ( -7860 ,-13384 ,-12940 ) ) and not( t1.c2 between '4s7ht' and 'mj' );").Check(testkit.Rows("<nil> <nil> <nil> <nil>"))
// Again, a simpler reproduce.
tk.MustQuery("select /*+ inl_join (t1, t2) */ t2.c5 from t1 right join t2 on t1.c2 = t2.c5 where not( t1.c2 between '4s7ht' and 'mj' );").Check(testkit.Rows())
}

func TestIssueRaceWhenBuildingExecutorConcurrently(t *testing.T) {
// issue: https://github.com/pingcap/tidb/issues/41412
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, c int, index idx_a(a), index idx_b(b))")
for i := 0; i < 2000; i++ {
v := i * 100
tk.MustExec("insert into t values(?, ?, ?)", v, v, v)
}
tk.MustQuery("select /*+ inl_merge_join(t1, t2) */ * from t t1 right join t t2 on t1.a = t2.b and t1.c = t2.c")
}

func TestIssue42298(t *testing.T) {
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)")
tk.MustExec("alter table t add column b int")
res := tk.MustQuery("admin show ddl job queries limit 268430000")
require.Greater(t, len(res.Rows()), 0, len(res.Rows()))
res = tk.MustQuery("admin show ddl job queries limit 999 offset 268430000")
require.Zero(t, len(res.Rows()), len(res.Rows()))
}

func TestIssue42662(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
tk.Session().GetSessionVars().ConnectionID = 12345
tk.Session().GetSessionVars().MemTracker = memory.NewTracker(memory.LabelForSession, -1)
tk.Session().GetSessionVars().MemTracker.SessionID = 12345
tk.Session().GetSessionVars().MemTracker.IsRootTrackerOfSess = true

sm := &testkit.MockSessionManager{
PS: []*util.ProcessInfo{tk.Session().ShowProcess()},
}
sm.Conn = make(map[uint64]session.Session)
sm.Conn[tk.Session().GetSessionVars().ConnectionID] = tk.Session()
dom.ServerMemoryLimitHandle().SetSessionManager(sm)
go dom.ServerMemoryLimitHandle().Run()

tk.MustExec("use test")
tk.MustQuery("select connection_id()").Check(testkit.Rows("12345"))
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (a int, b int, c int)")
tk.MustExec("create table t2 (a int, b int, c int)")
tk.MustExec("insert into t1 values (1, 1, 1), (1, 2, 2), (2, 1, 3), (2, 2, 4)")
tk.MustExec("insert into t2 values (1, 1, 1), (1, 2, 2), (2, 1, 3), (2, 2, 4)")
// set tidb_server_memory_limit to 1.6GB, tidb_server_memory_limit_sess_min_size to 128MB
tk.MustExec("set global tidb_server_memory_limit='1600MB'")
tk.MustExec("set global tidb_server_memory_limit_sess_min_size=128*1024*1024")
tk.MustExec("set global tidb_mem_oom_action = 'cancel'")
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/executor/issue42662_1", `return(true)`))
// tk.Session() should be marked as MemoryTop1Tracker but not killed.
tk.MustQuery("select /*+ hash_join(t1)*/ * from t1 join t2 on t1.a = t2.a and t1.b = t2.b")

// try to trigger the kill top1 logic
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/util/servermemorylimit/issue42662_2", `return(true)`))
time.Sleep(1 * time.Second)

// no error should be returned
tk.MustQuery("select count(*) from t1")
tk.MustQuery("select count(*) from t1")

require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/executor/issue42662_1"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/util/servermemorylimit/issue42662_2"))
}
>>>>>>> d6a5054b723 (*: global memory controller should not kill session whose mem less than limit_sess_min_size (#42803))
9 changes: 9 additions & 0 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -310,6 +310,15 @@ func (w *buildWorker) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chun
return
}
})
failpoint.Inject("issue42662_1", func(val failpoint.Value) {
if val.(bool) {
if w.hashJoinCtx.sessCtx.GetSessionVars().ConnectionID != 0 {
// consume 170MB memory, this sql should be tracked into MemoryTop1Tracker
w.hashJoinCtx.memTracker.Consume(170 * 1024 * 1024)
}
return
}
})
sessVars := w.hashJoinCtx.sessCtx.GetSessionVars()
for {
if w.hashJoinCtx.finished.Load() {
Expand Down
12 changes: 10 additions & 2 deletions server/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -2121,12 +2121,20 @@ func (cc *clientConn) handleStmt(ctx context.Context, stmt ast.StmtNode, warns [
cc.audit(plugin.Starting)
rs, err := cc.ctx.ExecuteStmt(ctx, stmt)
reg.End()
// The session tracker detachment from global tracker is solved in the `rs.Close` in most cases.
// If the rs is nil, the detachment will be done in the `handleNoDelay`.
// - If rs is not nil, the statement tracker detachment from session tracker
// is done in the `rs.Close` in most cases.
// - If the rs is nil and err is not nil, the detachment will be done in
// the `handleNoDelay`.
if rs != nil {
defer terror.Call(rs.Close)
}
if err != nil {
// If error is returned during the planner phase or the executor.Open
// phase, the rs will be nil, and StmtCtx.MemTracker StmtCtx.DiskTracker
// will not be detached. We need to detach them manually.
if sv := cc.ctx.GetSessionVars(); sv != nil && sv.StmtCtx != nil {
sv.StmtCtx.DetachMemDiskTracker()
}
return true, err
}

Expand Down
13 changes: 13 additions & 0 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -1114,6 +1114,19 @@ func (sc *StatementContext) UseDynamicPartitionPrune() bool {
return sc.UseDynamicPruneMode
}

// DetachMemDiskTracker detaches the memory and disk tracker from the sessionTracker.
func (sc *StatementContext) DetachMemDiskTracker() {
if sc == nil {
return
}
if sc.MemTracker != nil {
sc.MemTracker.Detach()
}
if sc.DiskTracker != nil {
sc.DiskTracker.Detach()
}
}

// CopTasksDetails collects some useful information of cop-tasks during execution.
type CopTasksDetails struct {
NumCopTasks int
Expand Down
14 changes: 7 additions & 7 deletions testkit/mocksessionmanager.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ type MockSessionManager struct {
SerID uint64
TxnInfo []*txninfo.TxnInfo
Dom *domain.Domain
conn map[uint64]session.Session
Conn map[uint64]session.Session
mu sync.Mutex
}

Expand All @@ -44,8 +44,8 @@ func (msm *MockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
if len(msm.TxnInfo) > 0 {
return msm.TxnInfo
}
rs := make([]*txninfo.TxnInfo, 0, len(msm.conn))
for _, se := range msm.conn {
rs := make([]*txninfo.TxnInfo, 0, len(msm.Conn))
for _, se := range msm.Conn {
info := se.TxnInfo()
if info != nil {
rs = append(rs, info)
Expand All @@ -66,7 +66,7 @@ func (msm *MockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
return ret
}
msm.mu.Lock()
for connID, pi := range msm.conn {
for connID, pi := range msm.Conn {
ret[connID] = pi.ShowProcess()
}
msm.mu.Unlock()
Expand All @@ -89,7 +89,7 @@ func (msm *MockSessionManager) GetProcessInfo(id uint64) (*util.ProcessInfo, boo
}
msm.mu.Lock()
defer msm.mu.Unlock()
if sess := msm.conn[id]; sess != nil {
if sess := msm.Conn[id]; sess != nil {
return sess.ShowProcess(), true
}
if msm.Dom != nil {
Expand Down Expand Up @@ -130,7 +130,7 @@ func (*MockSessionManager) GetInternalSessionStartTSList() []uint64 {

// KillNonFlashbackClusterConn implement SessionManager interface.
func (msm *MockSessionManager) KillNonFlashbackClusterConn() {
for _, se := range msm.conn {
for _, se := range msm.Conn {
processInfo := se.ShowProcess()
ddl, ok := processInfo.StmtCtx.GetPlan().(*core.DDL)
if !ok {
Expand All @@ -148,7 +148,7 @@ func (msm *MockSessionManager) KillNonFlashbackClusterConn() {
// CheckOldRunningTxn implement SessionManager interface.
func (msm *MockSessionManager) CheckOldRunningTxn(job2ver map[int64]int64, job2ids map[int64]string) {
msm.mu.Lock()
for _, se := range msm.conn {
for _, se := range msm.Conn {
session.RemoveLockDDLJobs(se, job2ver, job2ids)
}
msm.mu.Unlock()
Expand Down
6 changes: 3 additions & 3 deletions testkit/testkit.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,10 +69,10 @@ func NewTestKit(t testing.TB, store kv.Storage) *TestKit {
mockSm, ok := sm.(*MockSessionManager)
if ok {
mockSm.mu.Lock()
if mockSm.conn == nil {
mockSm.conn = make(map[uint64]session.Session)
if mockSm.Conn == nil {
mockSm.Conn = make(map[uint64]session.Session)
}
mockSm.conn[tk.session.GetSessionVars().ConnectionID] = tk.session
mockSm.Conn[tk.session.GetSessionVars().ConnectionID] = tk.session
mockSm.mu.Unlock()
}
tk.session.SetSessionManager(sm)
Expand Down
5 changes: 5 additions & 0 deletions util/memory/tracker.go
Original file line number Diff line number Diff line change
Expand Up @@ -302,6 +302,9 @@ func (t *Tracker) AttachTo(parent *Tracker) {

// Detach de-attach the tracker child from its parent, then set its parent property as nil
func (t *Tracker) Detach() {
if t == nil {
return
}
parent := t.getParent()
if parent == nil {
return
Expand Down Expand Up @@ -446,6 +449,7 @@ func (t *Tracker) Consume(bs int64) {
currentAction = nextAction
nextAction = currentAction.GetFallback()
}
logutil.BgLogger().Warn("global memory controller, lastAction", zap.Any("action", currentAction))
currentAction.Action(tracker)
}
}
Expand All @@ -471,6 +475,7 @@ func (t *Tracker) Consume(bs int64) {
}
oldTracker = MemUsageTop1Tracker.Load()
}
logutil.BgLogger().Error("global memory controller, update the Top1 session", zap.Int64("memUsage", memUsage), zap.Uint64("conn", sessionRootTracker.SessionID), zap.Uint64("limitSessMinSize", limitSessMinSize))
}
}

Expand Down
1 change: 1 addition & 0 deletions util/servermemorylimit/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"//util",
"//util/logutil",
"//util/memory",
"@com_github_pingcap_failpoint//:failpoint",
"@org_uber_go_atomic//:atomic",
"@org_uber_go_zap//:zap",
],
Expand Down
Loading

0 comments on commit f66bd48

Please sign in to comment.