Skip to content

Commit

Permalink
*: fix memory leak after dom.Close (#28027)
Browse files Browse the repository at this point in the history
  • Loading branch information
lance6716 authored Sep 14, 2021
1 parent 1bad364 commit 310fef1
Show file tree
Hide file tree
Showing 6 changed files with 66 additions and 4 deletions.
2 changes: 2 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -412,6 +412,8 @@ func (d *ddl) close() {
d.sessPool.close()
}

variable.UnregisterStatistics(d)

logutil.BgLogger().Info("[ddl] DDL closed", zap.String("ID", d.uuid), zap.Duration("take time", time.Since(startTime)))
}

Expand Down
9 changes: 8 additions & 1 deletion domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,8 @@ type Domain struct {
serverID uint64
serverIDSession *concurrency.Session
isLostConnectionToPD sync2.AtomicInt32 // !0: true, 0: false.

onClose func()
}

// loadInfoSchema loads infoschema at startTS.
Expand Down Expand Up @@ -627,13 +629,17 @@ func (do *Domain) Close() {
do.cancel()
do.wg.Wait()
do.sysSessionPool.Close()
variable.UnregisterStatistics(do.bindHandle)
if do.onClose != nil {
do.onClose()
}
logutil.BgLogger().Info("domain closed", zap.Duration("take time", time.Since(startTime)))
}

const resourceIdleTimeout = 3 * time.Minute // resources in the ResourcePool will be recycled after idleTimeout

// NewDomain creates a new domain. Should not create multiple domains for the same store.
func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duration, idxUsageSyncLease time.Duration, factory pools.Factory) *Domain {
func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duration, idxUsageSyncLease time.Duration, factory pools.Factory, onClose func()) *Domain {
capacity := 200 // capacity of the sysSessionPool size
do := &Domain{
store: store,
Expand All @@ -643,6 +649,7 @@ func NewDomain(store kv.Storage, ddlLease time.Duration, statsLease time.Duratio
infoCache: infoschema.NewCache(16),
slowQuery: newTopNSlowQueries(30, time.Hour*24*7, 500),
indexUsageSyncLease: idxUsageSyncLease,
onClose: onClose,
}

do.SchemaValidator = NewSchemaValidator(ddlLease, do)
Expand Down
4 changes: 2 additions & 2 deletions domain/domain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ func SubTestInfo(t *testing.T) {
Storage: s,
pdAddrs: []string{cluster.Members[0].GRPCAddr()}}
ddlLease := 80 * time.Millisecond
dom := NewDomain(mockStore, ddlLease, 0, 0, mockFactory)
dom := NewDomain(mockStore, ddlLease, 0, 0, mockFactory, nil)
defer func() {
dom.Close()
err := s.Close()
Expand Down Expand Up @@ -160,7 +160,7 @@ func SubTestDomain(t *testing.T) {
require.NoError(t, err)

ddlLease := 80 * time.Millisecond
dom := NewDomain(store, ddlLease, 0, 0, mockFactory)
dom := NewDomain(store, ddlLease, 0, 0, mockFactory, nil)
err = dom.Init(ddlLease, sysMockFactory)
require.NoError(t, err)

Expand Down
31 changes: 31 additions & 0 deletions session/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,15 @@ import (
"fmt"
"os"
"path"
"runtime"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"

"github.com/docker/go-units"
. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
Expand Down Expand Up @@ -90,6 +92,7 @@ var _ = Suite(&testClusteredSuite{})
var _ = SerialSuites(&testClusteredSerialSuite{})
var _ = SerialSuites(&testTxnStateSerialSuite{})
var _ = SerialSuites(&testStatisticsSuite{})
var _ = SerialSuites(&testTiDBAsLibrary{})

type testSessionSuiteBase struct {
cluster testutils.Cluster
Expand Down Expand Up @@ -124,6 +127,8 @@ type testStatisticsSuite struct {
testSessionSuiteBase
}

type testTiDBAsLibrary struct{}

func clearStorage(store kv.Storage) error {
txn, err := store.Begin()
if err != nil {
Expand Down Expand Up @@ -5654,3 +5659,29 @@ func (s *testSessionSuite) TestLocalTemporaryTableUpdate(c *C) {
tk.MustQuery("select * from tmp1").Check(testkit.Rows())
}
}

func (s *testTiDBAsLibrary) TestMemoryLeak(c *C) {
initAndCloseTiDB := func() {
store, err := mockstore.NewMockStore(mockstore.WithStoreType(mockstore.EmbedUnistore))
c.Assert(err, IsNil)
defer store.Close()

dom, err := session.BootstrapSession(store)
//nolint:staticcheck
defer dom.Close()
c.Assert(err, IsNil)
}

runtime.GC()
memStat := runtime.MemStats{}
runtime.ReadMemStats(&memStat)
oldHeapInUse := memStat.HeapInuse

for i := 0; i < 10; i++ {
initAndCloseTiDB()
}

runtime.GC()
runtime.ReadMemStats(&memStat)
c.Assert(memStat.HeapInuse-oldHeapInUse, Less, uint64(150*units.MiB))
}
5 changes: 4 additions & 1 deletion session/tidb.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,10 @@ func (dm *domainMap) Get(store kv.Storage) (d *domain.Domain, err error) {
zap.Stringer("index usage sync lease", idxUsageSyncLease))
factory := createSessionFunc(store)
sysFactory := createSessionWithDomainFunc(store)
d = domain.NewDomain(store, ddlLease, statisticLease, idxUsageSyncLease, factory)
onClose := func() {
dm.Delete(store)
}
d = domain.NewDomain(store, ddlLease, statisticLease, idxUsageSyncLease, factory, onClose)
err1 = d.Init(ddlLease, sysFactory)
if err1 != nil {
// If we don't clean it, there are some dirty data when retrying the function of Init.
Expand Down
19 changes: 19 additions & 0 deletions sessionctx/variable/statusvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,25 @@ func RegisterStatistics(s Statistics) {
statisticsListLock.Unlock()
}

// UnregisterStatistics unregisters statistics.
func UnregisterStatistics(s Statistics) {
statisticsListLock.Lock()
defer statisticsListLock.Unlock()
idx := -1
for i := range statisticsList {
if statisticsList[i] == s {
idx = i
}
}
if idx < 0 {
return
}
last := len(statisticsList) - 1
statisticsList[idx] = statisticsList[last]
statisticsList[last] = nil
statisticsList = statisticsList[:last]
}

// GetStatusVars gets registered statistics status variables.
// TODO: Refactor this function to avoid repeated memory allocation / dealloc
func GetStatusVars(vars *SessionVars) (map[string]*StatusVal, error) {
Expand Down

0 comments on commit 310fef1

Please sign in to comment.