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

util: avoid column allocator reuse the column hold huge memory (#32554) #32578

Merged
merged 2 commits into from
Apr 25, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
6 changes: 2 additions & 4 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1142,8 +1142,7 @@ func createSessionFunc(store kv.Storage) pools.Factory {
}
se.sessionVars.CommonGlobalLoaded = true
se.sessionVars.InRestrictedSQL = true
// TODO: Remove this line after fixing https://github.com/pingcap/tidb/issues/30880
// Chunk RPC protocol may have memory leak issue not solved.
// Internal session uses default format to prevent memory leak problem.
se.sessionVars.EnableChunkRPC = false
return se, nil
}
Expand All @@ -1165,8 +1164,7 @@ func createSessionWithDomainFunc(store kv.Storage) func(*domain.Domain) (pools.R
}
se.sessionVars.CommonGlobalLoaded = true
se.sessionVars.InRestrictedSQL = true
// TODO: Remove this line after fixing https://github.com/pingcap/tidb/issues/30880
// Chunk RPC protocol may have memory leak issue not solved.
// Internal session uses default format to prevent memory leak problem.
se.sessionVars.EnableChunkRPC = false
return se, nil
}
Expand Down
3 changes: 3 additions & 0 deletions util/chunk/alloc.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,9 @@ func (alloc *poolColumnAllocator) init() {
}

func (alloc *poolColumnAllocator) put(col *Column) {
if col.avoidReusing {
return
}
typeSize := col.typeSize()
if typeSize <= 0 {
return
Expand Down
51 changes: 51 additions & 0 deletions util/chunk/alloc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -151,3 +151,54 @@ func TestNoDuplicateColumnReuse(t *testing.T) {
}
}
}

func TestAvoidColumnReuse(t *testing.T) {
// For issue: https://github.com/pingcap/tidb/issues/31981
// Some chunk columns are references to rpc message.
// So when reusing Chunk, we should ignore them.

fieldTypes := []*types.FieldType{
{Tp: mysql.TypeVarchar},
{Tp: mysql.TypeJSON},
{Tp: mysql.TypeFloat},
{Tp: mysql.TypeNewDecimal},
{Tp: mysql.TypeDouble},
{Tp: mysql.TypeLonglong},
{Tp: mysql.TypeTimestamp},
{Tp: mysql.TypeDatetime},
}
alloc := NewAllocator()
for i := 0; i < maxFreeChunks+10; i++ {
chk := alloc.Alloc(fieldTypes, 5, 10)
for _, col := range chk.columns {
col.avoidReusing = true
}
}
alloc.Reset()

a := alloc.columnAlloc
// Make sure no duplicated column in the pool.
for _, p := range a.pool {
require.True(t, p.empty())
}

// test decoder will set avoid reusing flag.
chk := alloc.Alloc(fieldTypes, 5, 1024)
for i := 0; i <= 10; i++ {
for _, col := range chk.columns {
col.AppendNull()
}
}
codec := &Codec{fieldTypes}
buf := codec.Encode(chk)

decoder := NewDecoder(
NewChunkWithCapacity(fieldTypes, 0),
fieldTypes,
)
decoder.Reset(buf)
decoder.ReuseIntermChk(chk)
for _, col := range chk.columns {
require.True(t, col.avoidReusing)
}
}
3 changes: 3 additions & 0 deletions util/chunk/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,9 @@ func (c *Codec) decodeColumn(buffer []byte, col *Column, ordinal int) (remained

// decode data.
col.data = buffer[:numDataBytes:numDataBytes]
// The column reference the data of the grpc response, the memory of the grpc message cannot be GCed if we reuse
// this column. Thus, we set `avoidReusing` to true.
col.avoidReusing = true
return buffer[numDataBytes:]
}

Expand Down
2 changes: 2 additions & 0 deletions util/chunk/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,8 @@ type Column struct {
offsets []int64 // used for varLen column. Row i starts from data[offsets[i]]
data []byte
elemBuf []byte

avoidReusing bool // avoid reusing the Column by allocator
}

// ColumnAllocator defines an allocator for Column.
Expand Down