From 27a56180bd20a4151458ea60cd3667f2de598264 Mon Sep 17 00:00:00 2001 From: Lonng Date: Mon, 22 Apr 2019 13:30:14 +0800 Subject: [PATCH 01/21] *: replace gofail with the new failpoint implementation (#10184) --- Makefile | 42 ++++++++++---------- ddl/column.go | 23 ++++++----- ddl/ddl.go | 10 +++-- ddl/fail_test.go | 6 +-- ddl/failtest/fail_db_test.go | 50 +++++++++++++++--------- ddl/index.go | 55 ++++++++++++++------------- ddl/reorg.go | 12 ++---- ddl/serial_test.go | 30 ++++++++------- ddl/table.go | 45 +++++++++++++--------- domain/domain.go | 10 +++-- domain/domain_test.go | 6 +-- executor/aggregate.go | 19 +++++---- executor/seqtest/seq_executor_test.go | 20 ++++++---- executor/write_test.go | 6 +-- go.mod | 2 +- go.sum | 12 +++++- owner/fail_test.go | 6 +-- owner/manager.go | 24 ++++++++---- planner/core/rule_column_pruning.go | 10 +++-- planner/failtest/fail_test.go | 8 ++-- session/session.go | 12 +++--- session/session_fail_test.go | 30 ++++++++------- session/session_test.go | 6 +-- session/txn.go | 41 +++++++++++--------- store/mockstore/mocktikv/rpc.go | 52 ++++++++++++++----------- store/tikv/2pc.go | 10 +++-- store/tikv/2pc_fail_test.go | 32 ++++++++++------ store/tikv/coprocessor.go | 11 +++--- store/tikv/gcworker/gc_worker_test.go | 14 +++---- store/tikv/region_request.go | 41 ++++++++++---------- store/tikv/sql_fail_test.go | 12 +++--- store/tikv/store_fail_test.go | 6 +-- store/tikv/txn.go | 12 +++--- tablecodec/tablecodec_test.go | 8 ++-- util/codec/decimal.go | 10 +++-- 35 files changed, 394 insertions(+), 299 deletions(-) diff --git a/Makefile b/Makefile index e33389531a3c3..d69a0cb39aae8 100644 --- a/Makefile +++ b/Makefile @@ -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')" @@ -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; \ @@ -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") @@ -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 @@ -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 diff --git a/ddl/column.go b/ddl/column.go index d20d5390e8a2d..6c22425a6784e 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -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" @@ -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 { @@ -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. diff --git a/ddl/ddl.go b/ddl/ddl.go index 5729a75584860..c1df201bd9315 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -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" @@ -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) diff --git a/ddl/fail_test.go b/ddl/fail_test.go index b8f6eb54bd5fb..1e8f6ef2df4bd 100644 --- a/ddl/fail_test.go +++ b/ddl/fail_test.go @@ -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" @@ -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) } } } diff --git a/ddl/failtest/fail_db_test.go b/ddl/failtest/fail_db_test.go index 660530fb76c7f..23ef4d5d9d676 100644 --- a/ddl/failtest/fail_db_test.go +++ b/ddl/failtest/fail_db_test.go @@ -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" @@ -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) @@ -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)") @@ -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") @@ -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") @@ -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++ { @@ -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() { @@ -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") @@ -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)) @@ -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 diff --git a/ddl/index.go b/ddl/index.go index 057d022aa2c0f..2164d763ac685 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -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" @@ -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 { @@ -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() { @@ -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()) @@ -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) diff --git a/ddl/reorg.go b/ddl/reorg.go index 2334beb26c6d6..716254addbf3d 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" @@ -312,8 +313,6 @@ func (d *ddlCtx) GetTableMaxRowID(startTS uint64, tbl table.PhysicalTable) (maxR return maxRowID, false, nil } -var gofailOnceGuard bool - // getTableRange gets the start and end handle of a table (or partition). func getTableRange(d *ddlCtx, tbl table.PhysicalTable, snapshotVer uint64, priority int) (startHandle, endHandle int64, err error) { startHandle = math.MinInt64 @@ -375,12 +374,9 @@ func getReorgInfo(d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table) (*re } logutil.Logger(ddlLogCtx).Info("[ddl] job get table range", zap.Int64("jobID", job.ID), zap.Int64("physicalTableID", pid), zap.Int64("startHandle", start), zap.Int64("endHandle", end)) - // gofail: var errorUpdateReorgHandle bool - // if errorUpdateReorgHandle && !gofailOnceGuard { - // // only return error once. - // gofailOnceGuard = true - // return &info, errors.New("occur an error when update reorg handle.") - // } + failpoint.Inject("errorUpdateReorgHandle", func() (*reorgInfo, error) { + return &info, errors.New("occur an error when update reorg handle") + }) err = t.UpdateDDLReorgHandle(job, start, end, pid) if err != nil { return &info, errors.Trace(err) diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 1b1a50313b357..665337cdd31a5 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -22,7 +22,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/ddl" @@ -69,8 +69,10 @@ func (s *testSerialSuite) TearDownSuite(c *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") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/errorMockPanic", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/errorMockPanic"), IsNil) + }() tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -377,8 +379,8 @@ func (s *testSerialSuite) TestRecoverTableByJobIDFail(c *C) { hook := &ddl.TestDDLCallback{} hook.OnJobRunBeforeExported = func(job *model.Job) { if job.Type == model.ActionRecoverTable { - gofail.Enable("github.com/pingcap/tidb/store/tikv/mockCommitError", `return(true)`) - gofail.Enable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockCommitError", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr", `return(true)`), IsNil) } } origHook := s.dom.DDL().GetHook() @@ -387,8 +389,8 @@ func (s *testSerialSuite) TestRecoverTableByJobIDFail(c *C) { // 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/mockRecoverTableCommitErr") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockCommitError"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr"), IsNil) // make sure enable GC after recover table. enable, err := gcutil.CheckGCEnable(tk.Se) @@ -437,8 +439,8 @@ func (s *testSerialSuite) TestRecoverTableByTableNameFail(c *C) { hook := &ddl.TestDDLCallback{} hook.OnJobRunBeforeExported = func(job *model.Job) { if job.Type == model.ActionRecoverTable { - gofail.Enable("github.com/pingcap/tidb/store/tikv/mockCommitError", `return(true)`) - gofail.Enable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/mockCommitError", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr", `return(true)`), IsNil) } } origHook := s.dom.DDL().GetHook() @@ -447,8 +449,8 @@ func (s *testSerialSuite) TestRecoverTableByTableNameFail(c *C) { // 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/mockRecoverTableCommitErr") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/mockCommitError"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockRecoverTableCommitErr"), IsNil) // make sure enable GC after recover table. enable, err := gcutil.CheckGCEnable(tk.Se) @@ -464,8 +466,10 @@ func (s *testSerialSuite) TestRecoverTableByTableNameFail(c *C) { func (s *testSerialSuite) TestCancelJobByErrorCountLimit(c *C) { tk := testkit.NewTestKit(c, s.store) - gofail.Enable("github.com/pingcap/tidb/ddl/mockExceedErrorLimit", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/ddl/mockExceedErrorLimit") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/ddl/mockExceedErrorLimit", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/mockExceedErrorLimit"), IsNil) + }() tk.MustExec("use test") tk.MustExec("drop table if exists t") _, err := tk.Exec("create table t (a int)") diff --git a/ddl/table.go b/ddl/table.go index 7cb63bf6e6915..179b11235b26b 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -20,6 +20,7 @@ import ( "sync/atomic" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl/util" @@ -36,10 +37,11 @@ import ( ) func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { - // gofail: var mockExceedErrorLimit bool - // if mockExceedErrorLimit { - // return ver, errors.New("mock do job error") - // } + failpoint.Inject("mockExceedErrorLimit", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(ver, errors.New("mock do job error")) + } + }) schemaID := job.SchemaID tbInfo := &model.TableInfo{} @@ -257,11 +259,12 @@ func (w *worker) onRecoverTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in return ver, errors.Trace(err) } - // gofail: var mockRecoverTableCommitErr bool - // if mockRecoverTableCommitErr && mockRecoverTableCommitErrOnce { - // mockRecoverTableCommitErrOnce = false - // kv.MockCommitErrorEnable() - // } + failpoint.Inject("mockRecoverTableCommitErr", func(val failpoint.Value) { + if val.(bool) && mockRecoverTableCommitErrOnce { + mockRecoverTableCommitErrOnce = false + kv.MockCommitErrorEnable() + } + }) ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) if err != nil { @@ -402,11 +405,12 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro job.State = model.JobStateCancelled return ver, errors.Trace(err) } - // gofail: var truncateTableErr bool - // if truncateTableErr { - // job.State = model.JobStateCancelled - // return ver, errors.New("occur an error after dropping table.") - // } + failpoint.Inject("truncateTableErr", func(val failpoint.Value) { + if val.(bool) { + job.State = model.JobStateCancelled + failpoint.Return(ver, errors.New("occur an error after dropping table")) + } + }) var oldPartitionIDs []int64 if tblInfo.GetPartitionInfo() != nil { @@ -564,11 +568,14 @@ func onRenameTable(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - // gofail: var renameTableErr bool - // if renameTableErr { - // job.State = model.JobStateCancelled - // return ver, errors.New("occur an error after renaming table.") - // } + + failpoint.Inject("renameTableErr", func(val failpoint.Value) { + if val.(bool) { + job.State = model.JobStateCancelled + failpoint.Return(ver, errors.New("occur an error after renaming table")) + } + }) + tblInfo.Name = tableName err = t.CreateTableOrView(newSchemaID, tblInfo) if err != nil { diff --git a/domain/domain.go b/domain/domain.go index c7f18052ac240..3e08c8e3ac733 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -26,6 +26,7 @@ import ( "github.com/ngaut/pools" "github.com/ngaut/sync2" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" @@ -300,10 +301,11 @@ func (do *Domain) GetScope(status string) variable.ScopeFlag { // Reload reloads InfoSchema. // It's public in order to do the test. func (do *Domain) Reload() error { - // gofail: var ErrorMockReloadFailed bool - // if ErrorMockReloadFailed { - // return errors.New("mock reload failed") - // } + failpoint.Inject("ErrorMockReloadFailed", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("mock reload failed")) + } + }) // Lock here for only once at the same time. do.m.Lock() diff --git a/domain/domain_test.go b/domain/domain_test.go index a85462c4740c8..d01fa94ad65ff 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -20,7 +20,7 @@ import ( "github.com/ngaut/pools" . "github.com/pingcap/check" "github.com/pingcap/errors" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -86,7 +86,7 @@ func (*testSuite) TestT(c *C) { succ := dom.SchemaValidator.Check(ts, schemaVer, nil) c.Assert(succ, Equals, ResultSucc) - gofail.Enable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed", `return(true)`), IsNil) err = dom.Reload() c.Assert(err, NotNil) succ = dom.SchemaValidator.Check(ts, schemaVer, nil) @@ -98,7 +98,7 @@ func (*testSuite) TestT(c *C) { ts = ver.Ver succ = dom.SchemaValidator.Check(ts, schemaVer, nil) c.Assert(succ, Equals, ResultUnknown) - gofail.Disable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed"), IsNil) err = dom.Reload() c.Assert(err, IsNil) succ = dom.SchemaValidator.Check(ts, schemaVer, nil) diff --git a/executor/aggregate.go b/executor/aggregate.go index c5e88a68e7469..fde12cb6bab8f 100644 --- a/executor/aggregate.go +++ b/executor/aggregate.go @@ -20,6 +20,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/executor/aggfuncs" "github.com/pingcap/tidb/expression" @@ -607,10 +608,11 @@ func (e *HashAggExec) parallelExec(ctx context.Context, chk *chunk.Chunk) error e.prepared = true } - // gofail: var parallelHashAggError bool - // if parallelHashAggError { - // return errors.New("HashAggExec.parallelExec error") - // } + failpoint.Inject("parallelHashAggError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("HashAggExec.parallelExec error")) + } + }) for !chk.IsFull() { e.finalInputCh <- chk @@ -684,10 +686,11 @@ func (e *HashAggExec) execute(ctx context.Context) (err error) { return err } - // gofail: var unparallelHashAggError bool - // if unparallelHashAggError { - // return errors.New("HashAggExec.unparallelExec error") - // } + failpoint.Inject("unparallelHashAggError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(errors.New("HashAggExec.unparallelExec error")) + } + }) // no more data. if e.childResult.NumRows() == 0 { diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 2dd10fbfdc444..7bc877411f3de 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -31,7 +31,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser" "github.com/pingcap/parser/model" @@ -136,8 +136,10 @@ func (s *seqTestSuite) TestEarlyClose(c *C) { } // Goroutine should not leak when error happen. - gofail.Enable("github.com/pingcap/tidb/store/tikv/handleTaskOnceError", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/store/tikv/handleTaskOnceError") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/handleTaskOnceError", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/handleTaskOnceError"), IsNil) + }() rss, err := tk.Se.Execute(ctx, "select * from earlyclose") c.Assert(err, IsNil) rs := rss[0] @@ -661,8 +663,10 @@ func (s *seqTestSuite) TestParallelHashAggClose(c *C) { // └─TableScan_10 | 3.00 | cop | table:t, range:[-inf,+inf], keep order:fa$se, stats:pseudo | // Goroutine should not leak when error happen. - gofail.Enable("github.com/pingcap/tidb/executor/parallelHashAggError", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/executor/parallelHashAggError") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/parallelHashAggError", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/parallelHashAggError"), IsNil) + }() ctx := context.Background() rss, err := tk.Se.Execute(ctx, "select sum(a) from (select cast(t.a as signed) as a, b from t) t group by b;") c.Assert(err, IsNil) @@ -680,8 +684,10 @@ func (s *seqTestSuite) TestUnparallelHashAggClose(c *C) { tk.MustExec("insert into t values(1,1),(2,2)") // Goroutine should not leak when error happen. - gofail.Enable("github.com/pingcap/tidb/executor/unparallelHashAggError", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/executor/unparallelHashAggError") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/unparallelHashAggError", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/executor/unparallelHashAggError"), IsNil) + }() ctx := context.Background() rss, err := tk.Se.Execute(ctx, "select sum(distinct a) from (select cast(t.a as signed) as a, b from t) t group by b;") c.Assert(err, IsNil) diff --git a/executor/write_test.go b/executor/write_test.go index b553bde6ba788..7bb7906f9a214 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -21,7 +21,7 @@ import ( "sync/atomic" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" @@ -2472,9 +2472,9 @@ func (s *testSuite2) TestAutoIDInRetry(c *C) { tk.MustExec("insert into t values (),()") tk.MustExec("insert into t values ()") - gofail.Enable("github.com/pingcap/tidb/session/mockCommitRetryForAutoID", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockCommitRetryForAutoID", `return(true)`), IsNil) tk.MustExec("commit") - gofail.Disable("github.com/pingcap/tidb/session/mockCommitRetryForAutoID") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockCommitRetryForAutoID"), IsNil) tk.MustExec("insert into t values ()") tk.MustQuery(`select * from t`).Check(testkit.Rows("1", "2", "3", "4", "5")) diff --git a/go.mod b/go.mod index ecbd19cb3b8c6..234034a2a05ea 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/philhofer/fwd v1.0.0 // indirect github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8 github.com/pingcap/errors v0.11.1 - github.com/pingcap/gofail v0.0.0-20181217135706-6a951c1e42c3 + github.com/pingcap/failpoint v0.0.0-20190418063555-9c3d850fd17c github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 diff --git a/go.sum b/go.sum index a53c14279e76e..16924fb8faad6 100644 --- a/go.sum +++ b/go.sum @@ -3,6 +3,7 @@ github.com/BurntSushi/toml v0.3.1 h1:WXkYYl6Yr3qBf1K79EBnL4mak0OimBfB0XUf9Vl28OQ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/StackExchange/wmi v0.0.0-20180725035823-b12b22c5341f h1:5ZfJxyXo8KyX8DgGXC5B7ILL8y51fci/qYz2B4j8iLY= github.com/StackExchange/wmi v0.0.0-20180725035823-b12b22c5341f/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= +github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQlvB2AYWme2bIB18r/SipGiMEVJYE9U0z+MGoU/LtQ= @@ -44,6 +45,7 @@ github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-ole/go-ole v1.2.1 h1:2lOsA72HgjxAuMlKpFiCbHTvu44PIVkZ5hqm3RSdI/E= github.com/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8= +github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4 h1:3DFRjZdCDhzvxDf0U6/1qAryzOqD7Y5iAj0DJRRl1bs= github.com/go-sql-driver/mysql v0.0.0-20170715192408-3955978caca4/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/gogo/protobuf v0.0.0-20180717141946-636bf0302bc9/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= @@ -62,6 +64,7 @@ github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db h1:woRePGFeVFfLKN/pO github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c h1:964Od4U6p2jUkFxvCydnIczKteheJEzHRToSGK3Bnlw= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/shlex v0.0.0-20181106134648-c34317bd91bf/go.mod h1:RpwtwJQFrIEPstU94h88MWPXP2ektJZ8cZ0YntAmXiE= github.com/gorilla/context v1.1.1 h1:AWwleXJkX/nhcU9bZSnZoi3h/qGYqQAGhq6zZe/aQW8= github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= github.com/gorilla/mux v1.6.2 h1:Pgr17XVTNXAk3q/r4CpKzC5xBM/qW1uVLV+IhRZpIIk= @@ -95,6 +98,7 @@ github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7 h1:7KAv7KMGTTqSmYZtNdc github.com/ngaut/pools v0.0.0-20180318154953-b7bc8c42aac7/go.mod h1:iWMfgwqYW+e8n5lC/jjNEhwcjbRDpl5NT7n2h+4UNcI= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef h1:K0Fn+DoFqNqktdZtdV3bPQ/0cuYh2H4rkg0tytX/07k= github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef/go.mod h1:7WjlapSfwQyo6LNmIvEWzsW1hbBQfpUO4JWnuQRmva8= +github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2xBGjv0j365Q= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= @@ -104,6 +108,7 @@ 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/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= github.com/philhofer/fwd v1.0.0 h1:UbZqGr5Y38ApvM/V/jEljVxwocdweyH+vmYvRPBnbqQ= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8 h1:USx2/E1bX46VG32FIw034Au6seQ2fY9NEILmNh/UlQg= @@ -111,8 +116,8 @@ github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuM github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.1 h1:BXFZ6MdDd2U1uJUa2sRAWTmm+nieEzuyYM0R4aUTcC8= github.com/pingcap/errors v0.11.1/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/gofail v0.0.0-20181217135706-6a951c1e42c3 h1:04yuCf5NMvLU8rB2m4Qs3rynH7EYpMno3lHkewIOdMo= -github.com/pingcap/gofail v0.0.0-20181217135706-6a951c1e42c3/go.mod h1:DazNTg0PTldtpsQiT9I5tVJwV1onHMKBBgXzmJUlMns= +github.com/pingcap/failpoint v0.0.0-20190418063555-9c3d850fd17c h1:orH/u6aM2R59OoqzGYViSrNEnnWAS6G3gObq74zriS8= +github.com/pingcap/failpoint v0.0.0-20190418063555-9c3d850fd17c/go.mod h1:fdAkVXuIXHAPZ7a280nj9bRORfK9NuSsOguvBH0+W6c= github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e h1:P73/4dPCL96rGrobssy1nVy2VaVpNCuLpCbr+FEaTA8= github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 h1:32oF1/8lVnBR2JVcCAnKPQATTOX0+ckRDFpjQk4Ngno= @@ -182,6 +187,7 @@ github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d h1:ggUgChAeyge4NZ4 github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18 h1:MPPkRncZLN9Kh4MEFmbnK4h3BD7AUmskWv2+EeZJCCs= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= +github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/multierr v1.1.0 h1:HoEmRHQPVSqub6w2z2d2EOVs2fjyFRGyofhKuyDq0QI= @@ -223,6 +229,8 @@ google.golang.org/grpc v0.0.0-20180607172857-7a6a684ca69e/go.mod h1:yo6s7OP7yaDg google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= google.golang.org/grpc v1.17.0 h1:TRJYBgMclJvGYn2rIMjj+h9KtMt5r1Ij7ODVRIZkwhk= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= +gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= +gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= diff --git a/owner/fail_test.go b/owner/fail_test.go index 62076a13bd61c..3b0ed1c1f2149 100644 --- a/owner/fail_test.go +++ b/owner/fail_test.go @@ -23,7 +23,7 @@ import ( "github.com/coreos/etcd/clientv3" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testleak" @@ -76,7 +76,7 @@ func (s *testSuite) TestFailNewSession(c *C) { cli.Close() } }() - gofail.Enable("github.com/pingcap/tidb/owner/closeClient", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/owner/closeClient", `return(true)`), IsNil) _, err = NewSession(context.Background(), "fail_new_serssion", cli, retryCnt, ManagerSessionTTL) isContextDone := terror.ErrorEqual(grpc.ErrClientConnClosing, err) || terror.ErrorEqual(context.Canceled, err) c.Assert(isContextDone, IsTrue, Commentf("err %v", err)) @@ -93,7 +93,7 @@ func (s *testSuite) TestFailNewSession(c *C) { cli.Close() } }() - gofail.Enable("github.com/pingcap/tidb/owner/closeGrpc", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/owner/closeGrpc", `return(true)`), IsNil) _, err = NewSession(context.Background(), "fail_new_serssion", cli, retryCnt, ManagerSessionTTL) isContextDone := terror.ErrorEqual(grpc.ErrClientConnClosing, err) || terror.ErrorEqual(context.Canceled, err) c.Assert(isContextDone, IsTrue, Commentf("err %v", err)) diff --git a/owner/manager.go b/owner/manager.go index 0957bc063e3c6..5f738d11a9328 100644 --- a/owner/manager.go +++ b/owner/manager.go @@ -28,6 +28,7 @@ import ( "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" "github.com/coreos/etcd/mvcc/mvccpb" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/util" @@ -142,15 +143,22 @@ func NewSession(ctx context.Context, logPrefix string, etcdCli *clientv3.Client, return etcdSession, errors.Trace(err) } - // gofail: var closeClient bool - // if closeClient { - // etcdCli.Close() - // } + failpoint.Inject("closeClient", func(val failpoint.Value) { + if val.(bool) { + if err := etcdCli.Close(); err != nil { + failpoint.Return(etcdSession, errors.Trace(err)) + } + } + }) + + failpoint.Inject("closeGrpc", func(val failpoint.Value) { + if val.(bool) { + if err := etcdCli.ActiveConnection().Close(); err != nil { + failpoint.Return(etcdSession, errors.Trace(err)) + } + } + }) - // gofail: var closeGrpc bool - // if closeGrpc { - // etcdCli.ActiveConnection().Close() - // } startTime := time.Now() etcdSession, err = concurrency.NewSession(etcdCli, concurrency.WithTTL(ttl), concurrency.WithContext(ctx)) diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 77651fffef1f8..987e0288f5f53 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -15,6 +15,7 @@ package core import ( "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -31,10 +32,11 @@ func (s *columnPruner) optimize(lp LogicalPlan) (LogicalPlan, error) { } func getUsedList(usedCols []*expression.Column, schema *expression.Schema) ([]bool, error) { - // gofail: var enableGetUsedListErr bool - // if enableGetUsedListErr { - // return nil, errors.New("getUsedList failed, triggered by gofail enableGetUsedListErr") - // } + failpoint.Inject("enableGetUsedListErr", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(nil, errors.New("getUsedList failed, triggered by gofail enableGetUsedListErr")) + } + }) used := make([]bool, schema.Len()) for _, col := range usedCols { diff --git a/planner/failtest/fail_test.go b/planner/failtest/fail_test.go index 54605ad25cd16..393ac9ae399c9 100644 --- a/planner/failtest/fail_test.go +++ b/planner/failtest/fail_test.go @@ -17,7 +17,7 @@ import ( "testing" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/session" @@ -77,8 +77,10 @@ func (s *testFailPointSuit) TestColumnPruningError(c *C) { tk.MustQuery(`select a from t;`).Check(testkit.Rows(`1`)) // test the injected fail point - gofail.Enable("github.com/pingcap/tidb/planner/core/enableGetUsedListErr", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/executor/enableGetUsedListErr") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/planner/core/enableGetUsedListErr", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/planner/core/enableGetUsedListErr"), IsNil) + }() err := tk.ExecToErr(`select a from t;`) c.Assert(err.Error(), Equals, "getUsedList failed, triggered by gofail enableGetUsedListErr") } diff --git a/session/session.go b/session/session.go index 4a02acc0c55a4..fe388b9cc728d 100644 --- a/session/session.go +++ b/session/session.go @@ -32,6 +32,7 @@ import ( "github.com/ngaut/pools" "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/auth" @@ -326,11 +327,12 @@ func (s *session) doCommit(ctx context.Context) error { } // mockCommitError and mockGetTSErrorInRetry use to test PR #8743. - // gofail: var mockCommitError bool - // if mockCommitError && mockCommitErrorOnce { - // mockCommitErrorOnce = false - // return kv.ErrRetryable - // } + failpoint.Inject("mockCommitError", func(val failpoint.Value) { + if val.(bool) && mockCommitErrorOnce { + mockCommitErrorOnce = false + failpoint.Return(kv.ErrRetryable) + } + }) if s.sessionVars.BinlogClient != nil { prewriteValue := binloginfo.GetPrewriteValue(s, false) diff --git a/session/session_fail_test.go b/session/session_fail_test.go index 37a37418b789e..3c9c91259948b 100644 --- a/session/session_fail_test.go +++ b/session/session_fail_test.go @@ -17,7 +17,7 @@ import ( "context" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/util/testkit" ) @@ -28,11 +28,11 @@ func (s *testSessionSuite) TestFailStatementCommit(c *C) { tk.MustExec("begin") tk.MustExec("insert into t values (1)") - gofail.Enable("github.com/pingcap/tidb/session/mockStmtCommitError", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockStmtCommitError", `return(true)`), IsNil) _, err := tk.Exec("insert into t values (2),(3),(4),(5)") c.Assert(err, NotNil) - gofail.Disable("github.com/pingcap/tidb/session/mockStmtCommitError") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockStmtCommitError"), IsNil) _, err = tk.Exec("select * from t") c.Assert(err, NotNil) @@ -67,12 +67,12 @@ func (s *testSessionSuite) TestFailStatementCommitInRetry(c *C) { tk.MustExec("insert into t values (2),(3),(4),(5)") tk.MustExec("insert into t values (6)") - gofail.Enable("github.com/pingcap/tidb/session/mockCommitError8942", `return(true)`) - gofail.Enable("github.com/pingcap/tidb/session/mockStmtCommitError", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockCommitError8942", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockStmtCommitError", `return(true)`), IsNil) _, err := tk.Exec("commit") c.Assert(err, NotNil) - gofail.Disable("github.com/pingcap/tidb/session/mockCommitError8942") - gofail.Disable("github.com/pingcap/tidb/session/mockStmtCommitError") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockCommitError8942"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockStmtCommitError"), IsNil) tk.MustExec("insert into t values (6)") tk.MustQuery(`select * from t`).Check(testkit.Rows("6")) @@ -93,25 +93,29 @@ func (s *testSessionSuite) TestGetTSFailDirtyState(c *C) { } func (s *testSessionSuite) TestGetTSFailDirtyStateInretry(c *C) { - defer gofail.Disable("github.com/pingcap/tidb/session/mockCommitError") - defer gofail.Disable("github.com/pingcap/tidb/session/mockGetTSErrorInRetry") + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockCommitError"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tidb/session/mockGetTSErrorInRetry"), IsNil) + }() tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t (id int)") - gofail.Enable("github.com/pingcap/tidb/session/mockCommitError", `return(true)`) - gofail.Enable("github.com/pingcap/tidb/session/mockGetTSErrorInRetry", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockCommitError", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/session/mockGetTSErrorInRetry", `return(true)`), IsNil) tk.MustExec("insert into t values (2)") tk.MustQuery(`select * from t`).Check(testkit.Rows("2")) } func (s *testSessionSuite) TestRetryPreparedSleep(c *C) { - defer gofail.Disable("github.com/pingcap/tidb/store/tmpMaxTxnTime") + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tmpMaxTxnTime"), IsNil) + }() tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t (c1 int)") tk.MustExec("insert t values (11)") - gofail.Enable("github.com/pingcap/tidb/store/tmpMaxTxnTime", `return(2)->return(0)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tmpMaxTxnTime", `return(2)->return(0)`), IsNil) tk.MustExec("begin") tk.MustExec("update t set c1=? where c1=11;", 21) tk.MustExec("insert into t select sleep(3)") diff --git a/session/session_test.go b/session/session_test.go index 4cc72f5379494..d3de1403beaa3 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -22,7 +22,7 @@ import ( "time" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser" "github.com/pingcap/parser/auth" "github.com/pingcap/parser/model" @@ -1716,7 +1716,7 @@ func (s *testSchemaSuite) TestLoadSchemaFailed(c *C) { tk2.MustExec("begin") // Make sure loading information schema is failed and server is invalid. - gofail.Enable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed", `return(true)`), IsNil) err := domain.GetDomain(tk.Se).Reload() c.Assert(err, NotNil) @@ -1737,7 +1737,7 @@ func (s *testSchemaSuite) TestLoadSchemaFailed(c *C) { c.Assert(err, IsNil) c.Assert(ver, NotNil) - gofail.Disable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed") + failpoint.Disable("github.com/pingcap/tidb/domain/ErrorMockReloadFailed") time.Sleep(lease * 2) tk.MustExec("drop table if exists t;") diff --git a/session/txn.go b/session/txn.go index 214f7a0eddf15..df802741310ae 100644 --- a/session/txn.go +++ b/session/txn.go @@ -21,6 +21,7 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" @@ -177,17 +178,19 @@ func (st *TxnState) Commit(ctx context.Context) error { } // mockCommitError8942 is used for PR #8942. - // gofail: var mockCommitError8942 bool - // if mockCommitError8942 { - // return kv.ErrRetryable - // } + failpoint.Inject("mockCommitError8942", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(kv.ErrRetryable) + } + }) // mockCommitRetryForAutoID is used to mock an commit retry for adjustAutoIncrementDatum. - // gofail: var mockCommitRetryForAutoID bool - // if mockCommitRetryForAutoID && !mockAutoIDRetry() { - // enableMockAutoIDRetry() - // return kv.ErrRetryable - // } + failpoint.Inject("mockCommitRetryForAutoID", func(val failpoint.Value) { + if val.(bool) && !mockAutoIDRetry() { + enableMockAutoIDRetry() + failpoint.Return(kv.ErrRetryable) + } + }) return st.Transaction.Commit(ctx) } @@ -355,11 +358,12 @@ func (tf *txnFuture) wait() (kv.Transaction, error) { // Then mockGetTSErrorInRetry will return retryable error when first retry. // Before PR #8743, we don't cleanup txn after meet error such as error like: PD server timeout[try again later] // This may cause duplicate data to be written. - // gofail: var mockGetTSErrorInRetry bool - // if mockGetTSErrorInRetry && mockGetTSErrorInRetryOnce && !mockCommitErrorOnce { - // mockGetTSErrorInRetryOnce = false - // return nil, errors.Errorf("PD server timeout[try again later]") - // } + failpoint.Inject("mockGetTSErrorInRetry", func(val failpoint.Value) { + if val.(bool) && mockGetTSErrorInRetryOnce && !mockCommitErrorOnce { + mockGetTSErrorInRetryOnce = false + failpoint.Return(nil, errors.Errorf("PD server timeout[try again later]")) + } + }) startTS, err := tf.future.Wait() if err == nil { @@ -391,11 +395,12 @@ func (s *session) StmtCommit() error { st := &s.txn var count int err := kv.WalkMemBuffer(st.buf, func(k kv.Key, v []byte) error { + failpoint.Inject("mockStmtCommitError", func(val failpoint.Value) { + if val.(bool) { + count++ + } + }) - // gofail: var mockStmtCommitError bool - // if mockStmtCommitError { - // count++ - // } if count > 3 { return errors.New("mock stmt commit error") } diff --git a/store/mockstore/mocktikv/rpc.go b/store/mockstore/mocktikv/rpc.go index 7edf236cc7eea..8700b4e2b75ab 100644 --- a/store/mockstore/mocktikv/rpc.go +++ b/store/mockstore/mocktikv/rpc.go @@ -22,6 +22,7 @@ import ( "github.com/golang/protobuf/proto" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/debugpb" "github.com/pingcap/kvproto/pkg/errorpb" @@ -578,10 +579,12 @@ func (c *RPCClient) checkArgs(ctx context.Context, addr string) (*rpcHandler, er // SendRequest sends a request to mock cluster. func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.Request, timeout time.Duration) (*tikvrpc.Response, error) { - // gofail: var rpcServerBusy bool - // if rpcServerBusy { - // return tikvrpc.GenRegionErrorResp(req, &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}) - // } + failpoint.Inject("rpcServerBusy", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(tikvrpc.GenRegionErrorResp(req, &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}})) + } + }) + handler, err := c.checkArgs(ctx, addr) if err != nil { return nil, err @@ -613,31 +616,34 @@ func (c *RPCClient) SendRequest(ctx context.Context, addr string, req *tikvrpc.R } resp.Prewrite = handler.handleKvPrewrite(r) case tikvrpc.CmdCommit: - // gofail: var rpcCommitResult string - // switch rpcCommitResult { - // case "timeout": - // return nil, errors.New("timeout") - // case "notLeader": - // return &tikvrpc.Response{ - // Type: tikvrpc.CmdCommit, - // Commit: &kvrpcpb.CommitResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, - // }, nil - // case "keyError": - // return &tikvrpc.Response{ - // Type: tikvrpc.CmdCommit, - // Commit: &kvrpcpb.CommitResponse{Error: &kvrpcpb.KeyError{}}, - // }, nil - // } + failpoint.Inject("rpcCommitResult", func(val failpoint.Value) { + switch val.(string) { + case "timeout": + failpoint.Return(nil, errors.New("timeout")) + case "notLeader": + failpoint.Return(&tikvrpc.Response{ + Type: tikvrpc.CmdCommit, + Commit: &kvrpcpb.CommitResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, + }, nil) + case "keyError": + failpoint.Return(&tikvrpc.Response{ + Type: tikvrpc.CmdCommit, + Commit: &kvrpcpb.CommitResponse{Error: &kvrpcpb.KeyError{}}, + }, nil) + } + }) + r := req.Commit if err := handler.checkRequest(reqCtx, r.Size()); err != nil { resp.Commit = &kvrpcpb.CommitResponse{RegionError: err} return resp, nil } resp.Commit = handler.handleKvCommit(r) - // gofail: var rpcCommitTimeout bool - // if rpcCommitTimeout { - // return nil, undeterminedErr - // } + failpoint.Inject("rpcCommitTimeout", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(nil, undeterminedErr) + } + }) case tikvrpc.CmdCleanup: r := req.Cleanup if err := handler.checkRequest(reqCtx, r.Size()); err != nil { diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index c6a5590b80aa6..bcc8b968ac8e4 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -22,6 +22,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" @@ -734,10 +735,11 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) error { return errors.Trace(err) } - // gofail: var tmpMaxTxnTime uint64 - // if tmpMaxTxnTime > 0 { - // c.maxTxnTimeUse = tmpMaxTxnTime - // } + failpoint.Inject("tmpMaxTxnTime", func(val failpoint.Value) { + if tmpMaxTxnTime := uint64(val.(int)); tmpMaxTxnTime > 0 { + c.maxTxnTimeUse = tmpMaxTxnTime + } + }) if c.store.oracle.IsExpired(c.startTS, c.maxTxnTimeUse) { err = errors.Errorf("conn%d txn takes too much time, txnStartTS: %d, comm: %d", diff --git a/store/tikv/2pc_fail_test.go b/store/tikv/2pc_fail_test.go index 9cc1d9e698244..49624f5be5197 100644 --- a/store/tikv/2pc_fail_test.go +++ b/store/tikv/2pc_fail_test.go @@ -18,7 +18,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/kv" ) @@ -26,8 +26,10 @@ import ( // TestFailCommitPrimaryRpcErrors tests rpc errors are handled properly when // committing primary region task. func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) { - gofail.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("timeout")`) - defer gofail.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("timeout")`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + }() // The rpc error will be wrapped to ErrResultUndetermined. t1 := s.begin(c) err := t1.Set([]byte("a"), []byte("a1")) @@ -44,8 +46,10 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRpcErrors(c *C) { // TestFailCommitPrimaryRegionError tests RegionError is handled properly when // committing primary region task. func (s *testCommitterSuite) TestFailCommitPrimaryRegionError(c *C) { - gofail.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("notLeader")`) - defer gofail.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("notLeader")`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + }() // Ensure it returns the original error without wrapped to ErrResultUndetermined // if it exceeds max retry timeout on RegionError. t2 := s.begin(c) @@ -59,8 +63,10 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRegionError(c *C) { // TestFailCommitPrimaryRPCErrorThenRegionError tests the case when commit first // receive a rpc timeout, then region errors afterwrards. func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenRegionError(c *C) { - gofail.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `1*return("timeout")->return("notLeader")`) - defer gofail.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `1*return("timeout")->return("notLeader")`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + }() // The region error will be wrapped to ErrResultUndetermined. t1 := s.begin(c) err := t1.Set([]byte("a"), []byte("a1")) @@ -73,8 +79,10 @@ func (s *testCommitterSuite) TestFailCommitPrimaryRPCErrorThenRegionError(c *C) // TestFailCommitPrimaryKeyError tests KeyError is handled properly when // committing primary region task. func (s *testCommitterSuite) TestFailCommitPrimaryKeyError(c *C) { - gofail.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("keyError")`) - defer gofail.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult", `return("keyError")`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitResult"), IsNil) + }() // Ensure it returns the original error without wrapped to ErrResultUndetermined // if it meets KeyError. t3 := s.begin(c) @@ -86,8 +94,10 @@ func (s *testCommitterSuite) TestFailCommitPrimaryKeyError(c *C) { } func (s *testCommitterSuite) TestFailCommitTimeout(c *C) { - gofail.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcCommitTimeout"), IsNil) + }() txn := s.begin(c) err := txn.Set([]byte("a"), []byte("a1")) c.Assert(err, IsNil) diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index 83f87da8dc165..b7aa4a39e907f 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -27,6 +27,7 @@ import ( "github.com/cznic/mathutil" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" @@ -636,11 +637,11 @@ func (worker *copIteratorWorker) handleTask(bo *Backoffer, task *copTask, respCh // handleTaskOnce handles single copTask, successful results are send to channel. // If error happened, returns error. If region split or meet lock, returns the remain tasks. func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch chan<- *copResponse) ([]*copTask, error) { - - // gofail: var handleTaskOnceError bool - // if handleTaskOnceError { - // return nil, errors.New("mock handleTaskOnce error") - // } + failpoint.Inject("handleTaskOnceError", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(nil, errors.New("mock handleTaskOnce error")) + } + }) sender := NewRegionRequestSender(worker.store.regionCache, worker.store.client) req := &tikvrpc.Request{ diff --git a/store/tikv/gcworker/gc_worker_test.go b/store/tikv/gcworker/gc_worker_test.go index 579a8b9b456f5..2245159cbd8bd 100644 --- a/store/tikv/gcworker/gc_worker_test.go +++ b/store/tikv/gcworker/gc_worker_test.go @@ -21,7 +21,7 @@ import ( "time" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" @@ -174,23 +174,23 @@ func (s *testGCWorkerSuite) TestDoGCForOneRegion(c *C) { c.Assert(regionErr, IsNil) c.Assert(err, IsNil) - gofail.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("timeout")`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("timeout")`), IsNil) regionErr, err = taskWorker.doGCForRegion(bo, 20, loc.Region) c.Assert(regionErr, IsNil) c.Assert(err, NotNil) - gofail.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult"), IsNil) - gofail.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("GCNotLeader")`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("GCNotLeader")`), IsNil) regionErr, err = taskWorker.doGCForRegion(bo, 20, loc.Region) c.Assert(regionErr.GetNotLeader(), NotNil) c.Assert(err, IsNil) - gofail.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult"), IsNil) - gofail.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("GCServerIsBusy")`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult", `return("GCServerIsBusy")`), IsNil) regionErr, err = taskWorker.doGCForRegion(bo, 20, loc.Region) c.Assert(regionErr.GetServerIsBusy(), NotNil) c.Assert(err, IsNil) - gofail.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreSendReqResult"), IsNil) } func (s *testGCWorkerSuite) TestDoGC(c *C) { diff --git a/store/tikv/region_request.go b/store/tikv/region_request.go index 9791a348785ae..a10efc645fcaf 100644 --- a/store/tikv/region_request.go +++ b/store/tikv/region_request.go @@ -18,6 +18,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/kv" @@ -67,26 +68,26 @@ func (s *RegionRequestSender) SendReq(bo *Backoffer, req *tikvrpc.Request, regio // SendReqCtx sends a request to tikv server and return response and RPCCtx of this RPC. func (s *RegionRequestSender) SendReqCtx(bo *Backoffer, req *tikvrpc.Request, regionID RegionVerID, timeout time.Duration) (*tikvrpc.Response, *RPCContext, error) { - - // gofail: var tikvStoreSendReqResult string - // switch tikvStoreSendReqResult { - // case "timeout": - // return nil, nil, errors.New("timeout") - // case "GCNotLeader": - // if req.Type == tikvrpc.CmdGC { - // return &tikvrpc.Response{ - // Type: tikvrpc.CmdGC, - // GC: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, - // }, nil, nil - // } - // case "GCServerIsBusy": - // if req.Type == tikvrpc.CmdGC { - // return &tikvrpc.Response{ - // Type: tikvrpc.CmdGC, - // GC: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}}, - // }, nil, nil - // } - // } + failpoint.Inject("tikvStoreSendReqResult", func(val failpoint.Value) { + switch val.(string) { + case "timeout": + failpoint.Return(nil, nil, errors.New("timeout")) + case "GCNotLeader": + if req.Type == tikvrpc.CmdGC { + failpoint.Return(&tikvrpc.Response{ + Type: tikvrpc.CmdGC, + GC: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{NotLeader: &errorpb.NotLeader{}}}, + }, nil, nil) + } + case "GCServerIsBusy": + if req.Type == tikvrpc.CmdGC { + failpoint.Return(&tikvrpc.Response{ + Type: tikvrpc.CmdGC, + GC: &kvrpcpb.GCResponse{RegionError: &errorpb.Error{ServerIsBusy: &errorpb.ServerIsBusy{}}}, + }, nil, nil) + } + } + }) for { ctx, err := s.regionCache.GetRPCContext(bo, regionID) diff --git a/store/tikv/sql_fail_test.go b/store/tikv/sql_fail_test.go index 95076a331177f..28fd144853c39 100644 --- a/store/tikv/sql_fail_test.go +++ b/store/tikv/sql_fail_test.go @@ -20,7 +20,7 @@ import ( "time" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/session" @@ -52,14 +52,16 @@ func (s *testSQLSuite) TearDownSuite(c *C) { } func (s *testSQLSuite) TestInsertSleepOverMaxTxnTime(c *C) { - defer gofail.Disable("github.com/pingcap/tidb/store/tmpMaxTxnTime") + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tmpMaxTxnTime"), IsNil) + }() se, err := session.CreateSession4Test(s.store) c.Assert(err, IsNil) _, err = se.Execute(context.Background(), "drop table if exists test.t") c.Assert(err, IsNil) _, err = se.Execute(context.Background(), "create table test.t(a int)") c.Assert(err, IsNil) - gofail.Enable("github.com/pingcap/tidb/store/tmpMaxTxnTime", `return(2)->return(0)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tmpMaxTxnTime", `return(2)->return(0)`), IsNil) start := time.Now() _, err = se.Execute(context.Background(), "insert into test.t (a) select sleep(3)") c.Assert(err, IsNil) @@ -73,11 +75,11 @@ func (s *testSQLSuite) TestFailBusyServerCop(c *C) { var wg sync.WaitGroup wg.Add(2) - gofail.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy", `return(true)`), IsNil) go func() { defer wg.Done() time.Sleep(time.Millisecond * 100) - gofail.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy"), IsNil) }() go func() { diff --git a/store/tikv/store_fail_test.go b/store/tikv/store_fail_test.go index 00bf9ece25df0..17b9038a82a99 100644 --- a/store/tikv/store_fail_test.go +++ b/store/tikv/store_fail_test.go @@ -19,7 +19,7 @@ import ( "time" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" ) func (s *testStoreSuite) TestFailBusyServerKV(c *C) { @@ -33,11 +33,11 @@ func (s *testStoreSuite) TestFailBusyServerKV(c *C) { var wg sync.WaitGroup wg.Add(2) - gofail.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy", `return(true)`) + c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy", `return(true)`), IsNil) go func() { defer wg.Done() time.Sleep(time.Millisecond * 100) - gofail.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy") + c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/mockstore/mocktikv/rpcServerBusy"), IsNil) }() go func() { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 8727bf4cf82ed..a6254e3bb589c 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -20,6 +20,7 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" @@ -220,11 +221,12 @@ func (txn *tikvTxn) Commit(ctx context.Context) error { } defer txn.close() - // gofail: var mockCommitError bool - // if mockCommitError && kv.IsMockCommitErrorEnable() { - // kv.MockCommitErrorDisable() - // return errors.New("mock commit error") - // } + failpoint.Inject("mockCommitError", func(val failpoint.Value) { + if val.(bool) && kv.IsMockCommitErrorEnable() { + kv.MockCommitErrorDisable() + failpoint.Return(errors.New("mock commit error")) + } + }) metrics.TiKVTxnCmdCounter.WithLabelValues("set").Add(float64(txn.setCnt)) metrics.TiKVTxnCmdCounter.WithLabelValues("commit").Inc() diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index 6285505e5b77d..644a51dd95af0 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -20,7 +20,7 @@ import ( "time" . "github.com/pingcap/check" - gofail "github.com/pingcap/gofail/runtime" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/stmtctx" @@ -282,8 +282,10 @@ func (s *testTableCodecSuite) TestCutKey(c *C) { } func (s *testTableCodecSuite) TestDecodeBadDecical(c *C) { - gofail.Enable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal", `return(true)`) - defer gofail.Disable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal") + c.Assert(failpoint.Enable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal", `return(true)`), IsNil) + defer func() { + c.Assert(failpoint.Disable("github.com/pingcap/tidb/util/codec/errorInDecodeDecimal"), IsNil) + }() dec := types.NewDecFromStringForTest("0.111") b, err := codec.EncodeDecimal(nil, dec, 0, 0) c.Assert(err, IsNil) diff --git a/util/codec/decimal.go b/util/codec/decimal.go index 8fcb849c70d23..abf7130a7f4b9 100644 --- a/util/codec/decimal.go +++ b/util/codec/decimal.go @@ -15,6 +15,7 @@ package codec import ( "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb/types" ) @@ -31,10 +32,11 @@ func EncodeDecimal(b []byte, dec *types.MyDecimal, precision, frac int) ([]byte, // DecodeDecimal decodes bytes to decimal. func DecodeDecimal(b []byte) ([]byte, *types.MyDecimal, int, int, error) { - // gofail: var errorInDecodeDecimal bool - // if errorInDecodeDecimal { - // return b, nil, 0, 0, errors.New("gofail error") - // } + failpoint.Inject("errorInDecodeDecimal", func(val failpoint.Value) { + if val.(bool) { + failpoint.Return(b, nil, 0, 0, errors.New("gofail error")) + } + }) if len(b) < 3 { return b, nil, 0, 0, errors.New("insufficient bytes to decode value") From 92bdfb524d06b51028acaebaa54150be497fc221 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 22 Apr 2019 13:49:54 +0800 Subject: [PATCH 02/21] store/helper: make test stable (#10211) --- store/helper/helper_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/store/helper/helper_test.go b/store/helper/helper_test.go index aa803a0ae3626..db98ede717a00 100644 --- a/store/helper/helper_test.go +++ b/store/helper/helper_test.go @@ -67,7 +67,7 @@ func (s *HelperTestSuite) SetUpSuite(c *C) { mockTikvStore, err := mockstore.NewMockTikvStore(mockstore.WithMVCCStore(mvccStore)) s.store = &mockStore{ mockTikvStore.(tikv.Storage), - []string{"127.0.0.1:10090/"}, + []string{"127.0.0.1:10100/"}, } c.Assert(err, IsNil) } @@ -87,7 +87,7 @@ func (s *HelperTestSuite) mockPDHTTPServer(c *C) { router.HandleFunc("/pd/api/v1/hotspot/regions/read", s.mockHotRegionResponse) serverMux := http.NewServeMux() serverMux.Handle("/", router) - server := &http.Server{Addr: "127.0.0.1:10090", Handler: serverMux} + server := &http.Server{Addr: "127.0.0.1:10100", Handler: serverMux} err := server.ListenAndServe() c.Assert(err, IsNil) } From d6396daa77743fb234650d28c862f2d84299da9d Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 22 Apr 2019 14:23:33 +0800 Subject: [PATCH 03/21] *: lazy eval explain id and tracker label (#10139) --- distsql/distsql.go | 3 +- distsql/distsql_test.go | 11 ++++- distsql/request_builder.go | 3 +- distsql/select_result.go | 5 ++- executor/benchmark_test.go | 3 +- executor/builder.go | 11 ++++- executor/distsql.go | 28 +++++++----- executor/executor.go | 9 ++-- executor/executor_pkg_test.go | 3 +- executor/executor_required_rows_test.go | 15 ++++--- executor/index_lookup_join.go | 5 ++- executor/join.go | 10 ++++- executor/load_data.go | 5 ++- executor/merge_join.go | 6 ++- executor/pkg_test.go | 17 +++---- executor/prepared.go | 5 ++- executor/sort.go | 10 +++-- executor/table_reader.go | 12 +++-- executor/table_readers_required_rows_test.go | 5 ++- planner/core/common_plans.go | 11 ++--- planner/core/explain.go | 4 +- planner/core/plan.go | 10 +++-- util/chunk/list.go | 7 ++- util/memory/tracker.go | 12 ++--- util/memory/tracker_test.go | 47 ++++++++++---------- util/stringutil/string_util.go | 30 +++++++++++++ 26 files changed, 192 insertions(+), 95 deletions(-) diff --git a/distsql/distsql.go b/distsql/distsql.go index 9c808fe0c30d6..36dabda4270c1 100644 --- a/distsql/distsql.go +++ b/distsql/distsql.go @@ -15,6 +15,7 @@ package distsql import ( "context" + "fmt" "github.com/pingcap/errors" "github.com/pingcap/tidb/kv" @@ -82,7 +83,7 @@ func Select(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, fie // The difference from Select is that SelectWithRuntimeStats will set copPlanIDs into selectResult, // which can help selectResult to collect runtime stats. func SelectWithRuntimeStats(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []string) (SelectResult, error) { + fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []fmt.Stringer) (SelectResult, error) { sr, err := Select(ctx, sctx, kvReq, fieldTypes, fb) if err != nil { return sr, err diff --git a/distsql/distsql_test.go b/distsql/distsql_test.go index a60ef28911b62..478881e47720f 100644 --- a/distsql/distsql_test.go +++ b/distsql/distsql_test.go @@ -15,6 +15,7 @@ package distsql import ( "context" + "fmt" "sync" "testing" "time" @@ -31,6 +32,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/execdetails" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -63,7 +65,12 @@ func (s *testSuite) createSelectNormal(batch, totalRows int, c *C, planIDs []str if planIDs == nil { response, err = Select(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false)) } else { - response, err = SelectWithRuntimeStats(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false), planIDs) + var planIDFuncs []fmt.Stringer + for i := range planIDs { + idx := i + planIDFuncs = append(planIDFuncs, stringutil.StringerStr(planIDs[idx])) + } + response, err = SelectWithRuntimeStats(context.TODO(), s.sctx, request, colTypes, statistics.NewQueryFeedback(0, nil, 0, false), planIDFuncs) } c.Assert(err, IsNil) @@ -115,7 +122,7 @@ func (s *testSuite) TestSelectWithRuntimeStats(c *C) { c.Fatal("invalid copPlanIDs") } for i := range planIDs { - if response.copPlanIDs[i] != planIDs[i] { + if response.copPlanIDs[i].String() != planIDs[i] { c.Fatal("invalid copPlanIDs") } } diff --git a/distsql/request_builder.go b/distsql/request_builder.go index 824ddf4cb76dc..06734fbc45d06 100644 --- a/distsql/request_builder.go +++ b/distsql/request_builder.go @@ -14,6 +14,7 @@ package distsql import ( + "fmt" "math" "github.com/pingcap/parser/mysql" @@ -43,7 +44,7 @@ func (builder *RequestBuilder) Build() (*kv.Request, error) { } // SetMemTracker sets a memTracker for this request. -func (builder *RequestBuilder) SetMemTracker(sctx sessionctx.Context, label string) *RequestBuilder { +func (builder *RequestBuilder) SetMemTracker(sctx sessionctx.Context, label fmt.Stringer) *RequestBuilder { t := memory.NewTracker(label, sctx.GetSessionVars().MemQuotaDistSQL) t.AttachTo(sctx.GetSessionVars().StmtCtx.MemTracker) builder.Request.MemTracker = t diff --git a/distsql/select_result.go b/distsql/select_result.go index 2e3ca4913404d..f428d8640e108 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -15,6 +15,7 @@ package distsql import ( "context" + "fmt" "time" "github.com/pingcap/errors" @@ -74,7 +75,7 @@ type selectResult struct { // copPlanIDs contains all copTasks' planIDs, // which help to collect copTasks' runtime stats. - copPlanIDs []string + copPlanIDs []fmt.Stringer memTracker *memory.Tracker } @@ -207,7 +208,7 @@ func (r *selectResult) updateCopRuntimeStats(callee string) { detail.NumProducedRows != nil && detail.NumIterations != nil { planID := r.copPlanIDs[i] r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl. - RecordOneCopTask(planID, callee, detail) + RecordOneCopTask(planID.String(), callee, detail) } } } diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 976fc6d58c4b5..60b0c5b244f97 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/stringutil" ) var ( @@ -127,7 +128,7 @@ func (mds *mockDataSource) Next(ctx context.Context, req *chunk.RecordBatch) err } func buildMockDataSource(opt mockDataSourceParameters) *mockDataSource { - baseExec := newBaseExecutor(opt.ctx, opt.schema, "") + baseExec := newBaseExecutor(opt.ctx, opt.schema, stringutil.StringerStr("")) m := &mockDataSource{baseExec, opt, nil, nil, 0} types := m.retTypes() colData := make([][]interface{}, len(types)) diff --git a/executor/builder.go b/executor/builder.go index 3f500b808eaa5..53eff17bb27ce 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -16,6 +16,7 @@ package executor import ( "bytes" "context" + "fmt" "math" "sort" "strings" @@ -46,6 +47,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" ) @@ -666,9 +668,14 @@ func (b *executorBuilder) buildReplace(vals *InsertValues) Executor { return replaceExec } +var ( + grantStmtLabel fmt.Stringer = stringutil.StringerStr("GrantStmt") + revokeStmtLabel fmt.Stringer = stringutil.StringerStr("RevokeStmt") +) + func (b *executorBuilder) buildGrant(grant *ast.GrantStmt) Executor { e := &GrantExec{ - baseExecutor: newBaseExecutor(b.ctx, nil, "GrantStmt"), + baseExecutor: newBaseExecutor(b.ctx, nil, grantStmtLabel), Privs: grant.Privs, ObjectType: grant.ObjectType, Level: grant.Level, @@ -681,7 +688,7 @@ func (b *executorBuilder) buildGrant(grant *ast.GrantStmt) Executor { func (b *executorBuilder) buildRevoke(revoke *ast.RevokeStmt) Executor { e := &RevokeExec{ - baseExecutor: newBaseExecutor(b.ctx, nil, "RevokeStmt"), + baseExecutor: newBaseExecutor(b.ctx, nil, revokeStmtLabel), ctx: b.ctx, Privs: revoke.Privs, ObjectType: revoke.ObjectType, diff --git a/executor/distsql.go b/executor/distsql.go index dc7309428df3b..e93d67cb16670 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -15,6 +15,7 @@ package executor import ( "context" + "fmt" "math" "runtime" "sort" @@ -41,6 +42,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" "go.uber.org/zap" ) @@ -258,7 +260,7 @@ func (e *IndexReaderExecutor) Close() error { err := e.result.Close() e.result = nil if e.runtimeStats != nil { - copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.plans[0].ExplainID()) + copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.plans[0].ExplainID().String()) copStats.SetRowNum(e.feedback.Actual()) } e.ctx.StoreQueryFeedback(e.feedback) @@ -299,6 +301,8 @@ func (e *IndexReaderExecutor) Open(ctx context.Context) error { return e.open(ctx, kvRanges) } +var indexReaderDistSQLTrackerLabel fmt.Stringer = stringutil.StringerStr("IndexReaderDistSQLTracker") + func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) error { var err error if e.corColInFilter { @@ -320,7 +324,7 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange) SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetMemTracker(e.ctx, "IndexReaderDistSQLTracker"). + SetMemTracker(e.ctx, indexReaderDistSQLTrackerLabel). Build() if err != nil { e.feedback.Invalidate() @@ -442,6 +446,8 @@ func (e *IndexLookUpExecutor) startWorkers(ctx context.Context, initBatchSize in return nil } +var indexLookupDistSQLTrackerLabel fmt.Stringer = stringutil.StringerStr("IndexLookupDistSQLTracker") + // startIndexWorker launch a background goroutine to fetch handles, send the results to workCh. func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []kv.KeyRange, workCh chan<- *lookupTableTask, initBatchSize int) error { if e.runtimeStats != nil { @@ -456,7 +462,7 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k SetKeepOrder(e.keepOrder). SetStreaming(e.indexStreaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetMemTracker(e.ctx, "IndexLookupDistSQLTracker"). + SetMemTracker(e.ctx, indexLookupDistSQLTrackerLabel). Build() if err != nil { return err @@ -492,9 +498,9 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k logutil.Logger(ctx).Error("close Select result failed", zap.Error(err)) } if e.runtimeStats != nil { - copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.idxPlans[len(e.idxPlans)-1].ExplainID()) + copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.idxPlans[len(e.idxPlans)-1].ExplainID().String()) copStats.SetRowNum(count) - copStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.tblPlans[0].ExplainID()) + copStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.tblPlans[0].ExplainID().String()) copStats.SetRowNum(count) } e.ctx.StoreQueryFeedback(e.feedback) @@ -505,6 +511,8 @@ func (e *IndexLookUpExecutor) startIndexWorker(ctx context.Context, kvRanges []k return nil } +var tableWorkerLabel fmt.Stringer = stringutil.StringerStr("tableWorker") + // startTableWorker launchs some background goroutines which pick tasks from workCh and execute the task. func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-chan *lookupTableTask) { lookupConcurrencyLimit := e.ctx.GetSessionVars().IndexLookupConcurrency @@ -517,7 +525,7 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha keepOrder: e.keepOrder, handleIdx: e.handleIdx, isCheckOp: e.isCheckOp, - memTracker: memory.NewTracker("tableWorker", -1), + memTracker: memory.NewTracker(tableWorkerLabel, -1), } worker.memTracker.AttachTo(e.memTracker) ctx1, cancel := context.WithCancel(ctx) @@ -531,7 +539,7 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha func (e *IndexLookUpExecutor) buildTableReader(ctx context.Context, handles []int64) (Executor, error) { tableReaderExec := &TableReaderExecutor{ - baseExecutor: newBaseExecutor(e.ctx, e.schema, e.id+"_tableReader"), + baseExecutor: newBaseExecutor(e.ctx, e.schema, stringutil.MemoizeStr(func() string { return e.id.String() + "_tableReader" })), table: e.table, dagPB: e.tableRequest, streaming: e.tableStreaming, @@ -565,7 +573,7 @@ func (e *IndexLookUpExecutor) Close() error { e.memTracker.Detach() e.memTracker = nil if e.runtimeStats != nil { - copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.idxPlans[0].ExplainID()) + copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.idxPlans[0].ExplainID().String()) copStats.SetRowNum(e.feedback.Actual()) } return nil @@ -852,8 +860,8 @@ func GetLackHandles(expectedHandles []int64, obtainedHandlesMap map[int64]struct return diffHandles } -func getPhysicalPlanIDs(plans []plannercore.PhysicalPlan) []string { - planIDs := make([]string, 0, len(plans)) +func getPhysicalPlanIDs(plans []plannercore.PhysicalPlan) []fmt.Stringer { + planIDs := make([]fmt.Stringer, 0, len(plans)) for _, p := range plans { planIDs = append(planIDs, p.ExplainID()) } diff --git a/executor/executor.go b/executor/executor.go index 619d16a370b83..89ae7cf7a5707 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -46,6 +46,7 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/stringutil" "go.uber.org/zap" ) @@ -76,7 +77,7 @@ var ( type baseExecutor struct { ctx sessionctx.Context - id string + id fmt.Stringer schema *expression.Schema initCap int maxChunkSize int @@ -130,7 +131,7 @@ func (e *baseExecutor) Next(ctx context.Context, req *chunk.RecordBatch) error { return nil } -func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id string, children ...Executor) baseExecutor { +func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id fmt.Stringer, children ...Executor) baseExecutor { e := baseExecutor{ children: children, ctx: ctx, @@ -140,7 +141,7 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id strin maxChunkSize: ctx.GetSessionVars().MaxChunkSize, } if ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { - e.runtimeStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.id) + e.runtimeStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.id.String()) } if schema != nil { cols := schema.Columns @@ -1291,7 +1292,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars := ctx.GetSessionVars() sc := &stmtctx.StatementContext{ TimeZone: vars.Location(), - MemTracker: memory.NewTracker(s.Text(), vars.MemQuotaQuery), + MemTracker: memory.NewTracker(stringutil.MemoizeStr(s.Text), vars.MemQuotaQuery), } switch config.GetGlobalConfig().OOMAction { case config.OOMActionCancel: diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index bff4f3eae7396..4ded2edf33048 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" ) var _ = Suite(&testExecSuite{}) @@ -92,7 +93,7 @@ func (s *testExecSuite) TestShowProcessList(c *C) { // Compose executor. e := &ShowExec{ - baseExecutor: newBaseExecutor(sctx, schema, ""), + baseExecutor: newBaseExecutor(sctx, schema, stringutil.StringerStr("")), Tp: ast.ShowProcessList, } diff --git a/executor/executor_required_rows_test.go b/executor/executor_required_rows_test.go index 96273d628bfef..1cf40fec57a2b 100644 --- a/executor/executor_required_rows_test.go +++ b/executor/executor_required_rows_test.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/stringutil" ) type requiredRowsDataSource struct { @@ -62,7 +63,7 @@ func newRequiredRowsDataSource(ctx sessionctx.Context, totalRows int, expectedRo cols[i] = &expression.Column{Index: i, RetType: retTypes[i]} } schema := expression.NewSchema(cols...) - baseExec := newBaseExecutor(ctx, schema, "") + baseExec := newBaseExecutor(ctx, schema, stringutil.StringerStr("")) return &requiredRowsDataSource{baseExec, totalRows, 0, ctx, expectedRowsRet, 0, defaultGenerator} } @@ -190,7 +191,7 @@ func (s *testExecSuite) TestLimitRequiredRows(c *C) { func buildLimitExec(ctx sessionctx.Context, src Executor, offset, count int) Executor { n := mathutil.Min(count, ctx.GetSessionVars().MaxChunkSize) - base := newBaseExecutor(ctx, src.Schema(), "", src) + base := newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src) base.initCap = n limitExec := &LimitExec{ baseExecutor: base, @@ -205,7 +206,7 @@ func defaultCtx() sessionctx.Context { ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize ctx.GetSessionVars().MemQuotaSort = variable.DefTiDBMemQuotaSort - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker("", ctx.GetSessionVars().MemQuotaQuery) + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(stringutil.StringerStr(""), ctx.GetSessionVars().MemQuotaQuery) ctx.GetSessionVars().SnapshotTS = uint64(1) return ctx } @@ -273,7 +274,7 @@ func (s *testExecSuite) TestSortRequiredRows(c *C) { func buildSortExec(sctx sessionctx.Context, byItems []*plannercore.ByItems, src Executor) Executor { sortExec := SortExec{ - baseExecutor: newBaseExecutor(sctx, src.Schema(), "", src), + baseExecutor: newBaseExecutor(sctx, src.Schema(), stringutil.StringerStr(""), src), ByItems: byItems, schema: src.Schema(), } @@ -380,7 +381,7 @@ func (s *testExecSuite) TestTopNRequiredRows(c *C) { func buildTopNExec(ctx sessionctx.Context, offset, count int, byItems []*plannercore.ByItems, src Executor) Executor { sortExec := SortExec{ - baseExecutor: newBaseExecutor(ctx, src.Schema(), "", src), + baseExecutor: newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src), ByItems: byItems, schema: src.Schema(), } @@ -473,7 +474,7 @@ func (s *testExecSuite) TestSelectionRequiredRows(c *C) { func buildSelectionExec(ctx sessionctx.Context, filters []expression.Expression, src Executor) Executor { return &SelectionExec{ - baseExecutor: newBaseExecutor(ctx, src.Schema(), "", src), + baseExecutor: newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src), filters: filters, } } @@ -591,7 +592,7 @@ func (s *testExecSuite) TestProjectionParallelRequiredRows(c *C) { func buildProjectionExec(ctx sessionctx.Context, exprs []expression.Expression, src Executor, numWorkers int) Executor { return &ProjectionExec{ - baseExecutor: newBaseExecutor(ctx, src.Schema(), "", src), + baseExecutor: newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src), numWorkers: int64(numWorkers), evaluatorSuit: expression.NewEvaluatorSuite(exprs, false), } diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 8dab9ba4681cb..e49baf3c6290a 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mvmap" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" "go.uber.org/zap" ) @@ -366,7 +367,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { encodedLookUpKeys: chunk.NewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeBlob)}, ow.ctx.GetSessionVars().MaxChunkSize), lookupMap: mvmap.NewMVMap(), } - task.memTracker = memory.NewTracker(fmt.Sprintf("lookup join task %p", task), -1) + task.memTracker = memory.NewTracker(stringutil.MemoizeStr(func() string { return fmt.Sprintf("lookup join task %p", task) }), -1) task.memTracker.AttachTo(ow.parentMemTracker) ow.increaseBatchSize() @@ -564,7 +565,7 @@ func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTa } defer terror.Call(innerExec.Close) innerResult := chunk.NewList(innerExec.retTypes(), iw.ctx.GetSessionVars().MaxChunkSize, iw.ctx.GetSessionVars().MaxChunkSize) - innerResult.GetMemTracker().SetLabel("inner result") + innerResult.GetMemTracker().SetLabel(innerResultLabel) innerResult.GetMemTracker().AttachTo(task.memTracker) for { err := innerExec.Next(ctx, chunk.NewRecordBatch(iw.executorChk)) diff --git a/executor/join.go b/executor/join.go index 98a1cd9318eb7..3352a3307161a 100644 --- a/executor/join.go +++ b/executor/join.go @@ -15,6 +15,7 @@ package executor import ( "context" + "fmt" "sync" "sync/atomic" "time" @@ -30,6 +31,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mvmap" + "github.com/pingcap/tidb/util/stringutil" ) var ( @@ -261,12 +263,14 @@ func (e *HashJoinExec) wait4Inner() (finished bool, err error) { return false, nil } +var innerResultLabel fmt.Stringer = stringutil.StringerStr("innerResult") + // fetchInnerRows fetches all rows from inner executor, and append them to // e.innerResult. func (e *HashJoinExec) fetchInnerRows(ctx context.Context) error { e.innerResult = chunk.NewList(e.innerExec.retTypes(), e.initCap, e.maxChunkSize) e.innerResult.GetMemTracker().AttachTo(e.memTracker) - e.innerResult.GetMemTracker().SetLabel("innerResult") + e.innerResult.GetMemTracker().SetLabel(innerResultLabel) var err error for { if e.finished.Load().(bool) { @@ -611,6 +615,8 @@ func (e *NestedLoopApplyExec) Close() error { return e.outerExec.Close() } +var innerListLabel fmt.Stringer = stringutil.StringerStr("innerList") + // Open implements the Executor interface. func (e *NestedLoopApplyExec) Open(ctx context.Context) error { err := e.outerExec.Open(ctx) @@ -626,7 +632,7 @@ func (e *NestedLoopApplyExec) Open(ctx context.Context) error { e.memTracker = memory.NewTracker(e.id, e.ctx.GetSessionVars().MemQuotaNestedLoopApply) e.memTracker.AttachTo(e.ctx.GetSessionVars().StmtCtx.MemTracker) - e.innerList.GetMemTracker().SetLabel("innerList") + e.innerList.GetMemTracker().SetLabel(innerListLabel) e.innerList.GetMemTracker().AttachTo(e.memTracker) return nil diff --git a/executor/load_data.go b/executor/load_data.go index b03a912213323..25ec97b830f25 100644 --- a/executor/load_data.go +++ b/executor/load_data.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/stringutil" "go.uber.org/zap" ) @@ -37,9 +38,11 @@ type LoadDataExec struct { loadDataInfo *LoadDataInfo } +var insertValuesLabel fmt.Stringer = stringutil.StringerStr("InsertValues") + // NewLoadDataInfo returns a LoadDataInfo structure, and it's only used for tests now. func NewLoadDataInfo(ctx sessionctx.Context, row []types.Datum, tbl table.Table, cols []*table.Column) *LoadDataInfo { - insertVal := &InsertValues{baseExecutor: newBaseExecutor(ctx, nil, "InsertValues"), Table: tbl} + insertVal := &InsertValues{baseExecutor: newBaseExecutor(ctx, nil, insertValuesLabel), Table: tbl} return &LoadDataInfo{ row: row, InsertValues: insertVal, diff --git a/executor/merge_join.go b/executor/merge_join.go index 4db3784f4e7a4..2eca140bed902 100644 --- a/executor/merge_join.go +++ b/executor/merge_join.go @@ -15,6 +15,7 @@ package executor import ( "context" + "fmt" "time" "github.com/pingcap/errors" @@ -22,6 +23,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/stringutil" ) // MergeJoinExec implements the merge join algorithm. @@ -206,6 +208,8 @@ func (e *MergeJoinExec) Close() error { return e.baseExecutor.Close() } +var innerTableLabel fmt.Stringer = stringutil.StringerStr("innerTable") + // Open implements the Executor Open interface. func (e *MergeJoinExec) Open(ctx context.Context) error { if err := e.baseExecutor.Open(ctx); err != nil { @@ -221,7 +225,7 @@ func (e *MergeJoinExec) Open(ctx context.Context) error { e.childrenResults = append(e.childrenResults, child.newFirstChunk()) } - e.innerTable.memTracker = memory.NewTracker("innerTable", -1) + e.innerTable.memTracker = memory.NewTracker(innerTableLabel, -1) e.innerTable.memTracker.AttachTo(e.memTracker) return nil diff --git a/executor/pkg_test.go b/executor/pkg_test.go index ccea2e3882b16..3893cfaf188c6 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -17,6 +17,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/stringutil" "github.com/spaolacci/murmur3" ) @@ -63,7 +64,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { con := &expression.Constant{Value: types.NewDatum(6), RetType: types.NewFieldType(mysql.TypeLong)} outerSchema := expression.NewSchema(col0) outerExec := &MockExec{ - baseExecutor: newBaseExecutor(sctx, outerSchema, ""), + baseExecutor: newBaseExecutor(sctx, outerSchema, stringutil.StringerStr("")), Rows: []chunk.MutRow{ chunk.MutRowFromDatums(types.MakeDatums(1)), chunk.MutRowFromDatums(types.MakeDatums(2)), @@ -74,7 +75,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { }} innerSchema := expression.NewSchema(col1) innerExec := &MockExec{ - baseExecutor: newBaseExecutor(sctx, innerSchema, ""), + baseExecutor: newBaseExecutor(sctx, innerSchema, stringutil.StringerStr("")), Rows: []chunk.MutRow{ chunk.MutRowFromDatums(types.MakeDatums(1)), chunk.MutRowFromDatums(types.MakeDatums(2)), @@ -90,7 +91,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ - baseExecutor: newBaseExecutor(sctx, joinSchema, ""), + baseExecutor: newBaseExecutor(sctx, joinSchema, stringutil.StringerStr("")), outerExec: outerExec, innerExec: innerExec, outerFilter: []expression.Expression{outerFilter}, @@ -121,7 +122,7 @@ func prepareOneColChildExec(sctx sessionctx.Context, rowCount int) Executor { col0 := &expression.Column{Index: 0, RetType: types.NewFieldType(mysql.TypeLong)} schema := expression.NewSchema(col0) exec := &MockExec{ - baseExecutor: newBaseExecutor(sctx, schema, ""), + baseExecutor: newBaseExecutor(sctx, schema, stringutil.StringerStr("")), Rows: make([]chunk.MutRow, rowCount)} for i := 0; i < len(exec.Rows); i++ { exec.Rows[i] = chunk.MutRowFromDatums(types.MakeDatums(i % 10)) @@ -137,7 +138,7 @@ func buildExec4RadixHashJoin(sctx sessionctx.Context, rowCount int) *RadixHashJo col1 := &expression.Column{Index: 0, RetType: types.NewFieldType(mysql.TypeLong)} joinSchema := expression.NewSchema(col0, col1) hashJoinExec := &HashJoinExec{ - baseExecutor: newBaseExecutor(sctx, joinSchema, "HashJoin", childExec0, childExec1), + baseExecutor: newBaseExecutor(sctx, joinSchema, stringutil.StringerStr("HashJoin"), childExec0, childExec1), concurrency: 4, joinType: 0, // InnerJoin innerIdx: 0, @@ -163,7 +164,7 @@ func (s *pkgTestSuite) TestRadixPartition(c *C) { defer func() { sv.L2CacheSize, sv.EnableRadixJoin, sv.MaxChunkSize = originL2CacheSize, originEnableRadixJoin, originMaxChunkSize }() - sv.StmtCtx.MemTracker = memory.NewTracker("RootMemTracker", variable.DefTiDBMemQuotaHashJoin) + sv.StmtCtx.MemTracker = memory.NewTracker(stringutil.StringerStr("RootMemTracker"), variable.DefTiDBMemQuotaHashJoin) ctx := context.Background() err := hashJoinExec.Open(ctx) @@ -249,7 +250,7 @@ func BenchmarkPartitionInnerRows(b *testing.B) { defer func() { sv.L2CacheSize, sv.EnableRadixJoin, sv.MaxChunkSize = originL2CacheSize, originEnableRadixJoin, originMaxChunkSize }() - sv.StmtCtx.MemTracker = memory.NewTracker("RootMemTracker", variable.DefTiDBMemQuotaHashJoin) + sv.StmtCtx.MemTracker = memory.NewTracker(stringutil.StringerStr("RootMemTracker"), variable.DefTiDBMemQuotaHashJoin) ctx := context.Background() hashJoinExec.Open(ctx) @@ -273,7 +274,7 @@ func (s *pkgTestSuite) TestParallelBuildHashTable4RadixJoin(c *C) { sv.L2CacheSize = 100 sv.EnableRadixJoin = true sv.MaxChunkSize = 100 - sv.StmtCtx.MemTracker = memory.NewTracker("RootMemTracker", variable.DefTiDBMemQuotaHashJoin) + sv.StmtCtx.MemTracker = memory.NewTracker(stringutil.StringerStr("RootMemTracker"), variable.DefTiDBMemQuotaHashJoin) ctx := context.Background() err := hashJoinExec.Open(ctx) diff --git a/executor/prepared.go b/executor/prepared.go index d169d2a4e57e8..ecbcaede4753e 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -32,6 +32,7 @@ import ( "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/sqlexec" + "github.com/pingcap/tidb/util/stringutil" "go.uber.org/zap" ) @@ -85,9 +86,11 @@ type PrepareExec struct { Fields []*ast.ResultField } +var prepareStmtLabel = stringutil.StringerStr("PrepareStmt") + // NewPrepareExec creates a new PrepareExec. func NewPrepareExec(ctx sessionctx.Context, is infoschema.InfoSchema, sqlTxt string) *PrepareExec { - base := newBaseExecutor(ctx, nil, "PrepareStmt") + base := newBaseExecutor(ctx, nil, prepareStmtLabel) base.initCap = chunk.ZeroCapacity return &PrepareExec{ baseExecutor: base, diff --git a/executor/sort.go b/executor/sort.go index 48bacd10249ad..8e2e221a6828a 100644 --- a/executor/sort.go +++ b/executor/sort.go @@ -16,6 +16,7 @@ package executor import ( "container/heap" "context" + "fmt" "sort" "time" @@ -25,8 +26,11 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/stringutil" ) +var rowChunksLabel fmt.Stringer = stringutil.StringerStr("rowChunks") + // SortExec represents sorting executor. type SortExec struct { baseExecutor @@ -104,7 +108,7 @@ func (e *SortExec) fetchRowChunks(ctx context.Context) error { fields := e.retTypes() e.rowChunks = chunk.NewList(fields, e.initCap, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) - e.rowChunks.GetMemTracker().SetLabel("rowChunks") + e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) for { chk := e.children[0].newFirstChunk() err := e.children[0].Next(ctx, chunk.NewRecordBatch(chk)) @@ -273,7 +277,7 @@ func (e *TopNExec) loadChunksUntilTotalLimit(ctx context.Context) error { e.chkHeap = &topNChunkHeap{e} e.rowChunks = chunk.NewList(e.retTypes(), e.initCap, e.maxChunkSize) e.rowChunks.GetMemTracker().AttachTo(e.memTracker) - e.rowChunks.GetMemTracker().SetLabel("rowChunks") + e.rowChunks.GetMemTracker().SetLabel(rowChunksLabel) for uint64(e.rowChunks.Len()) < e.totalLimit { srcChk := e.children[0].newFirstChunk() // adjust required rows by total limit @@ -351,7 +355,7 @@ func (e *TopNExec) doCompaction() error { newRowPtr := newRowChunks.AppendRow(e.rowChunks.GetRow(rowPtr)) newRowPtrs = append(newRowPtrs, newRowPtr) } - newRowChunks.GetMemTracker().SetLabel("rowChunks") + newRowChunks.GetMemTracker().SetLabel(rowChunksLabel) e.memTracker.ReplaceChild(e.rowChunks.GetMemTracker(), newRowChunks.GetMemTracker()) e.rowChunks = newRowChunks diff --git a/executor/table_reader.go b/executor/table_reader.go index 5f2785419b92e..a441e00755668 100644 --- a/executor/table_reader.go +++ b/executor/table_reader.go @@ -15,6 +15,7 @@ package executor import ( "context" + "fmt" "time" "github.com/opentracing/opentracing-go" @@ -28,6 +29,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -37,11 +39,11 @@ var _ Executor = &TableReaderExecutor{} // selectResultHook is used to hack distsql.SelectWithRuntimeStats safely for testing. type selectResultHook struct { selectResultFunc func(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []string) (distsql.SelectResult, error) + fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []fmt.Stringer) (distsql.SelectResult, error) } func (sr selectResultHook) SelectResult(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []string) (distsql.SelectResult, error) { + fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []fmt.Stringer) (distsql.SelectResult, error) { if sr.selectResultFunc == nil { return distsql.SelectWithRuntimeStats(ctx, sctx, kvReq, fieldTypes, fb, copPlanIDs) } @@ -145,13 +147,15 @@ func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.RecordBatch) func (e *TableReaderExecutor) Close() error { err := e.resultHandler.Close() if e.runtimeStats != nil { - copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.plans[0].ExplainID()) + copStats := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.plans[0].ExplainID().String()) copStats.SetRowNum(e.feedback.Actual()) } e.ctx.StoreQueryFeedback(e.feedback) return err } +var tableReaderDistSQLTrackerLabel fmt.Stringer = stringutil.StringerStr("TableReaderDistSQLTracker") + // buildResp first builds request and sends it to tikv using distsql.Select. It uses SelectResut returned by the callee // to fetch all results. func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Range) (distsql.SelectResult, error) { @@ -162,7 +166,7 @@ func (e *TableReaderExecutor) buildResp(ctx context.Context, ranges []*ranger.Ra SetKeepOrder(e.keepOrder). SetStreaming(e.streaming). SetFromSessionVars(e.ctx.GetSessionVars()). - SetMemTracker(e.ctx, "TableReaderDistSQLTracker"). + SetMemTracker(e.ctx, tableReaderDistSQLTrackerLabel). Build() if err != nil { return nil, err diff --git a/executor/table_readers_required_rows_test.go b/executor/table_readers_required_rows_test.go index e1fa48e019109..1565323faa035 100644 --- a/executor/table_readers_required_rows_test.go +++ b/executor/table_readers_required_rows_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -102,7 +103,7 @@ func mockDistsqlSelectCtxGet(ctx context.Context) (totalRows int, expectedRowsRe } func mockSelectResult(ctx context.Context, sctx sessionctx.Context, kvReq *kv.Request, - fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []string) (distsql.SelectResult, error) { + fieldTypes []*types.FieldType, fb *statistics.QueryFeedback, copPlanIDs []fmt.Stringer) (distsql.SelectResult, error) { totalRows, expectedRowsRet := mockDistsqlSelectCtxGet(ctx) return &requiredRowsSelectResult{ retTypes: fieldTypes, @@ -142,7 +143,7 @@ func buildMockBaseExec(sctx sessionctx.Context) baseExecutor { cols[i] = &expression.Column{Index: i, RetType: retTypes[i]} } schema := expression.NewSchema(cols...) - baseExec := newBaseExecutor(sctx, schema, "") + baseExec := newBaseExecutor(sctx, schema, stringutil.StringerStr("")) return baseExec } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 6beeeeba536fe..7eb85daddb217 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -554,15 +554,16 @@ func (e *Explain) explainPlanInRowFormat(p PhysicalPlan, taskType, indent string func (e *Explain) prepareOperatorInfo(p PhysicalPlan, taskType string, indent string, isLastChild bool) { operatorInfo := p.ExplainInfo() count := string(strconv.AppendFloat([]byte{}, p.statsInfo().RowCount, 'f', 2, 64)) - row := []string{e.prettyIdentifier(p.ExplainID(), indent, isLastChild), count, taskType, operatorInfo} + explainID := p.ExplainID().String() + row := []string{e.prettyIdentifier(explainID, indent, isLastChild), count, taskType, operatorInfo} if e.Analyze { runtimeStatsColl := e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl // There maybe some mock information for cop task to let runtimeStatsColl.Exists(p.ExplainID()) is true. // So check copTaskExecDetail first and print the real cop task information if it's not empty. - if runtimeStatsColl.ExistsCopStats(p.ExplainID()) { - row = append(row, runtimeStatsColl.GetCopStats(p.ExplainID()).String()) - } else if runtimeStatsColl.ExistsRootStats(p.ExplainID()) { - row = append(row, runtimeStatsColl.GetRootStats(p.ExplainID()).String()) + if runtimeStatsColl.ExistsCopStats(explainID) { + row = append(row, runtimeStatsColl.GetCopStats(explainID).String()) + } else if runtimeStatsColl.ExistsRootStats(explainID) { + row = append(row, runtimeStatsColl.GetRootStats(explainID).String()) } else { row = append(row, "time:0ns, loops:0, rows:0") } diff --git a/planner/core/explain.go b/planner/core/explain.go index ee26383570401..18fe8f8f998df 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -130,12 +130,12 @@ func (p *PhysicalTableScan) ExplainInfo() string { // ExplainInfo implements PhysicalPlan interface. func (p *PhysicalTableReader) ExplainInfo() string { - return "data:" + p.tablePlan.ExplainID() + return "data:" + p.tablePlan.ExplainID().String() } // ExplainInfo implements PhysicalPlan interface. func (p *PhysicalIndexReader) ExplainInfo() string { - return "index:" + p.indexPlan.ExplainID() + return "index:" + p.indexPlan.ExplainID().String() } // ExplainInfo implements PhysicalPlan interface. diff --git a/planner/core/plan.go b/planner/core/plan.go index 082c4b2832022..a141115e36140 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -16,12 +16,14 @@ package core import ( "fmt" "math" + "strconv" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -34,7 +36,7 @@ type Plan interface { // Get the ID. ID() int // Get the ID in explain statement - ExplainID() string + ExplainID() fmt.Stringer // replaceExprColumns replace all the column reference in the plan's expression node. replaceExprColumns(replace map[string]*expression.Column) @@ -258,8 +260,10 @@ func (p *basePlan) statsInfo() *property.StatsInfo { return p.stats } -func (p *basePlan) ExplainID() string { - return fmt.Sprintf("%s_%d", p.tp, p.id) +func (p *basePlan) ExplainID() fmt.Stringer { + return stringutil.MemoizeStr(func() string { + return p.tp + "_" + strconv.Itoa(p.id) + }) } // Schema implements Plan Schema interface. diff --git a/util/chunk/list.go b/util/chunk/list.go index f3dd06f613767..cf3ca1c5ab2dc 100644 --- a/util/chunk/list.go +++ b/util/chunk/list.go @@ -14,9 +14,12 @@ package chunk import ( + "fmt" + "github.com/pingcap/errors" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tidb/util/stringutil" ) // List holds a slice of chunks, use to append rows with max chunk size properly handled. @@ -39,13 +42,15 @@ type RowPtr struct { RowIdx uint32 } +var chunkListLabel fmt.Stringer = stringutil.StringerStr("chunk.List") + // NewList creates a new List with field types, init chunk size and max chunk size. func NewList(fieldTypes []*types.FieldType, initChunkSize, maxChunkSize int) *List { l := &List{ fieldTypes: fieldTypes, initChunkSize: initChunkSize, maxChunkSize: maxChunkSize, - memTracker: memory.NewTracker("chunk.List", -1), + memTracker: memory.NewTracker(chunkListLabel, -1), consumedIdx: -1, } return l diff --git a/util/memory/tracker.go b/util/memory/tracker.go index 54de745d4301f..10be4a9257505 100644 --- a/util/memory/tracker.go +++ b/util/memory/tracker.go @@ -43,10 +43,10 @@ type Tracker struct { children []*Tracker // The children memory trackers } - label string // Label of this "Tracker". - bytesConsumed int64 // Consumed bytes. - bytesLimit int64 // Negative value means no limit. - maxConsumed int64 // max number of bytes consumed during execution. + label fmt.Stringer // Label of this "Tracker". + bytesConsumed int64 // Consumed bytes. + bytesLimit int64 // Negative value means no limit. + maxConsumed int64 // max number of bytes consumed during execution. actionOnExceed ActionOnExceed parent *Tracker // The parent memory tracker. } @@ -54,7 +54,7 @@ type Tracker struct { // NewTracker creates a memory tracker. // 1. "label" is the label used in the usage string. // 2. "bytesLimit < 0" means no limit. -func NewTracker(label string, bytesLimit int64) *Tracker { +func NewTracker(label fmt.Stringer, bytesLimit int64) *Tracker { return &Tracker{ label: label, bytesLimit: bytesLimit, @@ -68,7 +68,7 @@ func (t *Tracker) SetActionOnExceed(a ActionOnExceed) { } // SetLabel sets the label of a Tracker. -func (t *Tracker) SetLabel(label string) { +func (t *Tracker) SetLabel(label fmt.Stringer) { t.label = label } diff --git a/util/memory/tracker_test.go b/util/memory/tracker_test.go index 8868571279500..11c6be4848c75 100644 --- a/util/memory/tracker_test.go +++ b/util/memory/tracker_test.go @@ -22,6 +22,7 @@ import ( "github.com/cznic/mathutil" . "github.com/pingcap/check" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/testleak" ) @@ -42,14 +43,14 @@ func (s *testSuite) SetUpTest(c *C) { testleak.BeforeTest() } func (s *testSuite) TearDownTest(c *C) { testleak.AfterTest(c)() } func (s *testSuite) TestSetLabel(c *C) { - tracker := NewTracker("old label", -1) - c.Assert(tracker.label, Equals, "old label") + tracker := NewTracker(stringutil.StringerStr("old label"), -1) + c.Assert(tracker.label.String(), Equals, "old label") c.Assert(tracker.BytesConsumed(), Equals, int64(0)) c.Assert(tracker.bytesLimit, Equals, int64(-1)) c.Assert(tracker.parent, IsNil) c.Assert(len(tracker.mu.children), Equals, 0) - tracker.SetLabel("new label") - c.Assert(tracker.label, Equals, "new label") + tracker.SetLabel(stringutil.StringerStr("new label")) + c.Assert(tracker.label.String(), Equals, "new label") c.Assert(tracker.BytesConsumed(), Equals, int64(0)) c.Assert(tracker.bytesLimit, Equals, int64(-1)) c.Assert(tracker.parent, IsNil) @@ -57,7 +58,7 @@ func (s *testSuite) TestSetLabel(c *C) { } func (s *testSuite) TestConsume(c *C) { - tracker := NewTracker("tracker", -1) + tracker := NewTracker(stringutil.StringerStr("tracker"), -1) c.Assert(tracker.BytesConsumed(), Equals, int64(0)) tracker.Consume(100) @@ -84,7 +85,7 @@ func (s *testSuite) TestConsume(c *C) { } func (s *testSuite) TestOOMAction(c *C) { - tracker := NewTracker("oom tracker", 100) + tracker := NewTracker(stringutil.StringerStr("oom tracker"), 100) action := &mockAction{} tracker.SetActionOnExceed(action) @@ -102,9 +103,9 @@ func (a *mockAction) Action(t *Tracker) { } func (s *testSuite) TestAttachTo(c *C) { - oldParent := NewTracker("old parent", -1) - newParent := NewTracker("new parent", -1) - child := NewTracker("child", -1) + oldParent := NewTracker(stringutil.StringerStr("old parent"), -1) + newParent := NewTracker(stringutil.StringerStr("new parent"), -1) + child := NewTracker(stringutil.StringerStr("child"), -1) child.Consume(100) child.AttachTo(oldParent) c.Assert(child.BytesConsumed(), Equals, int64(100)) @@ -124,11 +125,11 @@ func (s *testSuite) TestAttachTo(c *C) { } func (s *testSuite) TestReplaceChild(c *C) { - oldChild := NewTracker("old child", -1) + oldChild := NewTracker(stringutil.StringerStr("old child"), -1) oldChild.Consume(100) - newChild := NewTracker("new child", -1) + newChild := NewTracker(stringutil.StringerStr("new child"), -1) newChild.Consume(500) - parent := NewTracker("parent", -1) + parent := NewTracker(stringutil.StringerStr("parent"), -1) oldChild.AttachTo(parent) c.Assert(parent.BytesConsumed(), Equals, int64(100)) @@ -155,12 +156,12 @@ func (s *testSuite) TestReplaceChild(c *C) { } func (s *testSuite) TestToString(c *C) { - parent := NewTracker("parent", -1) + parent := NewTracker(stringutil.StringerStr("parent"), -1) - child1 := NewTracker("child 1", 1000) - child2 := NewTracker("child 2", -1) - child3 := NewTracker("child 3", -1) - child4 := NewTracker("child 4", -1) + child1 := NewTracker(stringutil.StringerStr("child 1"), 1000) + child2 := NewTracker(stringutil.StringerStr("child 2"), -1) + child3 := NewTracker(stringutil.StringerStr("child 3"), -1) + child4 := NewTracker(stringutil.StringerStr("child 4"), -1) child1.AttachTo(parent) child2.AttachTo(parent) @@ -193,10 +194,10 @@ func (s *testSuite) TestToString(c *C) { } func (s *testSuite) TestMaxConsumed(c *C) { - r := NewTracker("root", -1) - c1 := NewTracker("child 1", -1) - c2 := NewTracker("child 2", -1) - cc1 := NewTracker("child of child 1", -1) + r := NewTracker(stringutil.StringerStr("root"), -1) + c1 := NewTracker(stringutil.StringerStr("child 1"), -1) + c2 := NewTracker(stringutil.StringerStr("child 2"), -1) + cc1 := NewTracker(stringutil.StringerStr("child of child 1"), -1) c1.AttachTo(r) c2.AttachTo(r) @@ -220,9 +221,9 @@ func (s *testSuite) TestMaxConsumed(c *C) { } func BenchmarkConsume(b *testing.B) { - tracker := NewTracker("root", -1) + tracker := NewTracker(stringutil.StringerStr("root"), -1) b.RunParallel(func(pb *testing.PB) { - childTracker := NewTracker("child", -1) + childTracker := NewTracker(stringutil.StringerStr("child"), -1) childTracker.AttachTo(tracker) for pb.Next() { childTracker.Consume(256 << 20) diff --git a/util/stringutil/string_util.go b/util/stringutil/string_util.go index 75aafa0b536dc..6e69f61d2bd52 100644 --- a/util/stringutil/string_util.go +++ b/util/stringutil/string_util.go @@ -14,6 +14,7 @@ package stringutil import ( + "fmt" "strings" "unicode/utf8" @@ -250,3 +251,32 @@ func IsExactMatch(patTypes []byte) bool { func Copy(src string) string { return string(hack.Slice(src)) } + +// stringerFunc defines string func implement fmt.Stringer. +type stringerFunc func() string + +// String implements fmt.Stringer +func (l stringerFunc) String() string { + return l() +} + +// MemoizeStr returns memoized version of stringFunc. +func MemoizeStr(l func() string) fmt.Stringer { + var result string + return stringerFunc(func() string { + if result != "" { + return result + } + result = l() + return result + }) +} + +// StringerStr defines a alias to normal string. +// implement fmt.Stringer +type StringerStr string + +// String implements fmt.Stringer +func (i StringerStr) String() string { + return string(i) +} From e4aee765872ae67770863d5a64d9259a08bd6695 Mon Sep 17 00:00:00 2001 From: HuaiyuXu Date: Mon, 22 Apr 2019 15:12:16 +0800 Subject: [PATCH 04/21] executor, util: fix wrong behavior of group_concat(distinct) (#10108) --- executor/aggfuncs/func_group_concat.go | 15 ++++++++++----- executor/aggregate_test.go | 8 ++++++++ 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/executor/aggfuncs/func_group_concat.go b/executor/aggfuncs/func_group_concat.go index b269bb0f51b5e..636b2bb69006d 100644 --- a/executor/aggfuncs/func_group_concat.go +++ b/executor/aggfuncs/func_group_concat.go @@ -21,6 +21,8 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/set" ) @@ -143,7 +145,8 @@ func (e *groupConcat) GetTruncated() *int32 { type partialResult4GroupConcatDistinct struct { basePartialResult4GroupConcat - valSet set.StringSet + valSet set.StringSet + encodeBytesBuffer []byte } type groupConcatDistinct struct { @@ -167,6 +170,7 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI v, isNull := "", false for _, row := range rowsInGroup { p.valsBuf.Reset() + p.encodeBytesBuffer = p.encodeBytesBuffer[:0] for _, arg := range e.args { v, isNull, err = arg.EvalString(sctx, row) if err != nil { @@ -175,16 +179,17 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI if isNull { break } + p.encodeBytesBuffer = codec.EncodeBytes(p.encodeBytesBuffer, hack.Slice(v)) p.valsBuf.WriteString(v) } if isNull { continue } - joinedVals := p.valsBuf.String() - if p.valSet.Exist(joinedVals) { + joinedVal := string(p.encodeBytesBuffer) + if p.valSet.Exist(joinedVal) { continue } - p.valSet.Insert(joinedVals) + p.valSet.Insert(joinedVal) // write separator if p.buffer == nil { p.buffer = &bytes.Buffer{} @@ -192,7 +197,7 @@ func (e *groupConcatDistinct) UpdatePartialResult(sctx sessionctx.Context, rowsI p.buffer.WriteString(e.sep) } // write values - p.buffer.WriteString(joinedVals) + p.buffer.WriteString(p.valsBuf.String()) } if p.buffer != nil { return e.truncatePartialResultIfNeed(sctx, p.buffer) diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index f5ff52767b33a..f1c86de5e1616 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -694,3 +694,11 @@ func (s *testSuite1) TestAggJSON(c *C) { `"hello"`, )) } + +func (s *testSuite1) TestIssue10098(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec(`drop table if exists t;`) + tk.MustExec("create table t(a char(10), b char(10))") + tk.MustExec("insert into t values('1', '222'), ('12', '22')") + tk.MustQuery("select group_concat(distinct a, b) from t").Check(testkit.Rows("1222,1222")) +} From 5fcc7e2a61d3dbee5eb5dd3d43f1d3cf02294dd8 Mon Sep 17 00:00:00 2001 From: goroutine Date: Mon, 22 Apr 2019 15:32:15 +0800 Subject: [PATCH 05/21] executor: tiny clean up by removing unnecessary string creation (#10213) * executor: clean up by remove unnecessary string creation * check if id is nil --- executor/benchmark_test.go | 3 +-- executor/executor.go | 4 +++- executor/executor_pkg_test.go | 3 +-- executor/executor_required_rows_test.go | 15 +++++++-------- executor/pkg_test.go | 8 ++++---- executor/table_readers_required_rows_test.go | 3 +-- 6 files changed, 17 insertions(+), 19 deletions(-) diff --git a/executor/benchmark_test.go b/executor/benchmark_test.go index 60b0c5b244f97..e13e0eaf6f137 100644 --- a/executor/benchmark_test.go +++ b/executor/benchmark_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/stringutil" ) var ( @@ -128,7 +127,7 @@ func (mds *mockDataSource) Next(ctx context.Context, req *chunk.RecordBatch) err } func buildMockDataSource(opt mockDataSourceParameters) *mockDataSource { - baseExec := newBaseExecutor(opt.ctx, opt.schema, stringutil.StringerStr("")) + baseExec := newBaseExecutor(opt.ctx, opt.schema, nil) m := &mockDataSource{baseExec, opt, nil, nil, 0} types := m.retTypes() colData := make([][]interface{}, len(types)) diff --git a/executor/executor.go b/executor/executor.go index 89ae7cf7a5707..59676432f0927 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -141,7 +141,9 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id fmt.S maxChunkSize: ctx.GetSessionVars().MaxChunkSize, } if ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { - e.runtimeStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.id.String()) + if e.id != nil { + e.runtimeStats = e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(e.id.String()) + } } if schema != nil { cols := schema.Columns diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index 4ded2edf33048..c2c274455c7e2 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/ranger" - "github.com/pingcap/tidb/util/stringutil" ) var _ = Suite(&testExecSuite{}) @@ -93,7 +92,7 @@ func (s *testExecSuite) TestShowProcessList(c *C) { // Compose executor. e := &ShowExec{ - baseExecutor: newBaseExecutor(sctx, schema, stringutil.StringerStr("")), + baseExecutor: newBaseExecutor(sctx, schema, nil), Tp: ast.ShowProcessList, } diff --git a/executor/executor_required_rows_test.go b/executor/executor_required_rows_test.go index 1cf40fec57a2b..5cdd3cfe2898e 100644 --- a/executor/executor_required_rows_test.go +++ b/executor/executor_required_rows_test.go @@ -33,7 +33,6 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/memory" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/tidb/util/stringutil" ) type requiredRowsDataSource struct { @@ -63,7 +62,7 @@ func newRequiredRowsDataSource(ctx sessionctx.Context, totalRows int, expectedRo cols[i] = &expression.Column{Index: i, RetType: retTypes[i]} } schema := expression.NewSchema(cols...) - baseExec := newBaseExecutor(ctx, schema, stringutil.StringerStr("")) + baseExec := newBaseExecutor(ctx, schema, nil) return &requiredRowsDataSource{baseExec, totalRows, 0, ctx, expectedRowsRet, 0, defaultGenerator} } @@ -191,7 +190,7 @@ func (s *testExecSuite) TestLimitRequiredRows(c *C) { func buildLimitExec(ctx sessionctx.Context, src Executor, offset, count int) Executor { n := mathutil.Min(count, ctx.GetSessionVars().MaxChunkSize) - base := newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src) + base := newBaseExecutor(ctx, src.Schema(), nil, src) base.initCap = n limitExec := &LimitExec{ baseExecutor: base, @@ -206,7 +205,7 @@ func defaultCtx() sessionctx.Context { ctx.GetSessionVars().InitChunkSize = variable.DefInitChunkSize ctx.GetSessionVars().MaxChunkSize = variable.DefMaxChunkSize ctx.GetSessionVars().MemQuotaSort = variable.DefTiDBMemQuotaSort - ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(stringutil.StringerStr(""), ctx.GetSessionVars().MemQuotaQuery) + ctx.GetSessionVars().StmtCtx.MemTracker = memory.NewTracker(nil, ctx.GetSessionVars().MemQuotaQuery) ctx.GetSessionVars().SnapshotTS = uint64(1) return ctx } @@ -274,7 +273,7 @@ func (s *testExecSuite) TestSortRequiredRows(c *C) { func buildSortExec(sctx sessionctx.Context, byItems []*plannercore.ByItems, src Executor) Executor { sortExec := SortExec{ - baseExecutor: newBaseExecutor(sctx, src.Schema(), stringutil.StringerStr(""), src), + baseExecutor: newBaseExecutor(sctx, src.Schema(), nil, src), ByItems: byItems, schema: src.Schema(), } @@ -381,7 +380,7 @@ func (s *testExecSuite) TestTopNRequiredRows(c *C) { func buildTopNExec(ctx sessionctx.Context, offset, count int, byItems []*plannercore.ByItems, src Executor) Executor { sortExec := SortExec{ - baseExecutor: newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src), + baseExecutor: newBaseExecutor(ctx, src.Schema(), nil, src), ByItems: byItems, schema: src.Schema(), } @@ -474,7 +473,7 @@ func (s *testExecSuite) TestSelectionRequiredRows(c *C) { func buildSelectionExec(ctx sessionctx.Context, filters []expression.Expression, src Executor) Executor { return &SelectionExec{ - baseExecutor: newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src), + baseExecutor: newBaseExecutor(ctx, src.Schema(), nil, src), filters: filters, } } @@ -592,7 +591,7 @@ func (s *testExecSuite) TestProjectionParallelRequiredRows(c *C) { func buildProjectionExec(ctx sessionctx.Context, exprs []expression.Expression, src Executor, numWorkers int) Executor { return &ProjectionExec{ - baseExecutor: newBaseExecutor(ctx, src.Schema(), stringutil.StringerStr(""), src), + baseExecutor: newBaseExecutor(ctx, src.Schema(), nil, src), numWorkers: int64(numWorkers), evaluatorSuit: expression.NewEvaluatorSuite(exprs, false), } diff --git a/executor/pkg_test.go b/executor/pkg_test.go index 3893cfaf188c6..74a478aadce48 100644 --- a/executor/pkg_test.go +++ b/executor/pkg_test.go @@ -64,7 +64,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { con := &expression.Constant{Value: types.NewDatum(6), RetType: types.NewFieldType(mysql.TypeLong)} outerSchema := expression.NewSchema(col0) outerExec := &MockExec{ - baseExecutor: newBaseExecutor(sctx, outerSchema, stringutil.StringerStr("")), + baseExecutor: newBaseExecutor(sctx, outerSchema, nil), Rows: []chunk.MutRow{ chunk.MutRowFromDatums(types.MakeDatums(1)), chunk.MutRowFromDatums(types.MakeDatums(2)), @@ -75,7 +75,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { }} innerSchema := expression.NewSchema(col1) innerExec := &MockExec{ - baseExecutor: newBaseExecutor(sctx, innerSchema, stringutil.StringerStr("")), + baseExecutor: newBaseExecutor(sctx, innerSchema, nil), Rows: []chunk.MutRow{ chunk.MutRowFromDatums(types.MakeDatums(1)), chunk.MutRowFromDatums(types.MakeDatums(2)), @@ -91,7 +91,7 @@ func (s *pkgTestSuite) TestNestedLoopApply(c *C) { make([]types.Datum, innerExec.Schema().Len()), []expression.Expression{otherFilter}, outerExec.retTypes(), innerExec.retTypes()) joinSchema := expression.NewSchema(col0, col1) join := &NestedLoopApplyExec{ - baseExecutor: newBaseExecutor(sctx, joinSchema, stringutil.StringerStr("")), + baseExecutor: newBaseExecutor(sctx, joinSchema, nil), outerExec: outerExec, innerExec: innerExec, outerFilter: []expression.Expression{outerFilter}, @@ -122,7 +122,7 @@ func prepareOneColChildExec(sctx sessionctx.Context, rowCount int) Executor { col0 := &expression.Column{Index: 0, RetType: types.NewFieldType(mysql.TypeLong)} schema := expression.NewSchema(col0) exec := &MockExec{ - baseExecutor: newBaseExecutor(sctx, schema, stringutil.StringerStr("")), + baseExecutor: newBaseExecutor(sctx, schema, nil), Rows: make([]chunk.MutRow, rowCount)} for i := 0; i < len(exec.Rows); i++ { exec.Rows[i] = chunk.MutRowFromDatums(types.MakeDatums(i % 10)) diff --git a/executor/table_readers_required_rows_test.go b/executor/table_readers_required_rows_test.go index 1565323faa035..21819329d6a82 100644 --- a/executor/table_readers_required_rows_test.go +++ b/executor/table_readers_required_rows_test.go @@ -30,7 +30,6 @@ import ( "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" - "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tipb/go-tipb" ) @@ -143,7 +142,7 @@ func buildMockBaseExec(sctx sessionctx.Context) baseExecutor { cols[i] = &expression.Column{Index: i, RetType: retTypes[i]} } schema := expression.NewSchema(cols...) - baseExec := newBaseExecutor(sctx, schema, stringutil.StringerStr("")) + baseExec := newBaseExecutor(sctx, schema, nil) return baseExec } From 2c35315a70267051e9a0fee9e210780a9acc33e2 Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 22 Apr 2019 16:04:40 +0800 Subject: [PATCH 06/21] remove grpc_client monitor (#10216) --- domain/domain.go | 3 --- store/tikv/kv.go | 8 +------- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 3e08c8e3ac733..45b8a1d5cec5a 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -22,7 +22,6 @@ import ( "unsafe" "github.com/coreos/etcd/clientv3" - grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/ngaut/pools" "github.com/ngaut/sync2" "github.com/pingcap/errors" @@ -592,8 +591,6 @@ func (do *Domain) Init(ddlLease time.Duration, sysFactory func(*Domain) (pools.R AutoSyncInterval: 30 * time.Second, DialTimeout: 5 * time.Second, DialOptions: []grpc.DialOption{ - grpc.WithUnaryInterceptor(grpc_prometheus.UnaryClientInterceptor), - grpc.WithStreamInterceptor(grpc_prometheus.StreamClientInterceptor), grpc.WithBackoffMaxDelay(time.Second * 3), grpc.WithKeepaliveParams(keepalive.ClientParameters{ Time: time.Duration(cfg.TiKVClient.GrpcKeepAliveTime) * time.Second, diff --git a/store/tikv/kv.go b/store/tikv/kv.go index fd942936fe727..cf6a3e920d7f5 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -24,7 +24,6 @@ import ( "time" "github.com/coreos/etcd/clientv3" - grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/pingcap/errors" pd "github.com/pingcap/pd/client" "github.com/pingcap/tidb/config" @@ -36,7 +35,6 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/util/logutil" "go.uber.org/zap" - "google.golang.org/grpc" ) type storeCache struct { @@ -55,11 +53,7 @@ func createEtcdKV(addrs []string, tlsConfig *tls.Config) (*clientv3.Client, erro Endpoints: addrs, AutoSyncInterval: 30 * time.Second, DialTimeout: 5 * time.Second, - DialOptions: []grpc.DialOption{ - grpc.WithUnaryInterceptor(grpc_prometheus.UnaryClientInterceptor), - grpc.WithStreamInterceptor(grpc_prometheus.StreamClientInterceptor), - }, - TLS: tlsConfig, + TLS: tlsConfig, }) if err != nil { return nil, errors.Trace(err) From b864fac504f9f059a39a7d477e923440f6d88a55 Mon Sep 17 00:00:00 2001 From: Keyi Xie Date: Mon, 22 Apr 2019 16:25:27 +0800 Subject: [PATCH 07/21] rm travis.yml && speed it up (#10208) --- .travis.yml | 25 ------------------------- Makefile | 2 +- 2 files changed, 1 insertion(+), 26 deletions(-) delete mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 669153cfe14d9..0000000000000 --- a/.travis.yml +++ /dev/null @@ -1,25 +0,0 @@ -language: go - -go_import_path: github.com/pingcap/tidb - -go: - - "1.12" - -env: - - TRAVIS_COVERAGE=1 - -# Run coverage tests. -matrix: - fast_finish: true - allow_failures: - - go: "1.12" - env: TRAVIS_COVERAGE=1 - -before_install: - # create /logs/unit-test for unit test. - - sudo mkdir /logs - - sudo touch /logs/unit-test - # See https://github.com/golang/go/issues/12933 - - bash gitcookie.sh -script: - - make gotest upload-coverage diff --git a/Makefile b/Makefile index d69a0cb39aae8..aa2ff28a6ee13 100644 --- a/Makefile +++ b/Makefile @@ -131,7 +131,7 @@ ifeq ("$(TRAVIS_COVERAGE)", "1") $(OVERALLS) -project=github.com/pingcap/tidb \ -covermode=count \ -ignore='.git,vendor,cmd,docs,LICENSES' \ - -concurrency=2 \ + -concurrency=4 \ -- -coverpkg=./... \ || { $(FAILPOINT_DISABLE); exit 1; } else From 085746c25405a983de5375a0de7308f60ccc852e Mon Sep 17 00:00:00 2001 From: HuaiyuXu Date: Mon, 22 Apr 2019 19:31:08 +0800 Subject: [PATCH 08/21] expression: check corCol in DisableParseJSONFlag4Expr (#10224) --- expression/util.go | 10 +++++++--- expression/util_test.go | 21 +++++++++++++++++++++ 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/expression/util.go b/expression/util.go index a7f2e0a2b5554..d11f3b45a54b8 100644 --- a/expression/util.go +++ b/expression/util.go @@ -572,13 +572,17 @@ func GetParamExpression(ctx sessionctx.Context, v *driver.ParamMarkerExpr) (Expr // DisableParseJSONFlag4Expr disables ParseToJSONFlag for `expr` except Column. // We should not *PARSE* a string as JSON under some scenarios. ParseToJSONFlag -// is 0 for JSON column yet, so we can skip it. Moreover, Column.RetType refers -// to the infoschema, if we modify it, data race may happen if another goroutine -// read from the infoschema at the same time. +// is 0 for JSON column yet(as well as JSON correlated column), so we can skip +// it. Moreover, Column.RetType refers to the infoschema, if we modify it, data +// race may happen if another goroutine read from the infoschema at the same +// time. func DisableParseJSONFlag4Expr(expr Expression) { if _, isColumn := expr.(*Column); isColumn { return } + if _, isCorCol := expr.(*CorrelatedColumn); isCorCol { + return + } expr.GetType().Flag &= ^mysql.ParseToJSONFlag } diff --git a/expression/util_test.go b/expression/util_test.go index f1e03ec971226..4aaef213bb483 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -90,6 +90,27 @@ func isLogicOrFunction(e Expression) bool { return false } +func (s *testUtilSuite) TestDisableParseJSONFlag4Expr(c *check.C) { + var expr Expression + expr = &Column{RetType: newIntFieldType()} + ft := expr.GetType() + ft.Flag |= mysql.ParseToJSONFlag + DisableParseJSONFlag4Expr(expr) + c.Assert(mysql.HasParseToJSONFlag(ft.Flag), check.IsTrue) + + expr = &CorrelatedColumn{Column: Column{RetType: newIntFieldType()}} + ft = expr.GetType() + ft.Flag |= mysql.ParseToJSONFlag + DisableParseJSONFlag4Expr(expr) + c.Assert(mysql.HasParseToJSONFlag(ft.Flag), check.IsTrue) + + expr = &ScalarFunction{RetType: newIntFieldType()} + ft = expr.GetType() + ft.Flag |= mysql.ParseToJSONFlag + DisableParseJSONFlag4Expr(expr) + c.Assert(mysql.HasParseToJSONFlag(ft.Flag), check.IsFalse) +} + func BenchmarkExtractColumns(b *testing.B) { conditions := []Expression{ newFunction(ast.EQ, newColumn(0), newColumn(1)), From 03aa5e2e51e87236c77e01fb6872cff734d7b226 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Mon, 22 Apr 2019 19:51:22 +0800 Subject: [PATCH 09/21] *: remove go.mod replace check in circle CI (#10228) The jenkins CI already run this check, it's redundant for circle CI to check again --- circle.yml | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/circle.yml b/circle.yml index 42c7eff03c96d..a090f35bb3654 100644 --- a/circle.yml +++ b/circle.yml @@ -9,7 +9,4 @@ jobs: - checkout - run: name: "Build & Test" - command: make dev - - run: - name: "Check go mod replace is removed" - command: ./tools/check/check_parser_replace.sh + command: make dev \ No newline at end of file From 6ba69f2aadb1ed53fa935bb43c1751f29e2bc9b7 Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 22 Apr 2019 20:51:46 +0800 Subject: [PATCH 10/21] plan, executor: pre with labels for metrics in conn/session (#10232) --- server/conn.go | 160 +++++++++++++++++++++++++++++++++++++++------ session/session.go | 103 +++++++++++++++++++++++++---- 2 files changed, 229 insertions(+), 34 deletions(-) diff --git a/server/conn.go b/server/conn.go index dc1581e1f30c5..dd79ae5ca24e5 100644 --- a/server/conn.go +++ b/server/conn.go @@ -75,6 +75,47 @@ const ( connStatusWaitShutdown // Notified by server to close. ) +var ( + queryTotalCounterComSleepOK = metrics.QueryTotalCounter.WithLabelValues("Sleep", "OK") + queryTotalCounterComSleepError = metrics.QueryTotalCounter.WithLabelValues("Sleep", "Error") + queryTotalCounterComQuitOK = metrics.QueryTotalCounter.WithLabelValues("Quit", "OK") + queryTotalCounterComQuitError = metrics.QueryTotalCounter.WithLabelValues("Quit", "Error") + queryTotalCounterComInitDBOK = metrics.QueryTotalCounter.WithLabelValues("InitDB", "OK") + queryTotalCounterComInitDBError = metrics.QueryTotalCounter.WithLabelValues("InitDB", "Error") + queryTotalCounterComQueryOK = metrics.QueryTotalCounter.WithLabelValues("Query", "OK") + queryTotalCounterComQueryError = metrics.QueryTotalCounter.WithLabelValues("Query", "Error") + queryTotalCounterComPingOK = metrics.QueryTotalCounter.WithLabelValues("Ping", "OK") + queryTotalCounterComPingError = metrics.QueryTotalCounter.WithLabelValues("Ping", "Error") + queryTotalCounterComFieldListOK = metrics.QueryTotalCounter.WithLabelValues("FieldList", "OK") + queryTotalCounterComFieldListError = metrics.QueryTotalCounter.WithLabelValues("FieldList", "Error") + queryTotalCounterComPrepareOK = metrics.QueryTotalCounter.WithLabelValues("StmtPrepare", "OK") + queryTotalCounterComPrepareError = metrics.QueryTotalCounter.WithLabelValues("StmtPrepare", "Error") + queryTotalCounterComExecuteOK = metrics.QueryTotalCounter.WithLabelValues("StmtExecute", "OK") + queryTotalCounterComExecuteError = metrics.QueryTotalCounter.WithLabelValues("StmtExecute", "Error") + queryTotalCounterComFetchOK = metrics.QueryTotalCounter.WithLabelValues("StmtFetch", "OK") + queryTotalCounterComFetchError = metrics.QueryTotalCounter.WithLabelValues("StmtFetch", "Error") + queryTotalCounterComCloseOK = metrics.QueryTotalCounter.WithLabelValues("StmtClose", "OK") + queryTotalCounterComCloseError = metrics.QueryTotalCounter.WithLabelValues("StmtClose", "Error") + queryTotalCounterComSendLongDataOK = metrics.QueryTotalCounter.WithLabelValues("StmtSendLongData", "OK") + queryTotalCounterComSendLongDataError = metrics.QueryTotalCounter.WithLabelValues("StmtSendLongData", "Error") + queryTotalCounterComResetOK = metrics.QueryTotalCounter.WithLabelValues("StmtReset", "OK") + queryTotalCounterComResetError = metrics.QueryTotalCounter.WithLabelValues("StmtReset", "Error") + queryTotalCounterComSetOptionOK = metrics.QueryTotalCounter.WithLabelValues("SetOption", "OK") + queryTotalCounterComSetOptionError = metrics.QueryTotalCounter.WithLabelValues("SetOption", "Error") + + queryDurationHistogramUse = metrics.QueryDurationHistogram.WithLabelValues("Use") + queryDurationHistogramShow = metrics.QueryDurationHistogram.WithLabelValues("Show") + queryDurationHistogramBegin = metrics.QueryDurationHistogram.WithLabelValues("Begin") + queryDurationHistogramCommit = metrics.QueryDurationHistogram.WithLabelValues("Commit") + queryDurationHistogramRollback = metrics.QueryDurationHistogram.WithLabelValues("Rollback") + queryDurationHistogramInsert = metrics.QueryDurationHistogram.WithLabelValues("Insert") + queryDurationHistogramReplace = metrics.QueryDurationHistogram.WithLabelValues("Replace") + queryDurationHistogramDelete = metrics.QueryDurationHistogram.WithLabelValues("Delete") + queryDurationHistogramUpdate = metrics.QueryDurationHistogram.WithLabelValues("Update") + queryDurationHistogramSelect = metrics.QueryDurationHistogram.WithLabelValues("Select") + queryDurationHistogramGeneral = metrics.QueryDurationHistogram.WithLabelValues(metrics.LblGeneral) +) + // newClientConn creates a *clientConn object. func newClientConn(s *Server) *clientConn { return &clientConn{ @@ -668,53 +709,130 @@ func errStrForLog(err error) string { } func (cc *clientConn) addMetrics(cmd byte, startTime time.Time, err error) { - var label string switch cmd { case mysql.ComSleep: - label = "Sleep" + if err != nil { + queryTotalCounterComSleepError.Inc() + } else { + queryTotalCounterComSleepOK.Inc() + } case mysql.ComQuit: - label = "Quit" + if err != nil { + queryTotalCounterComQuitError.Inc() + } else { + queryTotalCounterComQuitOK.Inc() + } case mysql.ComQuery: if cc.ctx.Value(sessionctx.LastExecuteDDL) != nil { // Don't take DDL execute time into account. // It's already recorded by other metrics in ddl package. return } - label = "Query" + if err != nil { + queryTotalCounterComQueryError.Inc() + } else { + queryTotalCounterComQueryOK.Inc() + } case mysql.ComPing: - label = "Ping" + if err != nil { + queryTotalCounterComPingError.Inc() + } else { + queryTotalCounterComPingOK.Inc() + } case mysql.ComInitDB: - label = "InitDB" + if err != nil { + queryTotalCounterComInitDBError.Inc() + } else { + queryTotalCounterComInitDBOK.Inc() + } case mysql.ComFieldList: - label = "FieldList" + if err != nil { + queryTotalCounterComFieldListError.Inc() + } else { + queryTotalCounterComFieldListOK.Inc() + } case mysql.ComStmtPrepare: - label = "StmtPrepare" + if err != nil { + queryTotalCounterComPrepareError.Inc() + } else { + queryTotalCounterComPrepareOK.Inc() + } case mysql.ComStmtExecute: - label = "StmtExecute" + if err != nil { + queryTotalCounterComExecuteError.Inc() + } else { + queryTotalCounterComExecuteOK.Inc() + } case mysql.ComStmtFetch: - label = "StmtFetch" + if err != nil { + queryTotalCounterComFetchError.Inc() + } else { + queryTotalCounterComFetchOK.Inc() + } case mysql.ComStmtClose: - label = "StmtClose" + if err != nil { + queryTotalCounterComCloseError.Inc() + } else { + queryTotalCounterComCloseOK.Inc() + } case mysql.ComStmtSendLongData: - label = "StmtSendLongData" + if err != nil { + queryTotalCounterComSendLongDataError.Inc() + } else { + queryTotalCounterComSendLongDataOK.Inc() + } case mysql.ComStmtReset: - label = "StmtReset" + if err != nil { + queryTotalCounterComResetError.Inc() + } else { + queryTotalCounterComResetOK.Inc() + } case mysql.ComSetOption: - label = "SetOption" + if err != nil { + queryTotalCounterComSetOptionError.Inc() + } else { + queryTotalCounterComSetOptionOK.Inc() + } default: - label = strconv.Itoa(int(cmd)) - } - if err != nil { - metrics.QueryTotalCounter.WithLabelValues(label, "Error").Inc() - } else { - metrics.QueryTotalCounter.WithLabelValues(label, "OK").Inc() + label := strconv.Itoa(int(cmd)) + if err != nil { + metrics.QueryTotalCounter.WithLabelValues(label, "ERROR").Inc() + } else { + metrics.QueryTotalCounter.WithLabelValues(label, "OK").Inc() + } } stmtType := cc.ctx.GetSessionVars().StmtCtx.StmtType sqlType := metrics.LblGeneral if stmtType != "" { sqlType = stmtType } - metrics.QueryDurationHistogram.WithLabelValues(sqlType).Observe(time.Since(startTime).Seconds()) + + switch sqlType { + case "Use": + queryDurationHistogramUse.Observe(time.Since(startTime).Seconds()) + case "Show": + queryDurationHistogramShow.Observe(time.Since(startTime).Seconds()) + case "Begin": + queryDurationHistogramBegin.Observe(time.Since(startTime).Seconds()) + case "Commit": + queryDurationHistogramCommit.Observe(time.Since(startTime).Seconds()) + case "Rollback": + queryDurationHistogramRollback.Observe(time.Since(startTime).Seconds()) + case "Insert": + queryDurationHistogramInsert.Observe(time.Since(startTime).Seconds()) + case "Replace": + queryDurationHistogramReplace.Observe(time.Since(startTime).Seconds()) + case "Delete": + queryDurationHistogramDelete.Observe(time.Since(startTime).Seconds()) + case "Update": + queryDurationHistogramUpdate.Observe(time.Since(startTime).Seconds()) + case "Select": + queryDurationHistogramSelect.Observe(time.Since(startTime).Seconds()) + case metrics.LblGeneral: + queryDurationHistogramGeneral.Observe(time.Since(startTime).Seconds()) + default: + metrics.QueryDurationHistogram.WithLabelValues(sqlType).Observe(time.Since(startTime).Seconds()) + } } // dispatch handles client request based on command which is the first byte of the data. diff --git a/session/session.go b/session/session.go index fe388b9cc728d..f8da577411ed9 100644 --- a/session/session.go +++ b/session/session.go @@ -69,6 +69,33 @@ import ( "go.uber.org/zap" ) +var ( + statementPerTransactionInternalOK = metrics.StatementPerTransaction.WithLabelValues(metrics.LblInternal, "ok") + statementPerTransactionInternalError = metrics.StatementPerTransaction.WithLabelValues(metrics.LblInternal, "error") + statementPerTransactionGeneralOK = metrics.StatementPerTransaction.WithLabelValues(metrics.LblGeneral, "ok") + statementPerTransactionGeneralError = metrics.StatementPerTransaction.WithLabelValues(metrics.LblGeneral, "error") + transactionDurationInternalOK = metrics.TransactionDuration.WithLabelValues(metrics.LblInternal, "ok") + transactionDurationInternalError = metrics.TransactionDuration.WithLabelValues(metrics.LblInternal, "error") + transactionDurationGeneralOK = metrics.TransactionDuration.WithLabelValues(metrics.LblGeneral, "ok") + transactionDurationGeneralError = metrics.TransactionDuration.WithLabelValues(metrics.LblGeneral, "error") + + transactionCounterInternalOK = metrics.TransactionCounter.WithLabelValues(metrics.LblInternal, metrics.LblOK) + transactionCounterInternalErr = metrics.TransactionCounter.WithLabelValues(metrics.LblInternal, metrics.LblError) + transactionCounterGeneralOK = metrics.TransactionCounter.WithLabelValues(metrics.LblGeneral, metrics.LblOK) + transactionCounterGeneralErr = metrics.TransactionCounter.WithLabelValues(metrics.LblGeneral, metrics.LblError) + + transactionRollbackCounterInternal = metrics.TransactionCounter.WithLabelValues(metrics.LblInternal, metrics.LblRollback) + transactionRollbackCounterGeneral = metrics.TransactionCounter.WithLabelValues(metrics.LblGeneral, metrics.LblRollback) + + sessionExecuteRunDurationInternal = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblInternal) + sessionExecuteRunDurationGeneral = metrics.SessionExecuteRunDuration.WithLabelValues(metrics.LblGeneral) + + sessionExecuteCompileDurationInternal = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblInternal) + sessionExecuteCompileDurationGeneral = metrics.SessionExecuteCompileDuration.WithLabelValues(metrics.LblGeneral) + sessionExecuteParseDurationInternal = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblInternal) + sessionExecuteParseDurationGeneral = metrics.SessionExecuteParseDuration.WithLabelValues(metrics.LblGeneral) +) + // Session context type Session interface { sessionctx.Context @@ -396,11 +423,9 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { err = s.retry(ctx, uint(maxRetryCount)) } } - label := s.getSQLLabel() counter := s.sessionVars.TxnCtx.StatementCount duration := time.Since(s.GetSessionVars().TxnCtx.CreateTime).Seconds() - metrics.StatementPerTransaction.WithLabelValues(label, metrics.RetLabel(err)).Observe(float64(counter)) - metrics.TransactionDuration.WithLabelValues(label, metrics.RetLabel(err)).Observe(float64(duration)) + s.recordOnTransactionExecution(err, counter, duration) s.cleanRetryInfo() if isoLevelOneShot := &s.sessionVars.TxnIsolationLevelOneShot; isoLevelOneShot.State != 0 { @@ -446,12 +471,8 @@ func (s *session) CommitTxn(ctx context.Context) error { s.sessionVars.StmtCtx.MergeExecDetails(nil, commitDetail) } stmt.LogSlowQuery(s.sessionVars.TxnCtx.StartTS, err == nil) - label := metrics.LblOK - if err != nil { - label = metrics.LblError - } s.sessionVars.TxnCtx.Cleanup() - metrics.TransactionCounter.WithLabelValues(s.getSQLLabel(), label).Inc() + s.recordTransactionCounter(err) return err } @@ -463,7 +484,11 @@ func (s *session) RollbackTxn(ctx context.Context) { if s.txn.Valid() { terror.Log(s.txn.Rollback()) - metrics.TransactionCounter.WithLabelValues(s.getSQLLabel(), metrics.LblRollback).Inc() + if s.isInternal() { + transactionRollbackCounterInternal.Inc() + } else { + transactionRollbackCounterGeneral.Inc() + } } s.cleanRetryInfo() s.txn.changeToInvalid() @@ -515,6 +540,10 @@ func (s *session) getSQLLabel() string { return metrics.LblGeneral } +func (s *session) isInternal() bool { + return s.sessionVars.InRestrictedSQL +} + func (s *session) isRetryableError(err error) bool { if SchemaChangedWithoutRetry { return kv.IsRetryableError(err) @@ -917,7 +946,11 @@ func (s *session) executeStatement(ctx context.Context, connID uint64, stmtNode } return nil, err } - metrics.SessionExecuteRunDuration.WithLabelValues(s.getSQLLabel()).Observe(time.Since(startTime).Seconds()) + if s.isInternal() { + sessionExecuteRunDurationInternal.Observe(time.Since(startTime).Seconds()) + } else { + sessionExecuteRunDurationGeneral.Observe(time.Since(startTime).Seconds()) + } if recordSet != nil { recordSets = append(recordSets, recordSet) @@ -956,8 +989,12 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec zap.String("sql", sql)) return nil, util.SyntaxError(err) } - label := s.getSQLLabel() - metrics.SessionExecuteParseDuration.WithLabelValues(label).Observe(time.Since(startTS).Seconds()) + isInternal := s.isInternal() + if isInternal { + sessionExecuteParseDurationInternal.Observe(time.Since(startTS).Seconds()) + } else { + sessionExecuteParseDurationGeneral.Observe(time.Since(startTS).Seconds()) + } compiler := executor.Compiler{Ctx: s} for _, stmtNode := range stmtNodes { @@ -977,7 +1014,11 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec zap.String("sql", sql)) return nil, err } - metrics.SessionExecuteCompileDuration.WithLabelValues(label).Observe(time.Since(startTS).Seconds()) + if isInternal { + sessionExecuteCompileDurationInternal.Observe(time.Since(startTS).Seconds()) + } else { + sessionExecuteCompileDurationGeneral.Observe(time.Since(startTS).Seconds()) + } s.currentPlan = stmt.Plan // Step3: Execute the physical plan. @@ -1710,3 +1751,39 @@ func logQuery(query string, vars *variable.SessionVars) { zap.String("sql", query+vars.GetExecuteArgumentsInfo())) } } + +func (s *session) recordOnTransactionExecution(err error, counter int, duration float64) { + if s.isInternal() { + if err != nil { + statementPerTransactionInternalError.Observe(float64(counter)) + transactionDurationInternalError.Observe(duration) + } else { + statementPerTransactionInternalOK.Observe(float64(counter)) + transactionDurationInternalOK.Observe(duration) + } + } else { + if err != nil { + statementPerTransactionGeneralError.Observe(float64(counter)) + transactionDurationGeneralError.Observe(duration) + } else { + statementPerTransactionGeneralOK.Observe(float64(counter)) + transactionDurationGeneralOK.Observe(duration) + } + } +} + +func (s *session) recordTransactionCounter(err error) { + if s.isInternal() { + if err != nil { + transactionCounterInternalErr.Inc() + } else { + transactionCounterInternalOK.Inc() + } + } else { + if err != nil { + transactionCounterGeneralErr.Inc() + } else { + transactionCounterGeneralOK.Inc() + } + } +} From e59539505aa9d25e08a0716ec212ed95fdabeeaa Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 22 Apr 2019 21:50:59 +0800 Subject: [PATCH 11/21] plan, executor: pre with labels for metrics in plan/executor package (#10231) --- executor/builder.go | 30 ++++++++++++++++------- executor/compiler.go | 38 ++++++++++++++++++++++++++++- executor/seqtest/prepared_test.go | 3 +++ metrics/server.go | 5 ++++ planner/core/common_plans.go | 8 +++++- planner/core/point_get_plan_test.go | 1 + planner/core/prepare_test.go | 1 + 7 files changed, 75 insertions(+), 11 deletions(-) diff --git a/executor/builder.go b/executor/builder.go index 53eff17bb27ce..a003f9c57f899 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -52,6 +52,18 @@ import ( "github.com/pingcap/tipb/go-tipb" ) +var ( + executorCounterMergeJoinExec = metrics.ExecutorCounter.WithLabelValues("MergeJoinExec") + executorCountHashJoinExec = metrics.ExecutorCounter.WithLabelValues("HashJoinExec") + executorCounterHashAggExec = metrics.ExecutorCounter.WithLabelValues("HashAggExec") + executorStreamAggExec = metrics.ExecutorCounter.WithLabelValues("StreamAggExec") + executorCounterSortExec = metrics.ExecutorCounter.WithLabelValues("SortExec") + executorCounterTopNExec = metrics.ExecutorCounter.WithLabelValues("TopNExec") + executorCounterNestedLoopApplyExec = metrics.ExecutorCounter.WithLabelValues("NestedLoopApplyExec") + executorCounterIndexLookUpJoin = metrics.ExecutorCounter.WithLabelValues("IndexLookUpJoin") + executorCounterIndexLookUpExecutor = metrics.ExecutorCounter.WithLabelValues("IndexLookUpExecutor") +) + // executorBuilder builds an Executor from a Plan. // The InfoSchema must not change during execution. type executorBuilder struct { @@ -882,7 +894,7 @@ func (b *executorBuilder) buildMergeJoin(v *plannercore.PhysicalMergeJoin) Execu return nil } - metrics.ExecutorCounter.WithLabelValues("MergeJoinExec").Inc() + executorCounterMergeJoinExec.Inc() return e } @@ -948,7 +960,7 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo e.joiners[i] = newJoiner(b.ctx, v.JoinType, v.InnerChildIdx == 0, defaultValues, v.OtherConditions, lhsTypes, rhsTypes) } - metrics.ExecutorCounter.WithLabelValues("HashJoinExec").Inc() + executorCountHashJoinExec.Inc() if e.ctx.GetSessionVars().EnableRadixJoin { return &RadixHashJoinExec{HashJoinExec: e} } @@ -1031,7 +1043,7 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor } } - metrics.ExecutorCounter.WithLabelValues("HashAggExec").Inc() + executorCounterHashAggExec.Inc() return e } @@ -1059,7 +1071,7 @@ func (b *executorBuilder) buildStreamAgg(v *plannercore.PhysicalStreamAgg) Execu } } - metrics.ExecutorCounter.WithLabelValues("StreamAggExec").Inc() + executorStreamAggExec.Inc() return e } @@ -1153,7 +1165,7 @@ func (b *executorBuilder) buildSort(v *plannercore.PhysicalSort) Executor { ByItems: v.ByItems, schema: v.Schema(), } - metrics.ExecutorCounter.WithLabelValues("SortExec").Inc() + executorCounterSortExec.Inc() return &sortExec } @@ -1167,7 +1179,7 @@ func (b *executorBuilder) buildTopN(v *plannercore.PhysicalTopN) Executor { ByItems: v.ByItems, schema: v.Schema(), } - metrics.ExecutorCounter.WithLabelValues("TopNExec").Inc() + executorCounterTopNExec.Inc() return &TopNExec{ SortExec: sortExec, limit: &plannercore.PhysicalLimit{Count: v.Count, Offset: v.Offset}, @@ -1207,7 +1219,7 @@ func (b *executorBuilder) buildApply(apply *plannercore.PhysicalApply) *NestedLo joiner: tupleJoiner, outerSchema: apply.OuterSchema, } - metrics.ExecutorCounter.WithLabelValues("NestedLoopApplyExec").Inc() + executorCounterNestedLoopApplyExec.Inc() return e } @@ -1618,7 +1630,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) } e.innerCtx.keyCols = innerKeyCols e.joinResult = e.newFirstChunk() - metrics.ExecutorCounter.WithLabelValues("IndexLookUpJoin").Inc() + executorCounterIndexLookUpJoin.Inc() return e } @@ -1825,7 +1837,7 @@ func (b *executorBuilder) buildIndexLookUpReader(v *plannercore.PhysicalIndexLoo ts := v.TablePlans[0].(*plannercore.PhysicalTableScan) ret.ranges = is.Ranges - metrics.ExecutorCounter.WithLabelValues("IndexLookUpExecutor").Inc() + executorCounterIndexLookUpExecutor.Inc() sctx := b.ctx.GetSessionVars().StmtCtx sctx.IndexIDs = append(sctx.IndexIDs, is.Index.ID) sctx.TableIDs = append(sctx.TableIDs, ts.Table.ID) diff --git a/executor/compiler.go b/executor/compiler.go index 6070626337a0c..23b2239475ca9 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -29,6 +29,19 @@ import ( "go.uber.org/zap" ) +var ( + stmtNodeCounterUse = metrics.StmtNodeCounter.WithLabelValues("Use") + stmtNodeCounterShow = metrics.StmtNodeCounter.WithLabelValues("Show") + stmtNodeCounterBegin = metrics.StmtNodeCounter.WithLabelValues("Begin") + stmtNodeCounterCommit = metrics.StmtNodeCounter.WithLabelValues("Commit") + stmtNodeCounterRollback = metrics.StmtNodeCounter.WithLabelValues("Rollback") + stmtNodeCounterInsert = metrics.StmtNodeCounter.WithLabelValues("Insert") + stmtNodeCounterReplace = metrics.StmtNodeCounter.WithLabelValues("Replace") + stmtNodeCounterDelete = metrics.StmtNodeCounter.WithLabelValues("Delete") + stmtNodeCounterUpdate = metrics.StmtNodeCounter.WithLabelValues("Update") + stmtNodeCounterSelect = metrics.StmtNodeCounter.WithLabelValues("Select") +) + // Compiler compiles an ast.StmtNode to a physical plan. type Compiler struct { Ctx sessionctx.Context @@ -124,7 +137,30 @@ func CountStmtNode(stmtNode ast.StmtNode, inRestrictedSQL bool) { } typeLabel := GetStmtLabel(stmtNode) - metrics.StmtNodeCounter.WithLabelValues(typeLabel).Inc() + switch typeLabel { + case "Use": + stmtNodeCounterUse.Inc() + case "Show": + stmtNodeCounterShow.Inc() + case "Begin": + stmtNodeCounterBegin.Inc() + case "Commit": + stmtNodeCounterCommit.Inc() + case "Rollback": + stmtNodeCounterRollback.Inc() + case "Insert": + stmtNodeCounterInsert.Inc() + case "Replace": + stmtNodeCounterReplace.Inc() + case "Delete": + stmtNodeCounterDelete.Inc() + case "Update": + stmtNodeCounterUpdate.Inc() + case "Select": + stmtNodeCounterSelect.Inc() + default: + metrics.StmtNodeCounter.WithLabelValues(typeLabel).Inc() + } if !config.GetGlobalConfig().Status.RecordQPSbyDB { return diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index 8443145bf3849..cbc4d95475b03 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -428,6 +428,7 @@ func (s *seqTestSuite) TestPreparedInsert(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = orgMemGuardRatio plannercore.PreparedPlanCacheMaxMemory = orgMaxMemory }() + metrics.ResettablePlanCacheCounterFortTest = true metrics.PlanCacheCounter.Reset() counter := metrics.PlanCacheCounter.WithLabelValues("prepare") pb := &dto.Metric{} @@ -510,6 +511,7 @@ func (s *seqTestSuite) TestPreparedUpdate(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = orgMemGuardRatio plannercore.PreparedPlanCacheMaxMemory = orgMaxMemory }() + metrics.ResettablePlanCacheCounterFortTest = true metrics.PlanCacheCounter.Reset() counter := metrics.PlanCacheCounter.WithLabelValues("prepare") pb := &dto.Metric{} @@ -569,6 +571,7 @@ func (s *seqTestSuite) TestPreparedDelete(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = orgMemGuardRatio plannercore.PreparedPlanCacheMaxMemory = orgMaxMemory }() + metrics.ResettablePlanCacheCounterFortTest = true metrics.PlanCacheCounter.Reset() counter := metrics.PlanCacheCounter.WithLabelValues("prepare") pb := &dto.Metric{} diff --git a/metrics/server.go b/metrics/server.go index 15d082153a106..11a7ef47fbebf 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -21,6 +21,11 @@ import ( "github.com/prometheus/client_golang/prometheus" ) +var ( + // ResettablePlanCacheCounterFortTest be used to support reset counter in test. + ResettablePlanCacheCounterFortTest = false +) + // Metrics var ( QueryDurationHistogram = prometheus.NewHistogramVec( diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 7eb85daddb217..56a554c269c40 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -35,6 +35,8 @@ import ( "github.com/pingcap/tidb/util/ranger" ) +var planCacheCounter = metrics.PlanCacheCounter.WithLabelValues("prepare") + // ShowDDL is for showing DDL information. type ShowDDL struct { baseSchemaProducer @@ -200,7 +202,11 @@ func (e *Execute) getPhysicalPlan(ctx sessionctx.Context, is infoschema.InfoSche if prepared.UseCache { cacheKey = NewPSTMTPlanCacheKey(sessionVars, e.ExecID, prepared.SchemaVersion) if cacheValue, exists := ctx.PreparedPlanCache().Get(cacheKey); exists { - metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() + if metrics.ResettablePlanCacheCounterFortTest { + metrics.PlanCacheCounter.WithLabelValues("prepare").Inc() + } else { + planCacheCounter.Inc() + } plan := cacheValue.(*PSTMTPlanCacheValue).Plan err := e.rebuildRange(plan) if err != nil { diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index a485bd488aac8..b2f8858b36b5b 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -68,6 +68,7 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { tk.MustQuery("explain delete from t where a = 1").Check(testkit.Rows( "Point_Get_1 1.00 root table:t, handle:1", )) + metrics.ResettablePlanCacheCounterFortTest = true metrics.PlanCacheCounter.Reset() counter := metrics.PlanCacheCounter.WithLabelValues("prepare") pb := &dto.Metric{} diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index 5c030feed78ae..f5af8954eec94 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -154,6 +154,7 @@ func (s *testPlanSuite) TestPrepareCacheDeferredFunction(c *C) { var cnt [2]float64 var planStr [2]string + metrics.ResettablePlanCacheCounterFortTest = true metrics.PlanCacheCounter.Reset() counter := metrics.PlanCacheCounter.WithLabelValues("prepare") for i := 0; i < 2; i++ { From 49abe19aaf3a97e39052364dd667c0b531654919 Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 22 Apr 2019 22:16:11 +0800 Subject: [PATCH 12/21] store: pre with labels for metrics in store package (#10229) --- store/tikv/2pc.go | 9 +++++++-- store/tikv/backoff.go | 34 +++++++++++++++++++++++++++++++- store/tikv/coprocessor.go | 4 +++- store/tikv/lock_resolver.go | 31 +++++++++++++++++++---------- store/tikv/rawkv.go | 33 +++++++++++++++++++++---------- store/tikv/region_cache.go | 30 ++++++++++++++++++++++++---- store/tikv/snapshot.go | 12 +++++++++--- store/tikv/txn.go | 39 +++++++++++++++++++++++++------------ 8 files changed, 149 insertions(+), 43 deletions(-) diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index bcc8b968ac8e4..c078bac7f1c8e 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -45,6 +45,11 @@ const ( actionCleanup twoPhaseCommitAction = 3 ) +var ( + tikvSecondaryLockCleanupFailureCounterCommit = metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("commit") + tikvSecondaryLockCleanupFailureCounterRollback = metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("rollback") +) + func (ca twoPhaseCommitAction) String() string { switch ca { case actionPrewrite: @@ -303,7 +308,7 @@ func (c *twoPhaseCommitter) doActionOnKeys(bo *Backoffer, action twoPhaseCommitA zap.Uint64("conn", c.connID), zap.Stringer("action type", action), zap.Error(e)) - metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("commit").Inc() + tikvSecondaryLockCleanupFailureCounterCommit.Inc() } }() } else { @@ -681,7 +686,7 @@ func (c *twoPhaseCommitter) execute(ctx context.Context) error { cleanupKeysCtx := context.WithValue(context.Background(), txnStartKey, ctx.Value(txnStartKey)) err := c.cleanupKeys(NewBackoffer(cleanupKeysCtx, cleanupMaxBackoff).WithVars(c.txn.vars), c.keys) if err != nil { - metrics.TiKVSecondaryLockCleanupFailureCounter.WithLabelValues("rollback").Inc() + tikvSecondaryLockCleanupFailureCounterRollback.Inc() logutil.Logger(ctx).Info("2PC cleanup failed", zap.Error(err), zap.Uint64("txnStartTS", c.startTS)) diff --git a/store/tikv/backoff.go b/store/tikv/backoff.go index 8d139903bfb32..84dc8e79580d2 100644 --- a/store/tikv/backoff.go +++ b/store/tikv/backoff.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/util/logutil" + "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -43,6 +44,37 @@ const ( DecorrJitter ) +var ( + tikvBackoffCounterRPC = metrics.TiKVBackoffCounter.WithLabelValues("tikvRPC") + tikvBackoffCounterLock = metrics.TiKVBackoffCounter.WithLabelValues("txnLock") + tikvBackoffCounterLockFast = metrics.TiKVBackoffCounter.WithLabelValues("tikvLockFast") + tikvBackoffCounterPD = metrics.TiKVBackoffCounter.WithLabelValues("pdRPC") + tikvBackoffCounterRegionMiss = metrics.TiKVBackoffCounter.WithLabelValues("regionMiss") + tikvBackoffCounterUpdateLeader = metrics.TiKVBackoffCounter.WithLabelValues("updateLeader") + tikvBackoffCounterServerBusy = metrics.TiKVBackoffCounter.WithLabelValues("serverBusy") + tikvBackoffCounterEmpty = metrics.TiKVBackoffCounter.WithLabelValues("") +) + +func (t backoffType) Counter() prometheus.Counter { + switch t { + case boTiKVRPC: + return tikvBackoffCounterRPC + case BoTxnLock: + return tikvBackoffCounterLock + case boTxnLockFast: + return tikvBackoffCounterLockFast + case BoPDRPC: + return tikvBackoffCounterPD + case BoRegionMiss: + return tikvBackoffCounterRegionMiss + case BoUpdateLeader: + return tikvBackoffCounterUpdateLeader + case boServerBusy: + return tikvBackoffCounterServerBusy + } + return tikvBackoffCounterEmpty +} + // NewBackoffFn creates a backoff func which implements exponential backoff with // optional jitters. // See http://www.awsarchitectureblog.com/2015/03/backoff.html @@ -222,7 +254,7 @@ func (b *Backoffer) Backoff(typ backoffType, err error) error { default: } - metrics.TiKVBackoffCounter.WithLabelValues(typ.String()).Inc() + typ.Counter().Inc() // Lazy initialize. if b.fn == nil { b.fn = make(map[backoffType]func(context.Context) int) diff --git a/store/tikv/coprocessor.go b/store/tikv/coprocessor.go index b7aa4a39e907f..0855b01552506 100644 --- a/store/tikv/coprocessor.go +++ b/store/tikv/coprocessor.go @@ -40,6 +40,8 @@ import ( "go.uber.org/zap" ) +var tikvTxnRegionsNumHistogramWithCoprocessor = metrics.TiKVTxnRegionsNumHistogram.WithLabelValues("coprocessor") + // CopClient is coprocessor client. type CopClient struct { store *tikvStore @@ -280,7 +282,7 @@ func buildCopTasks(bo *Backoffer, cache *RegionCache, ranges *copRanges, desc bo zap.Int("range len", rangesLen), zap.Int("task len", len(tasks))) } - metrics.TiKVTxnRegionsNumHistogram.WithLabelValues("coprocessor").Observe(float64(len(tasks))) + tikvTxnRegionsNumHistogramWithCoprocessor.Observe(float64(len(tasks))) return tasks, nil } diff --git a/store/tikv/lock_resolver.go b/store/tikv/lock_resolver.go index d50b1a4eadf2a..944026bcb13a0 100644 --- a/store/tikv/lock_resolver.go +++ b/store/tikv/lock_resolver.go @@ -33,6 +33,17 @@ import ( // ResolvedCacheSize is max number of cached txn status. const ResolvedCacheSize = 2048 +var ( + tikvLockResolverCountWithBatchResolve = metrics.TiKVLockResolverCounter.WithLabelValues("batch_resolve") + tikvLockResolverCountWithExpired = metrics.TiKVLockResolverCounter.WithLabelValues("expired") + tikvLockResolverCountWithNotExpired = metrics.TiKVLockResolverCounter.WithLabelValues("not_expired") + tikvLockResolverCountWithResolve = metrics.TiKVLockResolverCounter.WithLabelValues("resolve") + tikvLockResolverCountWithQueryTxnStatus = metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status") + tikvLockResolverCountWithQueryTxnStatusCommitted = metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status_committed") + tikvLockResolverCountWithQueryTxnStatusRolledBack = metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status_rolled_back") + tikvLockResolverCountWithResolveLocks = metrics.TiKVLockResolverCounter.WithLabelValues("query_resolve_locks") +) + // LockResolver resolves locks and also caches resolved txn status. type LockResolver struct { store Storage @@ -163,15 +174,15 @@ func (lr *LockResolver) BatchResolveLocks(bo *Backoffer, locks []*Lock, loc Regi return true, nil } - metrics.TiKVLockResolverCounter.WithLabelValues("batch_resolve").Inc() + tikvLockResolverCountWithBatchResolve.Inc() var expiredLocks []*Lock for _, l := range locks { if lr.store.GetOracle().IsExpired(l.TxnID, l.TTL) { - metrics.TiKVLockResolverCounter.WithLabelValues("expired").Inc() + tikvLockResolverCountWithExpired.Inc() expiredLocks = append(expiredLocks, l) } else { - metrics.TiKVLockResolverCounter.WithLabelValues("not_expired").Inc() + tikvLockResolverCountWithNotExpired.Inc() } } if len(expiredLocks) != len(locks) { @@ -259,15 +270,15 @@ func (lr *LockResolver) ResolveLocks(bo *Backoffer, locks []*Lock) (ok bool, err return true, nil } - metrics.TiKVLockResolverCounter.WithLabelValues("resolve").Inc() + tikvLockResolverCountWithResolve.Inc() var expiredLocks []*Lock for _, l := range locks { if lr.store.GetOracle().IsExpired(l.TxnID, l.TTL) { - metrics.TiKVLockResolverCounter.WithLabelValues("expired").Inc() + tikvLockResolverCountWithExpired.Inc() expiredLocks = append(expiredLocks, l) } else { - metrics.TiKVLockResolverCounter.WithLabelValues("not_expired").Inc() + tikvLockResolverCountWithNotExpired.Inc() } } if len(expiredLocks) == 0 { @@ -312,7 +323,7 @@ func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte return s, nil } - metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status").Inc() + tikvLockResolverCountWithQueryTxnStatus.Inc() var status TxnStatus req := &tikvrpc.Request{ @@ -353,9 +364,9 @@ func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte } if cmdResp.CommitVersion != 0 { status = TxnStatus(cmdResp.GetCommitVersion()) - metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status_committed").Inc() + tikvLockResolverCountWithQueryTxnStatusCommitted.Inc() } else { - metrics.TiKVLockResolverCounter.WithLabelValues("query_txn_status_rolled_back").Inc() + tikvLockResolverCountWithQueryTxnStatusRolledBack.Inc() } lr.saveResolved(txnID, status) return status, nil @@ -363,7 +374,7 @@ func (lr *LockResolver) getTxnStatus(bo *Backoffer, txnID uint64, primary []byte } func (lr *LockResolver) resolveLock(bo *Backoffer, l *Lock, status TxnStatus, cleanRegions map[RegionVerID]struct{}) error { - metrics.TiKVLockResolverCounter.WithLabelValues("query_resolve_locks").Inc() + tikvLockResolverCountWithResolveLocks.Inc() for { loc, err := lr.store.GetRegionCache().LocateKey(bo, l.Key) if err != nil { diff --git a/store/tikv/rawkv.go b/store/tikv/rawkv.go index c787ec4654955..692cf77807d10 100644 --- a/store/tikv/rawkv.go +++ b/store/tikv/rawkv.go @@ -33,6 +33,19 @@ var ( ErrMaxScanLimitExceeded = errors.New("limit should be less than MaxRawKVScanLimit") ) +var ( + tikvRawkvCmdHistogramWithGet = metrics.TiKVRawkvCmdHistogram.WithLabelValues("get") + tikvRawkvCmdHistogramWithBatchGet = metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_get") + tikvRawkvCmdHistogramWithBatchPut = metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_put") + tikvRawkvCmdHistogramWithDelete = metrics.TiKVRawkvCmdHistogram.WithLabelValues("delete") + tikvRawkvCmdHistogramWithBatchDelete = metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_delete") + tikvRawkvCmdHistogramWithRawScan = metrics.TiKVRawkvCmdHistogram.WithLabelValues("raw_scan") + tikvRawkvCmdHistogramWithRawReversScan = metrics.TiKVRawkvCmdHistogram.WithLabelValues("raw_reverse_scan") + + tikvRawkvSizeHistogramWithKey = metrics.TiKVRawkvSizeHistogram.WithLabelValues("key") + tikvRawkvSizeHistogramWithValue = metrics.TiKVRawkvSizeHistogram.WithLabelValues("value") +) + const ( // rawBatchPutSize is the maximum size limit for rawkv each batch put request. rawBatchPutSize = 16 * 1024 @@ -81,7 +94,7 @@ func (c *RawKVClient) ClusterID() uint64 { // Get queries value with the key. When the key does not exist, it returns `nil, nil`. func (c *RawKVClient) Get(key []byte) ([]byte, error) { start := time.Now() - defer func() { metrics.TiKVRawkvCmdHistogram.WithLabelValues("get").Observe(time.Since(start).Seconds()) }() + defer func() { tikvRawkvCmdHistogramWithGet.Observe(time.Since(start).Seconds()) }() req := &tikvrpc.Request{ Type: tikvrpc.CmdRawGet, @@ -110,7 +123,7 @@ func (c *RawKVClient) Get(key []byte) ([]byte, error) { func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { start := time.Now() defer func() { - metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_get").Observe(time.Since(start).Seconds()) + tikvRawkvCmdHistogramWithBatchGet.Observe(time.Since(start).Seconds()) }() bo := NewBackoffer(context.Background(), rawkvMaxBackoff) @@ -139,9 +152,9 @@ func (c *RawKVClient) BatchGet(keys [][]byte) ([][]byte, error) { // Put stores a key-value pair to TiKV. func (c *RawKVClient) Put(key, value []byte) error { start := time.Now() - defer func() { metrics.TiKVRawkvCmdHistogram.WithLabelValues("put").Observe(time.Since(start).Seconds()) }() - metrics.TiKVRawkvSizeHistogram.WithLabelValues("key").Observe(float64(len(key))) - metrics.TiKVRawkvSizeHistogram.WithLabelValues("value").Observe(float64(len(value))) + defer func() { tikvRawkvCmdHistogramWithBatchPut.Observe(time.Since(start).Seconds()) }() + tikvRawkvSizeHistogramWithKey.Observe(float64(len(key))) + tikvRawkvSizeHistogramWithValue.Observe(float64(len(value))) if len(value) == 0 { return errors.New("empty value is not supported") @@ -172,7 +185,7 @@ func (c *RawKVClient) Put(key, value []byte) error { func (c *RawKVClient) BatchPut(keys, values [][]byte) error { start := time.Now() defer func() { - metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_put").Observe(time.Since(start).Seconds()) + tikvRawkvCmdHistogramWithBatchPut.Observe(time.Since(start).Seconds()) }() if len(keys) != len(values) { @@ -191,7 +204,7 @@ func (c *RawKVClient) BatchPut(keys, values [][]byte) error { // Delete deletes a key-value pair from TiKV. func (c *RawKVClient) Delete(key []byte) error { start := time.Now() - defer func() { metrics.TiKVRawkvCmdHistogram.WithLabelValues("delete").Observe(time.Since(start).Seconds()) }() + defer func() { tikvRawkvCmdHistogramWithDelete.Observe(time.Since(start).Seconds()) }() req := &tikvrpc.Request{ Type: tikvrpc.CmdRawDelete, @@ -217,7 +230,7 @@ func (c *RawKVClient) Delete(key []byte) error { func (c *RawKVClient) BatchDelete(keys [][]byte) error { start := time.Now() defer func() { - metrics.TiKVRawkvCmdHistogram.WithLabelValues("batch_delete").Observe(time.Since(start).Seconds()) + tikvRawkvCmdHistogramWithBatchDelete.Observe(time.Since(start).Seconds()) }() bo := NewBackoffer(context.Background(), rawkvMaxBackoff) @@ -275,7 +288,7 @@ func (c *RawKVClient) DeleteRange(startKey []byte, endKey []byte) error { // `Scan(append(startKey, '\0'), append(endKey, '\0'), limit)`. func (c *RawKVClient) Scan(startKey, endKey []byte, limit int) (keys [][]byte, values [][]byte, err error) { start := time.Now() - defer func() { metrics.TiKVRawkvCmdHistogram.WithLabelValues("raw_scan").Observe(time.Since(start).Seconds()) }() + defer func() { tikvRawkvCmdHistogramWithRawScan.Observe(time.Since(start).Seconds()) }() if limit > MaxRawKVScanLimit { return nil, nil, errors.Trace(ErrMaxScanLimitExceeded) @@ -320,7 +333,7 @@ func (c *RawKVClient) Scan(startKey, endKey []byte, limit int) (keys [][]byte, v func (c *RawKVClient) ReverseScan(startKey, endKey []byte, limit int) (keys [][]byte, values [][]byte, err error) { start := time.Now() defer func() { - metrics.TiKVRawkvCmdHistogram.WithLabelValues("raw_reverse_scan").Observe(time.Since(start).Seconds()) + tikvRawkvCmdHistogramWithRawReversScan.Observe(time.Since(start).Seconds()) }() if limit > MaxRawKVScanLimit { diff --git a/store/tikv/region_cache.go b/store/tikv/region_cache.go index d75c7bad23db5..a48207c7b88fb 100644 --- a/store/tikv/region_cache.go +++ b/store/tikv/region_cache.go @@ -36,6 +36,16 @@ const ( rcDefaultRegionCacheTTL = time.Minute * 10 ) +var ( + tikvRegionCacheCounterWithDropRegionFromCacheOK = metrics.TiKVRegionCacheCounter.WithLabelValues("drop_region_from_cache", "ok") + tikvRegionCacheCounterWithGetRegionByIDOK = metrics.TiKVRegionCacheCounter.WithLabelValues("get_region_by_id", "ok") + tikvRegionCacheCounterWithGetRegionByIDError = metrics.TiKVRegionCacheCounter.WithLabelValues("get_region_by_id", "err") + tikvRegionCacheCounterWithGetRegionOK = metrics.TiKVRegionCacheCounter.WithLabelValues("get_region", "ok") + tikvRegionCacheCounterWithGetRegionError = metrics.TiKVRegionCacheCounter.WithLabelValues("get_region", "err") + tikvRegionCacheCounterWithGetStoreOK = metrics.TiKVRegionCacheCounter.WithLabelValues("get_store", "ok") + tikvRegionCacheCounterWithGetStoreError = metrics.TiKVRegionCacheCounter.WithLabelValues("get_store", "err") +) + // CachedRegion encapsulates {Region, TTL} type CachedRegion struct { region *Region @@ -368,7 +378,7 @@ func (c *RegionCache) dropRegionFromCache(verID RegionVerID) { if !ok { return } - metrics.TiKVRegionCacheCounter.WithLabelValues("drop_region_from_cache", metrics.RetLabel(nil)).Inc() + tikvRegionCacheCounterWithDropRegionFromCacheOK.Inc() c.mu.sorted.Delete(newBtreeItem(r.region)) delete(c.mu.regions, verID) } @@ -393,7 +403,11 @@ func (c *RegionCache) loadRegion(bo *Backoffer, key []byte, isEndKey bool) (*Reg } else { meta, leader, err = c.pdClient.GetRegion(bo.ctx, key) } - metrics.TiKVRegionCacheCounter.WithLabelValues("get_region", metrics.RetLabel(err)).Inc() + if err != nil { + tikvRegionCacheCounterWithGetRegionError.Inc() + } else { + tikvRegionCacheCounterWithGetRegionOK.Inc() + } if err != nil { backoffErr = errors.Errorf("loadRegion from PD failed, key: %q, err: %v", key, err) continue @@ -431,7 +445,11 @@ func (c *RegionCache) loadRegionByID(bo *Backoffer, regionID uint64) (*Region, e } } meta, leader, err := c.pdClient.GetRegionByID(bo.ctx, regionID) - metrics.TiKVRegionCacheCounter.WithLabelValues("get_region_by_id", metrics.RetLabel(err)).Inc() + if err != nil { + tikvRegionCacheCounterWithGetRegionByIDError.Inc() + } else { + tikvRegionCacheCounterWithGetRegionByIDOK.Inc() + } if err != nil { backoffErr = errors.Errorf("loadRegion from PD failed, regionID: %v, err: %v", regionID, err) continue @@ -492,7 +510,11 @@ func (c *RegionCache) ClearStoreByID(id uint64) { func (c *RegionCache) loadStoreAddr(bo *Backoffer, id uint64) (string, error) { for { store, err := c.pdClient.GetStore(bo.ctx, id) - metrics.TiKVRegionCacheCounter.WithLabelValues("get_store", metrics.RetLabel(err)).Inc() + if err != nil { + tikvRegionCacheCounterWithGetStoreError.Inc() + } else { + tikvRegionCacheCounterWithGetStoreOK.Inc() + } if err != nil { if errors.Cause(err) == context.Canceled { return "", errors.Trace(err) diff --git a/store/tikv/snapshot.go b/store/tikv/snapshot.go index cca2990219ceb..a0cf8367996f4 100644 --- a/store/tikv/snapshot.go +++ b/store/tikv/snapshot.go @@ -41,6 +41,12 @@ const ( batchGetSize = 5120 ) +var ( + tikvTxnCmdCounterWithBatchGet = metrics.TiKVTxnCmdCounter.WithLabelValues("batch_get") + tikvTxnCmdHistogramWithBatchGet = metrics.TiKVTxnCmdHistogram.WithLabelValues("batch_get") + tikvTxnRegionsNumHistogramWithSnapshot = metrics.TiKVTxnRegionsNumHistogram.WithLabelValues("snapshot") +) + // tikvSnapshot implements the kv.Snapshot interface. type tikvSnapshot struct { store *tikvStore @@ -73,9 +79,9 @@ func (s *tikvSnapshot) BatchGet(keys []kv.Key) (map[string][]byte, error) { if len(keys) == 0 { return m, nil } - metrics.TiKVTxnCmdCounter.WithLabelValues("batch_get").Inc() + tikvTxnCmdCounterWithBatchGet.Inc() start := time.Now() - defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("batch_get").Observe(time.Since(start).Seconds()) }() + defer func() { tikvTxnCmdHistogramWithBatchGet.Observe(time.Since(start).Seconds()) }() // We want [][]byte instead of []kv.Key, use some magic to save memory. bytesKeys := *(*[][]byte)(unsafe.Pointer(&keys)) @@ -110,7 +116,7 @@ func (s *tikvSnapshot) batchGetKeysByRegions(bo *Backoffer, keys [][]byte, colle return errors.Trace(err) } - metrics.TiKVTxnRegionsNumHistogram.WithLabelValues("snapshot").Observe(float64(len(groups))) + tikvTxnRegionsNumHistogramWithSnapshot.Observe(float64(len(groups))) var batches []batchKeys for id, g := range groups { diff --git a/store/tikv/txn.go b/store/tikv/txn.go index a6254e3bb589c..9d1c335b8fcbf 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -34,6 +34,21 @@ var ( _ kv.Transaction = (*tikvTxn)(nil) ) +var ( + tikvTxnCmdCountWithGet = metrics.TiKVTxnCmdCounter.WithLabelValues("get") + tikvTxnCmdHistogramWithGet = metrics.TiKVTxnCmdHistogram.WithLabelValues("get") + tikvTxnCmdCountWithSeek = metrics.TiKVTxnCmdCounter.WithLabelValues("seek") + tikvTxnCmdHistogramWithSeek = metrics.TiKVTxnCmdHistogram.WithLabelValues("seek") + tikvTxnCmdCountWithSeekReverse = metrics.TiKVTxnCmdCounter.WithLabelValues("seek_reverse") + tikvTxnCmdHistogramWithSeekReverse = metrics.TiKVTxnCmdHistogram.WithLabelValues("seek_reverse") + tikvTxnCmdCountWithDelete = metrics.TiKVTxnCmdCounter.WithLabelValues("delete") + tikvTxnCmdCountWithSet = metrics.TiKVTxnCmdCounter.WithLabelValues("set") + tikvTxnCmdCountWithCommit = metrics.TiKVTxnCmdCounter.WithLabelValues("commit") + tikvTxnCmdHistogramWithCommit = metrics.TiKVTxnCmdHistogram.WithLabelValues("commit") + tikvTxnCmdCountWithRollback = metrics.TiKVTxnCmdCounter.WithLabelValues("rollback") + tikvTxnCmdHistogramWithLockKeys = metrics.TiKVTxnCmdCounter.WithLabelValues("lock_keys") +) + // tikvTxn implements kv.Transaction. type tikvTxn struct { snapshot *tikvSnapshot @@ -108,9 +123,9 @@ func (txn *tikvTxn) Reset() { // Get implements transaction interface. func (txn *tikvTxn) Get(k kv.Key) ([]byte, error) { - metrics.TiKVTxnCmdCounter.WithLabelValues("get").Inc() + tikvTxnCmdCountWithGet.Inc() start := time.Now() - defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("get").Observe(time.Since(start).Seconds()) }() + defer func() { tikvTxnCmdHistogramWithGet.Observe(time.Since(start).Seconds()) }() ret, err := txn.us.Get(k) if kv.IsErrNotFound(err) { @@ -172,26 +187,26 @@ func (txn *tikvTxn) String() string { } func (txn *tikvTxn) Iter(k kv.Key, upperBound kv.Key) (kv.Iterator, error) { - metrics.TiKVTxnCmdCounter.WithLabelValues("seek").Inc() + tikvTxnCmdCountWithSeek.Inc() start := time.Now() - defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("seek").Observe(time.Since(start).Seconds()) }() + defer func() { tikvTxnCmdHistogramWithSeek.Observe(time.Since(start).Seconds()) }() return txn.us.Iter(k, upperBound) } // IterReverse creates a reversed Iterator positioned on the first entry which key is less than k. func (txn *tikvTxn) IterReverse(k kv.Key) (kv.Iterator, error) { - metrics.TiKVTxnCmdCounter.WithLabelValues("seek_reverse").Inc() + tikvTxnCmdCountWithSeekReverse.Inc() start := time.Now() defer func() { - metrics.TiKVTxnCmdHistogram.WithLabelValues("seek_reverse").Observe(time.Since(start).Seconds()) + tikvTxnCmdHistogramWithSeekReverse.Observe(time.Since(start).Seconds()) }() return txn.us.IterReverse(k) } func (txn *tikvTxn) Delete(k kv.Key) error { - metrics.TiKVTxnCmdCounter.WithLabelValues("delete").Inc() + tikvTxnCmdCountWithDelete.Inc() txn.dirty = true return txn.us.Delete(k) @@ -228,10 +243,10 @@ func (txn *tikvTxn) Commit(ctx context.Context) error { } }) - metrics.TiKVTxnCmdCounter.WithLabelValues("set").Add(float64(txn.setCnt)) - metrics.TiKVTxnCmdCounter.WithLabelValues("commit").Inc() + tikvTxnCmdCountWithSet.Add(float64(txn.setCnt)) + tikvTxnCmdCountWithCommit.Inc() start := time.Now() - defer func() { metrics.TiKVTxnCmdHistogram.WithLabelValues("commit").Observe(time.Since(start).Seconds()) }() + defer func() { tikvTxnCmdHistogramWithCommit.Observe(time.Since(start).Seconds()) }() // connID is used for log. var connID uint64 @@ -292,13 +307,13 @@ func (txn *tikvTxn) Rollback() error { } txn.close() logutil.Logger(context.Background()).Debug("[kv] rollback txn", zap.Uint64("txnStartTS", txn.StartTS())) - metrics.TiKVTxnCmdCounter.WithLabelValues("rollback").Inc() + tikvTxnCmdCountWithRollback.Inc() return nil } func (txn *tikvTxn) LockKeys(keys ...kv.Key) error { - metrics.TiKVTxnCmdCounter.WithLabelValues("lock_keys").Inc() + tikvTxnCmdHistogramWithLockKeys.Inc() txn.mu.Lock() for _, key := range keys { txn.lockKeys = append(txn.lockKeys, key) From 66ba872857b85c17845718184db2d473cf92f51a Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 22 Apr 2019 22:52:54 +0800 Subject: [PATCH 13/21] session: Adjust cop-tasks' information in slow log to let it can be parsed by `pt-query` correctly (#10230) --- sessionctx/variable/session.go | 36 +++++++++++++++++++++-------- sessionctx/variable/session_test.go | 4 ++-- 2 files changed, 28 insertions(+), 12 deletions(-) diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 03104eb888e2c..5c82edf72e0c1 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -899,10 +899,22 @@ const ( SlowLogStatsInfoStr = "Stats" // SlowLogNumCopTasksStr is the number of cop-tasks. SlowLogNumCopTasksStr = "Num_cop_tasks" - // SlowLogCopProcessStr includes some useful information about cop-tasks' process time. - SlowLogCopProcessStr = "Cop_process" - // SlowLogCopWaitStr includes some useful information about cop-tasks' wait time. - SlowLogCopWaitStr = "Cop_wait" + // SlowLogCopProcAvg is the average process time of all cop-tasks. + SlowLogCopProcAvg = "Cop_proc_avg" + // SlowLogCopProcP90 is the p90 process time of all cop-tasks. + SlowLogCopProcP90 = "Cop_proc_p90" + // SlowLogCopProcMax is the max process time of all cop-tasks. + SlowLogCopProcMax = "Cop_proc_max" + // SlowLogCopProcAddr is the address of TiKV where the cop-task which cost max process time run. + SlowLogCopProcAddr = "Cop_proc_addr" + // SlowLogCopWaitAvg is the average wait time of all cop-tasks. + SlowLogCopWaitAvg = "Cop_wait_avg" + // SlowLogCopWaitP90 is the p90 wait time of all cop-tasks. + SlowLogCopWaitP90 = "Cop_wait_p90" + // SlowLogCopWaitMax is the max wait time of all cop-tasks. + SlowLogCopWaitMax = "Cop_wait_max" + // SlowLogCopWaitAddr is the address of TiKV where the cop-task which cost wait process time run. + SlowLogCopWaitAddr = "Cop_wait_addr" // SlowLogMemMax is the max number bytes of memory used in this statement. SlowLogMemMax = "Mem_max" ) @@ -972,12 +984,16 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe } if copTasks != nil { buf.WriteString(SlowLogRowPrefixStr + SlowLogNumCopTasksStr + SlowLogSpaceMarkStr + strconv.FormatInt(int64(copTasks.NumCopTasks), 10) + "\n") - buf.WriteString(SlowLogRowPrefixStr + SlowLogCopProcessStr + SlowLogSpaceMarkStr + - fmt.Sprintf("Avg_time: %v P90_time: %v Max_time: %v Max_addr: %v", copTasks.AvgProcessTime, - copTasks.P90ProcessTime, copTasks.MaxProcessTime, copTasks.MaxProcessAddress) + "\n") - buf.WriteString(SlowLogRowPrefixStr + SlowLogCopWaitStr + SlowLogSpaceMarkStr + - fmt.Sprintf("Avg_time: %v P90_time: %v Max_time: %v Max_Addr: %v", copTasks.AvgWaitTime, - copTasks.P90WaitTime, copTasks.MaxWaitTime, copTasks.MaxWaitAddress) + "\n") + buf.WriteString(SlowLogRowPrefixStr + fmt.Sprintf("%v%v%v %v%v%v %v%v%v %v%v%v", + SlowLogCopProcAvg, SlowLogSpaceMarkStr, copTasks.AvgProcessTime.Seconds(), + SlowLogCopProcP90, SlowLogSpaceMarkStr, copTasks.P90ProcessTime.Seconds(), + SlowLogCopProcMax, SlowLogSpaceMarkStr, copTasks.MaxProcessTime.Seconds(), + SlowLogCopProcAddr, SlowLogSpaceMarkStr, copTasks.MaxProcessAddress) + "\n") + buf.WriteString(SlowLogRowPrefixStr + fmt.Sprintf("%v%v%v %v%v%v %v%v%v %v%v%v", + SlowLogCopWaitAvg, SlowLogSpaceMarkStr, copTasks.AvgWaitTime.Seconds(), + SlowLogCopWaitP90, SlowLogSpaceMarkStr, copTasks.P90WaitTime.Seconds(), + SlowLogCopWaitMax, SlowLogSpaceMarkStr, copTasks.MaxWaitTime.Seconds(), + SlowLogCopWaitAddr, SlowLogSpaceMarkStr, copTasks.MaxWaitAddress) + "\n") } if memMax > 0 { buf.WriteString(SlowLogRowPrefixStr + SlowLogMemMax + SlowLogSpaceMarkStr + strconv.FormatInt(memMax, 10) + "\n") diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 9cec3189c8742..c8d78344e165c 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -130,8 +130,8 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) { # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 # Stats: t1:pseudo # Num_cop_tasks: 10 -# Cop_process: Avg_time: 1s P90_time: 2s Max_time: 3s Max_addr: 10.6.131.78 -# Cop_wait: Avg_time: 10ms P90_time: 20ms Max_time: 30ms Max_Addr: 10.6.131.79 +# Cop_proc_avg: 1 Cop_proc_p90: 2 Cop_proc_max: 3 Cop_proc_addr: 10.6.131.78 +# Cop_wait_avg: 0.01 Cop_wait_p90: 0.02 Cop_wait_max: 0.03 Cop_wait_addr: 10.6.131.79 # Mem_max: 2333 select * from t;` sql := "select * from t" From bd457f126b59a3ec3f2191d905ee848c73ed8cd4 Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 22 Apr 2019 23:04:28 +0800 Subject: [PATCH 14/21] tikv/client: remove unused grpc metric in tikv-client (#10233) --- go.mod | 2 +- store/tikv/client.go | 18 ++++++------------ 2 files changed, 7 insertions(+), 13 deletions(-) diff --git a/go.mod b/go.mod index 234034a2a05ea..d6140988c2326 100644 --- a/go.mod +++ b/go.mod @@ -30,7 +30,7 @@ require ( github.com/gorilla/mux v1.6.2 github.com/gorilla/websocket v1.4.0 // indirect github.com/grpc-ecosystem/go-grpc-middleware v1.0.0 - github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 + github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0 // indirect github.com/grpc-ecosystem/grpc-gateway v1.5.1 // indirect github.com/jonboulle/clockwork v0.1.0 // indirect github.com/klauspost/cpuid v0.0.0-20170728055534-ae7887de9fa5 diff --git a/store/tikv/client.go b/store/tikv/client.go index 6b8f69aa4495a..b514525ddd435 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -22,9 +22,7 @@ import ( "sync/atomic" "time" - "github.com/grpc-ecosystem/go-grpc-middleware" "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing" - "github.com/grpc-ecosystem/go-grpc-prometheus" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/coprocessor" "github.com/pingcap/kvproto/pkg/debugpb" @@ -207,18 +205,14 @@ func (a *connArray) Init(addr string, security config.Security) error { opt = grpc.WithTransportCredentials(credentials.NewTLS(tlsConfig)) } - unaryInterceptor := grpc_prometheus.UnaryClientInterceptor - streamInterceptor := grpc_prometheus.StreamClientInterceptor cfg := config.GetGlobalConfig() + var ( + unaryInterceptor grpc.UnaryClientInterceptor + streamInterceptor grpc.StreamClientInterceptor + ) if cfg.OpenTracing.Enable { - unaryInterceptor = grpc_middleware.ChainUnaryClient( - unaryInterceptor, - grpc_opentracing.UnaryClientInterceptor(), - ) - streamInterceptor = grpc_middleware.ChainStreamClient( - streamInterceptor, - grpc_opentracing.StreamClientInterceptor(), - ) + unaryInterceptor = grpc_opentracing.UnaryClientInterceptor() + streamInterceptor = grpc_opentracing.StreamClientInterceptor() } allowBatch := cfg.TiKVClient.MaxBatchSize > 0 From f26edbf15c30a3a720dfe9a10254bf7140589396 Mon Sep 17 00:00:00 2001 From: HuaiyuXu Date: Mon, 22 Apr 2019 23:26:24 +0800 Subject: [PATCH 15/21] executor: use EncodeBytes in countOriginDistinct (#10225) --- executor/aggfuncs/func_count.go | 4 +++- executor/aggregate_test.go | 8 ++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/executor/aggfuncs/func_count.go b/executor/aggfuncs/func_count.go index 50e0b592f8c0b..e5033c9399c5f 100644 --- a/executor/aggfuncs/func_count.go +++ b/executor/aggfuncs/func_count.go @@ -10,6 +10,8 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/set" ) @@ -329,7 +331,7 @@ func (e *countOriginalWithDistinct) evalAndEncode( if err != nil || isNull { break } - encodedBytes = appendString(encodedBytes, buf, val) + encodedBytes = codec.EncodeBytes(encodedBytes, hack.Slice(val)) default: return nil, false, errors.Errorf("unsupported column type for encode %d", tp) } diff --git a/executor/aggregate_test.go b/executor/aggregate_test.go index f1c86de5e1616..06826616d5564 100644 --- a/executor/aggregate_test.go +++ b/executor/aggregate_test.go @@ -695,6 +695,14 @@ func (s *testSuite1) TestAggJSON(c *C) { )) } +func (s *testSuite1) TestIssue10099(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10), b char(10))") + tk.MustExec("insert into t values('1', '222'), ('12', '22')") + tk.MustQuery("select count(distinct a, b) from t").Check(testkit.Rows("2")) +} + func (s *testSuite1) TestIssue10098(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec(`drop table if exists t;`) From 58d1bb8cb33731cfcc81a9a1ffb984797d9aad03 Mon Sep 17 00:00:00 2001 From: Lingyu Song Date: Tue, 23 Apr 2019 10:05:53 +0800 Subject: [PATCH 16/21] privilege: support authentication with active roles (#10125) --- executor/show.go | 16 +++--- executor/simple.go | 3 +- infoschema/tables.go | 10 ++-- planner/core/logical_plan_builder.go | 1 - planner/core/optimizer.go | 5 +- planner/core/point_get_plan.go | 4 +- planner/optimize.go | 4 +- privilege/privilege.go | 2 +- privilege/privileges/cache.go | 51 ++++++++++++++----- privilege/privileges/cache_test.go | 64 ++++++++++++++++++----- privilege/privileges/privileges.go | 6 +-- privilege/privileges/privileges_test.go | 68 +++++++++++++++++++++---- 12 files changed, 174 insertions(+), 60 deletions(-) diff --git a/executor/show.go b/executor/show.go index ed0403c214710..fb5dee5b70e41 100644 --- a/executor/show.go +++ b/executor/show.go @@ -247,10 +247,10 @@ func (e *ShowExec) fetchShowProcessList() error { return nil } - loginUser := e.ctx.GetSessionVars().User + loginUser, activeRoles := e.ctx.GetSessionVars().User, e.ctx.GetSessionVars().ActiveRoles var hasProcessPriv bool if pm := privilege.GetPrivilegeManager(e.ctx); pm != nil { - if pm.RequestVerification("", "", "", mysql.ProcessPriv) { + if pm.RequestVerification(activeRoles, "", "", "", mysql.ProcessPriv) { hasProcessPriv = true } } @@ -284,11 +284,12 @@ func (e *ShowExec) fetchShowTables() error { } // sort for tables tableNames := make([]string, 0, len(e.is.SchemaTables(e.DBName))) + activeRoles := e.ctx.GetSessionVars().ActiveRoles var tableTypes = make(map[string]string) for _, v := range e.is.SchemaTables(e.DBName) { // Test with mysql.AllPrivMask means any privilege would be OK. // TODO: Should consider column privileges, which also make a table visible. - if checker != nil && !checker.RequestVerification(e.DBName.O, v.Meta().Name.O, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(activeRoles, e.DBName.O, v.Meta().Name.O, "", mysql.AllPrivMask) { continue } tableNames = append(tableNames, v.Meta().Name.O) @@ -332,8 +333,9 @@ func (e *ShowExec) fetchShowTableStatus() error { return errors.Trace(err) } + activeRoles := e.ctx.GetSessionVars().ActiveRoles for _, row := range rows { - if checker != nil && !checker.RequestVerification(e.DBName.O, row.GetString(0), "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(activeRoles, e.DBName.O, row.GetString(0), "", mysql.AllPrivMask) { continue } e.result.AppendRow(row) @@ -356,7 +358,8 @@ func (e *ShowExec) fetchShowColumns() error { return errors.Trace(err) } checker := privilege.GetPrivilegeManager(e.ctx) - if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.RequestVerification(e.DBName.O, tb.Meta().Name.O, "", mysql.AllPrivMask) { + activeRoles := e.ctx.GetSessionVars().ActiveRoles + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.RequestVerification(activeRoles, e.DBName.O, tb.Meta().Name.O, "", mysql.AllPrivMask) { return e.tableAccessDenied("SELECT", tb.Meta().Name.O) } @@ -438,7 +441,8 @@ func (e *ShowExec) fetchShowIndex() error { } checker := privilege.GetPrivilegeManager(e.ctx) - if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.RequestVerification(e.DBName.O, tb.Meta().Name.O, "", mysql.AllPrivMask) { + activeRoles := e.ctx.GetSessionVars().ActiveRoles + if checker != nil && e.ctx.GetSessionVars().User != nil && !checker.RequestVerification(activeRoles, e.DBName.O, tb.Meta().Name.O, "", mysql.AllPrivMask) { return e.tableAccessDenied("SELECT", tb.Meta().Name.O) } diff --git a/executor/simple.go b/executor/simple.go index 5cd1ef5259d95..d4199c4950858 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -630,7 +630,8 @@ func (e *SimpleExec) executeSetPwd(s *ast.SetPwdStmt) error { h = e.ctx.GetSessionVars().User.AuthHostname } else { checker := privilege.GetPrivilegeManager(e.ctx) - if checker != nil && !checker.RequestVerification("", "", "", mysql.SuperPriv) { + activeRoles := e.ctx.GetSessionVars().ActiveRoles + if checker != nil && !checker.RequestVerification(activeRoles, "", "", "", mysql.SuperPriv) { return ErrDBaccessDenied.GenWithStackByArgs(u, h, "mysql") } u = s.User.Username diff --git a/infoschema/tables.go b/infoschema/tables.go index 46cd30826d936..4a848f64ba828 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -642,7 +642,7 @@ func dataForProcesslist(ctx sessionctx.Context) [][]types.Datum { loginUser := ctx.GetSessionVars().User var hasProcessPriv bool if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - if pm.RequestVerification("", "", "", mysql.ProcessPriv) { + if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) { hasProcessPriv = true } } @@ -905,7 +905,7 @@ func dataForViews(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.Da if charset == "" { charset = mysql.DefaultCharset } - if checker != nil && !checker.RequestVerification(schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } record := types.MakeDatums( @@ -949,7 +949,7 @@ func dataForTables(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types.D createOptions := "" - if checker != nil && !checker.RequestVerification(schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } @@ -1029,7 +1029,7 @@ func dataForIndexes(ctx sessionctx.Context, schemas []*model.DBInfo) ([][]types. var rows [][]types.Datum for _, schema := range schemas { for _, tb := range schema.Tables { - if checker != nil && !checker.RequestVerification(schema.Name.L, tb.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, tb.Name.L, "", mysql.AllPrivMask) { continue } @@ -1091,7 +1091,7 @@ func dataForColumns(ctx sessionctx.Context, schemas []*model.DBInfo) [][]types.D var rows [][]types.Datum for _, schema := range schemas { for _, table := range schema.Tables { - if checker != nil && !checker.RequestVerification(schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { + if checker != nil && !checker.RequestVerification(ctx.GetSessionVars().ActiveRoles, schema.Name.L, table.Name.L, "", mysql.AllPrivMask) { continue } diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 3d156b4b830c1..d00f28d561647 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -2265,7 +2265,6 @@ func (b *PlanBuilder) BuildDataSourceFromView(dbName model.CIStr, tableInfo *mod if err != nil { return nil, err } - originalVisitInfo := b.visitInfo b.visitInfo = make([]visitInfo, 0) selectLogicalPlan, err := b.Build(selectNode) diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 3809fb57d5576..1baf596854a66 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/auth" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/planner/property" @@ -83,9 +84,9 @@ func BuildLogicalPlan(ctx sessionctx.Context, node ast.Node, is infoschema.InfoS } // CheckPrivilege checks the privilege for a user. -func CheckPrivilege(pm privilege.Manager, vs []visitInfo) error { +func CheckPrivilege(activeRoles []*auth.RoleIdentity, pm privilege.Manager, vs []visitInfo) error { for _, v := range vs { - if !pm.RequestVerification(v.db, v.table, v.column, v.privilege) { + if !pm.RequestVerification(activeRoles, v.db, v.table, v.column, v.privilege) { if v.err == nil { return ErrPrivilegeCheckFail } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index dcfb295656237..0b36025af5d7a 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -98,7 +98,7 @@ func (p *PointGetPlan) StatsCount() float64 { return 1 } -// StatsCount will return the the RowCount of property.StatsInfo for this plan. +// statsInfo will return the the RowCount of property.StatsInfo for this plan. func (p *PointGetPlan) statsInfo() *property.StatsInfo { if p.stats == nil { p.stats = &property.StatsInfo{} @@ -241,7 +241,7 @@ func checkFastPlanPrivilege(ctx sessionctx.Context, fastPlan *PointGetPlan, chec } dbName := ctx.GetSessionVars().CurrentDB for _, checkType := range checkTypes { - if !pm.RequestVerification(dbName, fastPlan.TblInfo.Name.L, "", checkType) { + if !pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, dbName, fastPlan.TblInfo.Name.L, "", checkType) { return errors.New("privilege check fail") } } diff --git a/planner/optimize.go b/planner/optimize.go index a23c2d362610d..820983d505b97 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -40,12 +40,12 @@ func Optimize(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) ( } ctx.GetSessionVars().StmtCtx.Tables = builder.GetDBTableInfo() - + activeRoles := ctx.GetSessionVars().ActiveRoles // Check privilege. Maybe it's better to move this to the Preprocess, but // we need the table information to check privilege, which is collected // into the visitInfo in the logical plan builder. if pm := privilege.GetPrivilegeManager(ctx); pm != nil { - if err := plannercore.CheckPrivilege(pm, builder.GetVisitInfo()); err != nil { + if err := plannercore.CheckPrivilege(activeRoles, pm, builder.GetVisitInfo()); err != nil { return nil, err } } diff --git a/privilege/privilege.go b/privilege/privilege.go index 332e8229ea93a..0b33276284024 100644 --- a/privilege/privilege.go +++ b/privilege/privilege.go @@ -39,7 +39,7 @@ type Manager interface { // If table is not "", check global/db/table scope privileges. // priv should be a defined constant like CreatePriv, if pass AllPrivMask to priv, // this means any privilege would be OK. - RequestVerification(db, table, column string, priv mysql.PrivilegeType) bool + RequestVerification(activeRole []*auth.RoleIdentity, db, table, column string, priv mysql.PrivilegeType) bool // RequestVerificationWithUser verifies specific user privilege for the request. RequestVerificationWithUser(db, table, column string, priv mysql.PrivilegeType, user *auth.UserIdentity) bool diff --git a/privilege/privileges/cache.go b/privilege/privileges/cache.go index 85121d30440e4..3cba12a3b6316 100644 --- a/privilege/privileges/cache.go +++ b/privilege/privileges/cache.go @@ -656,29 +656,52 @@ func (p *MySQLPrivilege) matchColumns(user, host, db, table, column string) *col } // RequestVerification checks whether the user have sufficient privileges to do the operation. -func (p *MySQLPrivilege) RequestVerification(user, host, db, table, column string, priv mysql.PrivilegeType) bool { - record1 := p.matchUser(user, host) - if record1 != nil && record1.Privileges&priv > 0 { +func (p *MySQLPrivilege) RequestVerification(activeRoles []*auth.RoleIdentity, user, host, db, table, column string, priv mysql.PrivilegeType) bool { + roleList := p.FindAllRole(activeRoles) + roleList = append(roleList, &auth.RoleIdentity{Username: user, Hostname: host}) + + var userPriv, dbPriv, tablePriv, columnPriv mysql.PrivilegeType + for _, r := range roleList { + userRecord := p.matchUser(r.Username, r.Hostname) + if userRecord != nil { + userPriv |= userRecord.Privileges + } + } + if userPriv&priv > 0 { return true } - record2 := p.matchDB(user, host, db) - if record2 != nil && record2.Privileges&priv > 0 { + for _, r := range roleList { + dbRecord := p.matchDB(r.Username, r.Hostname, db) + if dbRecord != nil { + dbPriv |= dbRecord.Privileges + } + } + if dbPriv&priv > 0 { return true } - record3 := p.matchTables(user, host, db, table) - if record3 != nil { - if record3.TablePriv&priv > 0 { - return true - } - if column != "" && record3.ColumnPriv&priv > 0 { - return true + for _, r := range roleList { + tableRecord := p.matchTables(r.Username, r.Hostname, db, table) + if tableRecord != nil { + tablePriv |= tableRecord.TablePriv + if column != "" { + columnPriv |= tableRecord.ColumnPriv + } } } + if tablePriv&priv > 0 || columnPriv&priv > 0 { + return true + } - record4 := p.matchColumns(user, host, db, table, column) - if record4 != nil && record4.ColumnPriv&priv > 0 { + columnPriv = 0 + for _, r := range roleList { + columnRecord := p.matchColumns(r.Username, r.Hostname, db, table, column) + if columnRecord != nil { + columnPriv |= columnRecord.ColumnPriv + } + } + if columnPriv&priv > 0 { return true } diff --git a/privilege/privileges/cache_test.go b/privilege/privileges/cache_test.go index fe9e6c740035e..6a3ffbdfa9fc1 100644 --- a/privilege/privileges/cache_test.go +++ b/privilege/privileges/cache_test.go @@ -15,6 +15,7 @@ package privileges_test import ( . "github.com/pingcap/check" + "github.com/pingcap/parser/auth" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" @@ -155,6 +156,7 @@ func (s *testCacheSuite) TestLoadDefaultRoleTable(c *C) { func (s *testCacheSuite) TestPatternMatch(c *C) { se, err := session.CreateSession4Test(s.store) + activeRoles := make([]*auth.RoleIdentity, 0) c.Assert(err, IsNil) defer se.Close() mustExec(c, se, "USE MYSQL;") @@ -163,20 +165,20 @@ func (s *testCacheSuite) TestPatternMatch(c *C) { var p privileges.MySQLPrivilege err = p.LoadUserTable(se) c.Assert(err, IsNil) - c.Assert(p.RequestVerification("root", "10.0.1", "test", "", "", mysql.SelectPriv), IsTrue) - c.Assert(p.RequestVerification("root", "10.0.1.118", "test", "", "", mysql.SelectPriv), IsTrue) - c.Assert(p.RequestVerification("root", "localhost", "test", "", "", mysql.SelectPriv), IsFalse) - c.Assert(p.RequestVerification("root", "127.0.0.1", "test", "", "", mysql.SelectPriv), IsFalse) - c.Assert(p.RequestVerification("root", "114.114.114.114", "test", "", "", mysql.SelectPriv), IsFalse) - c.Assert(p.RequestVerification("root", "114.114.114.114", "test", "", "", mysql.PrivilegeType(0)), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "root", "10.0.1", "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "root", "10.0.1.118", "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "root", "localhost", "test", "", "", mysql.SelectPriv), IsFalse) + c.Assert(p.RequestVerification(activeRoles, "root", "127.0.0.1", "test", "", "", mysql.SelectPriv), IsFalse) + c.Assert(p.RequestVerification(activeRoles, "root", "114.114.114.114", "test", "", "", mysql.SelectPriv), IsFalse) + c.Assert(p.RequestVerification(activeRoles, "root", "114.114.114.114", "test", "", "", mysql.PrivilegeType(0)), IsTrue) mustExec(c, se, "TRUNCATE TABLE mysql.user") mustExec(c, se, `INSERT INTO mysql.user VALUES ("", "root", "", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "Y", "N")`) p = privileges.MySQLPrivilege{} err = p.LoadUserTable(se) c.Assert(err, IsNil) - c.Assert(p.RequestVerification("root", "", "test", "", "", mysql.SelectPriv), IsTrue) - c.Assert(p.RequestVerification("root", "notnull", "test", "", "", mysql.SelectPriv), IsFalse) + c.Assert(p.RequestVerification(activeRoles, "root", "", "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "root", "notnull", "test", "", "", mysql.SelectPriv), IsFalse) // Pattern match for DB. mustExec(c, se, "TRUNCATE TABLE mysql.user") @@ -184,11 +186,12 @@ func (s *testCacheSuite) TestPatternMatch(c *C) { mustExec(c, se, `INSERT INTO mysql.db (user,host,db,select_priv) values ('genius', '%', 'te%', 'Y')`) err = p.LoadDBTable(se) c.Assert(err, IsNil) - c.Assert(p.RequestVerification("genius", "127.0.0.1", "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "genius", "127.0.0.1", "test", "", "", mysql.SelectPriv), IsTrue) } func (s *testCacheSuite) TestCaseInsensitive(c *C) { se, err := session.CreateSession4Test(s.store) + activeRoles := make([]*auth.RoleIdentity, 0) c.Assert(err, IsNil) defer se.Close() mustExec(c, se, "CREATE DATABASE TCTrain;") @@ -199,9 +202,9 @@ func (s *testCacheSuite) TestCaseInsensitive(c *C) { err = p.LoadDBTable(se) c.Assert(err, IsNil) // DB and Table names are case insensitive in MySQL. - c.Assert(p.RequestVerification("genius", "127.0.0.1", "TCTrain", "TCTrainOrder", "", mysql.SelectPriv), IsTrue) - c.Assert(p.RequestVerification("genius", "127.0.0.1", "TCTRAIN", "TCTRAINORDER", "", mysql.SelectPriv), IsTrue) - c.Assert(p.RequestVerification("genius", "127.0.0.1", "tctrain", "tctrainorder", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "genius", "127.0.0.1", "TCTrain", "TCTrainOrder", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "genius", "127.0.0.1", "TCTRAIN", "TCTRAINORDER", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "genius", "127.0.0.1", "tctrain", "tctrainorder", "", mysql.SelectPriv), IsTrue) } func (s *testCacheSuite) TestLoadRoleGraph(c *C) { @@ -234,6 +237,40 @@ func (s *testCacheSuite) TestLoadRoleGraph(c *C) { c.Assert(graph["root@%"].Find("r_1", "%"), Equals, false) } +func (s *testCacheSuite) TestRoleGraphBFS(c *C) { + se, err := session.CreateSession4Test(s.store) + c.Assert(err, IsNil) + defer se.Close() + mustExec(c, se, `CREATE ROLE r_1, r_2, r_3, r_4, r_5, r_6;`) + mustExec(c, se, `GRANT r_2 TO r_1;`) + mustExec(c, se, `GRANT r_3 TO r_2;`) + mustExec(c, se, `GRANT r_4 TO r_3;`) + mustExec(c, se, `GRANT r_1 TO r_4;`) + mustExec(c, se, `GRANT r_5 TO r_3, r_6;`) + + var p privileges.MySQLPrivilege + err = p.LoadRoleGraph(se) + c.Assert(err, IsNil) + + activeRoles := make([]*auth.RoleIdentity, 0) + ret := p.FindAllRole(activeRoles) + c.Assert(len(ret), Equals, 0) + activeRoles = append(activeRoles, &auth.RoleIdentity{Username: "r_1", Hostname: "%"}) + ret = p.FindAllRole(activeRoles) + c.Assert(len(ret), Equals, 5) + + activeRoles = make([]*auth.RoleIdentity, 0) + activeRoles = append(activeRoles, &auth.RoleIdentity{Username: "r_6", Hostname: "%"}) + ret = p.FindAllRole(activeRoles) + c.Assert(len(ret), Equals, 2) + + activeRoles = make([]*auth.RoleIdentity, 0) + activeRoles = append(activeRoles, &auth.RoleIdentity{Username: "r_3", Hostname: "%"}) + activeRoles = append(activeRoles, &auth.RoleIdentity{Username: "r_6", Hostname: "%"}) + ret = p.FindAllRole(activeRoles) + c.Assert(len(ret), Equals, 6) +} + func (s *testCacheSuite) TestAbnormalMySQLTable(c *C) { store, err := mockstore.NewMockTikvStore() c.Assert(err, IsNil) @@ -306,8 +343,9 @@ func (s *testCacheSuite) TestAbnormalMySQLTable(c *C) { var p privileges.MySQLPrivilege err = p.LoadUserTable(se) c.Assert(err, IsNil) + activeRoles := make([]*auth.RoleIdentity, 0) // MySQL mysql.user table schema is not identical to TiDB, check it doesn't break privilege. - c.Assert(p.RequestVerification("root", "localhost", "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(p.RequestVerification(activeRoles, "root", "localhost", "test", "", "", mysql.SelectPriv), IsTrue) // Absent of those tables doesn't cause error. mustExec(c, se, "DROP TABLE mysql.db;") diff --git a/privilege/privileges/privileges.go b/privilege/privileges/privileges.go index 25e35b17027e7..3535e8c464ccf 100644 --- a/privilege/privileges/privileges.go +++ b/privilege/privileges/privileges.go @@ -40,7 +40,7 @@ type UserPrivileges struct { } // RequestVerification implements the Manager interface. -func (p *UserPrivileges) RequestVerification(db, table, column string, priv mysql.PrivilegeType) bool { +func (p *UserPrivileges) RequestVerification(activeRoles []*auth.RoleIdentity, db, table, column string, priv mysql.PrivilegeType) bool { if SkipWithGrant { return true } @@ -56,7 +56,7 @@ func (p *UserPrivileges) RequestVerification(db, table, column string, priv mysq } mysqlPriv := p.Handle.Get() - return mysqlPriv.RequestVerification(p.user, p.host, db, table, column, priv) + return mysqlPriv.RequestVerification(activeRoles, p.user, p.host, db, table, column, priv) } // RequestVerificationWithUser implements the Manager interface. @@ -76,7 +76,7 @@ func (p *UserPrivileges) RequestVerificationWithUser(db, table, column string, p } mysqlPriv := p.Handle.Get() - return mysqlPriv.RequestVerification(user.Username, user.Hostname, db, table, column, priv) + return mysqlPriv.RequestVerification(nil, user.Username, user.Hostname, db, table, column, priv) } // GetEncodedPassword implements the Manager interface. diff --git a/privilege/privileges/privileges_test.go b/privilege/privileges/privileges_test.go index 85dc130203fc6..5a53d780652c1 100644 --- a/privilege/privileges/privileges_test.go +++ b/privilege/privileges/privileges_test.go @@ -105,39 +105,80 @@ func (s *testPrivilegeSuite) TearDownTest(c *C) { func (s *testPrivilegeSuite) TestCheckDBPrivilege(c *C) { rootSe := newSession(c, s.store, s.dbName) mustExec(c, rootSe, `CREATE USER 'testcheck'@'localhost';`) + mustExec(c, rootSe, `CREATE USER 'testcheck_tmp'@'localhost';`) se := newSession(c, s.store, s.dbName) + activeRoles := make([]*auth.RoleIdentity, 0) c.Assert(se.Auth(&auth.UserIdentity{Username: "testcheck", Hostname: "localhost"}, nil, nil), IsTrue) pc := privilege.GetPrivilegeManager(se) - c.Assert(pc.RequestVerification("test", "", "", mysql.SelectPriv), IsFalse) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.SelectPriv), IsFalse) mustExec(c, rootSe, `GRANT SELECT ON *.* TO 'testcheck'@'localhost';`) - c.Assert(pc.RequestVerification("test", "", "", mysql.SelectPriv), IsTrue) - c.Assert(pc.RequestVerification("test", "", "", mysql.UpdatePriv), IsFalse) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.UpdatePriv), IsFalse) mustExec(c, rootSe, `GRANT Update ON test.* TO 'testcheck'@'localhost';`) - c.Assert(pc.RequestVerification("test", "", "", mysql.UpdatePriv), IsTrue) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.UpdatePriv), IsTrue) + + activeRoles = append(activeRoles, &auth.RoleIdentity{Username: "testcheck", Hostname: "localhost"}) + mustExec(c, rootSe, `GRANT 'testcheck'@'localhost' TO 'testcheck_tmp'@'localhost';`) + se2 := newSession(c, s.store, s.dbName) + c.Assert(se2.Auth(&auth.UserIdentity{Username: "testcheck_tmp", Hostname: "localhost"}, nil, nil), IsTrue) + pc = privilege.GetPrivilegeManager(se2) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.UpdatePriv), IsTrue) } func (s *testPrivilegeSuite) TestCheckTablePrivilege(c *C) { rootSe := newSession(c, s.store, s.dbName) mustExec(c, rootSe, `CREATE USER 'test1'@'localhost';`) + mustExec(c, rootSe, `CREATE USER 'test1_tmp'@'localhost';`) se := newSession(c, s.store, s.dbName) + activeRoles := make([]*auth.RoleIdentity, 0) c.Assert(se.Auth(&auth.UserIdentity{Username: "test1", Hostname: "localhost"}, nil, nil), IsTrue) pc := privilege.GetPrivilegeManager(se) - c.Assert(pc.RequestVerification("test", "test", "", mysql.SelectPriv), IsFalse) + c.Assert(pc.RequestVerification(activeRoles, "test", "test", "", mysql.SelectPriv), IsFalse) mustExec(c, rootSe, `GRANT SELECT ON *.* TO 'test1'@'localhost';`) - c.Assert(pc.RequestVerification("test", "test", "", mysql.SelectPriv), IsTrue) - c.Assert(pc.RequestVerification("test", "test", "", mysql.UpdatePriv), IsFalse) + c.Assert(pc.RequestVerification(activeRoles, "test", "test", "", mysql.SelectPriv), IsTrue) + c.Assert(pc.RequestVerification(activeRoles, "test", "test", "", mysql.UpdatePriv), IsFalse) mustExec(c, rootSe, `GRANT Update ON test.* TO 'test1'@'localhost';`) - c.Assert(pc.RequestVerification("test", "test", "", mysql.UpdatePriv), IsTrue) - c.Assert(pc.RequestVerification("test", "test", "", mysql.IndexPriv), IsFalse) + c.Assert(pc.RequestVerification(activeRoles, "test", "test", "", mysql.UpdatePriv), IsTrue) + c.Assert(pc.RequestVerification(activeRoles, "test", "test", "", mysql.IndexPriv), IsFalse) + + activeRoles = append(activeRoles, &auth.RoleIdentity{Username: "test1", Hostname: "localhost"}) + se2 := newSession(c, s.store, s.dbName) + mustExec(c, rootSe, `GRANT 'test1'@'localhost' TO 'test1_tmp'@'localhost';`) + c.Assert(se2.Auth(&auth.UserIdentity{Username: "test1_tmp", Hostname: "localhost"}, nil, nil), IsTrue) + pc2 := privilege.GetPrivilegeManager(se2) + c.Assert(pc2.RequestVerification(activeRoles, "test", "test", "", mysql.SelectPriv), IsTrue) + c.Assert(pc2.RequestVerification(activeRoles, "test", "test", "", mysql.UpdatePriv), IsTrue) + c.Assert(pc2.RequestVerification(activeRoles, "test", "test", "", mysql.IndexPriv), IsFalse) mustExec(c, rootSe, `GRANT Index ON test.test TO 'test1'@'localhost';`) - c.Assert(pc.RequestVerification("test", "test", "", mysql.IndexPriv), IsTrue) + c.Assert(pc.RequestVerification(activeRoles, "test", "test", "", mysql.IndexPriv), IsTrue) + c.Assert(pc2.RequestVerification(activeRoles, "test", "test", "", mysql.IndexPriv), IsTrue) +} + +func (s *testPrivilegeSuite) TestCheckPrivilegeWithRoles(c *C) { + rootSe := newSession(c, s.store, s.dbName) + mustExec(c, rootSe, `CREATE USER 'test_role'@'localhost';`) + mustExec(c, rootSe, `CREATE ROLE r_1, r_2, r_3;`) + mustExec(c, rootSe, `GRANT r_1, r_2, r_3 TO 'test_role'@'localhost';`) + + se := newSession(c, s.store, s.dbName) + c.Assert(se.Auth(&auth.UserIdentity{Username: "test_role", Hostname: "localhost"}, nil, nil), IsTrue) + mustExec(c, se, `SET ROLE r_1, r_2;`) + + mustExec(c, rootSe, `GRANT SELECT ON test.* TO r_1;`) + pc := privilege.GetPrivilegeManager(se) + activeRoles := se.GetSessionVars().ActiveRoles + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.SelectPriv), IsTrue) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.UpdatePriv), IsFalse) + mustExec(c, rootSe, `GRANT UPDATE ON test.* TO r_2;`) + c.Assert(pc.RequestVerification(activeRoles, "test", "", "", mysql.UpdatePriv), IsTrue) } func (s *testPrivilegeSuite) TestShowGrants(c *C) { @@ -445,6 +486,13 @@ func (s *testPrivilegeSuite) TestCreateDropUser(c *C) { c.Assert(se.Auth(&auth.UserIdentity{Username: "tcd2", Hostname: "localhost", AuthUsername: "tcd2", AuthHostname: "%"}, nil, nil), IsTrue) mustExec(c, se, `DROP USER tcd1`) mustExec(c, se, `CREATE USER tcd1`) + + // should pass + mustExec(c, se, `GRANT tcd2 TO tcd1`) + c.Assert(se.Auth(&auth.UserIdentity{Username: "tcd1", Hostname: "localhost", AuthUsername: "tcd1", AuthHostname: "%"}, nil, nil), IsTrue) + mustExec(c, se, `SET ROLE tcd2;`) + mustExec(c, se, `CREATE USER tcd3`) + mustExec(c, se, `DROP USER tcd3`) } func (s *testPrivilegeSuite) TestShowCreateTable(c *C) { From 13b8d4884dc2770f0e6c934abddaa225b52ebaa9 Mon Sep 17 00:00:00 2001 From: Lonng Date: Tue, 23 Apr 2019 10:44:26 +0800 Subject: [PATCH 17/21] go.mod: update failpoint to latest version (#10235) Signed-off-by: Lonng --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index d6140988c2326..9ac0eaa65e863 100644 --- a/go.mod +++ b/go.mod @@ -47,7 +47,7 @@ require ( github.com/philhofer/fwd v1.0.0 // indirect github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8 github.com/pingcap/errors v0.11.1 - github.com/pingcap/failpoint v0.0.0-20190418063555-9c3d850fd17c + github.com/pingcap/failpoint v0.0.0-20190422094118-d8535965f59b github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 diff --git a/go.sum b/go.sum index 16924fb8faad6..a284ae14f56b8 100644 --- a/go.sum +++ b/go.sum @@ -116,8 +116,8 @@ github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuM github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.1 h1:BXFZ6MdDd2U1uJUa2sRAWTmm+nieEzuyYM0R4aUTcC8= github.com/pingcap/errors v0.11.1/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/failpoint v0.0.0-20190418063555-9c3d850fd17c h1:orH/u6aM2R59OoqzGYViSrNEnnWAS6G3gObq74zriS8= -github.com/pingcap/failpoint v0.0.0-20190418063555-9c3d850fd17c/go.mod h1:fdAkVXuIXHAPZ7a280nj9bRORfK9NuSsOguvBH0+W6c= +github.com/pingcap/failpoint v0.0.0-20190422094118-d8535965f59b h1:gFQVlQbthX4C2WKV/zqGBF3bZFr7oceKK9jGOVNkfws= +github.com/pingcap/failpoint v0.0.0-20190422094118-d8535965f59b/go.mod h1:fdAkVXuIXHAPZ7a280nj9bRORfK9NuSsOguvBH0+W6c= github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e h1:P73/4dPCL96rGrobssy1nVy2VaVpNCuLpCbr+FEaTA8= github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20190215154024-7f2fc73ef562 h1:32oF1/8lVnBR2JVcCAnKPQATTOX0+ckRDFpjQk4Ngno= From 9c8396e789dfa8681ae290a3f1f96cdedffc320e Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 23 Apr 2019 11:08:22 +0800 Subject: [PATCH 18/21] metrics,store/tikv: add a duration metric for tikv batch client unavailable (#10212) --- metrics/tikvclient.go | 10 ++++++++++ store/tikv/client.go | 2 ++ 2 files changed, 12 insertions(+) diff --git a/metrics/tikvclient.go b/metrics/tikvclient.go index 4a530c0512ae2..1955399698eb1 100644 --- a/metrics/tikvclient.go +++ b/metrics/tikvclient.go @@ -206,4 +206,14 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 30), Help: "batch wait duration", }) + + TiKVBatchClientUnavailable = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "tikvclient", + Name: "batch_client_unavailable_seconds", + // 1ms ~ 1000s + Buckets: prometheus.ExponentialBuckets(0.001, 2, 20), + Help: "batch client unavailable", + }) ) diff --git a/store/tikv/client.go b/store/tikv/client.go index b514525ddd435..3a52a5f67de09 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -131,6 +131,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) { } logutil.Logger(context.Background()).Error("batchRecvLoop error when receive", zap.Error(err)) + now := time.Now() for { // try to re-create the streaming in the loop. // Hold the lock to forbid batchSendLoop using the old client. c.clientLock.Lock() @@ -150,6 +151,7 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) { // TODO: Use a more smart backoff strategy. time.Sleep(time.Second) } + metrics.TiKVBatchClientUnavailable.Observe(time.Since(now).Seconds()) continue } From 5691ff3d7f84b630376dc3443997313792307d20 Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 23 Apr 2019 17:18:58 +0800 Subject: [PATCH 19/21] ddl: spead up ddl unit test (#10234) --- ddl/db_integration_test.go | 136 +++++++++++++++++++------------ ddl/db_partition_test.go | 34 ++++---- ddl/db_test.go | 161 +++++++++++++++++++++++-------------- 3 files changed, 204 insertions(+), 127 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 8ab7b9fbdca20..171a5128b27b4 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -46,7 +46,17 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -var _ = Suite(&testIntegrationSuite{}) +var _ = Suite(&testIntegrationSuite1{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite2{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite3{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite4{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite5{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite6{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite7{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite8{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite9{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite10{&testIntegrationSuite{}}) +var _ = Suite(&testIntegrationSuite11{&testIntegrationSuite{}}) type testIntegrationSuite struct { lease time.Duration @@ -58,19 +68,10 @@ type testIntegrationSuite struct { tk *testkit.TestKit } -func (s *testIntegrationSuite) TearDownTest(c *C) { - tk := testkit.NewTestKit(c, s.store) - tk.MustExec("use test") - r := tk.MustQuery("show tables") - for _, tb := range r.Rows() { - tableName := tb[0] - tk.MustExec(fmt.Sprintf("drop table %v", tableName)) - } -} - -func (s *testIntegrationSuite) SetUpSuite(c *C) { +func setupIntegrationSuite(s *testIntegrationSuite, c *C) { var err error s.lease = 50 * time.Millisecond + ddl.WaitTimeWhenErrorOccured = 0 s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) @@ -93,12 +94,47 @@ func (s *testIntegrationSuite) SetUpSuite(c *C) { s.tk = testkit.NewTestKit(c, s.store) } -func (s *testIntegrationSuite) TearDownSuite(c *C) { +func tearDownIntegrationSuiteTest(s *testIntegrationSuite, c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show tables") + for _, tb := range r.Rows() { + tableName := tb[0] + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } +} + +func tearDownIntegrationSuite(s *testIntegrationSuite, c *C) { s.dom.Close() s.store.Close() } -func (s *testIntegrationSuite) TestNoZeroDateMode(c *C) { +func (s *testIntegrationSuite) SetUpSuite(c *C) { + setupIntegrationSuite(s, c) +} + +func (s *testIntegrationSuite) TearDownSuite(c *C) { + tearDownIntegrationSuite(s, c) +} + +type testIntegrationSuite1 struct{ *testIntegrationSuite } +type testIntegrationSuite2 struct{ *testIntegrationSuite } + +func (s *testIntegrationSuite2) TearDownTest(c *C) { + tearDownIntegrationSuiteTest(s.testIntegrationSuite, c) +} + +type testIntegrationSuite3 struct{ *testIntegrationSuite } +type testIntegrationSuite4 struct{ *testIntegrationSuite } +type testIntegrationSuite5 struct{ *testIntegrationSuite } +type testIntegrationSuite6 struct{ *testIntegrationSuite } +type testIntegrationSuite7 struct{ *testIntegrationSuite } +type testIntegrationSuite8 struct{ *testIntegrationSuite } +type testIntegrationSuite9 struct{ *testIntegrationSuite } +type testIntegrationSuite10 struct{ *testIntegrationSuite } +type testIntegrationSuite11 struct{ *testIntegrationSuite } + +func (s *testIntegrationSuite6) TestNoZeroDateMode(c *C) { tk := testkit.NewTestKit(c, s.store) defer tk.MustExec("set session sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION';") @@ -112,7 +148,7 @@ func (s *testIntegrationSuite) TestNoZeroDateMode(c *C) { assertErrorCode(c, tk, "create table test_zero_date(a timestamp default 0);", mysql.ErrInvalidDefault) } -func (s *testIntegrationSuite) TestInvalidDefault(c *C) { +func (s *testIntegrationSuite7) TestInvalidDefault(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") @@ -127,7 +163,7 @@ func (s *testIntegrationSuite) TestInvalidDefault(c *C) { } // TestInvalidNameWhenCreateTable for issue #3848 -func (s *testIntegrationSuite) TestInvalidNameWhenCreateTable(c *C) { +func (s *testIntegrationSuite8) TestInvalidNameWhenCreateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") @@ -146,7 +182,7 @@ func (s *testIntegrationSuite) TestInvalidNameWhenCreateTable(c *C) { } // TestCreateTableIfNotExists for issue #6879 -func (s *testIntegrationSuite) TestCreateTableIfNotExists(c *C) { +func (s *testIntegrationSuite3) TestCreateTableIfNotExists(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") @@ -171,7 +207,7 @@ func (s *testIntegrationSuite) TestCreateTableIfNotExists(c *C) { } // for issue #9910 -func (s *testIntegrationSuite) TestCreateTableWithKeyWord(c *C) { +func (s *testIntegrationSuite2) TestCreateTableWithKeyWord(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test;") @@ -180,7 +216,7 @@ func (s *testIntegrationSuite) TestCreateTableWithKeyWord(c *C) { c.Assert(err, IsNil) } -func (s *testIntegrationSuite) TestUniqueKeyNullValue(c *C) { +func (s *testIntegrationSuite1) TestUniqueKeyNullValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test") tk.MustExec("create table t(a int primary key, b varchar(255))") @@ -194,7 +230,7 @@ func (s *testIntegrationSuite) TestUniqueKeyNullValue(c *C) { tk.MustExec("admin check index t b") } -func (s *testIntegrationSuite) TestEndIncluded(c *C) { +func (s *testIntegrationSuite4) TestEndIncluded(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test") @@ -208,7 +244,7 @@ func (s *testIntegrationSuite) TestEndIncluded(c *C) { } // TestModifyColumnAfterAddIndex Issue 5134 -func (s *testIntegrationSuite) TestModifyColumnAfterAddIndex(c *C) { +func (s *testIntegrationSuite3) TestModifyColumnAfterAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table city (city VARCHAR(2) KEY);") @@ -216,7 +252,7 @@ func (s *testIntegrationSuite) TestModifyColumnAfterAddIndex(c *C) { tk.MustExec(`insert into city values ("abc"), ("abd");`) } -func (s *testIntegrationSuite) TestIssue2293(c *C) { +func (s *testIntegrationSuite9) TestIssue2293(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t_issue_2293 (a int)") @@ -226,7 +262,7 @@ func (s *testIntegrationSuite) TestIssue2293(c *C) { tk.MustQuery("select * from t_issue_2293").Check(testkit.Rows("1")) } -func (s *testIntegrationSuite) TestIssue6101(c *C) { +func (s *testIntegrationSuite2) TestIssue6101(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t1 (quantity decimal(2) unsigned);") @@ -242,7 +278,7 @@ func (s *testIntegrationSuite) TestIssue6101(c *C) { tk.MustExec("drop table t1") } -func (s *testIntegrationSuite) TestIssue3833(c *C) { +func (s *testIntegrationSuite4) TestIssue3833(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table issue3833 (b char(0))") @@ -251,7 +287,7 @@ func (s *testIntegrationSuite) TestIssue3833(c *C) { assertErrorCode(c, tk, "create table issue3833_2 (b char(0), index (b))", tmysql.ErrWrongKeyColumn) } -func (s *testIntegrationSuite) TestIssue2858And2717(c *C) { +func (s *testIntegrationSuite10) TestIssue2858And2717(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -268,7 +304,7 @@ func (s *testIntegrationSuite) TestIssue2858And2717(c *C) { tk.MustExec(`alter table t_issue_2858_hex alter column a set default 0x321`) } -func (s *testIntegrationSuite) TestIssue4432(c *C) { +func (s *testIntegrationSuite1) TestIssue4432(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -293,7 +329,7 @@ func (s *testIntegrationSuite) TestIssue4432(c *C) { tk.MustExec("drop table tx") } -func (s *testIntegrationSuite) TestMySQLErrorCode(c *C) { +func (s *testIntegrationSuite5) TestMySQLErrorCode(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test_db") @@ -410,7 +446,7 @@ func (s *testIntegrationSuite) TestMySQLErrorCode(c *C) { assertErrorCode(c, tk, sql, tmysql.ErrBadNull) } -func (s *testIntegrationSuite) TestTableDDLWithFloatType(c *C) { +func (s *testIntegrationSuite9) TestTableDDLWithFloatType(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists t") @@ -425,7 +461,7 @@ func (s *testIntegrationSuite) TestTableDDLWithFloatType(c *C) { s.tk.MustExec("drop table t") } -func (s *testIntegrationSuite) TestTableDDLWithTimeType(c *C) { +func (s *testIntegrationSuite10) TestTableDDLWithTimeType(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists t") @@ -448,7 +484,7 @@ func (s *testIntegrationSuite) TestTableDDLWithTimeType(c *C) { s.tk.MustExec("drop table t") } -func (s *testIntegrationSuite) TestUpdateMultipleTable(c *C) { +func (s *testIntegrationSuite2) TestUpdateMultipleTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database umt_db") tk.MustExec("use umt_db") @@ -507,7 +543,7 @@ func (s *testIntegrationSuite) TestUpdateMultipleTable(c *C) { tk.MustExec("drop database umt_db") } -func (s *testIntegrationSuite) TestNullGeneratedColumn(c *C) { +func (s *testIntegrationSuite7) TestNullGeneratedColumn(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -525,7 +561,7 @@ func (s *testIntegrationSuite) TestNullGeneratedColumn(c *C) { tk.MustExec("drop table t") } -func (s *testIntegrationSuite) TestChangingCharsetToUtf8(c *C) { +func (s *testIntegrationSuite9) TestChangingCharsetToUtf8(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -571,7 +607,7 @@ func (s *testIntegrationSuite) TestChangingCharsetToUtf8(c *C) { c.Assert(err, NotNil) } -func (s *testIntegrationSuite) TestChangingTableCharset(c *C) { +func (s *testIntegrationSuite10) TestChangingTableCharset(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("USE test") @@ -704,7 +740,7 @@ func (s *testIntegrationSuite) TestChangingTableCharset(c *C) { checkCharset() } -func (s *testIntegrationSuite) TestCaseInsensitiveCharsetAndCollate(c *C) { +func (s *testIntegrationSuite7) TestCaseInsensitiveCharsetAndCollate(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test_charset_collate") @@ -717,7 +753,7 @@ func (s *testIntegrationSuite) TestCaseInsensitiveCharsetAndCollate(c *C) { tk.MustExec("create table t4(id int) ENGINE=InnoDB DEFAULT CHARSET=Utf8mb4 COLLATE=utf8MB4_general_ci;") } -func (s *testIntegrationSuite) TestZeroFillCreateTable(c *C) { +func (s *testIntegrationSuite3) TestZeroFillCreateTable(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists abc;") @@ -742,7 +778,7 @@ func (s *testIntegrationSuite) TestZeroFillCreateTable(c *C) { c.Assert(mysql.HasUnsignedFlag(zCol.Flag), IsTrue) } -func (s *testIntegrationSuite) TestBitDefaultValue(c *C) { +func (s *testIntegrationSuite6) TestBitDefaultValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t_bit (c1 bit(10) default 250, c2 int);") @@ -792,7 +828,7 @@ func (s *testIntegrationSuite) TestBitDefaultValue(c *C) { );`) } -func (s *testIntegrationSuite) TestBackwardCompatibility(c *C) { +func (s *testIntegrationSuite5) TestBackwardCompatibility(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists test_backward_compatibility") defer tk.MustExec("drop database test_backward_compatibility") @@ -867,7 +903,7 @@ func (s *testIntegrationSuite) TestBackwardCompatibility(c *C) { tk.MustExec("admin check index t idx_b") } -func (s *testIntegrationSuite) TestMultiRegionGetTableEndHandle(c *C) { +func (s *testIntegrationSuite4) TestMultiRegionGetTableEndHandle(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") tk.MustExec("create database test_get_endhandle") @@ -938,7 +974,7 @@ func (s *testIntegrationSuite) checkGetMaxTableRowID(ctx *testMaxTableRowIDConte c.Assert(maxID, Equals, expectMaxID) } -func (s *testIntegrationSuite) TestGetTableEndHandle(c *C) { +func (s *testIntegrationSuite6) TestGetTableEndHandle(c *C) { // TestGetTableEndHandle test ddl.GetTableMaxRowID method, which will return the max row id of the table. tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_get_endhandle") @@ -1034,7 +1070,7 @@ func (s *testIntegrationSuite) getHistoryDDLJob(id int64) (*model.Job, error) { return job, errors.Trace(err) } -func (s *testIntegrationSuite) TestCreateTableTooLarge(c *C) { +func (s *testIntegrationSuite1) TestCreateTableTooLarge(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") @@ -1056,7 +1092,7 @@ func (s *testIntegrationSuite) TestCreateTableTooLarge(c *C) { atomic.StoreUint32(&ddl.TableColumnCountLimit, originLimit) } -func (s *testIntegrationSuite) TestChangeColumnPosition(c *C) { +func (s *testIntegrationSuite8) TestChangeColumnPosition(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") @@ -1107,7 +1143,7 @@ func (s *testIntegrationSuite) TestChangeColumnPosition(c *C) { c.Assert(createSQL, Equals, strings.Join(exceptedSQL, "\n")) } -func (s *testIntegrationSuite) TestAddIndexAfterAddColumn(c *C) { +func (s *testIntegrationSuite2) TestAddIndexAfterAddColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") @@ -1120,7 +1156,7 @@ func (s *testIntegrationSuite) TestAddIndexAfterAddColumn(c *C) { assertErrorCode(c, s.tk, sql, tmysql.ErrTooManyKeyParts) } -func (s *testIntegrationSuite) TestResolveCharset(c *C) { +func (s *testIntegrationSuite8) TestResolveCharset(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists resolve_charset") @@ -1150,7 +1186,7 @@ func (s *testIntegrationSuite) TestResolveCharset(c *C) { c.Assert(tbl.Meta().Charset, Equals, "binary") } -func (s *testIntegrationSuite) TestAddAnonymousIndex(c *C) { +func (s *testIntegrationSuite2) TestAddAnonymousIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("create table t_anonymous_index (c1 int, c2 int, C3 int)") @@ -1200,7 +1236,7 @@ func (s *testIntegrationSuite) TestAddAnonymousIndex(c *C) { c.Assert(t.Indices()[1].Meta().Name.String(), Equals, "primary_3") } -func (s *testIntegrationSuite) TestAddColumnTooMany(c *C) { +func (s *testIntegrationSuite1) TestAddColumnTooMany(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") count := int(atomic.LoadUint32(&ddl.TableColumnCountLimit) - 1) @@ -1215,7 +1251,7 @@ func (s *testIntegrationSuite) TestAddColumnTooMany(c *C) { assertErrorCode(c, s.tk, alterSQL, tmysql.ErrTooManyFields) } -func (s *testIntegrationSuite) TestAlterColumn(c *C) { +func (s *testIntegrationSuite4) TestAlterColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test_db") @@ -1333,7 +1369,7 @@ func (s *testIntegrationSuite) assertAlterErrorExec(c *C, sql string) { assertErrorCode(c, s.tk, sql, mysql.ErrAlterOperationNotSupportedReason) } -func (s *testIntegrationSuite) TestAlterAlgorithm(c *C) { +func (s *testIntegrationSuite3) TestAlterAlgorithm(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists t, t1") @@ -1397,7 +1433,7 @@ func (s *testIntegrationSuite) TestAlterAlgorithm(c *C) { s.tk.MustExec("alter table t default charset = utf8mb4, ALGORITHM=INSTANT") } -func (s *testIntegrationSuite) TestFulltextIndexIgnore(c *C) { +func (s *testIntegrationSuite5) TestFulltextIndexIgnore(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists t_ft") @@ -1413,7 +1449,7 @@ func (s *testIntegrationSuite) TestFulltextIndexIgnore(c *C) { c.Assert(r.Rows(), HasLen, 0) } -func (s *testIntegrationSuite) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { +func (s *testIntegrationSuite1) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists t") @@ -1548,7 +1584,7 @@ func (s *testIntegrationSuite) TestTreatOldVersionUTF8AsUTF8MB4(c *C) { ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) } -func (s *testIntegrationSuite) TestDefaultValueIsString(c *C) { +func (s *testIntegrationSuite3) TestDefaultValueIsString(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists t") diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index 60e07b6dff99a..3ebab9da7b640 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -42,7 +42,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testIntegrationSuite) TestCreateTableWithPartition(c *C) { +func (s *testIntegrationSuite9) TestCreateTableWithPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists tp;") @@ -232,7 +232,7 @@ func (s *testIntegrationSuite) TestCreateTableWithPartition(c *C) { assertErrorCode(c, tk, `create table t33 (a int, b int) partition by hash(a) partitions 0;`, tmysql.ErrNoParts) } -func (s *testIntegrationSuite) TestCreateTableWithHashPartition(c *C) { +func (s *testIntegrationSuite7) TestCreateTableWithHashPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists employees;") @@ -263,7 +263,7 @@ func (s *testIntegrationSuite) TestCreateTableWithHashPartition(c *C) { partition by hash( year(hired) ) partitions 4;`) } -func (s *testIntegrationSuite) TestCreateTableWithRangeColumnPartition(c *C) { +func (s *testIntegrationSuite10) TestCreateTableWithRangeColumnPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists log_message_1;") @@ -363,7 +363,7 @@ create table log_message_1 ( "partition p1 values less than (2, maxvalue))") } -func (s *testIntegrationSuite) TestCreateTableWithKeyPartition(c *C) { +func (s *testIntegrationSuite8) TestCreateTableWithKeyPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists tm1;") @@ -374,7 +374,7 @@ func (s *testIntegrationSuite) TestCreateTableWithKeyPartition(c *C) { partition by key(s1) partitions 10;`) } -func (s *testIntegrationSuite) TestAlterTableAddPartition(c *C) { +func (s *testIntegrationSuite5) TestAlterTableAddPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists employees;") @@ -481,7 +481,7 @@ func (s *testIntegrationSuite) TestAlterTableAddPartition(c *C) { assertErrorCode(c, tk, sql7, tmysql.ErrSameNamePartition) } -func (s *testIntegrationSuite) TestAlterTableDropPartition(c *C) { +func (s *testIntegrationSuite6) TestAlterTableDropPartition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists employees") @@ -618,7 +618,7 @@ func (s *testIntegrationSuite) TestAlterTableDropPartition(c *C) { assertErrorCode(c, tk, sql4, tmysql.ErrDropPartitionNonExistent) } -func (s *testIntegrationSuite) TestAddPartitionTooManyPartitions(c *C) { +func (s *testIntegrationSuite11) TestAddPartitionTooManyPartitions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") count := ddl.PartitionCountLimit @@ -645,7 +645,7 @@ func (s *testIntegrationSuite) TestAddPartitionTooManyPartitions(c *C) { tk.MustExec(sql2) sql3 := `alter table p2 add partition ( - partition p1025 values less than (1025) + partition p1025 values less than (1025) );` assertErrorCode(c, tk, sql3, tmysql.ErrTooManyPartitions) } @@ -675,7 +675,7 @@ func checkPartitionDelRangeDone(c *C, s *testIntegrationSuite, partitionPrefix k return hasOldPartitionData } -func (s *testIntegrationSuite) TestTruncatePartitionAndDropTable(c *C) { +func (s *testIntegrationSuite6) TestTruncatePartitionAndDropTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") // Test truncate common table. @@ -736,7 +736,7 @@ func (s *testIntegrationSuite) TestTruncatePartitionAndDropTable(c *C) { oldPID := oldTblInfo.Meta().Partition.Definitions[0].ID tk.MustExec("truncate table t3;") partitionPrefix := tablecodec.EncodeTablePrefix(oldPID) - hasOldPartitionData := checkPartitionDelRangeDone(c, s, partitionPrefix) + hasOldPartitionData := checkPartitionDelRangeDone(c, s.testIntegrationSuite, partitionPrefix) c.Assert(hasOldPartitionData, IsFalse) // Test drop table partition. @@ -773,7 +773,7 @@ func (s *testIntegrationSuite) TestTruncatePartitionAndDropTable(c *C) { oldPID = oldTblInfo.Meta().Partition.Definitions[1].ID tk.MustExec("drop table t4;") partitionPrefix = tablecodec.EncodeTablePrefix(oldPID) - hasOldPartitionData = checkPartitionDelRangeDone(c, s, partitionPrefix) + hasOldPartitionData = checkPartitionDelRangeDone(c, s.testIntegrationSuite, partitionPrefix) c.Assert(hasOldPartitionData, IsFalse) assertErrorCode(c, tk, "select * from t4;", tmysql.ErrNoSuchTable) @@ -830,7 +830,7 @@ func (s *testIntegrationSuite) TestTruncatePartitionAndDropTable(c *C) { } } -func (s *testIntegrationSuite) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { +func (s *testIntegrationSuite5) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") tk.MustExec("drop table if exists part1;") @@ -1033,7 +1033,7 @@ func (s *testIntegrationSuite) TestPartitionUniqueKeyNeedAllFieldsInPf(c *C) { assertErrorCode(c, tk, sql9, tmysql.ErrUniqueKeyNeedAllFieldsInPf) } -func (s *testIntegrationSuite) TestPartitionDropIndex(c *C) { +func (s *testIntegrationSuite3) TestPartitionDropIndex(c *C) { tk := testkit.NewTestKit(c, s.store) done := make(chan error, 1) tk.MustExec("use test_db") @@ -1113,7 +1113,7 @@ LOOP: tk.MustExec("drop table partition_drop_idx;") } -func (s *testIntegrationSuite) TestPartitionCancelAddIndex(c *C) { +func (s *testIntegrationSuite2) TestPartitionCancelAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test_db") @@ -1251,7 +1251,7 @@ func backgroundExecOnJobUpdatedExported(c *C, store kv.Storage, ctx sessionctx.C return hook.OnJobUpdatedExported, c3IdxInfo, checkErr } -func (s *testIntegrationSuite) TestPartitionAddIndex(c *C) { +func (s *testIntegrationSuite1) TestPartitionAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table partition_add_idx ( @@ -1305,7 +1305,7 @@ func testPartitionAddIndex(tk *testkit.TestKit, c *C) { tk.MustExec("drop table partition_add_idx") } -func (s *testIntegrationSuite) TestDropSchemaWithPartitionTable(c *C) { +func (s *testIntegrationSuite5) TestDropSchemaWithPartitionTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("drop database if exists test_db_with_partition") tk.MustExec("create database test_db_with_partition") @@ -1375,7 +1375,7 @@ func getPartitionTableRecordsNum(c *C, ctx sessionctx.Context, tbl table.Partiti return num } -func (s *testIntegrationSuite) TestPartitionErrorCode(c *C) { +func (s *testIntegrationSuite4) TestPartitionErrorCode(c *C) { tk := testkit.NewTestKit(c, s.store) // add partition tk.MustExec("set @@session.tidb_enable_table_partition = 1") diff --git a/ddl/db_test.go b/ddl/db_test.go index d7e00e1381491..c5867b6648576 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -58,9 +58,16 @@ const ( waitForCleanDataInterval = time.Millisecond * 100 ) -var _ = Suite(&testDBSuite{}) +var _ = Suite(&testDBSuite1{&testDBSuite{}}) +var _ = Suite(&testDBSuite2{&testDBSuite{}}) +var _ = Suite(&testDBSuite3{&testDBSuite{}}) +var _ = Suite(&testDBSuite4{&testDBSuite{}}) +var _ = Suite(&testDBSuite5{&testDBSuite{}}) +var _ = Suite(&testDBSuite6{&testDBSuite{}}) +var _ = Suite(&testDBSuite7{&testDBSuite{}}) +var _ = Suite(&testDBSuite8{&testDBSuite{}}) -const defaultBatchSize = 2048 +const defaultBatchSize = 1024 type testDBSuite struct { cluster *mocktikv.Cluster @@ -74,15 +81,15 @@ type testDBSuite struct { autoIDStep int64 } -func (s *testDBSuite) SetUpSuite(c *C) { +func setUpSuite(s *testDBSuite, c *C) { var err error - s.lease = 200 * time.Millisecond + s.lease = 100 * time.Millisecond session.SetSchemaLease(s.lease) session.SetStatsLease(0) s.schemaName = "test_db" s.autoIDStep = autoid.GetStep() - ddl.WaitTimeWhenErrorOccured = 1 * time.Microsecond + ddl.WaitTimeWhenErrorOccured = 0 s.cluster = mocktikv.NewCluster() mocktikv.BootstrapWithSingleStore(s.cluster) @@ -104,13 +111,30 @@ func (s *testDBSuite) SetUpSuite(c *C) { s.tk = testkit.NewTestKit(c, s.store) } -func (s *testDBSuite) TearDownSuite(c *C) { +func tearDownSuite(s *testDBSuite, c *C) { s.s.Execute(context.Background(), "drop database if exists test_db") s.s.Close() s.dom.Close() s.store.Close() } +func (s *testDBSuite) SetUpSuite(c *C) { + setUpSuite(s, c) +} + +func (s *testDBSuite) TearDownSuite(c *C) { + tearDownSuite(s, c) +} + +type testDBSuite1 struct{ *testDBSuite } +type testDBSuite2 struct{ *testDBSuite } +type testDBSuite3 struct{ *testDBSuite } +type testDBSuite4 struct{ *testDBSuite } +type testDBSuite5 struct{ *testDBSuite } +type testDBSuite6 struct{ *testDBSuite } +type testDBSuite7 struct{ *testDBSuite } +type testDBSuite8 struct{ *testDBSuite } + func assertErrorCode(c *C, tk *testkit.TestKit, sql string, errCode int) { _, err := tk.Exec(sql) c.Assert(err, NotNil) @@ -120,7 +144,7 @@ func assertErrorCode(c *C, tk *testkit.TestKit, sql string, errCode int) { c.Assert(tErr.ToSQLError().Code, DeepEquals, uint16(errCode), Commentf("MySQL code:%v", tErr.ToSQLError())) } -func (s *testDBSuite) TestAddIndexWithPK(c *C) { +func (s *testDBSuite6) TestAddIndexWithPK(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -143,7 +167,7 @@ func (s *testDBSuite) TestAddIndexWithPK(c *C) { s.tk.MustQuery("select * from test_add_index_with_pk2").Check(testkit.Rows("1 1 1 1", "2 2 2 2")) } -func (s *testDBSuite) TestRenameIndex(c *C) { +func (s *testDBSuite1) TestRenameIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) s.tk.MustExec("create table t (pk int primary key, c int default 1, c1 int default 1, unique key k1(c), key k2(c1))") @@ -197,7 +221,7 @@ func backgroundExec(s kv.Storage, sql string, done chan error) { done <- errors.Trace(err) } -func (s *testDBSuite) TestAddUniqueIndexRollback(c *C) { +func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t1") @@ -256,7 +280,7 @@ LOOP: s.mustExec(c, "drop table t1") } -func (s *testDBSuite) TestCancelAddIndex(c *C) { +func (s *testDBSuite3) TestCancelAddIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t1") @@ -322,7 +346,7 @@ LOOP: } // TestCancelAddIndex1 tests canceling ddl job when the add index worker is not started. -func (s *testDBSuite) TestCancelAddIndex1(c *C) { +func (s *testDBSuite4) TestCancelAddIndex1(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t") @@ -384,7 +408,7 @@ func (s *testDBSuite) TestCancelAddIndex1(c *C) { } // TestCancelDropIndex tests cancel ddl job which type is drop index. -func (s *testDBSuite) TestCancelDropIndex(c *C) { +func (s *testDBSuite5) TestCancelDropIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t") @@ -470,7 +494,7 @@ func (s *testDBSuite) TestCancelDropIndex(c *C) { } // TestCancelTruncateTable tests cancel ddl job which type is truncate table. -func (s *testDBSuite) TestCancelTruncateTable(c *C) { +func (s *testDBSuite7) TestCancelTruncateTable(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "create database if not exists test_truncate_table") @@ -516,7 +540,7 @@ func (s *testDBSuite) TestCancelTruncateTable(c *C) { } // TestCancelRenameIndex tests cancel ddl job which type is rename index. -func (s *testDBSuite) TestCancelRenameIndex(c *C) { +func (s *testDBSuite1) TestCancelRenameIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "create database if not exists test_rename_index") @@ -574,7 +598,7 @@ func (s *testDBSuite) TestCancelRenameIndex(c *C) { } // TestCancelDropTable tests cancel ddl job which type is drop table. -func (s *testDBSuite) TestCancelDropTableAndSchema(c *C) { +func (s *testDBSuite2) TestCancelDropTableAndSchema(c *C) { s.tk = testkit.NewTestKit(c, s.store) testCases := []struct { needAddTableOrDB bool @@ -661,7 +685,7 @@ func (s *testDBSuite) TestCancelDropTableAndSchema(c *C) { } } -func (s *testDBSuite) TestAddAnonymousIndex(c *C) { +func (s *testDBSuite3) TestAddAnonymousIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) s.mustExec(c, "create table t_anonymous_index (c1 int, c2 int, C3 int)") @@ -711,14 +735,14 @@ func (s *testDBSuite) TestAddAnonymousIndex(c *C) { c.Assert(t.Indices()[1].Meta().Name.String(), Equals, "primary_3") } -func (s *testDBSuite) testAlterLock(c *C) { +func (s *testDBSuite4) testAlterLock(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) s.mustExec(c, "create table t_index_lock (c1 int, c2 int, C3 int)") s.mustExec(c, "alter table t_indx_lock add index (c1, c2), lock=none") } -func (s *testDBSuite) TestAddMultiColumnsIndex(c *C) { +func (s *testDBSuite5) TestAddMultiColumnsIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -739,8 +763,11 @@ func (s *testDBSuite) TestAddMultiColumnsIndex(c *C) { s.tk.MustExec("admin check table test") } -func (s *testDBSuite) TestAddIndex(c *C) { +func (s *testDBSuite1) TestAddIndex1(c *C) { s.testAddIndex(c, false, "create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1))") +} + +func (s *testDBSuite2) TestAddIndex2(c *C) { s.testAddIndex(c, true, `create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range (c1) ( partition p0 values less than (3440), @@ -748,8 +775,14 @@ func (s *testDBSuite) TestAddIndex(c *C) { partition p2 values less than (122880), partition p3 values less than (204800), partition p4 values less than maxvalue)`) +} + +func (s *testDBSuite3) TestAddIndex3(c *C) { s.testAddIndex(c, true, `create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by hash (c1) partitions 4;`) +} + +func (s *testDBSuite4) TestAddIndex4(c *C) { s.testAddIndex(c, true, `create table test_add_index (c1 bigint, c2 bigint, c3 bigint, primary key(c1)) partition by range columns (c1) ( partition p0 values less than (3440), @@ -776,6 +809,7 @@ func (s *testDBSuite) testAddIndex(c *C, testPartition bool, createTableSQL stri sql := fmt.Sprintf("insert into test_add_index values (%d, %d, %d)", i, i, i) s.mustExec(c, sql) } + // Add some discrete rows. maxBatch := 20 batchCnt := 100 @@ -919,7 +953,7 @@ LOOP: } // TestCancelAddTableAndDropTablePartition tests cancel ddl job which type is add/drop table partition. -func (s *testDBSuite) TestCancelAddTableAndDropTablePartition(c *C) { +func (s *testDBSuite1) TestCancelAddTableAndDropTablePartition(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "create database if not exists test_partition_table") s.mustExec(c, "use test_partition_table") @@ -1006,7 +1040,7 @@ func (s *testDBSuite) TestCancelAddTableAndDropTablePartition(c *C) { s.dom.DDL().(ddl.DDLForTest).SetHook(originalHook) } -func (s *testDBSuite) TestDropIndex(c *C) { +func (s *testDBSuite2) TestDropIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) s.tk.MustExec("drop table if exists test_drop_index") @@ -1077,7 +1111,7 @@ LOOP: } // TestCancelDropColumn tests cancel ddl job which type is drop column. -func (s *testDBSuite) TestCancelDropColumn(c *C) { +func (s *testDBSuite3) TestCancelDropColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) s.mustExec(c, "drop table if exists test_drop_column") @@ -1201,7 +1235,7 @@ func checkDelRangeDone(c *C, ctx sessionctx.Context, idx table.Index) { c.Assert(handles, HasLen, 0, Commentf("take time %v", time.Since(startTime))) } -func (s *testDBSuite) TestAddIndexWithDupCols(c *C) { +func (s *testDBSuite4) TestAddIndexWithDupCols(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) err1 := infoschema.ErrColumnExists.GenWithStackByArgs("b") @@ -1227,7 +1261,7 @@ func (s *testDBSuite) showColumns(c *C, tableName string) [][]interface{} { return s.mustQuery(c, fmt.Sprintf("show columns from %s", tableName)) } -func (s *testDBSuite) TestCreateIndexType(c *C) { +func (s *testDBSuite5) TestCreateIndexType(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) sql := `CREATE TABLE test_index ( @@ -1242,7 +1276,7 @@ func (s *testDBSuite) TestCreateIndexType(c *C) { s.tk.MustExec(sql) } -func (s *testDBSuite) TestColumn(c *C) { +func (s *testDBSuite8) TestColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) s.tk.MustExec("create table t2 (c1 int, c2 int, c3 int)") @@ -1251,7 +1285,7 @@ func (s *testDBSuite) TestColumn(c *C) { s.tk.MustExec("drop table t2") } -func (s *testDBSuite) TestAddColumnTooMany(c *C) { +func (s *testDBSuite1) TestAddColumnTooMany(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") count := int(atomic.LoadUint32(&ddl.TableColumnCountLimit) - 1) @@ -1467,7 +1501,7 @@ LOOP: // TestDropColumn is for inserting value with a to-be-dropped column when do drop column. // Column info from schema in build-insert-plan should be public only, // otherwise they will not be consist with Table.Col(), then the server will panic. -func (s *testDBSuite) TestDropColumn(c *C) { +func (s *testDBSuite2) TestDropColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("create database drop_col_db") s.tk.MustExec("use drop_col_db") @@ -1494,7 +1528,7 @@ func (s *testDBSuite) TestDropColumn(c *C) { s.tk.MustExec("drop database drop_col_db") } -func (s *testDBSuite) TestPrimaryKey(c *C) { +func (s *testDBSuite3) TestPrimaryKey(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -1505,7 +1539,7 @@ func (s *testDBSuite) TestPrimaryKey(c *C) { c.Assert(ddl.ErrUnsupportedModifyPrimaryKey.Equal(err), IsTrue) } -func (s *testDBSuite) TestChangeColumn(c *C) { +func (s *testDBSuite4) TestChangeColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -1599,7 +1633,7 @@ func match(c *C, row []interface{}, expected ...interface{}) { } } -func (s *testDBSuite) TestCreateTableWithLike(c *C) { +func (s *testDBSuite5) TestCreateTableWithLike(c *C) { s.tk = testkit.NewTestKit(c, s.store) // for the same database s.tk.MustExec("create database ctwl_db") @@ -1659,7 +1693,7 @@ func (s *testDBSuite) TestCreateTableWithLike(c *C) { } // TestCreateTableWithLike2 tests create table with like when refer table have non-public column/index. -func (s *testDBSuite) TestCreateTableWithLike2(c *C) { +func (s *testDBSuite6) TestCreateTableWithLike2(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test_db") s.tk.MustExec("drop table if exists t1,t2;") @@ -1669,6 +1703,7 @@ func (s *testDBSuite) TestCreateTableWithLike2(c *C) { tbl1 := testGetTableByName(c, s.s, "test_db", "t1") doneCh := make(chan error, 2) hook := &ddl.TestDDLCallback{} + var onceChecker sync.Map hook.OnJobRunBeforeExported = func(job *model.Job) { if job.Type != model.ActionAddColumn && job.Type != model.ActionDropColumn && job.Type != model.ActionAddIndex && job.Type != model.ActionDropIndex { return @@ -1676,7 +1711,13 @@ func (s *testDBSuite) TestCreateTableWithLike2(c *C) { if job.TableID != tbl1.Meta().ID { return } + if job.SchemaState == model.StateDeleteOnly { + if _, ok := onceChecker.Load(job.ID); ok { + return + } + + onceChecker.Store(job.ID, true) go backgroundExec(s.store, "create table t2 like t1", doneCh) } } @@ -1691,7 +1732,6 @@ func (s *testDBSuite) TestCreateTableWithLike2(c *C) { c.Assert(err, IsNil) s.tk.MustExec("alter table t2 add column e int") t2Info := testGetTableByName(c, s.s, "test_db", "t2") - c.Assert(len(t2Info.Meta().Columns), Equals, 4) c.Assert(len(t2Info.Meta().Columns), Equals, len(t2Info.Cols())) } checkTbl2() @@ -1709,10 +1749,11 @@ func (s *testDBSuite) TestCreateTableWithLike2(c *C) { c.Assert(err, IsNil) s.tk.MustExec("alter table t2 add column e int") tbl2 := testGetTableByName(c, s.s, "test_db", "t2") - c.Assert(len(tbl2.Meta().Columns), Equals, 4) c.Assert(len(tbl2.Meta().Columns), Equals, len(tbl2.Cols())) - c.Assert(len(tbl2.Meta().Indices), Equals, 1) - c.Assert(tbl2.Meta().Indices[0].Name.L, Equals, "idx1") + + for i := 0; i < len(tbl2.Meta().Indices); i++ { + c.Assert(tbl2.Meta().Indices[i].State, Equals, model.StatePublic) + } } checkTbl2() @@ -1723,7 +1764,7 @@ func (s *testDBSuite) TestCreateTableWithLike2(c *C) { } -func (s *testDBSuite) TestCreateTable(c *C) { +func (s *testDBSuite1) TestCreateTable(c *C) { s.tk.MustExec("use test") s.tk.MustExec("CREATE TABLE `t` (`a` double DEFAULT 1.0 DEFAULT now() DEFAULT 2.0 );") s.tk.MustExec("CREATE TABLE IF NOT EXISTS `t` (`a` double DEFAULT 1.0 DEFAULT now() DEFAULT 2.0 );") @@ -1763,7 +1804,7 @@ func (s *testDBSuite) TestCreateTable(c *C) { c.Assert(err.Error(), Equals, "[types:1291]Column 'a' has duplicated value 'B' in ENUM") } -func (s *testDBSuite) TestTableForeignKey(c *C) { +func (s *testDBSuite2) TestTableForeignKey(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("create table t1 (a int, b int);") @@ -1777,7 +1818,7 @@ func (s *testDBSuite) TestTableForeignKey(c *C) { s.tk.MustExec("drop table if exists t1,t2,t3;") } -func (s *testDBSuite) TestTruncateTable(c *C) { +func (s *testDBSuite3) TestTruncateTable(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table truncate_table (c1 int, c2 int)") @@ -1824,12 +1865,12 @@ func (s *testDBSuite) TestTruncateTable(c *C) { c.Assert(hasOldTableData, IsFalse) } -func (s *testDBSuite) TestRenameTable(c *C) { +func (s *testDBSuite4) TestRenameTable(c *C) { isAlterTable := false s.testRenameTable(c, "rename table %s to %s", isAlterTable) } -func (s *testDBSuite) TestAlterTableRenameTable(c *C) { +func (s *testDBSuite5) TestAlterTableRenameTable(c *C) { isAlterTable := true s.testRenameTable(c, "alter table %s rename to %s", isAlterTable) } @@ -1931,7 +1972,7 @@ func (s *testDBSuite) testRenameTable(c *C, sql string, isAlterTable bool) { s.tk.MustExec("drop database test1") } -func (s *testDBSuite) TestRenameMultiTables(c *C) { +func (s *testDBSuite1) TestRenameMultiTables(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("create table t1(id int)") @@ -1946,7 +1987,7 @@ func (s *testDBSuite) TestRenameMultiTables(c *C) { s.tk.MustExec("drop table t1, t2") } -func (s *testDBSuite) TestAddNotNullColumn(c *C) { +func (s *testDBSuite2) TestAddNotNullColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test_db") // for different databases @@ -1972,7 +2013,7 @@ out: s.tk.MustExec("drop table tnn") } -func (s *testDBSuite) TestGeneratedColumnDDL(c *C) { +func (s *testDBSuite3) TestGeneratedColumnDDL(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") @@ -2047,7 +2088,7 @@ func (s *testDBSuite) TestGeneratedColumnDDL(c *C) { result.Check(testkit.Rows(`a int(11) YES `, `b bigint(20) YES VIRTUAL GENERATED`, `cnew bigint(20) YES `)) } -func (s *testDBSuite) TestComment(c *C) { +func (s *testDBSuite4) TestComment(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) s.tk.MustExec("drop table if exists ct, ct1") @@ -2077,7 +2118,7 @@ func (s *testDBSuite) TestComment(c *C) { s.tk.MustExec("drop table if exists ct, ct1") } -func (s *testDBSuite) TestRebaseAutoID(c *C) { +func (s *testDBSuite5) TestRebaseAutoID(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) @@ -2105,7 +2146,7 @@ func (s *testDBSuite) TestRebaseAutoID(c *C) { assertErrorCode(c, s.tk, "alter table tidb.test2 add column b int auto_increment key, auto_increment=10;", tmysql.ErrUnknown) } -func (s *testDBSuite) TestCheckColumnDefaultValue(c *C) { +func (s *testDBSuite7) TestCheckColumnDefaultValue(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test;") s.tk.MustExec("drop table if exists text_default_text;") @@ -2156,7 +2197,7 @@ func (s *testDBSuite) TestCheckColumnDefaultValue(c *C) { c.Assert(tblInfo.Meta().Columns[0].DefaultValue, Equals, `null`) } -func (s *testDBSuite) TestCharacterSetInColumns(c *C) { +func (s *testDBSuite1) TestCharacterSetInColumns(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("create database varchar_test;") defer s.tk.MustExec("drop database varchar_test;") @@ -2178,7 +2219,7 @@ func (s *testDBSuite) TestCharacterSetInColumns(c *C) { s.tk.MustExec("create table t15(id int) charset=utf8mb4;") } -func (s *testDBSuite) TestAddNotNullColumnWhileInsertOnDupUpdate(c *C) { +func (s *testDBSuite2) TestAddNotNullColumnWhileInsertOnDupUpdate(c *C) { tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use " + s.schemaName) tk2 := testkit.NewTestKit(c, s.store) @@ -2209,7 +2250,7 @@ func (s *testDBSuite) TestAddNotNullColumnWhileInsertOnDupUpdate(c *C) { c.Assert(tk2Err, IsNil) } -func (s *testDBSuite) TestColumnModifyingDefinition(c *C) { +func (s *testDBSuite3) TestColumnModifyingDefinition(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists test2;") @@ -2234,7 +2275,7 @@ func (s *testDBSuite) TestColumnModifyingDefinition(c *C) { assertErrorCode(c, s.tk, "alter table test2 change c1 a1 bigint not null;", tmysql.WarnDataTruncated) } -func (s *testDBSuite) TestCheckTooBigFieldLength(c *C) { +func (s *testDBSuite4) TestCheckTooBigFieldLength(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists tr_01;") @@ -2260,7 +2301,7 @@ func (s *testDBSuite) TestCheckTooBigFieldLength(c *C) { s.tk.MustExec("alter table tr_05 modify column a varchar(16000) charset utf8mb4;") } -func (s *testDBSuite) TestCheckConvertToCharacter(c *C) { +func (s *testDBSuite5) TestCheckConvertToCharacter(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test") s.tk.MustExec("drop table if exists t") @@ -2282,7 +2323,7 @@ func (s *testDBSuite) TestCheckConvertToCharacter(c *C) { c.Assert(t.Cols()[0].Charset, Equals, "binary") } -func (s *testDBSuite) TestModifyColumnRollBack(c *C) { +func (s *testDBSuite7) TestModifyColumnRollBack(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t1") @@ -2371,7 +2412,7 @@ LOOP: s.mustExec(c, "drop table t1") } -func (s *testDBSuite) TestModifyColumnNullToNotNull(c *C) { +func (s *testDBSuite1) TestModifyColumnNullToNotNull(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t1") @@ -2419,7 +2460,7 @@ func (s *testDBSuite) TestModifyColumnNullToNotNull(c *C) { s.mustExec(c, "drop table t1") } -func (s *testDBSuite) TestTransactionOnAddDropColumn(c *C) { +func (s *testDBSuite2) TestTransactionOnAddDropColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t1") @@ -2474,7 +2515,7 @@ func (s *testDBSuite) TestTransactionOnAddDropColumn(c *C) { s.tk.MustQuery("select a,b from t1 order by a").Check(testkit.Rows("1 1", "1 1", "1 1", "2 2", "2 2", "2 2")) } -func (s *testDBSuite) TestTransactionWithWriteOnlyColumn(c *C) { +func (s *testDBSuite3) TestTransactionWithWriteOnlyColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t1") @@ -2521,7 +2562,7 @@ func (s *testDBSuite) TestTransactionWithWriteOnlyColumn(c *C) { s.tk.MustQuery("select a from t1").Check(testkit.Rows("2")) } -func (s *testDBSuite) TestAddColumn2(c *C) { +func (s *testDBSuite4) TestAddColumn2(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.mustExec(c, "use test_db") s.mustExec(c, "drop table if exists t1") @@ -2591,7 +2632,7 @@ func (s *testDBSuite) TestAddColumn2(c *C) { s.tk.MustQuery("select a,b,_tidb_rowid from t2").Check(testkit.Rows("1 3 2")) } -func (s *testDBSuite) TestAddIndexForGeneratedColumn(c *C) { +func (s *testDBSuite5) TestAddIndexForGeneratedColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test_db") s.tk.MustExec("create table t(y year NOT NULL DEFAULT '2155')") @@ -2629,7 +2670,7 @@ func (s *testDBSuite) TestAddIndexForGeneratedColumn(c *C) { s.tk.MustExec("admin check table gcai_table") } -func (s *testDBSuite) TestIssue9100(c *C) { +func (s *testDBSuite6) TestIssue9100(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test_db") tk.MustExec("create table employ (a int, b int) partition by range (b) (partition p0 values less than (1));") @@ -2644,7 +2685,7 @@ func (s *testDBSuite) TestIssue9100(c *C) { c.Assert(err.Error(), Equals, "[ddl:1503]A UNIQUE INDEX must include all columns in the table's partitioning function") } -func (s *testDBSuite) TestModifyColumnCharset(c *C) { +func (s *testDBSuite1) TestModifyColumnCharset(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use test_db") s.tk.MustExec("create table t_mcc(a varchar(8) charset utf8, b varchar(8) charset utf8)") @@ -2672,7 +2713,7 @@ func (s *testDBSuite) TestModifyColumnCharset(c *C) { } -func (s *testDBSuite) TestAlterShardRowIDBits(c *C) { +func (s *testDBSuite2) TestAlterShardRowIDBits(c *C) { s.tk = testkit.NewTestKit(c, s.store) tk := s.tk From 95b66e48527490238549a22fed7cc300204ae658 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 23 Apr 2019 20:56:18 +0800 Subject: [PATCH 20/21] ddl: ddl-owner try to use memory infoSchema to check first (#10170) --- ddl/ddl.go | 10 ++++---- ddl/ddl_worker.go | 4 +-- ddl/schema.go | 65 +++++++++++++++++++++++++++++++++++++---------- ddl/table.go | 62 +++++++++++++++++++++++++++++++++++++------- 4 files changed, 112 insertions(+), 29 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index c1df201bd9315..2e49ee75b4e2f 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -280,9 +280,8 @@ type DDL interface { // ddl is used to handle the statements that define the structure or schema of the database. type ddl struct { - m sync.RWMutex - infoHandle *infoschema.Handle - quitCh chan struct{} + m sync.RWMutex + quitCh chan struct{} *ddlCtx workers map[workerType]*worker @@ -300,6 +299,7 @@ type ddlCtx struct { ddlEventCh chan<- *util.Event lease time.Duration // lease is schema lease. binlogCli *pumpcli.PumpsClient // binlogCli is used for Binlog. + infoHandle *infoschema.Handle // hook may be modified. mu struct { @@ -380,12 +380,12 @@ func newDDL(ctx context.Context, etcdCli *clientv3.Client, store kv.Storage, ownerManager: manager, schemaSyncer: syncer, binlogCli: binloginfo.GetPumpsClient(), + infoHandle: infoHandle, } ddlCtx.mu.hook = hook ddlCtx.mu.interceptor = &BaseInterceptor{} d := &ddl{ - infoHandle: infoHandle, - ddlCtx: ddlCtx, + ddlCtx: ddlCtx, } d.start(ctx, ctxPool) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 62ed299cc45b1..6a9ed39f5a01e 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -487,7 +487,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, switch job.Type { case model.ActionCreateSchema: - ver, err = onCreateSchema(t, job) + ver, err = onCreateSchema(d, t, job) case model.ActionDropSchema: ver, err = onDropSchema(t, job) case model.ActionCreateTable: @@ -523,7 +523,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionRebaseAutoID: ver, err = onRebaseAutoID(d.store, t, job) case model.ActionRenameTable: - ver, err = onRenameTable(t, job) + ver, err = onRenameTable(d, t, job) case model.ActionShardRowID: ver, err = w.onShardRowID(d, t, job) case model.ActionModifyTableComment: diff --git a/ddl/schema.go b/ddl/schema.go index ac875e158059b..51df4ef1f4b7b 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb/meta" ) -func onCreateSchema(t *meta.Meta, job *model.Job) (ver int64, _ error) { +func onCreateSchema(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { schemaID := job.SchemaID dbInfo := &model.DBInfo{} if err := job.DecodeArgs(dbInfo); err != nil { @@ -32,20 +32,13 @@ func onCreateSchema(t *meta.Meta, job *model.Job) (ver int64, _ error) { dbInfo.ID = schemaID dbInfo.State = model.StateNone - dbs, err := t.ListDatabases() + err := checkSchemaNotExists(d, t, schemaID, dbInfo) if err != nil { - return ver, errors.Trace(err) - } - - for _, db := range dbs { - if db.Name.L == dbInfo.Name.L { - if db.ID != schemaID { - // The database already exists, can't create it, we should cancel this job now. - job.State = model.JobStateCancelled - return ver, infoschema.ErrDatabaseExists.GenWithStackByArgs(db.Name) - } - dbInfo = db + if infoschema.ErrDatabaseExists.Equal(err) { + // The database already exists, can't create it, we should cancel this job now. + job.State = model.JobStateCancelled } + return ver, errors.Trace(err) } ver, err = updateSchemaVersion(t, job) @@ -70,6 +63,52 @@ func onCreateSchema(t *meta.Meta, job *model.Job) (ver int64, _ error) { } } +func checkSchemaNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, dbInfo *model.DBInfo) error { + // d.infoHandle maybe nil in some test. + if d.infoHandle == nil { + return checkSchemaNotExistsFromStore(t, schemaID, dbInfo) + } + // Try to use memory schema info to check first. + currVer, err := t.GetSchemaVersion() + if err != nil { + return err + } + is := d.infoHandle.Get() + if is.SchemaMetaVersion() == currVer { + return checkSchemaNotExistsFromInfoSchema(is, schemaID, dbInfo) + } + return checkSchemaNotExistsFromStore(t, schemaID, dbInfo) +} + +func checkSchemaNotExistsFromInfoSchema(is infoschema.InfoSchema, schemaID int64, dbInfo *model.DBInfo) error { + // Check database exists by name. + if is.SchemaExists(dbInfo.Name) { + return infoschema.ErrDatabaseExists.GenWithStackByArgs(dbInfo.Name) + } + // Check database exists by ID. + if _, ok := is.SchemaByID(schemaID); ok { + return infoschema.ErrDatabaseExists.GenWithStackByArgs(dbInfo.Name) + } + return nil +} + +func checkSchemaNotExistsFromStore(t *meta.Meta, schemaID int64, dbInfo *model.DBInfo) error { + dbs, err := t.ListDatabases() + if err != nil { + return errors.Trace(err) + } + + for _, db := range dbs { + if db.Name.L == dbInfo.Name.L { + if db.ID != schemaID { + return infoschema.ErrDatabaseExists.GenWithStackByArgs(db.Name) + } + dbInfo = db + } + } + return nil +} + func onDropSchema(t *meta.Meta, job *model.Job) (ver int64, _ error) { dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job) if err != nil { diff --git a/ddl/table.go b/ddl/table.go index 179b11235b26b..e0f5764d65d9d 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -52,9 +52,11 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) } tbInfo.State = model.StateNone - err := checkTableNotExists(t, job, schemaID, tbInfo.Name.L) + err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L) if err != nil { - job.State = model.JobStateCancelled + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableExists.Equal(err) { + job.State = model.JobStateCancelled + } return ver, errors.Trace(err) } @@ -96,11 +98,18 @@ func onCreateView(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) return ver, errors.Trace(err) } tbInfo.State = model.StateNone - err := checkTableNotExists(t, job, schemaID, tbInfo.Name.L) + err := checkTableNotExists(d, t, schemaID, tbInfo.Name.L) if err != nil { - if infoschema.ErrDatabaseNotExists.Equal(err) || !orReplace { + if infoschema.ErrDatabaseNotExists.Equal(err) { job.State = model.JobStateCancelled return ver, errors.Trace(err) + } else if infoschema.ErrTableExists.Equal(err) { + if !orReplace { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + } else { + return ver, errors.Trace(err) } } ver, err = updateSchemaVersion(t, job) @@ -193,8 +202,11 @@ func (w *worker) onRecoverTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver in return ver, errors.Trace(err) } - err = checkTableNotExists(t, job, schemaID, tblInfo.Name.L) + err = checkTableNotExists(d, t, schemaID, tblInfo.Name.L) if err != nil { + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableExists.Equal(err) { + job.State = model.JobStateCancelled + } return ver, errors.Trace(err) } @@ -529,7 +541,7 @@ func verifyNoOverflowShardBits(s *sessionPool, tbl table.Table, shardRowIDBits u return nil } -func onRenameTable(t *meta.Meta, job *model.Job) (ver int64, _ error) { +func onRenameTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { var oldSchemaID int64 var tableName model.CIStr if err := job.DecodeArgs(&oldSchemaID, &tableName); err != nil { @@ -543,9 +555,11 @@ func onRenameTable(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } newSchemaID := job.SchemaID - err = checkTableNotExists(t, job, newSchemaID, tableName.L) + err = checkTableNotExists(d, t, newSchemaID, tableName.L) if err != nil { - job.State = model.JobStateCancelled + if infoschema.ErrDatabaseNotExists.Equal(err) || infoschema.ErrTableExists.Equal(err) { + job.State = model.JobStateCancelled + } return ver, errors.Trace(err) } @@ -665,7 +679,37 @@ func onModifyTableCharsetAndCollate(t *meta.Meta, job *model.Job) (ver int64, _ return ver, nil } -func checkTableNotExists(t *meta.Meta, job *model.Job, schemaID int64, tableName string) error { +func checkTableNotExists(d *ddlCtx, t *meta.Meta, schemaID int64, tableName string) error { + // d.infoHandle maybe nil in some test. + if d.infoHandle == nil { + return checkTableNotExistsFromStore(t, schemaID, tableName) + } + // Try to use memory schema info to check first. + currVer, err := t.GetSchemaVersion() + if err != nil { + return err + } + is := d.infoHandle.Get() + if is.SchemaMetaVersion() == currVer { + return checkTableNotExistsFromInfoSchema(is, schemaID, tableName) + } + + return checkTableNotExistsFromStore(t, schemaID, tableName) +} + +func checkTableNotExistsFromInfoSchema(is infoschema.InfoSchema, schemaID int64, tableName string) error { + // Check this table's database. + schema, ok := is.SchemaByID(schemaID) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs("") + } + if is.TableExists(schema.Name, model.NewCIStr(tableName)) { + return infoschema.ErrTableExists.GenWithStackByArgs(tableName) + } + return nil +} + +func checkTableNotExistsFromStore(t *meta.Meta, schemaID int64, tableName string) error { // Check this table's database. tables, err := t.ListTables(schemaID) if err != nil { From 718871646b2dcf870a2d988aab673b7bfbadd815 Mon Sep 17 00:00:00 2001 From: lysu Date: Wed, 24 Apr 2019 09:51:02 +0800 Subject: [PATCH 21/21] executor: reduce alloc and lock-hold-time caused by err/warn count (#10223) --- executor/executor.go | 11 ++++---- sessionctx/stmtctx/stmtctx.go | 47 +++++++++++++++++++++++------------ 2 files changed, 36 insertions(+), 22 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 59676432f0927..cc0da1bdddf46 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1401,18 +1401,17 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { } else if vars.StmtCtx.InSelectStmt { sc.PrevAffectedRows = -1 } - err = vars.SetSystemVar("warning_count", fmt.Sprintf("%d", vars.StmtCtx.NumWarnings(false))) + errCount, warnCount := vars.StmtCtx.NumErrorWarnings() + err = vars.SetSystemVar("warning_count", warnCount) if err != nil { return err } - err = vars.SetSystemVar("error_count", fmt.Sprintf("%d", vars.StmtCtx.NumWarnings(true))) + err = vars.SetSystemVar("error_count", errCount) if err != nil { return err } - if s != nil { - // execute missed stmtID uses empty sql - sc.OriginalSQL = s.Text() - } + // execute missed stmtID uses empty sql + sc.OriginalSQL = s.Text() vars.StmtCtx = sc return } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index d36a8c6e49ca5..d22d65f3ff4a3 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -16,6 +16,7 @@ package stmtctx import ( "math" "sort" + "strconv" "sync" "time" @@ -94,6 +95,7 @@ type StatementContext struct { message string warnings []SQLWarn + errorCount uint16 histogramsNotLoad bool execDetails execdetails.ExecDetails allExecDetails []*execdetails.ExecDetails @@ -267,31 +269,42 @@ func (sc *StatementContext) WarningCount() uint16 { return wc } -// NumWarnings gets warning count. It's different from `WarningCount` in that -// `WarningCount` return the warning count of the last executed command, so if -// the last command is a SHOW statement, `WarningCount` return 0. On the other -// hand, `NumWarnings` always return number of warnings(or errors if `errOnly` -// is set). -func (sc *StatementContext) NumWarnings(errOnly bool) uint16 { - var wc uint16 +const zero = "0" + +// NumErrorWarnings gets warning and error count. +func (sc *StatementContext) NumErrorWarnings() (ec, wc string) { + var ( + ecNum uint16 + wcNum int + ) sc.mu.Lock() - defer sc.mu.Unlock() - if errOnly { - for _, warn := range sc.mu.warnings { - if warn.Level == WarnLevelError { - wc++ - } - } + ecNum = sc.mu.errorCount + wcNum = len(sc.mu.warnings) + sc.mu.Unlock() + + if ecNum == 0 { + ec = zero } else { - wc = uint16(len(sc.mu.warnings)) + ec = strconv.Itoa(int(ecNum)) } - return wc + + if wcNum == 0 { + wc = zero + } else { + wc = strconv.Itoa(wcNum) + } + return } // SetWarnings sets warnings. func (sc *StatementContext) SetWarnings(warns []SQLWarn) { sc.mu.Lock() sc.mu.warnings = warns + for _, w := range warns { + if w.Level == WarnLevelError { + sc.mu.errorCount++ + } + } sc.mu.Unlock() } @@ -318,6 +331,7 @@ func (sc *StatementContext) AppendError(warn error) { sc.mu.Lock() if len(sc.mu.warnings) < math.MaxUint16 { sc.mu.warnings = append(sc.mu.warnings, SQLWarn{WarnLevelError, warn}) + sc.mu.errorCount++ } sc.mu.Unlock() } @@ -377,6 +391,7 @@ func (sc *StatementContext) ResetForRetry() { sc.mu.copied = 0 sc.mu.touched = 0 sc.mu.message = "" + sc.mu.errorCount = 0 sc.mu.warnings = nil sc.mu.execDetails = execdetails.ExecDetails{} sc.mu.allExecDetails = make([]*execdetails.ExecDetails, 0, 4)