Skip to content

Commit

Permalink
ddl: fix can't cancel a job when add index meets panic (pingcap#8621)
Browse files Browse the repository at this point in the history
  • Loading branch information
winkyao authored Jan 2, 2019
1 parent 4616e8c commit b8b3a7f
Show file tree
Hide file tree
Showing 4 changed files with 142 additions and 8 deletions.
19 changes: 14 additions & 5 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -730,6 +730,11 @@ func (w *addIndexWorker) batchCheckUniqueKey(txn kv.Transaction, idxRecords []*i
// backfillIndexInTxn will add w.batchCnt indices once, default value of w.batchCnt is 128.
// TODO: make w.batchCnt can be modified by system variable.
func (w *addIndexWorker) backfillIndexInTxn(handleRange reorgIndexTask) (taskCtx addIndexTaskContext, errInTxn error) {
// gofail: var errorMockPanic bool
// if errorMockPanic {
// panic("panic test")
// }

oprStartTime := time.Now()
errInTxn = kv.RunInNewTxn(w.sessCtx.GetStore(), true, func(txn kv.Transaction) error {
taskCtx.addedCount = 0
Expand Down Expand Up @@ -792,13 +797,17 @@ func (w *addIndexWorker) handleBackfillTask(d *ddlCtx, task *reorgIndexTask) *ad
startTime := lastLogTime

for {
taskCtx, err := w.backfillIndexInTxn(handleRange)
if err == nil {
// Because reorgIndexTask may run a long time,
// we should check whether this ddl job is still runnable.
err = w.ddlWorker.isReorgRunnable(d)
// Give job chance to be canceled, if we not check it here,
// if there is panic in w.backfillIndexInTxn we will never cancel the job.
// Because reorgIndexTask may run a long time,
// we should check whether this ddl job is still runnable.
err := w.ddlWorker.isReorgRunnable(d)
if err != nil {
result.err = err
return result
}

taskCtx, err := w.backfillIndexInTxn(handleRange)
if err != nil {
result.err = err
return result
Expand Down
125 changes: 125 additions & 0 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
// 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 (
"context"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/errors"
gofail "github.com/pingcap/gofail/runtime"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testleak"
)

var _ = SerialSuites(&testSerialSuite{})

type testSerialSuite struct {
store kv.Storage
dom *domain.Domain
}

func (s *testSerialSuite) SetUpSuite(c *C) {
testleak.BeforeTest()
session.SetSchemaLease(200 * time.Millisecond)
session.SetStatsLease(0)

ddl.WaitTimeWhenErrorOccured = 1 * time.Microsecond
var err error
s.store, err = mockstore.NewMockTikvStore()
c.Assert(err, IsNil)

s.dom, err = session.BootstrapSession(s.store)
c.Assert(err, IsNil)
}

func (s *testSerialSuite) TearDownSuite(c *C) {
if s.dom != nil {
s.dom.Close()
}
if s.store != nil {
s.store.Close()
}
testleak.AfterTest(c)()
}

// TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started.
func (s *testSerialSuite) TestCancelAddIndexPanic(c *C) {
gofail.Enable("github.com/pingcap/tidb/ddl/errorMockPanic", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/errorMockPanic")
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(c1 int, c2 int)")
defer tk.MustExec("drop table t;")
for i := 0; i < 5; i++ {
tk.MustExec("insert into t values (?, ?)", i, i)
}
var checkErr error
oldReorgWaitTimeout := ddl.ReorgWaitTimeout
ddl.ReorgWaitTimeout = 50 * time.Millisecond
defer func() { ddl.ReorgWaitTimeout = oldReorgWaitTimeout }()
hook := &ddl.TestDDLCallback{}
hook.OnJobRunBeforeExported = func(job *model.Job) {
if job.Type == model.ActionAddIndex && job.State == model.JobStateRunning && job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 {
jobIDs := []int64{job.ID}
hookCtx := mock.NewContext()
hookCtx.Store = s.store
err := hookCtx.NewTxn(context.Background())
if err != nil {
checkErr = errors.Trace(err)
return
}
txn, err := hookCtx.Txn(true)
if err != nil {
checkErr = errors.Trace(err)
return
}
errs, err := admin.CancelJobs(txn, jobIDs)
if err != nil {
checkErr = errors.Trace(err)
return
}
if errs[0] != nil {
checkErr = errors.Trace(errs[0])
return
}
txn, err = hookCtx.Txn(true)
if err != nil {
checkErr = errors.Trace(err)
return
}
checkErr = txn.Commit(context.Background())
}
}
origHook := s.dom.DDL().GetHook()
defer s.dom.DDL().(ddl.DDLForTest).SetHook(origHook)
s.dom.DDL().(ddl.DDLForTest).SetHook(hook)
rs, err := tk.Exec("alter table t add index idx_c2(c2)")
if rs != nil {
rs.Close()
}
c.Assert(checkErr, IsNil)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:12]cancelled DDL job")
}
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,7 @@ require (
github.com/onsi/gomega v1.4.3 // indirect
github.com/opentracing/basictracer-go v1.0.0
github.com/opentracing/opentracing-go v1.0.2
github.com/pingcap/check v0.0.0-20181222140913-41d022e836db
github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8
github.com/pingcap/errors v0.11.0
github.com/pingcap/gofail v0.0.0-20181217135706-6a951c1e42c3
github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -100,8 +100,8 @@ github.com/opentracing/basictracer-go v1.0.0 h1:YyUAhaEfjoWXclZVJ9sGoNct7j4TVk7l
github.com/opentracing/basictracer-go v1.0.0/go.mod h1:QfBfYuafItcjQuMwinw9GhYKwFXS9KnPs5lxoYwgW74=
github.com/opentracing/opentracing-go v1.0.2 h1:3jA2P6O1F9UOrWVpwrIo17pu01KWvNWg4X946/Y5Zwg=
github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
github.com/pingcap/check v0.0.0-20181222140913-41d022e836db h1:yg93sLvBszRnzcd+Z5gkCUdYgud2scHYYxnRwljvRAM=
github.com/pingcap/check v0.0.0-20181222140913-41d022e836db/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ=
github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8 h1:USx2/E1bX46VG32FIw034Au6seQ2fY9NEILmNh/UlQg=
github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ=
github.com/pingcap/errors v0.11.0 h1:DCJQB8jrHbQ1VVlMFIrbj2ApScNNotVmkSNplu2yUt4=
github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8=
github.com/pingcap/gofail v0.0.0-20181217135706-6a951c1e42c3 h1:04yuCf5NMvLU8rB2m4Qs3rynH7EYpMno3lHkewIOdMo=
Expand Down

0 comments on commit b8b3a7f

Please sign in to comment.