-
Notifications
You must be signed in to change notification settings - Fork 5.8k
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
planner, executor: fix haven't track the memory usage of PointGet/BatchPointGet #21230
base: master
Are you sure you want to change the base?
Changes from 14 commits
f76abb3
5c81c3f
83f4248
7f58a62
e4aa904
291e2c6
c9f3966
87d0476
4ff8172
6ada448
b521483
93f09b1
5883956
08d4671
8fd842c
86a523a
93c21bd
97579d9
facbc93
006ac37
092914a
7a7e04a
65feff2
279e4c5
1430ee2
ab74f22
f2ae417
a5fa934
7d2359a
46013d0
47e7c0a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -32,6 +32,7 @@ import ( | |
"github.com/pingcap/tidb/util/codec" | ||
"github.com/pingcap/tidb/util/hack" | ||
"github.com/pingcap/tidb/util/math" | ||
"github.com/pingcap/tidb/util/memory" | ||
"github.com/pingcap/tidb/util/rowcodec" | ||
) | ||
|
||
|
@@ -68,6 +69,8 @@ type BatchPointGetExec struct { | |
|
||
snapshot kv.Snapshot | ||
stats *runtimeStatsWithSnapshot | ||
|
||
memTracker *memory.Tracker | ||
} | ||
|
||
// buildVirtualColumnInfo saves virtual column indices and sort them in definition order | ||
|
@@ -123,11 +126,18 @@ func (e *BatchPointGetExec) Open(context.Context) error { | |
} | ||
e.snapshot = snapshot | ||
e.batchGetter = batchGetter | ||
|
||
e.memTracker = memory.NewTracker(e.id, -1) | ||
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) | ||
return nil | ||
} | ||
|
||
// Close implements the Executor interface. | ||
func (e *BatchPointGetExec) Close() error { | ||
|
||
e.memTracker.Consume(-e.memTracker.BytesConsumed()) | ||
e.memTracker = nil | ||
|
||
if e.runtimeStats != nil && e.snapshot != nil { | ||
e.snapshot.DelOption(kv.CollectRuntimeStats) | ||
} | ||
|
@@ -192,6 +202,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { | |
continue | ||
} | ||
|
||
e.memTracker.Consume(types.EstimatedMemUsage(idxVals, 1)) | ||
physID := getPhysID(e.tblInfo, idxVals[e.partPos].GetInt64()) | ||
idxKey, err1 := EncodeUniqueIndexKey(e.ctx, e.tblInfo, e.idxInfo, idxVals, physID) | ||
if err1 != nil && !kv.ErrNotExist.Equal(err1) { | ||
|
@@ -321,6 +332,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { | |
tID = getPhysID(e.tblInfo, d.GetInt64()) | ||
} | ||
} | ||
e.memTracker.Consume(int64(cap(handle.Encoded()))) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Besides, There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There is no kv.Handle.MemoryUsage()... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I meant maybe we can add such a method. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
key := tablecodec.EncodeRowKeyWithHandle(tID, handle) | ||
keys[i] = key | ||
} | ||
|
@@ -349,6 +361,8 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error { | |
e.values = make([][]byte, 0, len(values)) | ||
for i, key := range keys { | ||
val := values[string(key)] | ||
e.memTracker.Consume(int64(cap(val))) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ditto, not all |
||
|
||
if len(val) == 0 { | ||
if e.idxInfo != nil && (!e.tblInfo.IsCommonHandle || !e.idxInfo.Primary) { | ||
return kv.ErrNotExist.GenWithStack("inconsistent extra index %s, handle %d not found in table", | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -32,6 +32,7 @@ import ( | |
"github.com/pingcap/tidb/util/chunk" | ||
"github.com/pingcap/tidb/util/codec" | ||
"github.com/pingcap/tidb/util/execdetails" | ||
"github.com/pingcap/tidb/util/memory" | ||
"github.com/pingcap/tidb/util/rowcodec" | ||
) | ||
|
||
|
@@ -81,6 +82,8 @@ type PointGetExecutor struct { | |
virtualColumnRetFieldTypes []*types.FieldType | ||
|
||
stats *runtimeStatsWithSnapshot | ||
|
||
memTracker *memory.Tracker | ||
} | ||
|
||
// Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field | ||
|
@@ -140,11 +143,18 @@ func (e *PointGetExecutor) Open(context.Context) error { | |
e.snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) | ||
} | ||
e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) | ||
|
||
e.memTracker = memory.NewTracker(e.id, -1) | ||
e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) | ||
return nil | ||
} | ||
|
||
// Close implements the Executor interface. | ||
func (e *PointGetExecutor) Close() error { | ||
|
||
e.memTracker.Consume(-e.memTracker.BytesConsumed()) | ||
e.memTracker = nil | ||
|
||
if e.runtimeStats != nil && e.snapshot != nil { | ||
e.snapshot.DelOption(kv.CollectRuntimeStats) | ||
} | ||
|
@@ -247,6 +257,8 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { | |
} | ||
return nil | ||
} | ||
e.memTracker.Consume(int64(cap(val))) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I think we need tracking usage of executor's significant memory consuming members, such as There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @zhaoxugang PTAL this comment. Rest LGTM. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. IMO the point_get_exec mabe should't track memory,because the memory tracker should track the data from store but the data from store is return to client immediately.and the idxKey, handleVal is only one for point_get, it's Constant level so should't track. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why we should not track memory at constant level ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You are right...I have done |
||
|
||
err = DecodeRowValToChunk(e.base().ctx, e.schema, e.tblInfo, e.handle, val, req, e.rowDecoder) | ||
if err != nil { | ||
return err | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -3145,6 +3145,32 @@ func (s *testSessionSuite2) TestStmtHints(c *C) { | |
c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) | ||
} | ||
|
||
// Test memory track in POINT_GET/BATCH_POINT_GET for issue 21653 | ||
func (s *testSessionSuite2) TestPointGetMemoryTracking(c *C) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I can't directly validate the memory usage,do you have some advices to do it? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In this case, what is the difference between integer and others, why need add tests for clustered index whose type is not an integer. |
||
tk := testkit.NewTestKit(c, s.store) | ||
tk.MustExec("use test") | ||
tk.MustExec("drop table if exists t1;") | ||
tk.MustExec("create table t1(a int primary key);") | ||
config.UpdateGlobal(func(conf *config.Config) { | ||
conf.OOMAction = config.OOMActionCancel | ||
}) | ||
tk.MustExec("insert into t1 values (1);") | ||
tk.MustQuery("select * from t1 where a = 1;").Check(testkit.Rows("1")) | ||
tk.MustExec("set tidb_mem_quota_query=1;") | ||
err := tk.QueryToErr("select * from t1 where a = 1;") | ||
wshwsh12 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
c.Assert(err, NotNil) | ||
c.Assert(err.Error(), Matches, "Out Of Memory Quota!.*") | ||
err = tk.QueryToErr("select * from t1 where a in (1,2,3);") | ||
c.Assert(err, NotNil) | ||
c.Assert(err.Error(), Matches, "Out Of Memory Quota!.*") | ||
tk.MustExec("set tidb_mem_quota_query=1024;") | ||
tk.MustQuery("select * from t1 where a = 1;").Check(testkit.Rows("1")) | ||
err = tk.QueryToErr("select * from t1 where a = 1;") | ||
c.Check(err, IsNil) | ||
err = tk.QueryToErr("select * from t1 where a in (1,2,3);") | ||
c.Check(err, IsNil) | ||
} | ||
|
||
func (s *testSessionSuite3) TestPessimisticLockOnPartition(c *C) { | ||
// This test checks that 'select ... for update' locks the partition instead of the table. | ||
// Cover a bug that table ID is used to encode the lock key mistakenly. | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think this line can be put at the beginning of the loop.
First,
datumsContainNull(idxVals)
will be true if ANY datum inidxVals
is Null.Second, a Null datum object still consuming memory.