Skip to content

Commit

Permalink
mockstore: introducing embedded unistore (pingcap#17156)
Browse files Browse the repository at this point in the history
  • Loading branch information
Zejun Li authored May 18, 2020
1 parent f06fc35 commit ac30f53
Show file tree
Hide file tree
Showing 86 changed files with 1,208 additions and 433 deletions.
15 changes: 5 additions & 10 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@ import (
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/cluster"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/stmtsummary"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -67,15 +66,11 @@ func (s *testSuite) SetUpSuite(c *C) {
flag.Lookup("mockTikv")
useMockTikv := *mockTikv
if useMockTikv {
cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
s.cluster = cluster

mvccStore := mocktikv.MustNewMVCCStore()
cluster.SetMvccStore(mvccStore)
store, err := mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
mockstore.WithMVCCStore(mvccStore),
store, err := mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
s.cluster = c
}),
)
c.Assert(err, IsNil)
s.store = store
Expand Down
2 changes: 2 additions & 0 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/util/logutil"
tracing "github.com/uber/jaeger-client-go/config"

"go.uber.org/zap"
)

Expand Down Expand Up @@ -64,6 +65,7 @@ var (
ValidStorage = map[string]bool{
"mocktikv": true,
"tikv": true,
"unistore": true,
}
// checkTableBeforeDrop enable to execute `admin check table` before `drop table`.
CheckTableBeforeDrop = false
Expand Down
2 changes: 1 addition & 1 deletion ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ func (s *testStateChangeSuiteBase) SetUpSuite(c *C) {
s.lease = 200 * time.Millisecond
ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond)
var err error
s.store, err = mockstore.NewMockTikvStore()
s.store, err = mockstore.NewMockStore()
c.Assert(err, IsNil)
session.SetSchemaLease(s.lease)
s.dom, err = session.BootstrapSession(s.store)
Expand Down
15 changes: 5 additions & 10 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,6 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/cluster"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/israce"
Expand Down Expand Up @@ -71,15 +70,11 @@ func setupIntegrationSuite(s *testIntegrationSuite, c *C) {
s.lease = 50 * time.Millisecond
ddl.SetWaitTimeWhenErrorOccurred(0)

cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
s.cluster = cluster

mvccStore := mocktikv.MustNewMVCCStore()
cluster.SetMvccStore(mvccStore)
s.store, err = mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
mockstore.WithMVCCStore(mvccStore),
s.store, err = mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
s.cluster = c
}),
)
c.Assert(err, IsNil)
session.SetSchemaLease(s.lease)
Expand Down
15 changes: 5 additions & 10 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/cluster"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -95,15 +94,11 @@ func setUpSuite(s *testDBSuite, c *C) {
s.autoIDStep = autoid.GetStep()
ddl.SetWaitTimeWhenErrorOccurred(0)

cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
s.cluster = cluster

mvccStore := mocktikv.MustNewMVCCStore()
cluster.SetMvccStore(mvccStore)
s.store, err = mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
mockstore.WithMVCCStore(mvccStore),
s.store, err = mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
s.cluster = c
}),
)
c.Assert(err, IsNil)

Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ func testNewDDLAndStart(ctx context.Context, c *C, options ...Option) *ddl {
}

func testCreateStore(c *C, name string) kv.Storage {
store, err := mockstore.NewMockTikvStore()
store, err := mockstore.NewMockStore()
c.Assert(err, IsNil)
return store
}
Expand Down
15 changes: 5 additions & 10 deletions ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,6 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/cluster"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
Expand Down Expand Up @@ -66,15 +65,11 @@ func (s *testFailDBSuite) SetUpSuite(c *C) {
s.lease = 200 * time.Millisecond
ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond)
var err error
cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
s.cluster = cluster

mvccStore := mocktikv.MustNewMVCCStore()
cluster.SetMvccStore(mvccStore)
s.store, err = mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
mockstore.WithMVCCStore(mvccStore),
s.store, err = mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
s.cluster = c
}),
)
c.Assert(err, IsNil)
session.SetSchemaLease(s.lease)
Expand Down
15 changes: 5 additions & 10 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/cluster"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/gcutil"
Expand Down Expand Up @@ -71,16 +70,12 @@ func (s *testSerialSuite) SetUpSuite(c *C) {

ddl.SetWaitTimeWhenErrorOccurred(1 * time.Microsecond)

cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
s.cluster = cluster

var err error
mvccStore := mocktikv.MustNewMVCCStore()
cluster.SetMvccStore(mvccStore)
s.store, err = mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
mockstore.WithMVCCStore(mvccStore),
s.store, err = mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
s.cluster = c
}),
)
c.Assert(err, IsNil)

Expand Down
2 changes: 1 addition & 1 deletion ddl/table_split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ type testDDLTableSplitSuite struct{}
var _ = Suite(&testDDLTableSplitSuite{})

func (s *testDDLTableSplitSuite) TestTableSplit(c *C) {
store, err := mockstore.NewMockTikvStore()
store, err := mockstore.NewMockStore()
c.Assert(err, IsNil)
defer store.Close()
session.SetSchemaLease(100 * time.Millisecond)
Expand Down
2 changes: 1 addition & 1 deletion ddl/util/syncer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ func TestSyncerSimple(t *testing.T) {
CheckVersFirstWaitTime = origin
}()

store, err := mockstore.NewMockTikvStore()
store, err := mockstore.NewMockStore()
if err != nil {
t.Fatal(err)
}
Expand Down
9 changes: 1 addition & 8 deletions domain/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
)

type dbTestSuite struct{}
Expand All @@ -30,13 +29,7 @@ var _ = Suite(&dbTestSuite{})
func (ts *dbTestSuite) TestIntegration(c *C) {
var err error
lease := 50 * time.Millisecond
cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
mvccStore := mocktikv.MustNewMVCCStore()
store, err := mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
mockstore.WithMVCCStore(mvccStore),
)
store, err := mockstore.NewMockStore()
c.Assert(err, IsNil)
session.SetSchemaLease(lease)
_, err = session.BootstrapSession(store)
Expand Down
4 changes: 2 additions & 2 deletions domain/domain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,7 +148,7 @@ func TestInfo(t *testing.T) {
testleak.AfterTestT(t)()
}()
ddlLease := 80 * time.Millisecond
s, err := mockstore.NewMockTikvStore()
s, err := mockstore.NewMockStore()
if err != nil {
t.Fatal(err)
}
Expand Down Expand Up @@ -293,7 +293,7 @@ func (msm *mockSessionManager) UpdateTLSConfig(cfg *tls.Config) {}

func (*testSuite) TestT(c *C) {
defer testleak.AfterTest(c)()
store, err := mockstore.NewMockTikvStore()
store, err := mockstore.NewMockStore()
c.Assert(err, IsNil)
ddlLease := 80 * time.Millisecond
dom := NewDomain(store, ddlLease, 0, mockFactory)
Expand Down
4 changes: 2 additions & 2 deletions domain/global_vars_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ func (gvcSuite *testGVCSuite) TestSimple(c *C) {
defer testleak.AfterTest(c)()
testleak.BeforeTest()

store, err := mockstore.NewMockTikvStore()
store, err := mockstore.NewMockStore()
c.Assert(err, IsNil)
defer store.Close()
ddlLease := 50 * time.Millisecond
Expand Down Expand Up @@ -173,7 +173,7 @@ func (gvcSuite *testGVCSuite) TestCheckEnableStmtSummary(c *C) {
defer testleak.AfterTest(c)()
testleak.BeforeTest()

store, err := mockstore.NewMockTikvStore()
store, err := mockstore.NewMockStore()
c.Assert(err, IsNil)
defer store.Close()
ddlLease := 50 * time.Millisecond
Expand Down
43 changes: 23 additions & 20 deletions executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import (
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/store/mockstore/cluster"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -181,10 +181,12 @@ func (s *testSuite1) TestAnalyzeTooLongColumns(c *C) {
}

func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) {
cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
store, err := mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
var cls cluster.Cluster
store, err := mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
cls = c
}),
)
c.Assert(err, IsNil)
defer store.Close()
Expand All @@ -207,7 +209,7 @@ func (s *testFastAnalyze) TestAnalyzeFastSample(c *C) {

// construct 5 regions split by {12, 24, 36, 48}
splitKeys := generateTableSplitKeyForInt(tid, []int{12, 24, 36, 48})
manipulateCluster(cluster, splitKeys)
manipulateCluster(cls, splitKeys)

for i := 0; i < 60; i++ {
tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i))
Expand Down Expand Up @@ -274,10 +276,12 @@ func checkHistogram(sc *stmtctx.StatementContext, hg *statistics.Histogram) (boo
}

func (s *testFastAnalyze) TestFastAnalyze(c *C) {
cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
store, err := mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
var cls cluster.Cluster
store, err := mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
cls = c
}),
)
c.Assert(err, IsNil)
defer store.Close()
Expand All @@ -303,7 +307,7 @@ func (s *testFastAnalyze) TestFastAnalyze(c *C) {

// construct 6 regions split by {10, 20, 30, 40, 50}
splitKeys := generateTableSplitKeyForInt(tid, []int{10, 20, 30, 40, 50})
manipulateCluster(cluster, splitKeys)
manipulateCluster(cls, splitKeys)

for i := 0; i < 20; i++ {
tk.MustExec(fmt.Sprintf(`insert into t values (%d, %d, "char")`, i*3, i*3))
Expand Down Expand Up @@ -477,14 +481,13 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) {
return cli
}

cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
mvccStore := mocktikv.MustNewMVCCStore()
cluster.SetMvccStore(mvccStore)
store, err := mockstore.NewMockTikvStore(
mockstore.WithHijackClient(hijackClient),
mockstore.WithCluster(cluster),
mockstore.WithMVCCStore(mvccStore),
var cls cluster.Cluster
store, err := mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
cls = c
}),
mockstore.WithClientHijacker(hijackClient),
)
c.Assert(err, IsNil)
defer store.Close()
Expand All @@ -505,7 +508,7 @@ func (s *testFastAnalyze) TestFastAnalyzeRetryRowCount(c *C) {
for i := 0; i < 30; i++ {
tk.MustExec(fmt.Sprintf("insert into retry_row_count values (%d)", i))
}
cluster.SplitTable(tid, 6)
cls.SplitTable(tid, 6)
// Flush the region cache first.
tk.MustQuery("select * from retry_row_count")
tk.MustExec("analyze table retry_row_count")
Expand Down
2 changes: 1 addition & 1 deletion executor/batch_point_get_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ type testBatchPointGetSuite struct {
}

func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) {
store, err := mockstore.NewMockTikvStore()
store, err := mockstore.NewMockStore()
if err != nil {
return nil, nil, errors.Trace(err)
}
Expand Down
13 changes: 6 additions & 7 deletions executor/chunk_size_control_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/mockstore/cluster"
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -133,14 +132,14 @@ func (s *testChunkSizeControlSuite) SetUpSuite(c *C) {
kit := new(testChunkSizeControlKit)
s.m[name] = kit
kit.client = &testSlowClient{regionDelay: make(map[uint64]time.Duration)}
cluster := mocktikv.NewCluster()
mocktikv.BootstrapWithSingleStore(cluster)
kit.cluster = cluster

var err error
kit.store, err = mockstore.NewMockTikvStore(
mockstore.WithCluster(cluster),
mockstore.WithHijackClient(func(c tikv.Client) tikv.Client {
kit.store, err = mockstore.NewMockStore(
mockstore.WithClusterInspector(func(c cluster.Cluster) {
mockstore.BootstrapWithSingleStore(c)
kit.cluster = c
}),
mockstore.WithClientHijacker(func(c tikv.Client) tikv.Client {
kit.client.Client = c
return kit.client
}),
Expand Down
Loading

0 comments on commit ac30f53

Please sign in to comment.