Skip to content

Commit

Permalink
*: replace gofail with the new failpoint implementation (pingcap#10184)
Browse files Browse the repository at this point in the history
  • Loading branch information
lonng authored Apr 22, 2019
1 parent 8ec262c commit 27a5618
Show file tree
Hide file tree
Showing 35 changed files with 394 additions and 299 deletions.
42 changes: 21 additions & 21 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@ PACKAGES := $$($(PACKAGE_LIST))
PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||'
FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go")

GOFAIL_ENABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs tools/bin/gofail enable)
GOFAIL_DISABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs tools/bin/gofail disable)
FAILPOINT_ENABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs tools/bin/failpoint-ctl enable)
FAILPOINT_DISABLE := $$(find $$PWD/ -type d | grep -vE "(\.git|tools)" | xargs tools/bin/failpoint-ctl disable)

LDFLAGS += -X "github.com/pingcap/parser/mysql.TiDBReleaseVersion=$(shell git describe --tags --dirty --always)"
LDFLAGS += -X "github.com/pingcap/tidb/util/printer.TiDBBuildTS=$(shell date -u '+%Y-%m-%d %I:%M:%S')"
Expand Down Expand Up @@ -123,7 +123,7 @@ ifeq ("$(TRAVIS_COVERAGE)", "1")
bash <(curl -s https://codecov.io/bash)
endif

gotest: gofail-enable
gotest: failpoint-enable
ifeq ("$(TRAVIS_COVERAGE)", "1")
@echo "Running in TRAVIS_COVERAGE mode."
@export log_level=error; \
Expand All @@ -133,27 +133,27 @@ ifeq ("$(TRAVIS_COVERAGE)", "1")
-ignore='.git,vendor,cmd,docs,LICENSES' \
-concurrency=2 \
-- -coverpkg=./... \
|| { $(GOFAIL_DISABLE); exit 1; }
|| { $(FAILPOINT_DISABLE); exit 1; }
else
@echo "Running in native mode."
@export log_level=error; \
$(GOTEST) -ldflags '$(TEST_LDFLAGS)' -cover $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; }
$(GOTEST) -ldflags '$(TEST_LDFLAGS)' -cover $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; }
endif
@$(GOFAIL_DISABLE)
@$(FAILPOINT_DISABLE)

race: gofail-enable
race: failpoint-enable
@export log_level=debug; \
$(GOTEST) -timeout 20m -race $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; }
@$(GOFAIL_DISABLE)
$(GOTEST) -timeout 20m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; }
@$(FAILPOINT_DISABLE)

leak: gofail-enable
leak: failpoint-enable
@export log_level=debug; \
$(GOTEST) -tags leak $(PACKAGES) || { $(GOFAIL_DISABLE); exit 1; }
@$(GOFAIL_DISABLE)
$(GOTEST) -tags leak $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; }
@$(FAILPOINT_DISABLE)

tikv_integration_test: gofail-enable
$(GOTEST) ./store/tikv/. -with-tikv=true || { $(GOFAIL_DISABLE); exit 1; }
@$(GOFAIL_DISABLE)
tikv_integration_test: failpoint-enable
$(GOTEST) ./store/tikv/. -with-tikv=true || { $(FAILPOINT_DISABLE); exit 1; }
@$(FAILPOINT_DISABLE)

RACE_FLAG =
ifeq ("$(WITH_RACE)", "1")
Expand Down Expand Up @@ -195,13 +195,13 @@ importer:
checklist:
cat checklist.md

gofail-enable: tools/bin/gofail
failpoint-enable: tools/bin/failpoint-ctl
# Converting gofail failpoints...
@$(GOFAIL_ENABLE)
@$(FAILPOINT_ENABLE)

gofail-disable: tools/bin/gofail
failpoint-disable: tools/bin/failpoint-ctl
# Restoring gofail failpoints...
@$(GOFAIL_DISABLE)
@$(FAILPOINT_DISABLE)

checkdep:
$(GO) list -f '{{ join .Imports "\n" }}' github.com/pingcap/tidb/store/tikv | grep ^github.com/pingcap/parser$$ || exit 0; exit 1
Expand Down Expand Up @@ -230,8 +230,8 @@ tools/bin/errcheck: tools/check/go.mod
cd tools/check; \
$(GO) build -o ../bin/errcheck github.com/kisielk/errcheck

tools/bin/gofail: go.mod
$(GO) build -o $@ github.com/pingcap/gofail
tools/bin/failpoint-ctl: go.mod
$(GO) build -o $@ github.com/pingcap/failpoint/failpoint-ctl

tools/bin/misspell:tools/check/go.mod
$(GO) get -u github.com/client9/misspell/cmd/misspell
Expand Down
23 changes: 13 additions & 10 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand Down Expand Up @@ -163,10 +164,11 @@ func onAddColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error)
return ver, nil
}

// gofail: var errorBeforeDecodeArgs bool
// if errorBeforeDecodeArgs {
// return ver, errors.New("occur an error before decode args")
// }
failpoint.Inject("errorBeforeDecodeArgs", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(ver, errors.New("occur an error before decode args"))
}
})

tblInfo, columnInfo, col, pos, offset, err := checkAddColumn(t, job)
if err != nil {
Expand Down Expand Up @@ -374,12 +376,13 @@ func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.Colu
}
}

// gofail: var uninitializedOffsetAndState bool
// if uninitializedOffsetAndState {
// if newCol.State != model.StatePublic {
// return ver, errors.New("the column state is wrong")
// }
// }
failpoint.Inject("uninitializedOffsetAndState", func(val failpoint.Value) {
if val.(bool) {
if newCol.State != model.StatePublic {
failpoint.Return(ver, errors.New("the column state is wrong"))
}
}
})

if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) && !mysql.HasPreventNullInsertFlag(oldCol.Flag) {
// Introduce the `mysql.HasPreventNullInsertFlag` flag to prevent users from inserting or updating null values.
Expand Down
10 changes: 6 additions & 4 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/coreos/etcd/clientv3"
"github.com/ngaut/pools"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand Down Expand Up @@ -504,10 +505,11 @@ func (d *ddl) genGlobalID() (int64, error) {
err := kv.RunInNewTxn(d.store, true, func(txn kv.Transaction) error {
var err error

// gofail: var mockGenGlobalIDFail bool
// if mockGenGlobalIDFail {
// return errors.New("gofail genGlobalID error")
// }
failpoint.Inject("mockGenGlobalIDFail", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(errors.New("gofail genGlobalID error"))
}
})

globalID, err = meta.NewMeta(txn).GenGlobalID()
return errors.Trace(err)
Expand Down
6 changes: 3 additions & 3 deletions ddl/fail_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import (
"context"

. "github.com/pingcap/check"
gofail "github.com/pingcap/gofail/runtime"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -52,10 +52,10 @@ func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) {
stateCnt++
} else if job.SchemaState == model.StateWriteReorganization {
if first {
gofail.Enable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs", `return(true)`)
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs", `return(true)`), IsNil)
first = false
} else {
gofail.Disable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs")
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/errorBeforeDecodeArgs"), IsNil)
}
}
}
Expand Down
50 changes: 31 additions & 19 deletions ddl/failtest/fail_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/errors"
gofail "github.com/pingcap/gofail/runtime"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/ddl"
Expand Down Expand Up @@ -92,9 +92,10 @@ func (s *testFailDBSuite) TearDownSuite(c *C) {

// TestHalfwayCancelOperations tests the case that the schema is correct after the execution of operations are cancelled halfway.
func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) {
gofail.Enable("github.com/pingcap/tidb/ddl/truncateTableErr", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/truncateTableErr")

c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/truncateTableErr", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/truncateTableErr"), IsNil)
}()
// test for truncating table
_, err := s.se.Execute(context.Background(), "create database cancel_job_db")
c.Assert(err, IsNil)
Expand Down Expand Up @@ -131,8 +132,11 @@ func (s *testFailDBSuite) TestHalfwayCancelOperations(c *C) {
c.Assert(err, IsNil)

// test for renaming table
gofail.Enable("github.com/pingcap/tidb/ddl/renameTableErr", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/renameTableErr")
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/renameTableErr", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/renameTableErr"), IsNil)
}()

_, err = s.se.Execute(context.Background(), "create table tx(a int)")
c.Assert(err, IsNil)
_, err = s.se.Execute(context.Background(), "insert into tx values(1)")
Expand Down Expand Up @@ -176,14 +180,16 @@ func (s *testFailDBSuite) TestInitializeOffsetAndState(c *C) {
tk.MustExec("create table t(a int, b int, c int)")
defer tk.MustExec("drop table t")

gofail.Enable("github.com/pingcap/tidb/ddl/uninitializedOffsetAndState", `return(true)`)
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/uninitializedOffsetAndState", `return(true)`), IsNil)
tk.MustExec("ALTER TABLE t MODIFY COLUMN b int FIRST;")
gofail.Disable("github.com/pingcap/tidb/ddl/uninitializedOffsetAndState")
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/uninitializedOffsetAndState"), IsNil)
}

func (s *testFailDBSuite) TestUpdateHandleFailed(c *C) {
gofail.Enable("github.com/pingcap/tidb/ddl/errorUpdateReorgHandle", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/errorUpdateReorgHandle")
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/errorUpdateReorgHandle", `1*return`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/errorUpdateReorgHandle"), IsNil)
}()
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_handle_failed")
defer tk.MustExec("drop database test_handle_failed")
Expand All @@ -197,8 +203,10 @@ func (s *testFailDBSuite) TestUpdateHandleFailed(c *C) {
}

func (s *testFailDBSuite) TestAddIndexFailed(c *C) {
gofail.Enable("github.com/pingcap/tidb/ddl/mockAddIndexErr", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/mockAddIndexErr")
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockAddIndexErr", `1*return`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockAddIndexErr"), IsNil)
}()
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists test_add_index_failed")
defer tk.MustExec("drop database test_add_index_failed")
Expand Down Expand Up @@ -242,7 +250,7 @@ func (s *testFailDBSuite) TestFailSchemaSyncer(c *C) {
c.Assert(ok, IsTrue)

// make reload failed.
gofail.Enable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed", `return(true)`)
c.Assert(failpoint.Enable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed", `return(true)`), IsNil)
mockSyncer.CloseSession()
// wait the schemaValidator is stopped.
for i := 0; i < 50; i++ {
Expand All @@ -256,7 +264,7 @@ func (s *testFailDBSuite) TestFailSchemaSyncer(c *C) {
_, err := tk.Exec("insert into t values(1)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[domain:1]Information schema is out of date.")
gofail.Disable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed")
c.Assert(failpoint.Disable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed"), IsNil)
// wait the schemaValidator is started.
for i := 0; i < 50; i++ {
if s.dom.SchemaValidator.IsStarted() {
Expand All @@ -270,7 +278,9 @@ func (s *testFailDBSuite) TestFailSchemaSyncer(c *C) {
}

func (s *testFailDBSuite) TestGenGlobalIDFail(c *C) {
defer gofail.Disable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail")
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail"), IsNil)
}()
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists gen_global_id_fail")
tk.MustExec("use gen_global_id_fail")
Expand Down Expand Up @@ -301,11 +311,11 @@ func (s *testFailDBSuite) TestGenGlobalIDFail(c *C) {

for idx, test := range testcases {
if test.mockErr {
gofail.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(true)`)
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(true)`), IsNil)
_, err := tk.Exec(test.sql)
c.Assert(err, NotNil, Commentf("the %dth test case '%s' fail", idx, test.sql))
} else {
gofail.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(false)`)
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockGenGlobalIDFail", `return(false)`), IsNil)
tk.MustExec(test.sql)
tk.MustExec(fmt.Sprintf("insert into %s values (%d, 42)", test.table, rand.Intn(65536)))
tk.MustExec(fmt.Sprintf("admin check table %s", test.table))
Expand Down Expand Up @@ -349,8 +359,10 @@ func (s *testFailDBSuite) TestAddIndexWorkerNum(c *C) {
ddl.TestCheckWorkerNumber = lastSetWorkerCnt
defer tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_worker_cnt=%d", originDDLAddIndexWorkerCnt))

gofail.Enable("github.com/pingcap/tidb/ddl/checkIndexWorkerNum", `return(true)`)
defer gofail.Disable("github.com/pingcap/tidb/ddl/checkIndexWorkerNum")
c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/checkIndexWorkerNum", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/checkIndexWorkerNum"), IsNil)
}()

testutil.SessionExecInGoroutine(c, s.store, "create index c3_index on test_add_index (c3)", done)
checkNum := 0
Expand Down
55 changes: 28 additions & 27 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand Down Expand Up @@ -758,10 +759,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")
// }
failpoint.Inject("errorMockPanic", func(val failpoint.Value) {
if val.(bool) {
panic("panic test")
}
})

oprStartTime := time.Now()
errInTxn = kv.RunInNewTxn(w.sessCtx.GetStore(), true, func(txn kv.Transaction) error {
Expand Down Expand Up @@ -861,8 +863,6 @@ func (w *addIndexWorker) handleBackfillTask(d *ddlCtx, task *reorgIndexTask) *ad
return result
}

var gofailMockAddindexErrOnceGuard bool

func (w *addIndexWorker) run(d *ddlCtx) {
logutil.Logger(ddlLogCtx).Info("[ddl] add index worker start", zap.Int("workerID", w.id))
defer func() {
Expand All @@ -881,13 +881,13 @@ func (w *addIndexWorker) run(d *ddlCtx) {
}

logutil.Logger(ddlLogCtx).Debug("[ddl] add index worker got task", zap.Int("workerID", w.id), zap.String("task", task.String()))
// gofail: var mockAddIndexErr bool
//if w.id == 0 && mockAddIndexErr && !gofailMockAddindexErrOnceGuard {
// gofailMockAddindexErrOnceGuard = true
// result := &addIndexResult{addedCount: 0, nextHandle: 0, err: errors.Errorf("mock add index error")}
// w.resultCh <- result
// continue
//}
failpoint.Inject("mockAddIndexErr", func() {
if w.id == 0 {
result := &addIndexResult{addedCount: 0, nextHandle: 0, err: errors.Errorf("mock add index error")}
w.resultCh <- result
failpoint.Continue()
}
})

// Dynamic change batch size.
w.batchCnt = int(variable.GetDDLReorgBatchSize())
Expand Down Expand Up @@ -1159,20 +1159,21 @@ func (w *worker) addPhysicalTableIndex(t table.PhysicalTable, indexInfo *model.I
closeAddIndexWorkers(workers)
}

// gofail: var checkIndexWorkerNum bool
// if checkIndexWorkerNum {
// num := int(atomic.LoadInt32(&TestCheckWorkerNumber))
// if num != 0 {
// if num > len(kvRanges) {
// if len(idxWorkers) != len(kvRanges) {
// return errors.Errorf("check index worker num error, len kv ranges is: %v, check index worker num is: %v, actual index num is: %v", len(kvRanges), num, len(idxWorkers))
// }
// } else if num != len(idxWorkers) {
// return errors.Errorf("check index worker num error, len kv ranges is: %v, check index worker num is: %v, actual index num is: %v", len(kvRanges), num, len(idxWorkers))
// }
// TestCheckWorkerNumCh <- struct{}{}
// }
//}
failpoint.Inject("checkIndexWorkerNum", func(val failpoint.Value) {
if val.(bool) {
num := int(atomic.LoadInt32(&TestCheckWorkerNumber))
if num != 0 {
if num > len(kvRanges) {
if len(idxWorkers) != len(kvRanges) {
failpoint.Return(errors.Errorf("check index worker num error, len kv ranges is: %v, check index worker num is: %v, actual index num is: %v", len(kvRanges), num, len(idxWorkers)))
}
} else if num != len(idxWorkers) {
failpoint.Return(errors.Errorf("check index worker num error, len kv ranges is: %v, check index worker num is: %v, actual index num is: %v", len(kvRanges), num, len(idxWorkers)))
}
TestCheckWorkerNumCh <- struct{}{}
}
}
})

logutil.Logger(ddlLogCtx).Info("[ddl] start add index workers to reorg index", zap.Int("workerCnt", len(idxWorkers)), zap.Int("regionCnt", len(kvRanges)), zap.Int64("startHandle", startHandle), zap.Int64("endHandle", endHandle))
remains, err := w.sendRangeTaskToWorkers(t, idxWorkers, reorgInfo, &totalAddedCount, kvRanges)
Expand Down
Loading

0 comments on commit 27a5618

Please sign in to comment.