Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into fix-sha…
Browse files Browse the repository at this point in the history
…re-bit
  • Loading branch information
crazycs520 committed Apr 9, 2019
2 parents b7ece2c + 7acbe52 commit 328ef4c
Show file tree
Hide file tree
Showing 42 changed files with 937 additions and 413 deletions.
2 changes: 1 addition & 1 deletion config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -274,7 +274,7 @@ enable = false
# WriteTimeout specifies how long it will wait for writing binlog to pump.
write-timeout = "15s"

# If IgnoreError is true, when writting binlog meets error, TiDB would stop writting binlog,
# If IgnoreError is true, when writing binlog meets error, TiDB would stop writing binlog,
# but still provide service.
ignore-error = false

Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -242,7 +242,7 @@ type DDL interface {
CreateView(ctx sessionctx.Context, stmt *ast.CreateViewStmt) error
CreateTableWithLike(ctx sessionctx.Context, ident, referIdent ast.Ident, ifNotExists bool) error
DropTable(ctx sessionctx.Context, tableIdent ast.Ident) (err error)
RestoreTable(ctx sessionctx.Context, tbInfo *model.TableInfo, schemaID, autoID, dropJobID int64, snapshotTS uint64) (err error)
RecoverTable(ctx sessionctx.Context, tbInfo *model.TableInfo, schemaID, autoID, dropJobID int64, snapshotTS uint64) (err error)
DropView(ctx sessionctx.Context, tableIdent ast.Ident) (err error)
CreateIndex(ctx sessionctx.Context, tableIdent ast.Ident, unique bool, indexName model.CIStr,
columnNames []*ast.IndexColName, indexOption *ast.IndexOption) error
Expand Down
6 changes: 3 additions & 3 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1207,7 +1207,7 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e
return errors.Trace(err)
}

func (d *ddl) RestoreTable(ctx sessionctx.Context, tbInfo *model.TableInfo, schemaID, autoID, dropJobID int64, snapshotTS uint64) (err error) {
func (d *ddl) RecoverTable(ctx sessionctx.Context, tbInfo *model.TableInfo, schemaID, autoID, dropJobID int64, snapshotTS uint64) (err error) {
is := d.GetInfoSchemaWithInterceptor(ctx)
// Check schema exist.
schema, ok := is.SchemaByID(schemaID)
Expand All @@ -1225,9 +1225,9 @@ func (d *ddl) RestoreTable(ctx sessionctx.Context, tbInfo *model.TableInfo, sche
job := &model.Job{
SchemaID: schemaID,
TableID: tbInfo.ID,
Type: model.ActionRestoreTable,
Type: model.ActionRecoverTable,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{tbInfo, autoID, dropJobID, snapshotTS, restoreTableCheckFlagNone},
Args: []interface{}{tbInfo, autoID, dropJobID, snapshotTS, recoverTableCheckFlagNone},
}
err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
Expand Down
16 changes: 8 additions & 8 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -285,8 +285,8 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) {
}
}
switch job.Type {
case model.ActionRestoreTable:
err = finishRestoreTable(w, t, job)
case model.ActionRecoverTable:
err = finishRecoverTable(w, t, job)
}
if err != nil {
return errors.Trace(err)
Expand All @@ -303,15 +303,15 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) {
return errors.Trace(err)
}

func finishRestoreTable(w *worker, t *meta.Meta, job *model.Job) error {
func finishRecoverTable(w *worker, t *meta.Meta, job *model.Job) error {
tbInfo := &model.TableInfo{}
var autoID, dropJobID, restoreTableCheckFlag int64
var autoID, dropJobID, recoverTableCheckFlag int64
var snapshotTS uint64
err := job.DecodeArgs(tbInfo, &autoID, &dropJobID, &snapshotTS, &restoreTableCheckFlag)
err := job.DecodeArgs(tbInfo, &autoID, &dropJobID, &snapshotTS, &recoverTableCheckFlag)
if err != nil {
return errors.Trace(err)
}
if restoreTableCheckFlag == restoreTableCheckFlagEnableGC {
if recoverTableCheckFlag == recoverTableCheckFlagEnableGC {
err = enableGC(w)
if err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -532,8 +532,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
ver, err = onAddTablePartition(t, job)
case model.ActionModifyTableCharsetAndCollate:
ver, err = onModifyTableCharsetAndCollate(t, job)
case model.ActionRestoreTable:
ver, err = w.onRestoreTable(d, t, job)
case model.ActionRecoverTable:
ver, err = w.onRecoverTable(d, t, job)
default:
// Invalid job, cancel it.
job.State = model.JobStateCancelled
Expand Down
94 changes: 47 additions & 47 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,10 +128,10 @@ func (s *testSerialSuite) TestCancelAddIndexPanic(c *C) {
c.Assert(err.Error(), Equals, "[ddl:12]cancelled DDL job")
}

func (s *testSerialSuite) TestRestoreTableByJobID(c *C) {
func (s *testSerialSuite) TestRecoverTableByJobID(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_restore")
tk.MustExec("use test_restore")
tk.MustExec("create database if not exists test_recover")
tk.MustExec("use test_recover")
tk.MustExec("drop table if exists t_recover")
tk.MustExec("create table t_recover (a int);")
defer func(originGC bool) {
Expand Down Expand Up @@ -162,19 +162,19 @@ func (s *testSerialSuite) TestRestoreTableByJobID(c *C) {
rows, err := session.GetRows4Test(context.Background(), tk.Se, rs)
c.Assert(err, IsNil)
row := rows[0]
c.Assert(row.GetString(1), Equals, "test_restore")
c.Assert(row.GetString(1), Equals, "test_recover")
c.Assert(row.GetString(3), Equals, "drop table")
jobID := row.GetInt64(0)

// if GC safe point is not exists in mysql.tidb
_, err = tk.Exec(fmt.Sprintf("admin restore table by job %d", jobID))
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", jobID))
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "can not get 'tikv_gc_safe_point'")
// set GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))

// if GC enable is not exists in mysql.tidb
_, err = tk.Exec(fmt.Sprintf("admin restore table by job %d", jobID))
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", jobID))
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'")

Expand All @@ -183,31 +183,31 @@ func (s *testSerialSuite) TestRestoreTableByJobID(c *C) {

// recover job is before GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeAfterDrop))
_, err = tk.Exec(fmt.Sprintf("admin restore table by job %d", jobID))
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", jobID))
c.Assert(err, NotNil)
c.Assert(strings.Contains(err.Error(), "snapshot is older than GC safe point"), Equals, true)

// set GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))
// if there is a new table with the same name, should return failed.
tk.MustExec("create table t_recover (a int);")
_, err = tk.Exec(fmt.Sprintf("admin restore table by job %d", jobID))
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", jobID))
c.Assert(err.Error(), Equals, infoschema.ErrTableExists.GenWithStackByArgs("t_recover").Error())

// drop the new table with the same name, then restore table.
// drop the new table with the same name, then recover table.
tk.MustExec("drop table t_recover")

// do restore table.
tk.MustExec(fmt.Sprintf("admin restore table by job %d", jobID))
// do recover table.
tk.MustExec(fmt.Sprintf("recover table by job %d", jobID))

// check recover table meta and data record.
tk.MustQuery("select * from t_recover;").Check(testkit.Rows("1", "2", "3"))
// check recover table autoID.
tk.MustExec("insert into t_recover values (4),(5),(6)")
tk.MustQuery("select * from t_recover;").Check(testkit.Rows("1", "2", "3", "4", "5", "6"))

// restore table by none exits job.
_, err = tk.Exec(fmt.Sprintf("admin restore table by job %d", 10000000))
// recover table by none exits job.
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", 10000000))
c.Assert(err, NotNil)

// Disable GC by manual first, then after recover table, the GC enable status should also be disabled.
Expand All @@ -221,11 +221,11 @@ func (s *testSerialSuite) TestRestoreTableByJobID(c *C) {
rows, err = session.GetRows4Test(context.Background(), tk.Se, rs)
c.Assert(err, IsNil)
row = rows[0]
c.Assert(row.GetString(1), Equals, "test_restore")
c.Assert(row.GetString(1), Equals, "test_recover")
c.Assert(row.GetString(3), Equals, "drop table")
jobID = row.GetInt64(0)

tk.MustExec(fmt.Sprintf("admin restore table by job %d", jobID))
tk.MustExec(fmt.Sprintf("recover table by job %d", jobID))

// check recover table meta and data record.
tk.MustQuery("select * from t_recover;").Check(testkit.Rows("1"))
Expand All @@ -238,10 +238,10 @@ func (s *testSerialSuite) TestRestoreTableByJobID(c *C) {
c.Assert(gcEnable, Equals, false)
}

func (s *testSerialSuite) TestRestoreTableByTableName(c *C) {
func (s *testSerialSuite) TestRecoverTableByTableName(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_restore")
tk.MustExec("use test_restore")
tk.MustExec("create database if not exists test_recover")
tk.MustExec("use test_recover")
tk.MustExec("drop table if exists t_recover, t_recover2")
tk.MustExec("create table t_recover (a int);")
defer func(originGC bool) {
Expand All @@ -268,14 +268,14 @@ func (s *testSerialSuite) TestRestoreTableByTableName(c *C) {
tk.MustExec("drop table t_recover")

// if GC safe point is not exists in mysql.tidb
_, err := tk.Exec("admin restore table t_recover")
_, err := tk.Exec("recover table t_recover")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "can not get 'tikv_gc_safe_point'")
// set GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))

// if GC enable is not exists in mysql.tidb
_, err = tk.Exec("admin restore table t_recover")
_, err = tk.Exec("recover table t_recover")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:-1]can not get 'tikv_gc_enable'")

Expand All @@ -284,22 +284,22 @@ func (s *testSerialSuite) TestRestoreTableByTableName(c *C) {

// recover job is before GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeAfterDrop))
_, err = tk.Exec("admin restore table t_recover")
_, err = tk.Exec("recover table t_recover")
c.Assert(err, NotNil)
c.Assert(strings.Contains(err.Error(), "snapshot is older than GC safe point"), Equals, true)

// set GC safe point
tk.MustExec(fmt.Sprintf(safePointSQL, timeBeforeDrop))
// if there is a new table with the same name, should return failed.
tk.MustExec("create table t_recover (a int);")
_, err = tk.Exec("admin restore table t_recover")
_, err = tk.Exec("recover table t_recover")
c.Assert(err.Error(), Equals, infoschema.ErrTableExists.GenWithStackByArgs("t_recover").Error())

// drop the new table with the same name, then restore table.
// drop the new table with the same name, then recover table.
tk.MustExec("rename table t_recover to t_recover2")

// do restore table.
tk.MustExec("admin restore table t_recover")
// do recover table.
tk.MustExec("recover table t_recover")

// check recover table meta and data record.
tk.MustQuery("select * from t_recover;").Check(testkit.Rows("1", "2", "3"))
Expand All @@ -309,8 +309,8 @@ func (s *testSerialSuite) TestRestoreTableByTableName(c *C) {
// check rebase auto id.
tk.MustQuery("select a,_tidb_rowid from t_recover;").Check(testkit.Rows("1 1", "2 2", "3 3", "4 5001", "5 5002", "6 5003"))

// restore table by none exits job.
_, err = tk.Exec(fmt.Sprintf("admin restore table by job %d", 10000000))
// recover table by none exits job.
_, err = tk.Exec(fmt.Sprintf("recover table by job %d", 10000000))
c.Assert(err, NotNil)

// Disable GC by manual first, then after recover table, the GC enable status should also be disabled.
Expand All @@ -320,7 +320,7 @@ func (s *testSerialSuite) TestRestoreTableByTableName(c *C) {
tk.MustExec("delete from t_recover where a > 1")
tk.MustExec("drop table t_recover")

tk.MustExec("admin restore table t_recover")
tk.MustExec("recover table t_recover")

// check recover table meta and data record.
tk.MustQuery("select * from t_recover;").Check(testkit.Rows("1"))
Expand All @@ -333,10 +333,10 @@ func (s *testSerialSuite) TestRestoreTableByTableName(c *C) {
c.Assert(gcEnable, Equals, false)
}

func (s *testSerialSuite) TestRestoreTableByJobIDFail(c *C) {
func (s *testSerialSuite) TestRecoverTableByJobIDFail(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_restore")
tk.MustExec("use test_restore")
tk.MustExec("create database if not exists test_recover")
tk.MustExec("use test_recover")
tk.MustExec("drop table if exists t_recover")
tk.MustExec("create table t_recover (a int);")
defer func(originGC bool) {
Expand Down Expand Up @@ -364,7 +364,7 @@ func (s *testSerialSuite) TestRestoreTableByJobIDFail(c *C) {
rows, err := session.GetRows4Test(context.Background(), tk.Se, rs)
c.Assert(err, IsNil)
row := rows[0]
c.Assert(row.GetString(1), Equals, "test_restore")
c.Assert(row.GetString(1), Equals, "test_recover")
c.Assert(row.GetString(3), Equals, "drop table")
jobID := row.GetInt64(0)

Expand All @@ -376,21 +376,21 @@ func (s *testSerialSuite) TestRestoreTableByJobIDFail(c *C) {
// set hook
hook := &ddl.TestDDLCallback{}
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.Type == model.ActionRestoreTable {
if job.Type == model.ActionRecoverTable {
gofail.Enable("github.com/pingcap/tidb/store/tikv/mockCommitError", `return(true)`)
gofail.Enable("github.com/pingcap/tidb/ddl/mockRestoreTableCommitErr", `return(true)`)
gofail.Enable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr", `return(true)`)
}
}
origHook := s.dom.DDL().GetHook()
defer s.dom.DDL().(ddl.DDLForTest).SetHook(origHook)
s.dom.DDL().(ddl.DDLForTest).SetHook(hook)

// do restore table.
tk.MustExec(fmt.Sprintf("admin restore table by job %d", jobID))
// do recover table.
tk.MustExec(fmt.Sprintf("recover table by job %d", jobID))
gofail.Disable("github.com/pingcap/tidb/store/tikv/mockCommitError")
gofail.Disable("github.com/pingcap/tidb/ddl/mockRestoreTableCommitErr")
gofail.Disable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr")

// make sure enable GC after restore table.
// make sure enable GC after recover table.
enable, err := gcutil.CheckGCEnable(tk.Se)
c.Assert(err, IsNil)
c.Assert(enable, Equals, true)
Expand All @@ -402,10 +402,10 @@ func (s *testSerialSuite) TestRestoreTableByJobIDFail(c *C) {
tk.MustQuery("select * from t_recover;").Check(testkit.Rows("1", "2", "3", "4", "5", "6"))
}

func (s *testSerialSuite) TestRestoreTableByTableNameFail(c *C) {
func (s *testSerialSuite) TestRecoverTableByTableNameFail(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_restore")
tk.MustExec("use test_restore")
tk.MustExec("create database if not exists test_recover")
tk.MustExec("use test_recover")
tk.MustExec("drop table if exists t_recover")
tk.MustExec("create table t_recover (a int);")
defer func(originGC bool) {
Expand Down Expand Up @@ -436,21 +436,21 @@ func (s *testSerialSuite) TestRestoreTableByTableNameFail(c *C) {
// set hook
hook := &ddl.TestDDLCallback{}
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.Type == model.ActionRestoreTable {
if job.Type == model.ActionRecoverTable {
gofail.Enable("github.com/pingcap/tidb/store/tikv/mockCommitError", `return(true)`)
gofail.Enable("github.com/pingcap/tidb/ddl/mockRestoreTableCommitErr", `return(true)`)
gofail.Enable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr", `return(true)`)
}
}
origHook := s.dom.DDL().GetHook()
defer s.dom.DDL().(ddl.DDLForTest).SetHook(origHook)
s.dom.DDL().(ddl.DDLForTest).SetHook(hook)

// do restore table.
tk.MustExec("admin restore table t_recover")
// do recover table.
tk.MustExec("recover table t_recover")
gofail.Disable("github.com/pingcap/tidb/store/tikv/mockCommitError")
gofail.Disable("github.com/pingcap/tidb/ddl/mockRestoreTableCommitErr")
gofail.Disable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr")

// make sure enable GC after restore table.
// make sure enable GC after recover table.
enable, err := gcutil.CheckGCEnable(tk.Se)
c.Assert(err, IsNil)
c.Assert(enable, Equals, true)
Expand Down
Loading

0 comments on commit 328ef4c

Please sign in to comment.