ddl: use the Options to reorganize the NewDDL paramaters (#11966)
This commit is contained in:
@ -59,7 +59,11 @@ func (s *testColumnChangeSuite) TearDownSuite(c *C) {
|
||||
}
|
||||
|
||||
func (s *testColumnChangeSuite) TestColumnChange(c *C) {
|
||||
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
// create table t (c1 int, c2 int);
|
||||
tblInfo := testTableInfo(c, d, "t", 2)
|
||||
|
||||
@ -45,7 +45,11 @@ type testColumnSuite struct {
|
||||
|
||||
func (s *testColumnSuite) SetUpSuite(c *C) {
|
||||
s.store = testCreateStore(c, "test_column")
|
||||
s.d = testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
s.d = newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
|
||||
s.dbInfo = testSchemaInfo(c, s.d, "test_column")
|
||||
testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo)
|
||||
@ -753,7 +757,11 @@ func (s *testColumnSuite) testGetColumn(t table.Table, name string, isExist bool
|
||||
}
|
||||
|
||||
func (s *testColumnSuite) TestAddColumn(c *C) {
|
||||
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
tblInfo := testTableInfo(c, d, "t", 3)
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -842,7 +850,11 @@ func (s *testColumnSuite) TestAddColumn(c *C) {
|
||||
}
|
||||
|
||||
func (s *testColumnSuite) TestDropColumn(c *C) {
|
||||
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
tblInfo := testTableInfo(c, d, "t", 4)
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -919,7 +931,11 @@ func (s *testColumnSuite) TestDropColumn(c *C) {
|
||||
}
|
||||
|
||||
func (s *testColumnSuite) TestModifyColumn(c *C) {
|
||||
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
tests := []struct {
|
||||
origin string
|
||||
|
||||
28
ddl/ddl.go
28
ddl/ddl.go
@ -23,7 +23,6 @@ import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/ngaut/pools"
|
||||
"github.com/pingcap/errors"
|
||||
"github.com/pingcap/failpoint"
|
||||
@ -346,21 +345,22 @@ func asyncNotifyEvent(d *ddlCtx, e *util.Event) {
|
||||
}
|
||||
|
||||
// NewDDL creates a new DDL.
|
||||
func NewDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage,
|
||||
infoHandle *infoschema.Handle, hook Callback, lease time.Duration, ctxPool *pools.ResourcePool) DDL {
|
||||
return newDDL(ctx, etcdCli, store, infoHandle, hook, lease, ctxPool)
|
||||
func NewDDL(ctx context.Context, options ...Option) DDL {
|
||||
return newDDL(ctx, options...)
|
||||
}
|
||||
|
||||
func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage,
|
||||
infoHandle *infoschema.Handle, hook Callback, lease time.Duration, ctxPool *pools.ResourcePool) *ddl {
|
||||
if hook == nil {
|
||||
hook = &BaseCallback{}
|
||||
func newDDL(ctx context.Context, options ...Option) *ddl {
|
||||
opt := &Options{
|
||||
Hook: &BaseCallback{},
|
||||
}
|
||||
for _, o := range options {
|
||||
o(opt)
|
||||
}
|
||||
id := uuid.NewV4().String()
|
||||
ctx, cancelFunc := context.WithCancel(ctx)
|
||||
var manager owner.Manager
|
||||
var syncer util.SchemaSyncer
|
||||
if etcdCli == nil {
|
||||
if etcdCli := opt.EtcdCli; etcdCli == nil {
|
||||
// The etcdCli is nil if the store is localstore which is only used for testing.
|
||||
// So we use mockOwnerManager and MockSchemaSyncer.
|
||||
manager = owner.NewMockManager(id, cancelFunc)
|
||||
@ -372,21 +372,21 @@ func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage,
|
||||
|
||||
ddlCtx := &ddlCtx{
|
||||
uuid: id,
|
||||
store: store,
|
||||
lease: lease,
|
||||
store: opt.Store,
|
||||
lease: opt.Lease,
|
||||
ddlJobDoneCh: make(chan struct{}, 1),
|
||||
ownerManager: manager,
|
||||
schemaSyncer: syncer,
|
||||
binlogCli: binloginfo.GetPumpsClient(),
|
||||
infoHandle: infoHandle,
|
||||
infoHandle: opt.InfoHandle,
|
||||
}
|
||||
ddlCtx.mu.hook = hook
|
||||
ddlCtx.mu.hook = opt.Hook
|
||||
ddlCtx.mu.interceptor = &BaseInterceptor{}
|
||||
d := &ddl{
|
||||
ddlCtx: ddlCtx,
|
||||
}
|
||||
|
||||
d.start(ctx, ctxPool)
|
||||
d.start(ctx, opt.ResourcePool)
|
||||
variable.RegisterStatistics(d)
|
||||
|
||||
metrics.DDLCounter.WithLabelValues(metrics.CreateDDLInstance).Inc()
|
||||
|
||||
@ -19,13 +19,11 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/log"
|
||||
"github.com/pingcap/parser/ast"
|
||||
"github.com/pingcap/parser/model"
|
||||
"github.com/pingcap/parser/terror"
|
||||
"github.com/pingcap/tidb/infoschema"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
"github.com/pingcap/tidb/meta"
|
||||
"github.com/pingcap/tidb/meta/autoid"
|
||||
@ -116,11 +114,6 @@ func testNewContext(d *ddl) sessionctx.Context {
|
||||
return ctx
|
||||
}
|
||||
|
||||
func testNewDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage,
|
||||
infoHandle *infoschema.Handle, hook Callback, lease time.Duration) *ddl {
|
||||
return newDDL(ctx, etcdCli, store, infoHandle, hook, lease, nil)
|
||||
}
|
||||
|
||||
func getSchemaVer(c *C, ctx sessionctx.Context) int64 {
|
||||
err := ctx.NewTxn(context.Background())
|
||||
c.Assert(err, IsNil)
|
||||
|
||||
@ -53,7 +53,11 @@ func (s *testDDLSuite) TestCheckOwner(c *C) {
|
||||
store := testCreateStore(c, "test_owner")
|
||||
defer store.Close()
|
||||
|
||||
d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d1 := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d1.Stop()
|
||||
time.Sleep(testLease)
|
||||
testCheckOwner(c, d1, true)
|
||||
@ -67,7 +71,11 @@ func (s *testDDLSuite) testRunWorker(c *C) {
|
||||
defer store.Close()
|
||||
|
||||
RunWorker = false
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
testCheckOwner(c, d, false)
|
||||
defer d.Stop()
|
||||
|
||||
@ -76,7 +84,11 @@ func (s *testDDLSuite) testRunWorker(c *C) {
|
||||
c.Assert(worker, IsNil)
|
||||
// Make sure the DDL job can be done and exit that goroutine.
|
||||
RunWorker = true
|
||||
d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d1 := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
testCheckOwner(c, d1, true)
|
||||
defer d1.Stop()
|
||||
worker = d1.generalWorker()
|
||||
@ -87,7 +99,11 @@ func (s *testDDLSuite) TestSchemaError(c *C) {
|
||||
store := testCreateStore(c, "test_schema_error")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -98,7 +114,11 @@ func (s *testDDLSuite) TestTableError(c *C) {
|
||||
store := testCreateStore(c, "test_table_error")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -139,7 +159,11 @@ func (s *testDDLSuite) TestViewError(c *C) {
|
||||
store := testCreateStore(c, "test_view_error")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
dbInfo := testSchemaInfo(c, d, "test")
|
||||
@ -162,7 +186,11 @@ func (s *testDDLSuite) TestViewError(c *C) {
|
||||
func (s *testDDLSuite) TestInvalidDDLJob(c *C) {
|
||||
store := testCreateStore(c, "test_invalid_ddl_job_type_error")
|
||||
defer store.Close()
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -181,7 +209,11 @@ func (s *testDDLSuite) TestForeignKeyError(c *C) {
|
||||
store := testCreateStore(c, "test_foreign_key_error")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -199,7 +231,11 @@ func (s *testDDLSuite) TestIndexError(c *C) {
|
||||
store := testCreateStore(c, "test_index_error")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -234,7 +270,11 @@ func (s *testDDLSuite) TestIndexError(c *C) {
|
||||
func (s *testDDLSuite) TestColumnError(c *C) {
|
||||
store := testCreateStore(c, "test_column_error")
|
||||
defer store.Close()
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
|
||||
@ -443,7 +483,11 @@ func (s *testDDLSuite) checkCancelDropColumn(c *C, d *ddl, schemaID int64, table
|
||||
func (s *testDDLSuite) TestCancelJob(c *C) {
|
||||
store := testCreateStore(c, "test_cancel_job")
|
||||
defer store.Close()
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
dbInfo := testSchemaInfo(c, d, "test_cancel_job")
|
||||
testCreateSchema(c, testNewContext(d), d, dbInfo)
|
||||
@ -850,7 +894,11 @@ func (s *testDDLSuite) TestBuildJobDependence(c *C) {
|
||||
func (s *testDDLSuite) TestParallelDDL(c *C) {
|
||||
store := testCreateStore(c, "test_parallel_ddl")
|
||||
defer store.Close()
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
err := ctx.NewTxn(context.Background())
|
||||
@ -1040,7 +1088,11 @@ func (s *testDDLSuite) TestDDLPackageExecuteSQL(c *C) {
|
||||
store := testCreateStore(c, "test_run_sql")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
testCheckOwner(c, d, true)
|
||||
defer d.Stop()
|
||||
worker := d.generalWorker()
|
||||
|
||||
@ -24,7 +24,11 @@ import (
|
||||
)
|
||||
|
||||
func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) {
|
||||
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
// create table t_fail (c1 int, c2 int);
|
||||
tblInfo := testTableInfo(c, d, "t_fail", 2)
|
||||
|
||||
@ -111,7 +111,11 @@ func getForeignKey(t table.Table, name string) *model.FKInfo {
|
||||
}
|
||||
|
||||
func (s *testForeighKeySuite) TestForeignKey(c *C) {
|
||||
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
s.d = d
|
||||
s.dbInfo = testSchemaInfo(c, d, "test_foreign")
|
||||
|
||||
@ -52,7 +52,11 @@ func (s *testIndexChangeSuite) TearDownSuite(c *C) {
|
||||
}
|
||||
|
||||
func (s *testIndexChangeSuite) TestIndexChange(c *C) {
|
||||
d := testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
// create table t (c1 int primary key, c2 int);
|
||||
tblInfo := testTableInfo(c, d, "t", 2)
|
||||
|
||||
78
ddl/options.go
Normal file
78
ddl/options.go
Normal file
@ -0,0 +1,78 @@
|
||||
// Copyright 2019 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://wwm.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.
|
||||
|
||||
package ddl
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/ngaut/pools"
|
||||
"github.com/pingcap/tidb/infoschema"
|
||||
"github.com/pingcap/tidb/kv"
|
||||
)
|
||||
|
||||
// Option represents an option to initialize the DDL module
|
||||
type Option func(*Options)
|
||||
|
||||
// Options represents all the options of the DDL module needs
|
||||
type Options struct {
|
||||
EtcdCli *clientv3.Client
|
||||
Store kv.Storage
|
||||
InfoHandle *infoschema.Handle
|
||||
Hook Callback
|
||||
Lease time.Duration
|
||||
ResourcePool *pools.ResourcePool
|
||||
}
|
||||
|
||||
// WithEtcdClient specifies the `clientv3.Client` of DDL used to request the etcd service
|
||||
func WithEtcdClient(client *clientv3.Client) Option {
|
||||
return func(options *Options) {
|
||||
options.EtcdCli = client
|
||||
}
|
||||
}
|
||||
|
||||
// WithStore specifies the `kv.Storage` of DDL used to request the KV service
|
||||
func WithStore(store kv.Storage) Option {
|
||||
return func(options *Options) {
|
||||
options.Store = store
|
||||
}
|
||||
}
|
||||
|
||||
// WithInfoHandle specifies the `infoschema.Handle`
|
||||
func WithInfoHandle(ih *infoschema.Handle) Option {
|
||||
return func(options *Options) {
|
||||
options.InfoHandle = ih
|
||||
}
|
||||
}
|
||||
|
||||
// WithHook specifies the `Callback` of DDL used to notify the outer module when events are triggered
|
||||
func WithHook(callback Callback) Option {
|
||||
return func(options *Options) {
|
||||
options.Hook = callback
|
||||
}
|
||||
}
|
||||
|
||||
// WithLease specifies the schema lease duration
|
||||
func WithLease(lease time.Duration) Option {
|
||||
return func(options *Options) {
|
||||
options.Lease = lease
|
||||
}
|
||||
}
|
||||
|
||||
// WithResourcePool specifies the `pools.ResourcePool` of DDL used
|
||||
func WithResourcePool(pools *pools.ResourcePool) Option {
|
||||
return func(options *Options) {
|
||||
options.ResourcePool = pools
|
||||
}
|
||||
}
|
||||
60
ddl/options_test.go
Normal file
60
ddl/options_test.go
Normal file
@ -0,0 +1,60 @@
|
||||
// Copyright 2019 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.
|
||||
|
||||
package ddl_test
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/coreos/etcd/clientv3"
|
||||
"github.com/ngaut/pools"
|
||||
. "github.com/pingcap/check"
|
||||
"github.com/pingcap/tidb/ddl"
|
||||
"github.com/pingcap/tidb/infoschema"
|
||||
"github.com/pingcap/tidb/util/mock"
|
||||
)
|
||||
|
||||
type ddlOptionsSuite struct{}
|
||||
|
||||
var _ = Suite(&ddlOptionsSuite{})
|
||||
|
||||
func (s *ddlOptionsSuite) TestOptions(c *C) {
|
||||
client, err := clientv3.NewFromURL("test")
|
||||
c.Assert(err, IsNil)
|
||||
callback := &ddl.BaseCallback{}
|
||||
lease := time.Second * 3
|
||||
store := &mock.Store{}
|
||||
infoHandle := infoschema.NewHandle(store)
|
||||
pools := &pools.ResourcePool{}
|
||||
|
||||
options := []ddl.Option{
|
||||
ddl.WithEtcdClient(client),
|
||||
ddl.WithHook(callback),
|
||||
ddl.WithLease(lease),
|
||||
ddl.WithStore(store),
|
||||
ddl.WithInfoHandle(infoHandle),
|
||||
ddl.WithResourcePool(pools),
|
||||
}
|
||||
|
||||
opt := &ddl.Options{}
|
||||
for _, o := range options {
|
||||
o(opt)
|
||||
}
|
||||
|
||||
c.Assert(opt.EtcdCli, Equals, client)
|
||||
c.Assert(opt.Hook, Equals, callback)
|
||||
c.Assert(opt.Lease, Equals, lease)
|
||||
c.Assert(opt.Store, Equals, store)
|
||||
c.Assert(opt.InfoHandle, Equals, infoHandle)
|
||||
c.Assert(opt.ResourcePool, Equals, pools)
|
||||
}
|
||||
@ -36,7 +36,11 @@ func (s *testDDLSuite) TestReorg(c *C) {
|
||||
store := testCreateStore(c, "test_reorg")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
|
||||
time.Sleep(testLease)
|
||||
@ -159,14 +163,22 @@ func (s *testDDLSuite) TestReorgOwner(c *C) {
|
||||
store := testCreateStore(c, "test_reorg_owner")
|
||||
defer store.Close()
|
||||
|
||||
d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d1 := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d1.Stop()
|
||||
|
||||
ctx := testNewContext(d1)
|
||||
|
||||
testCheckOwner(c, d1, true)
|
||||
|
||||
d2 := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d2 := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d2.Stop()
|
||||
|
||||
dbInfo := testSchemaInfo(c, d1, "test")
|
||||
|
||||
@ -125,7 +125,11 @@ func testCheckSchemaState(c *C, d *ddl, dbInfo *model.DBInfo, state model.Schema
|
||||
func (s *testSchemaSuite) TestSchema(c *C) {
|
||||
store := testCreateStore(c, "test_schema")
|
||||
defer store.Close()
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
ctx := testNewContext(d)
|
||||
dbInfo := testSchemaInfo(c, d, "test")
|
||||
@ -186,12 +190,20 @@ func (s *testSchemaSuite) TestSchemaWaitJob(c *C) {
|
||||
store := testCreateStore(c, "test_schema_wait")
|
||||
defer store.Close()
|
||||
|
||||
d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d1 := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d1.Stop()
|
||||
|
||||
testCheckOwner(c, d1, true)
|
||||
|
||||
d2 := testNewDDL(context.Background(), nil, store, nil, nil, testLease*4)
|
||||
d2 := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease*4),
|
||||
)
|
||||
defer d2.Stop()
|
||||
ctx := testNewContext(d2)
|
||||
|
||||
@ -240,7 +252,11 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) {
|
||||
store := testCreateStore(c, "test_schema_resume")
|
||||
defer store.Close()
|
||||
|
||||
d1 := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d1 := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d1.Stop()
|
||||
|
||||
testCheckOwner(c, d1, true)
|
||||
|
||||
@ -44,7 +44,11 @@ func (s *testStatSuite) TestStat(c *C) {
|
||||
store := testCreateStore(c, "test_stat")
|
||||
defer store.Close()
|
||||
|
||||
d := testNewDDL(context.Background(), nil, store, nil, nil, testLease)
|
||||
d := newDDL(
|
||||
context.Background(),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
|
||||
time.Sleep(testLease)
|
||||
|
||||
@ -308,7 +308,11 @@ func testGetTableWithError(d *ddl, schemaID, tableID int64) (table.Table, error)
|
||||
|
||||
func (s *testTableSuite) SetUpSuite(c *C) {
|
||||
s.store = testCreateStore(c, "test_table")
|
||||
s.d = testNewDDL(context.Background(), nil, s.store, nil, nil, testLease)
|
||||
s.d = newDDL(
|
||||
context.Background(),
|
||||
WithStore(s.store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
|
||||
s.dbInfo = testSchemaInfo(c, s.d, "test")
|
||||
testCreateSchema(c, testNewContext(s.d), s.d, s.dbInfo)
|
||||
|
||||
@ -55,7 +55,12 @@ func TestSyncerSimple(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.RandClient()
|
||||
ctx := goctx.Background()
|
||||
d := NewDDL(ctx, cli, store, nil, nil, testLease, nil)
|
||||
d := NewDDL(
|
||||
ctx,
|
||||
WithEtcdClient(cli),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
|
||||
// for init function
|
||||
@ -82,7 +87,12 @@ func TestSyncerSimple(t *testing.T) {
|
||||
t.Fatalf("client get global version result not match, err %v", err)
|
||||
}
|
||||
|
||||
d1 := NewDDL(ctx, cli, store, nil, nil, testLease, nil)
|
||||
d1 := NewDDL(
|
||||
ctx,
|
||||
WithEtcdClient(cli),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d1.Stop()
|
||||
if err = d1.SchemaSyncer().Init(ctx); err != nil {
|
||||
t.Fatalf("schema version syncer init failed %v", err)
|
||||
|
||||
@ -633,7 +633,15 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R
|
||||
ctx := context.Background()
|
||||
callback := &ddlCallback{do: do}
|
||||
d := do.ddl
|
||||
do.ddl = ddl.NewDDL(ctx, do.etcdClient, do.store, do.infoHandle, callback, ddlLease, sysCtxPool)
|
||||
do.ddl = ddl.NewDDL(
|
||||
ctx,
|
||||
ddl.WithEtcdClient(do.etcdClient),
|
||||
ddl.WithStore(do.store),
|
||||
ddl.WithInfoHandle(do.infoHandle),
|
||||
ddl.WithHook(callback),
|
||||
ddl.WithLease(ddlLease),
|
||||
ddl.WithResourcePool(sysCtxPool),
|
||||
)
|
||||
failpoint.Inject("MockReplaceDDL", func(val failpoint.Value) {
|
||||
if val.(bool) {
|
||||
if err := do.ddl.Stop(); err != nil {
|
||||
|
||||
@ -92,7 +92,13 @@ func TestInfo(t *testing.T) {
|
||||
dom.etcdClient = cli
|
||||
// Mock new DDL and init the schema syncer with etcd client.
|
||||
goCtx := context.Background()
|
||||
dom.ddl = ddl.NewDDL(goCtx, dom.GetEtcdClient(), s, dom.infoHandle, nil, ddlLease, nil)
|
||||
dom.ddl = ddl.NewDDL(
|
||||
goCtx,
|
||||
ddl.WithEtcdClient(dom.GetEtcdClient()),
|
||||
ddl.WithStore(s),
|
||||
ddl.WithInfoHandle(dom.infoHandle),
|
||||
ddl.WithLease(ddlLease),
|
||||
)
|
||||
err = failpoint.Enable("github.com/pingcap/tidb/domain/MockReplaceDDL", `return(true)`)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
|
||||
@ -58,7 +58,12 @@ func TestSingle(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
cli := clus.RandClient()
|
||||
ctx := goctx.Background()
|
||||
d := NewDDL(ctx, cli, store, nil, nil, testLease, nil)
|
||||
d := NewDDL(
|
||||
ctx,
|
||||
WithEtcdClient(cli),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d.Stop()
|
||||
|
||||
isOwner := checkOwner(d, true)
|
||||
@ -109,13 +114,23 @@ func TestCluster(t *testing.T) {
|
||||
defer clus.Terminate(t)
|
||||
|
||||
cli := clus.Client(0)
|
||||
d := NewDDL(goctx.Background(), cli, store, nil, nil, testLease, nil)
|
||||
d := NewDDL(
|
||||
goctx.Background(),
|
||||
WithEtcdClient(cli),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
isOwner := checkOwner(d, true)
|
||||
if !isOwner {
|
||||
t.Fatalf("expect true, got isOwner:%v", isOwner)
|
||||
}
|
||||
cli1 := clus.Client(1)
|
||||
d1 := NewDDL(goctx.Background(), cli1, store, nil, nil, testLease, nil)
|
||||
d1 := NewDDL(
|
||||
goctx.Background(),
|
||||
WithEtcdClient(cli1),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
isOwner = checkOwner(d1, false)
|
||||
if isOwner {
|
||||
t.Fatalf("expect false, got isOwner:%v", isOwner)
|
||||
@ -135,7 +150,12 @@ func TestCluster(t *testing.T) {
|
||||
|
||||
// d3 (not owner) stop
|
||||
cli3 := clus.Client(3)
|
||||
d3 := NewDDL(goctx.Background(), cli3, store, nil, nil, testLease, nil)
|
||||
d3 := NewDDL(
|
||||
goctx.Background(),
|
||||
WithEtcdClient(cli3),
|
||||
WithStore(store),
|
||||
WithLease(testLease),
|
||||
)
|
||||
defer d3.Stop()
|
||||
isOwner = checkOwner(d3, false)
|
||||
if isOwner {
|
||||
|
||||
Reference in New Issue
Block a user