Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

*: add memory tracker for mppIterator #40901

Merged
merged 8 commits into from
Feb 2, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 3 additions & 2 deletions distsql/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/trxevents"
"github.com/pingcap/tipb/go-tipb"
"github.com/tikv/client-go/v2/tikvrpc/interceptor"
Expand All @@ -38,11 +39,11 @@ import (
)

// DispatchMPPTasks dispatches all tasks and returns an iterator.
func DispatchMPPTasks(ctx context.Context, sctx sessionctx.Context, tasks []*kv.MPPDispatchRequest, fieldTypes []*types.FieldType, planIDs []int, rootID int, startTs uint64, mppQueryID kv.MPPQueryID) (SelectResult, error) {
func DispatchMPPTasks(ctx context.Context, sctx sessionctx.Context, tasks []*kv.MPPDispatchRequest, fieldTypes []*types.FieldType, planIDs []int, rootID int, startTs uint64, mppQueryID kv.MPPQueryID, memTracker *memory.Tracker) (SelectResult, error) {
ctx = WithSQLKvExecCounterInterceptor(ctx, sctx.GetSessionVars().StmtCtx)
_, allowTiFlashFallback := sctx.GetSessionVars().AllowFallbackToTiKV[kv.TiFlash]
ctx = SetTiFlashMaxThreadsInContext(ctx, sctx)
resp := sctx.GetMPPClient().DispatchMPPTasks(ctx, sctx.GetSessionVars().KVVars, tasks, allowTiFlashFallback, startTs, mppQueryID, sctx.GetSessionVars().ChooseMppVersion())
resp := sctx.GetMPPClient().DispatchMPPTasks(ctx, sctx.GetSessionVars().KVVars, tasks, allowTiFlashFallback, startTs, mppQueryID, sctx.GetSessionVars().ChooseMppVersion(), memTracker)
if resp == nil {
return nil, errors.New("client returns nil response")
}
Expand Down
2 changes: 2 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3408,7 +3408,9 @@ func (b *executorBuilder) buildMPPGather(v *plannercore.PhysicalTableReader) Exe
originalPlan: v.GetTablePlan(),
startTS: startTs,
mppQueryID: kv.MPPQueryID{QueryTs: getMPPQueryTS(b.ctx), LocalQueryID: getMPPQueryID(b.ctx), ServerID: domain.GetDomain(b.ctx).ServerID()},
memTracker: memory.NewTracker(v.ID(), -1),
}
gather.memTracker.AttachTo(b.ctx.GetSessionVars().StmtCtx.MemTracker)
return gather
}

Expand Down
5 changes: 4 additions & 1 deletion executor/mpp_gather.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tipb/go-tipb"
"go.uber.org/zap"
)
Expand Down Expand Up @@ -63,6 +64,8 @@ type MPPGather struct {
mppReqs []*kv.MPPDispatchRequest

respIter distsql.SelectResult

memTracker *memory.Tracker
}

func (e *MPPGather) appendMPPDispatchReq(pf *plannercore.Fragment) error {
Expand Down Expand Up @@ -146,7 +149,7 @@ func (e *MPPGather) Open(ctx context.Context) (err error) {
failpoint.Return(errors.Errorf("The number of tasks is not right, expect %d tasks but actually there are %d tasks", val.(int), len(e.mppReqs)))
}
})
e.respIter, err = distsql.DispatchMPPTasks(ctx, e.ctx, e.mppReqs, e.retFieldTypes, planIDs, e.id, e.startTS, e.mppQueryID)
e.respIter, err = distsql.DispatchMPPTasks(ctx, e.ctx, e.mppReqs, e.retFieldTypes, planIDs, e.id, e.startTS, e.mppQueryID, e.memTracker)
if err != nil {
return errors.Trace(err)
}
Expand Down
23 changes: 23 additions & 0 deletions executor/tiflashtest/tiflash_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1402,3 +1402,26 @@ func TestDisaggregatedTiFlashQuery(t *testing.T) {
" └─TableFullScan_20 10000.00 mpp[tiflash] table:t1, partition:p2 keep order:false, stats:pseudo"))
// tk.MustQuery("select * from t1 where c1 < 2").Check(testkit.Rows("1 1"))
}

func TestMPPMemoryTracker(t *testing.T) {
store := testkit.CreateMockStore(t, withMockTiFlash(2))
tk := testkit.NewTestKit(t, store)
tk.MustExec("set tidb_mem_quota_query = 1 << 30")
tk.MustExec("set global tidb_mem_oom_action = 'CANCEL'")
tk.MustExec("use test")
tk.MustExec("create table t(a int);")
tk.MustExec("insert into t values (1);")
tk.MustExec("alter table t set tiflash replica 1")
tb := external.GetTableByName(t, tk, "test", "t")
err := domain.GetDomain(tk.Session()).DDL().UpdateTableReplicaInfo(tk.Session(), tb.Meta().ID, true)
require.NoError(t, err)
tk.MustExec("set tidb_enforce_mpp = on;")
tk.MustQuery("select * from t").Check(testkit.Rows("1"))
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/store/copr/testMPPOOMPanic", `return(true)`))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/store/copr/testMPPOOMPanic"))
}()
err = tk.QueryToErr("select * from t")
require.NotNil(t, err)
require.True(t, strings.Contains(err.Error(), "Out Of Memory Quota!"))
}
3 changes: 2 additions & 1 deletion kv/mpp.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"time"

"github.com/pingcap/kvproto/pkg/mpp"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tipb/go-tipb"
)

Expand Down Expand Up @@ -151,7 +152,7 @@ type MPPClient interface {
// TODO:: This interface will be refined after we support more executors.
ConstructMPPTasks(context.Context, *MPPBuildTasksRequest, time.Duration) ([]MPPTaskMeta, error)
// DispatchMPPTasks dispatches ALL mpp requests at once, and returns an iterator that transfers the data.
DispatchMPPTasks(ctx context.Context, vars interface{}, reqs []*MPPDispatchRequest, needTriggerFallback bool, startTs uint64, mppQueryID MPPQueryID, mppVersion MppVersion) Response
DispatchMPPTasks(ctx context.Context, vars interface{}, reqs []*MPPDispatchRequest, needTriggerFallback bool, startTs uint64, mppQueryID MPPQueryID, mppVersion MppVersion, memTracker *memory.Tracker) Response
}

// MPPBuildTasksRequest request the stores allocation for a mpp plan fragment.
Expand Down
23 changes: 22 additions & 1 deletion store/copr/mpp.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package copr

import (
"context"
"fmt"
"io"
"strconv"
"sync"
Expand All @@ -35,6 +36,7 @@ import (
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
"github.com/pingcap/tidb/util/memory"
"github.com/tikv/client-go/v2/tikv"
"github.com/tikv/client-go/v2/tikvrpc"
"go.uber.org/zap"
Expand Down Expand Up @@ -164,6 +166,8 @@ type mppIterator struct {
mu sync.Mutex

enableCollectExecutionInfo bool

memTracker *memory.Tracker
}

func (m *mppIterator) run(ctx context.Context) {
Expand Down Expand Up @@ -202,6 +206,22 @@ func (m *mppIterator) sendError(err error) {
}

func (m *mppIterator) sendToRespCh(resp *mppResponse) (exit bool) {
defer func() {
if r := recover(); r != nil {
logutil.BgLogger().Error("mppIterator panic", zap.Stack("stack"), zap.Any("recover", r))
m.sendError(errors.New(fmt.Sprint(r)))
}
}()
if m.memTracker != nil {
respSize := resp.MemSize()
failpoint.Inject("testMPPOOMPanic", func(val failpoint.Value) {
if val.(bool) && respSize != 0 {
respSize = 1 << 30
}
})
m.memTracker.Consume(respSize)
defer m.memTracker.Consume(-respSize)
}
select {
case m.respChan <- resp:
case <-m.finishCh:
Expand Down Expand Up @@ -542,7 +562,7 @@ func (m *mppIterator) Next(ctx context.Context) (kv.ResultSubset, error) {
}

// DispatchMPPTasks dispatches all the mpp task and waits for the responses.
func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{}, dispatchReqs []*kv.MPPDispatchRequest, needTriggerFallback bool, startTs uint64, mppQueryID kv.MPPQueryID, mppVersion kv.MppVersion) kv.Response {
func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{}, dispatchReqs []*kv.MPPDispatchRequest, needTriggerFallback bool, startTs uint64, mppQueryID kv.MPPQueryID, mppVersion kv.MppVersion, memTracker *memory.Tracker) kv.Response {
vars := variables.(*tikv.Variables)
ctxChild, cancelFunc := context.WithCancel(ctx)
iter := &mppIterator{
Expand All @@ -558,6 +578,7 @@ func (c *MPPClient) DispatchMPPTasks(ctx context.Context, variables interface{},
vars: vars,
needTriggerFallback: needTriggerFallback,
enableCollectExecutionInfo: config.GetGlobalConfig().Instance.EnableCollectExecutionInfo.Load(),
memTracker: memTracker,
}
go iter.run(ctxChild)
return iter
Expand Down