diff --git a/ddl/db_cache_serial_test.go b/ddl/db_cache_serial_test.go
new file mode 100644
index 0000000000000..e024cef522682
--- /dev/null
+++ b/ddl/db_cache_serial_test.go
@@ -0,0 +1,97 @@
+// Copyright 2021 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,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package ddl_test
+
+import (
+	"testing"
+	"time"
+
+	"github.com/pingcap/tidb/ddl"
+	"github.com/pingcap/tidb/domain"
+	"github.com/pingcap/tidb/errno"
+	"github.com/pingcap/tidb/parser/model"
+	"github.com/pingcap/tidb/parser/terror"
+	"github.com/pingcap/tidb/session"
+	"github.com/pingcap/tidb/store/mockstore"
+	"github.com/pingcap/tidb/testkit"
+	"github.com/stretchr/testify/require"
+)
+
+func TestAlterTableCache(t *testing.T) {
+	store, err := mockstore.NewMockStore()
+	require.NoError(t, err)
+	session.SetSchemaLease(600 * time.Millisecond)
+	session.DisableStats4Test()
+	dom, err := session.BootstrapSession(store)
+	require.NoError(t, err)
+
+	dom.SetStatsUpdating(true)
+
+	clean := func() {
+		dom.Close()
+		err := store.Close()
+		require.NoError(t, err)
+	}
+	defer clean()
+	tk := testkit.NewTestKit(t, store)
+	tk2 := testkit.NewTestKit(t, store)
+
+	tk.MustExec("use test")
+	tk.MustExec("drop table if exists t1")
+	tk2.MustExec("use test")
+	/* Test of cache table */
+	tk.MustExec("create table t1 ( n int auto_increment primary key)")
+	tk.MustGetErrCode("alter table t1 ca", errno.ErrParse)
+	tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable)
+	tk.MustExec("alter table t1 cache")
+	checkTableCacheStatus(t, tk.Session(), "test", "t1", model.TableCacheStatusEnable)
+	tk.MustExec("drop table if exists t1")
+	/*Test can't skip schema checker*/
+	tk.MustExec("drop table if exists t1,t2")
+	tk.MustExec("CREATE TABLE t1 (a int)")
+	tk.MustExec("CREATE TABLE t2 (a int)")
+	tk.MustExec("begin")
+	tk.MustExec("insert into t1 set a=1;")
+	tk2.MustExec("alter table t1 cache;")
+	_, err = tk.Exec("commit")
+	require.True(t, terror.ErrorEqual(domain.ErrInfoSchemaChanged, err))
+	/* Test can skip schema checker */
+	tk.MustExec("begin")
+	tk.MustExec("drop table if exists t1")
+	tk.MustExec("CREATE TABLE t1 (a int)")
+	tk.MustExec("insert into t1 set a=2;")
+	tk2.MustExec("alter table t2 cache")
+	tk.MustExec("commit")
+	// Test if a table is not exists
+	tk.MustExec("drop table if exists t")
+	tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable)
+	tk.MustExec("create table t (a int)")
+	tk.MustExec("alter table t cache")
+	// Multiple alter cache is okay
+	tk.MustExec("alter table t cache")
+	tk.MustExec("alter table t cache")
+	// Test a temporary table
+	tk.MustExec("drop table if exists t")
+	tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)")
+	tk.MustExec("drop table if exists tmp1")
+	// local temporary table alter is not supported
+	tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation)
+	// test global temporary table
+	tk.MustExec("create global temporary table tmp1 " +
+		"(id int not null primary key, code int not null, value int default null, unique key code(code))" +
+		"on commit delete rows")
+	tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error())
+
+}
diff --git a/ddl/db_cache_test.go b/ddl/db_cache_test.go
index e92045ae131f2..dd475e1e8506f 100644
--- a/ddl/db_cache_test.go
+++ b/ddl/db_cache_test.go
@@ -15,69 +15,42 @@
 package ddl_test
 
 import (
-	. "github.com/pingcap/check"
-	"github.com/pingcap/tidb/ddl"
+	"testing"
+
 	"github.com/pingcap/tidb/domain"
 	"github.com/pingcap/tidb/errno"
 	"github.com/pingcap/tidb/parser/model"
-	"github.com/pingcap/tidb/parser/terror"
-	"github.com/pingcap/tidb/util/testkit"
+	"github.com/pingcap/tidb/session"
+	"github.com/pingcap/tidb/sessionctx"
+	"github.com/pingcap/tidb/table"
+	"github.com/pingcap/tidb/testkit"
+	"github.com/stretchr/testify/require"
 )
 
-// test alter table cache
-func (s *testDBSuite2) TestAlterTableCache(c *C) {
-	tk := testkit.NewTestKit(c, s.store)
-	tk2 := testkit.NewTestKit(c, s.store)
-	tk.MustExec("use test")
-	tk.MustExec("drop table if exists t1")
-	tk2.MustExec("use test")
-	/* Test of cache table */
-	tk.MustExec("create table t1 ( n int auto_increment primary key)")
-	tk.MustGetErrCode("alter table t1 ca", errno.ErrParse)
-	tk.MustGetErrCode("alter table t2 cache", errno.ErrNoSuchTable)
-	tk.MustExec("alter table t1 cache")
-	checkTableCacheStatus(c, tk.Se, "test", "t1", model.TableCacheStatusEnable)
-	tk.MustExec("drop table if exists t1")
-	/*Test can't skip schema checker*/
-	tk.MustExec("drop table if exists t1,t2")
-	tk.MustExec("CREATE TABLE t1 (a int)")
-	tk.MustExec("CREATE TABLE t2 (a int)")
-	tk.MustExec("begin")
-	tk.MustExec("insert into t1 set a=1;")
-	tk2.MustExec("alter table t1 cache;")
-	_, err := tk.Exec("commit")
-	c.Assert(terror.ErrorEqual(domain.ErrInfoSchemaChanged, err), IsTrue)
-	/* Test can skip schema checker */
-	tk.MustExec("begin")
-	tk.MustExec("drop table if exists t1")
-	tk.MustExec("CREATE TABLE t1 (a int)")
-	tk.MustExec("insert into t1 set a=2;")
-	tk2.MustExec("alter table t2 cache")
-	tk.MustExec("commit")
-	// Test if a table is not exists
-	tk.MustExec("drop table if exists t")
-	tk.MustGetErrCode("alter table t cache", errno.ErrNoSuchTable)
-	tk.MustExec("create table t (a int)")
-	tk.MustExec("alter table t cache")
-	// Multiple alter cache is okay
-	tk.MustExec("alter table t cache")
-	tk.MustExec("alter table t cache")
-	// Test a temporary table
-	tk.MustExec("drop table if exists t")
-	tk.MustExec("create temporary table t (id int primary key auto_increment, u int unique, v int)")
-	tk.MustExec("drop table if exists tmp1")
-	// local temporary table alter is not supported
-	tk.MustGetErrCode("alter table t cache", errno.ErrUnsupportedDDLOperation)
-	// test global temporary table
-	tk.MustExec("create global temporary table tmp1 " +
-		"(id int not null primary key, code int not null, value int default null, unique key code(code))" +
-		"on commit delete rows")
-	tk.MustGetErrMsg("alter table tmp1 cache", ddl.ErrOptOnTemporaryTable.GenWithStackByArgs("alter temporary table cache").Error())
+func checkTableCacheStatus(t *testing.T, se session.Session, dbName, tableName string, status model.TableCacheStatusType) {
+	tb := testGetTableByNameT(t, se, dbName, tableName)
+	dom := domain.GetDomain(se)
+	err := dom.Reload()
+	require.NoError(t, err)
+	require.Equal(t, status, tb.Meta().TableCacheStatusType)
+}
 
+func testGetTableByNameT(t *testing.T, ctx sessionctx.Context, db, table string) table.Table {
+	dom := domain.GetDomain(ctx)
+	// Make sure the table schema is the new schema.
+	err := dom.Reload()
+	require.NoError(t, err)
+	tbl, err := dom.InfoSchema().TableByName(model.NewCIStr(db), model.NewCIStr(table))
+	require.NoError(t, err)
+	return tbl
 }
 
-func (s *testDBSuite2) TestAlterPartitionCache(c *C) {
-	tk := testkit.NewTestKit(c, s.store)
+func TestAlterPartitionCache(t *testing.T) {
+	t.Parallel()
+	store, clean := testkit.CreateMockStore(t)
+	defer clean()
+
+	tk := testkit.NewTestKit(t, store)
 	tk.MustExec("use test;")
 	tk.MustExec("drop table if exists cache_partition_table;")
 	tk.MustExec("create table cache_partition_table (a int, b int) partition by hash(a) partitions 3;")
@@ -103,8 +76,12 @@ func (s *testDBSuite2) TestAlterPartitionCache(c *C) {
 	tk.MustExec("drop table if exists cache_partition_list_table;")
 }
 
-func (s *testDBSuite2) TestAlterViewTableCache(c *C) {
-	tk := testkit.NewTestKit(c, s.store)
+func TestAlterViewTableCache(t *testing.T) {
+	t.Parallel()
+	store, clean := testkit.CreateMockStore(t)
+	defer clean()
+
+	tk := testkit.NewTestKit(t, store)
 	tk.MustExec("use test;")
 	tk.MustExec("drop table if exists cache_view_t")
 	tk.MustExec("create table cache_view_t (id int);")
@@ -112,16 +89,20 @@ func (s *testDBSuite2) TestAlterViewTableCache(c *C) {
 	tk.MustGetErrCode("alter table v cache", errno.ErrWrongObject)
 }
 
-func (s *testDBSuite2) TestAlterTableNoCache(c *C) {
-	tk := testkit.NewTestKit(c, s.store)
+func TestAlterTableNoCache(t *testing.T) {
+	t.Parallel()
+	store, clean := testkit.CreateMockStore(t)
+	defer clean()
+
+	tk := testkit.NewTestKit(t, store)
 	tk.MustExec("use test")
 	tk.MustExec("drop table if exists nocache_t1")
 	/* Test of cache table */
 	tk.MustExec("create table nocache_t1 ( n int auto_increment primary key)")
 	tk.MustExec("alter table nocache_t1 cache")
-	checkTableCacheStatus(c, tk.Se, "test", "nocache_t1", model.TableCacheStatusEnable)
+	checkTableCacheStatus(t, tk.Session(), "test", "nocache_t1", model.TableCacheStatusEnable)
 	tk.MustExec("alter table nocache_t1 nocache")
-	checkTableCacheStatus(c, tk.Se, "test", "nocache_t1", model.TableCacheStatusDisable)
+	checkTableCacheStatus(t, tk.Session(), "test", "nocache_t1", model.TableCacheStatusDisable)
 	tk.MustExec("drop table if exists t1")
 	// Test if a table is not exists
 	tk.MustExec("drop table if exists nocache_t")
@@ -133,8 +114,12 @@ func (s *testDBSuite2) TestAlterTableNoCache(c *C) {
 	tk.MustExec("alter table nocache_t nocache")
 }
 
-func (s *testDBSuite2) TestIndexOnCacheTable(c *C) {
-	tk := testkit.NewTestKit(c, s.store)
+func TestIndexOnCacheTable(t *testing.T) {
+	t.Parallel()
+	store, clean := testkit.CreateMockStore(t)
+	defer clean()
+
+	tk := testkit.NewTestKit(t, store)
 	tk.MustExec("use test;")
 	/*Test cache table can't add/drop/rename index */
 	tk.MustExec("drop table if exists cache_index")
diff --git a/ddl/db_test.go b/ddl/db_test.go
index 173ffa2679e4b..11b7b997e2243 100644
--- a/ddl/db_test.go
+++ b/ddl/db_test.go
@@ -6446,14 +6446,6 @@ func checkTableLock(c *C, se session.Session, dbName, tableName string, lockTp m
 	}
 }
 
-func checkTableCacheStatus(c *C, se session.Session, dbName, tableName string, status model.TableCacheStatusType) {
-	tb := testGetTableByName(c, se, dbName, tableName)
-	dom := domain.GetDomain(se)
-	err := dom.Reload()
-	c.Assert(err, IsNil)
-	c.Assert(tb.Meta().TableCacheStatusType, Equals, status)
-}
-
 func (s *testDBSuite2) TestDDLWithInvalidTableInfo(c *C) {
 	tk := testkit.NewTestKit(c, s.store)