diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index 62eb19840cb24..bf45932628794 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -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) diff --git a/ddl/column_test.go b/ddl/column_test.go index b5a7dab8d7690..9bc32081ae402 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -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 diff --git a/ddl/ddl.go b/ddl/ddl.go index 87072f7bcd75c..066d05efd7d36 100644 --- a/ddl/ddl.go +++ b/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() diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 1a0d52296f85e..cfd4fae805d62 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -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) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 7b28a292f4403..5da6c2aeb56a4 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -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() diff --git a/ddl/fail_test.go b/ddl/fail_test.go index 1e8f6ef2df4bd..e55b6e664fe0c 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -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) diff --git a/ddl/foreign_key_test.go b/ddl/foreign_key_test.go index cc3ed65e3ca6a..8740e2a49be47 100644 --- a/ddl/foreign_key_test.go +++ b/ddl/foreign_key_test.go @@ -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") diff --git a/ddl/index_change_test.go b/ddl/index_change_test.go index 770b5a5bfba02..d80f3eee5ecec 100644 --- a/ddl/index_change_test.go +++ b/ddl/index_change_test.go @@ -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) diff --git a/ddl/options.go b/ddl/options.go new file mode 100644 index 0000000000000..729c0faa4d0da --- /dev/null +++ b/ddl/options.go @@ -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 + } +} diff --git a/ddl/options_test.go b/ddl/options_test.go new file mode 100644 index 0000000000000..9ea6e6523b709 --- /dev/null +++ b/ddl/options_test.go @@ -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) +} diff --git a/ddl/reorg_test.go b/ddl/reorg_test.go index 6ceff6b7687ac..3eb5e12d9906f 100644 --- a/ddl/reorg_test.go +++ b/ddl/reorg_test.go @@ -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") diff --git a/ddl/schema_test.go b/ddl/schema_test.go index bc7e06ace9317..b9c2485a08189 100644 --- a/ddl/schema_test.go +++ b/ddl/schema_test.go @@ -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) diff --git a/ddl/stat_test.go b/ddl/stat_test.go index a08313be4813f..37a60d8a8df72 100644 --- a/ddl/stat_test.go +++ b/ddl/stat_test.go @@ -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) diff --git a/ddl/table_test.go b/ddl/table_test.go index 335893cbe0c9b..cabb81eab7f18 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -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) diff --git a/ddl/util/syncer_test.go b/ddl/util/syncer_test.go index 9199ba2ac2857..6a7b8789bd2a8 100644 --- a/ddl/util/syncer_test.go +++ b/ddl/util/syncer_test.go @@ -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) diff --git a/domain/domain.go b/domain/domain.go index 4d08582adc6da..f3fa3578eab6d 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -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 { diff --git a/domain/domain_test.go b/domain/domain_test.go index 8353330c2dc2e..a82bcb41e655b 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -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) diff --git a/owner/manager_test.go b/owner/manager_test.go index a83e4dc699352..7d986b3cb2721 100644 --- a/owner/manager_test.go +++ b/owner/manager_test.go @@ -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 {