diff --git a/Makefile b/Makefile index 8023c2902e4f0..86bb29999ef66 100644 --- a/Makefile +++ b/Makefile @@ -120,10 +120,7 @@ race: parserlib leak: parserlib @export log_level=debug; \ - for dir in $(PACKAGES); do \ - echo $$dir; \ - $(GOTEST) -tags leak $$dir | awk 'END{if($$1=="FAIL") {exit 1}}' || exit 1; \ - done; + $(GOTEST) -tags leak $(PACKAGES) tikv_integration_test: parserlib $(GOTEST) ./store/tikv/. -with-tikv=true diff --git a/bootstrap_test.go b/bootstrap_test.go index 99b663f9f351a..c886cc517c6ff 100644 --- a/bootstrap_test.go +++ b/bootstrap_test.go @@ -41,7 +41,9 @@ func (s *testBootstrapSuite) SetUpSuite(c *C) { func (s *testBootstrapSuite) TestBootstrap(c *C) { defer testleak.AfterTest(c)() - store := newStoreWithBootstrap(c, s.dbName) + store, dom := newStoreWithBootstrap(c, s.dbName) + defer dom.Close() + defer store.Close() se := newSession(c, store, s.dbName) mustExecSQL(c, se, "USE mysql;") r := mustExecSQL(c, se, `select * from user;`) @@ -168,7 +170,7 @@ func (s *testBootstrapSuite) testBootstrapWithError(c *C) { // TestUpgrade tests upgrading func (s *testBootstrapSuite) TestUpgrade(c *C) { defer testleak.AfterTest(c)() - store := newStoreWithBootstrap(c, s.dbName) + store, _ := newStoreWithBootstrap(c, s.dbName) defer store.Close() se := newSession(c, store, s.dbName) mustExecSQL(c, se, "USE mysql;") @@ -211,7 +213,9 @@ func (s *testBootstrapSuite) TestUpgrade(c *C) { c.Assert(ver, Equals, int64(0)) // Create a new session then upgrade() will run automatically. - BootstrapSession(store) + dom1, err := BootstrapSession(store) + c.Assert(err, IsNil) + defer dom1.Close() se2 := newSession(c, store, s.dbName) r = mustExecSQL(c, se2, `SELECT VARIABLE_VALUE from mysql.TiDB where VARIABLE_NAME="tidb_server_version";`) row, err = r.Next() diff --git a/ddl/ddl_db_test.go b/ddl/ddl_db_test.go index 38052d1d95110..a040c0ef1ccc0 100644 --- a/ddl/ddl_db_test.go +++ b/ddl/ddl_db_test.go @@ -101,7 +101,6 @@ func (s *testDBSuite) testErrorCode(c *C, sql string, errCode int) { } func (s *testDBSuite) TestMySQLErrorCode(c *C) { - defer testleak.AfterTest(c) s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -598,7 +597,6 @@ func (s *testDBSuite) TestIssue2293(c *C) { } func (s *testDBSuite) TestCreateIndexType(c *C) { - defer testleak.AfterTest(c)() s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) sql := `CREATE TABLE test_index ( @@ -1019,11 +1017,13 @@ func match(c *C, row []interface{}, expected ...interface{}) { } func (s *testDBSuite) TestUpdateMultipleTable(c *C) { - defer testleak.AfterTest(c) + defer testleak.AfterTest(c)() store, err := tidb.NewStore("memory://update_multiple_table") c.Assert(err, IsNil) - _, err = tidb.BootstrapSession(store) + defer store.Close() + dom, err := tidb.BootstrapSession(store) c.Assert(err, IsNil) + defer dom.Close() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") tk.MustExec("create table t1 (c1 int, c2 int)") @@ -1081,7 +1081,6 @@ func (s *testDBSuite) TestUpdateMultipleTable(c *C) { } func (s *testDBSuite) TestCreateTableTooLarge(c *C) { - defer testleak.AfterTest(c) s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") @@ -1104,12 +1103,14 @@ func (s *testDBSuite) TestCreateTableTooLarge(c *C) { } func (s *testDBSuite) TestCreateTableWithLike(c *C) { - defer testleak.AfterTest(c) + defer testleak.AfterTest(c)() store, err := tidb.NewStore("memory://create_table_like") c.Assert(err, IsNil) + defer store.Close() s.tk = testkit.NewTestKit(c, store) - _, err = tidb.BootstrapSession(store) + dom, err := tidb.BootstrapSession(store) c.Assert(err, IsNil) + defer dom.Close() // for the same database s.tk.MustExec("use test") @@ -1153,7 +1154,6 @@ func (s *testDBSuite) TestCreateTableWithLike(c *C) { } func (s *testDBSuite) TestCreateTable(c *C) { - defer testleak.AfterTest(c) store, err := tidb.NewStore("memory://create_table") c.Assert(err, IsNil) s.tk = testkit.NewTestKit(c, store) @@ -1176,11 +1176,13 @@ func (s *testDBSuite) TestCreateTable(c *C) { } func (s *testDBSuite) TestTruncateTable(c *C) { - defer testleak.AfterTest(c) + defer testleak.AfterTest(c)() store, err := tidb.NewStore("memory://truncate_table") c.Assert(err, IsNil) - _, err = tidb.BootstrapSession(store) + defer store.Close() + dom, err := tidb.BootstrapSession(store) c.Assert(err, IsNil) + defer dom.Close() tk := testkit.NewTestKit(c, store) tk.MustExec("use test") tk.MustExec("create table t (c1 int, c2 int)") @@ -1236,11 +1238,13 @@ func (s *testDBSuite) TestAlterTableRenameTable(c *C) { } func (s *testDBSuite) testRenameTable(c *C, storeStr, sql string) { - defer testleak.AfterTest(c) + defer testleak.AfterTest(c)() store, err := tidb.NewStore("memory://" + storeStr) c.Assert(err, IsNil) - _, err = tidb.BootstrapSession(store) + defer store.Close() + dom, err := tidb.BootstrapSession(store) c.Assert(err, IsNil) + defer dom.Close() s.tk = testkit.NewTestKit(c, store) s.tk.MustExec("use test") @@ -1287,11 +1291,13 @@ func (s *testDBSuite) testRenameTable(c *C, storeStr, sql string) { } func (s *testDBSuite) TestRenameMultiTables(c *C) { - defer testleak.AfterTest(c) + defer testleak.AfterTest(c)() store, err := tidb.NewStore("memory://rename_multi_tables") c.Assert(err, IsNil) - _, err = tidb.BootstrapSession(store) + defer store.Close() + dom, err := tidb.BootstrapSession(store) c.Assert(err, IsNil) + defer dom.Close() s.tk = testkit.NewTestKit(c, store) s.tk.MustExec("use test") s.tk.MustExec("create table t1(id int)") @@ -1305,7 +1311,7 @@ func (s *testDBSuite) TestRenameMultiTables(c *C) { } func (s *testDBSuite) TestAddNotNullColumn(c *C) { - defer testleak.AfterTest(c) + defer testleak.AfterTest(c)() s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test_db") // for different databases @@ -1431,9 +1437,6 @@ func (s *testDBSuite) TestChangeColumnPosition(c *C) { } func (s *testDBSuite) TestGeneratedColumnDDL(c *C) { - defer func() { - testleak.AfterTest(c)() - }() s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") diff --git a/domain/domain.go b/domain/domain.go index fdd4381b1fb8c..a3dc82d003a0b 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -51,6 +51,7 @@ type Domain struct { sysSessionPool *pools.ResourcePool exit chan struct{} etcdClient *clientv3.Client + wg sync.WaitGroup MockReloadFailed MockFailure // It mocks reload failed. } @@ -348,6 +349,7 @@ func (do *Domain) Close() { do.etcdClient.Close() } do.sysSessionPool.Close() + do.wg.Wait() } type ddlCallback struct { @@ -539,6 +541,7 @@ func (do *Domain) UpdateTableStatsLoop(ctx context.Context) error { if lease <= 0 { return nil } + do.wg.Add(1) go do.updateStatsWorker(ctx, lease) return nil } @@ -558,6 +561,7 @@ func (do *Domain) updateStatsWorker(ctx context.Context, lease time.Duration) { log.Error("[stats] update stats info fail: ", errors.ErrorStack(err)) } case <-do.exit: + do.wg.Done() return // This channel is sent only by ddl owner or the drop stats executor. case t := <-statsHandle.DDLEventCh(): diff --git a/executor/executor_test.go b/executor/executor_test.go index 1d96181fc7212..7368ad1a34b32 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -100,8 +100,11 @@ func (s *testSuite) TearDownSuite(c *C) { atomic.StoreInt32(&expression.TurnOnNewExprEval, 0) } +func (s *testSuite) SetUpTest(c *C) { + testleak.BeforeTest() +} + func (s *testSuite) TearDownTest(c *C) { - testleak.AfterTest(c)() tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") r := tk.MustQuery("show tables") @@ -109,6 +112,7 @@ func (s *testSuite) TearDownTest(c *C) { tableName := tb[0] tk.MustExec(fmt.Sprintf("drop table %v", tableName)) } + testleak.AfterTest(c)() } func (s *testSuite) TestAdmin(c *C) { diff --git a/expression/integration_test.go b/expression/integration_test.go index fc50571047a36..a3e71e8e08fd7 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -22,6 +22,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb" "github.com/pingcap/tidb/context" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/mysql" "github.com/pingcap/tidb/terror" @@ -37,6 +38,7 @@ var _ = Suite(&testIntegrationSuite{}) type testIntegrationSuite struct { store kv.Storage + dom *domain.Domain ctx context.Context } @@ -51,10 +53,17 @@ func (s *testIntegrationSuite) cleanEnv(c *C) { } func (s *testIntegrationSuite) SetUpSuite(c *C) { - s.store, _ = newStoreWithBootstrap() + var err error + s.store, s.dom, err = newStoreWithBootstrap() + c.Assert(err, IsNil) s.ctx = mock.NewContext() } +func (s *testIntegrationSuite) TearDownSuite(c *C) { + s.dom.Close() + s.store.Close() +} + func (s *testIntegrationSuite) TestFuncREPEAT(c *C) { tk := testkit.NewTestKit(c, s.store) defer func() { diff --git a/expression/typeinferer_test.go b/expression/typeinferer_test.go index a61e08092cb5a..0e147ce1eb0f2 100644 --- a/expression/typeinferer_test.go +++ b/expression/typeinferer_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb" "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" @@ -39,9 +40,10 @@ type testTypeInferrerSuite struct { func (ts *testTypeInferrerSuite) TestInferType(c *C) { c.Skip("we re-implement this test in plan/typeinfer_test.go") - store, err := newStoreWithBootstrap() + store, dom, err := newStoreWithBootstrap() c.Assert(err, IsNil) defer store.Close() + defer dom.Close() testKit := testkit.NewTestKit(c, store) testKit.MustExec("use test") sql := `create table t ( @@ -370,9 +372,10 @@ func (ts *testTypeInferrerSuite) TestInferType(c *C) { func (s *testTypeInferrerSuite) TestColumnInfoModified(c *C) { defer testleak.AfterTest(c)() - store, err := newStoreWithBootstrap() + store, dom, err := newStoreWithBootstrap() c.Assert(err, IsNil) defer store.Close() + defer dom.Close() testKit := testkit.NewTestKit(c, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists tab0") @@ -387,9 +390,10 @@ func (s *testTypeInferrerSuite) TestColumnInfoModified(c *C) { func (s *testTypeInferrerSuite) TestIsHybridType(c *C) { defer testleak.AfterTest(c)() - store, err := newStoreWithBootstrap() + store, dom, err := newStoreWithBootstrap() c.Assert(err, IsNil) defer store.Close() + defer dom.Close() testKit := testkit.NewTestKit(c, store) testKit.MustExec("use test") testKit.MustExec("drop table if exists t") @@ -436,12 +440,12 @@ func (s *testTypeInferrerSuite) TestIsHybridType(c *C) { } } -func newStoreWithBootstrap() (kv.Storage, error) { +func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { store, err := tikv.NewMockTikvStore() if err != nil { - return nil, errors.Trace(err) + return nil, nil, errors.Trace(err) } tidb.SetSchemaLease(0) - _, err = tidb.BootstrapSession(store) - return store, errors.Trace(err) + dom, err := tidb.BootstrapSession(store) + return store, dom, errors.Trace(err) } diff --git a/new_session_test.go b/new_session_test.go index ab3c708967010..81a31e54d0870 100644 --- a/new_session_test.go +++ b/new_session_test.go @@ -44,9 +44,11 @@ type testSessionSuite struct { cluster *mocktikv.Cluster mvccStore *mocktikv.MvccStore store kv.Storage + dom *domain.Domain } func (s *testSessionSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) s.mvccStore = mocktikv.NewMvccStore() @@ -58,12 +60,17 @@ func (s *testSessionSuite) SetUpSuite(c *C) { s.store = store tidb.SetSchemaLease(0) tidb.SetStatsLease(0) - _, err = tidb.BootstrapSession(s.store) + s.dom, err = tidb.BootstrapSession(s.store) c.Assert(err, IsNil) } -func (s *testSessionSuite) TearDownTest(c *C) { +func (s *testSessionSuite) TearDownSuite(c *C) { + s.dom.Close() + s.store.Close() testleak.AfterTest(c)() +} + +func (s *testSessionSuite) TearDownTest(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) r := tk.MustQuery("show tables") for _, tb := range r.Rows() { @@ -563,10 +570,11 @@ type testSchemaSuite struct { mvccStore *mocktikv.MvccStore store kv.Storage lease time.Duration + dom *domain.Domain + checkLeak func() } func (s *testSchemaSuite) TearDownTest(c *C) { - testleak.AfterTest(c)() tk := testkit.NewTestKitWithInit(c, s.store) r := tk.MustQuery("show tables") for _, tb := range r.Rows() { @@ -576,6 +584,7 @@ func (s *testSchemaSuite) TearDownTest(c *C) { } func (s *testSchemaSuite) SetUpSuite(c *C) { + testleak.BeforeTest() s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) s.mvccStore = mocktikv.NewMvccStore() @@ -588,8 +597,15 @@ func (s *testSchemaSuite) SetUpSuite(c *C) { s.lease = 20 * time.Millisecond tidb.SetSchemaLease(s.lease) tidb.SetStatsLease(0) - _, err = tidb.BootstrapSession(s.store) + dom, err := tidb.BootstrapSession(s.store) c.Assert(err, IsNil) + s.dom = dom +} + +func (s *testSchemaSuite) TearDownSuite(c *C) { + s.dom.Close() + s.store.Close() + testleak.AfterTest(c)() } func (s *testSchemaSuite) TestSchemaCheckerSQL(c *C) { diff --git a/session_test.go b/session_test.go index 65bd02fe614f9..663d9237f9d98 100644 --- a/session_test.go +++ b/session_test.go @@ -956,7 +956,9 @@ func (s *test1435Suite) TestIssue1435(c *C) { defer testleak.AfterTest(c)() localstore.MockRemoteStore = true dbName := "test_issue1435" - store := newStoreWithBootstrap(c, dbName) + store, dom := newStoreWithBootstrap(c, dbName) + defer dom.Close() + defer store.Close() se := newSession(c, store, dbName) se1 := newSession(c, store, dbName) se2 := newSession(c, store, dbName) @@ -1043,9 +1045,6 @@ func (s *test1435Suite) TestIssue1435(c *C) { se.Close() se1.Close() se2.Close() - sessionctx.GetDomain(ctx).Close() - err = store.Close() - c.Assert(err, IsNil) localstore.MockRemoteStore = false } diff --git a/statistics/handle_test.go b/statistics/handle_test.go index f1276cec75bf9..f5ccdd5caef4e 100644 --- a/statistics/handle_test.go +++ b/statistics/handle_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/types" ) @@ -37,13 +38,16 @@ type testStatsCacheSuite struct { } func (s *testStatsCacheSuite) SetUpSuite(c *C) { + testleak.BeforeTest() var err error s.store, s.do, err = newStoreWithBootstrap() c.Assert(err, IsNil) } func (s *testStatsCacheSuite) TearDownSuite(c *C) { + s.do.Close() s.store.Close() + testleak.AfterTest(c)() } func cleanEnv(c *C, store kv.Storage, do *domain.Domain) { diff --git a/statistics/selectivity_test.go b/statistics/selectivity_test.go index ae5cce4d8447f..0866d5ff45b8f 100644 --- a/statistics/selectivity_test.go +++ b/statistics/selectivity_test.go @@ -24,7 +24,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/testkit" - "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/types" ) @@ -90,10 +89,10 @@ func mockStatsTable(tbl *model.TableInfo, rowCount int64) *statistics.Table { } func (s *testSelectivitySuite) TestSelectivity(c *C) { - defer testleak.AfterTest(c)() store, do, err := newStoreWithBootstrap() defer store.Close() c.Assert(err, IsNil) + defer do.Close() testKit := testkit.NewTestKit(c, store) testKit.MustExec("use test") diff --git a/statistics/update_test.go b/statistics/update_test.go index e0b29624cda03..ceb98e33bbc44 100644 --- a/statistics/update_test.go +++ b/statistics/update_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/types" ) @@ -30,13 +31,16 @@ type testStatsUpdateSuite struct { } func (s *testStatsUpdateSuite) SetUpSuite(c *C) { + testleak.BeforeTest() var err error s.store, s.do, err = newStoreWithBootstrap() c.Assert(err, IsNil) } func (s *testStatsUpdateSuite) TearDownSuite(c *C) { + s.do.Close() s.store.Close() + testleak.AfterTest(c)() } func (s *testStatsUpdateSuite) TestSingleSessionInsert(c *C) { diff --git a/store/localstore/goleveldb/goleveldb_test.go b/store/localstore/goleveldb/goleveldb_test.go index b6b84d967690a..278e361f39334 100644 --- a/store/localstore/goleveldb/goleveldb_test.go +++ b/store/localstore/goleveldb/goleveldb_test.go @@ -32,6 +32,14 @@ type testSuite struct { db engine.DB } +func (s *testSuite) SetUpSuite(c *C) { + testleak.BeforeTest() +} + +func (s *testSuite) TearDownSuite(c *C) { + testleak.AfterTest(c)() +} + func (s *testSuite) SetUpTest(c *C) { var ( d MemoryDriver @@ -46,7 +54,6 @@ func (s *testSuite) TearDownTest(c *C) { } func (s *testSuite) TestGetSet(c *C) { - defer testleak.AfterTest(c)() db := s.db b := db.NewBatch() @@ -76,7 +83,6 @@ func (s *testSuite) TestGetSet(c *C) { } func (s *testSuite) TestSeek(c *C) { - defer testleak.AfterTest(c)() b := s.db.NewBatch() b.Put([]byte("a"), []byte("1")) b.Put([]byte("b"), []byte("2")) @@ -110,7 +116,6 @@ func (s *testSuite) TestSeek(c *C) { } func (s *testSuite) TestPrevSeek(c *C) { - defer testleak.AfterTest(c)() b := s.db.NewBatch() b.Put([]byte("b"), []byte("1")) b.Put([]byte("c"), []byte("2")) diff --git a/tidb_test.go b/tidb_test.go index 9ffe1fbf44b25..79033b4e093de 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -256,8 +256,7 @@ func (s *testMainSuite) TestRetryOpenStore(c *C) { // TODO: Merge TestIssue1435 in session test. func (s *testMainSuite) TestSchemaValidity(c *C) { localstore.MockRemoteStore = true - store := newStoreWithBootstrap(c, s.dbName+"schema_validity") - defer store.Close() + store, _ := newStoreWithBootstrap(c, s.dbName+"schema_validity") dbName := "test_schema_validity" se := newSession(c, store, dbName) se1 := newSession(c, store, dbName) @@ -348,7 +347,8 @@ func (s *testMainSuite) TestSchemaValidity(c *C) { func (s *testMainSuite) TestSysSessionPoolGoroutineLeak(c *C) { // TODO: testleak package should be able to find this leak. - store := newStoreWithBootstrap(c, s.dbName+"goroutine_leak") + store, dom := newStoreWithBootstrap(c, s.dbName+"goroutine_leak") + defer dom.Close() defer store.Close() se, err := createSession(store) c.Assert(err, IsNil) @@ -396,11 +396,11 @@ func newStore(c *C, dbPath string) kv.Storage { return store } -func newStoreWithBootstrap(c *C, dbPath string) kv.Storage { +func newStoreWithBootstrap(c *C, dbPath string) (kv.Storage, *domain.Domain) { store := newStore(c, dbPath) - _, err := BootstrapSession(store) + dom, err := BootstrapSession(store) c.Assert(err, IsNil) - return store + return store, dom } var testConnID uint64 diff --git a/util/goroutine_pool/fake.go b/util/goroutine_pool/fake.go new file mode 100644 index 0000000000000..2465572c142a8 --- /dev/null +++ b/util/goroutine_pool/fake.go @@ -0,0 +1,30 @@ +// Copyright 2017 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. +// +build leak + +package gp + +import "time" + +type Pool struct{} + +// New returns a new *Pool object. +// When compile with leak flag, goroutine will not be reusing. +func New(idleTimeout time.Duration) *Pool { + return &Pool{} +} + +// Go run f() in a new goroutine. +func (pool *Pool) Go(f func()) { + go f() +} diff --git a/util/goroutine_pool/gp.go b/util/goroutine_pool/gp.go index ff97093148833..88818156511d9 100644 --- a/util/goroutine_pool/gp.go +++ b/util/goroutine_pool/gp.go @@ -10,6 +10,7 @@ // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. +// +build !leak package gp diff --git a/util/goroutine_pool/gp_test.go b/util/goroutine_pool/gp_test.go index 2952f182305a5..747576315768c 100644 --- a/util/goroutine_pool/gp_test.go +++ b/util/goroutine_pool/gp_test.go @@ -10,6 +10,7 @@ // distributed under the License is distributed on an "AS IS" BASIS, // See the License for the specific language governing permissions and // limitations under the License. +// +build !leak package gp diff --git a/util/testleak/leaktest.go b/util/testleak/leaktest.go index 2191122498ddd..c7ae906dac6ea 100644 --- a/util/testleak/leaktest.go +++ b/util/testleak/leaktest.go @@ -42,6 +42,7 @@ func interestingGoroutines() (gs []string) { strings.Contains(stack, "check.(*resultTracker).start") || strings.Contains(stack, "localstore.(*dbStore).scheduler") || strings.Contains(stack, "ddl.(*ddl).start") || + strings.Contains(stack, "ddl.(*delRange).startEmulator") || strings.Contains(stack, "domain.NewDomain") || strings.Contains(stack, "testing.(*T).Run") || strings.Contains(stack, "domain.(*Domain).LoadPrivilegeLoop") ||