Skip to content

Commit

Permalink
ddl: use the Options to reorganize the NewDDL paramaters (pingcap#11966)
Browse files Browse the repository at this point in the history
  • Loading branch information
lonng authored and winkyao committed Sep 2, 2019
1 parent bbb70be commit 5352e7c
Show file tree
Hide file tree
Showing 18 changed files with 354 additions and 59 deletions.
6 changes: 5 additions & 1 deletion ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
24 changes: 20 additions & 4 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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
Expand Down
28 changes: 14 additions & 14 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"sync"
"time"

"github.com/coreos/etcd/clientv3"
"github.com/ngaut/pools"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
Expand Down Expand Up @@ -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)
Expand All @@ -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()
Expand Down
7 changes: 0 additions & 7 deletions ddl/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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)
Expand Down
78 changes: 65 additions & 13 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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()

Expand All @@ -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()
Expand All @@ -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)

Expand All @@ -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)

Expand Down Expand Up @@ -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")
Expand All @@ -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)

Expand All @@ -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)

Expand All @@ -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)

Expand Down Expand Up @@ -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)

Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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())
Expand Down Expand Up @@ -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()
Expand Down
6 changes: 5 additions & 1 deletion ddl/fail_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
6 changes: 5 additions & 1 deletion ddl/foreign_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
Expand Down
6 changes: 5 additions & 1 deletion ddl/index_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Loading

0 comments on commit 5352e7c

Please sign in to comment.