Skip to content

Commit

Permalink
*: Fix HTTP api get TiFlash replica failed cause by concurrent DDL. (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored Jun 10, 2020
1 parent 792d958 commit e053ce0
Show file tree
Hide file tree
Showing 2 changed files with 62 additions and 4 deletions.
58 changes: 58 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
testddlutil "github.com/pingcap/tidb/ddl/testutil"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
Expand Down Expand Up @@ -194,6 +195,16 @@ func testGetTableByName(c *C, ctx sessionctx.Context, db, table string) table.Ta
return tbl
}

func testGetSchemaByName(c *C, ctx sessionctx.Context, db string) *model.DBInfo {
dom := domain.GetDomain(ctx)
// Make sure the table schema is the new schema.
err := dom.Reload()
c.Assert(err, IsNil)
dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(db))
c.Assert(ok, IsTrue)
return dbInfo
}

func (s *testDBSuite) testGetTable(c *C, name string) table.Table {
ctx := s.s.(sessionctx.Context)
return testGetTableByName(c, ctx, s.schemaName, name)
Expand Down Expand Up @@ -672,6 +683,53 @@ func (s *testDBSuite5) TestCancelTruncateTable(c *C) {
s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook)
}

func (s *testDBSuite5) TestParallelDropSchemaAndDropTable(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.mustExec(c, "create database if not exists test_drop_schema_table")
s.mustExec(c, "use test_drop_schema_table")
s.mustExec(c, "create table t(c1 int, c2 int)")
var checkErr error
hook := &ddl.TestDDLCallback{}
dbInfo := testGetSchemaByName(c, s.tk.Se, "test_drop_schema_table")
done := false
var wg sync.WaitGroup
tk2 := testkit.NewTestKit(c, s.store)
tk2.MustExec("use test_drop_schema_table")
hook.OnJobUpdatedExported = func(job *model.Job) {
if job.Type == model.ActionDropSchema && job.State == model.JobStateRunning &&
job.SchemaState == model.StateWriteOnly && job.SchemaID == dbInfo.ID && done == false {
wg.Add(1)
done = true
go func() {
_, checkErr = tk2.Exec("drop table t")
wg.Done()
}()
time.Sleep(5 * time.Millisecond)
}
}
originalHook := s.dom.DDL().GetHook()
s.dom.DDL().(ddl.DDLForTest).SetHook(hook)
s.mustExec(c, "drop database test_drop_schema_table")
s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook)
wg.Wait()
c.Assert(done, IsTrue)
c.Assert(checkErr, NotNil)
c.Assert(checkErr.Error(), Equals, "[schema:1051]Unknown table 'test_drop_schema_table.t'")

// Below behaviour is use to mock query `curl "http://$IP:10080/tiflash/replica"`
fn := func(jobs []*model.Job) (bool, error) {
return executor.GetDropOrTruncateTableInfoFromJobs(jobs, 0, s.dom, func(job *model.Job, info *model.TableInfo) (bool, error) {
return false, nil
})
}
err := s.tk.Se.NewTxn(context.Background())
c.Assert(err, IsNil)
txn, err := s.tk.Se.Txn(true)
c.Assert(err, IsNil)
err = admin.IterHistoryDDLJobs(txn, fn)
c.Assert(err, IsNil)
}

// TestCancelRenameIndex tests cancel ddl job which type is rename index.
func (s *testDBSuite1) TestCancelRenameIndex(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
Expand Down
8 changes: 4 additions & 4 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -480,10 +480,10 @@ func GetDropOrTruncateTableInfoFromJobs(jobs []*model.Job, gcSafePoint uint64, d
// Get table meta from snapshot infoSchema.
table, ok := snapInfo.TableByID(job.TableID)
if !ok {
return false, infoschema.ErrTableNotExists.GenWithStackByArgs(
fmt.Sprintf("(Schema ID %d)", job.SchemaID),
fmt.Sprintf("(Table ID %d)", job.TableID),
)
// The dropped/truncated DDL maybe execute failed that caused by the parallel DDL execution,
// then can't find the table from the snapshot info-schema. Should just ignore error here,
// see more in TestParallelDropSchemaAndDropTable.
continue
}
finish, err := fn(job, table.Meta())
if err != nil || finish {
Expand Down

0 comments on commit e053ce0

Please sign in to comment.